Skip to content

Commit

Permalink
security: Break dependency from v::config to v::security
Browse files Browse the repository at this point in the history
Split v::security_config out from v::security

This will allow v::security to depend on v::http

Signed-off-by: Ben Pope <ben@redpanda.com>
  • Loading branch information
BenPope committed Sep 29, 2023
1 parent a030134 commit 1177f6f
Show file tree
Hide file tree
Showing 10 changed files with 326 additions and 238 deletions.
1 change: 1 addition & 0 deletions src/v/cluster/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,7 @@ v_cc_library(
v::raft
Roaring::roaring
absl::flat_hash_map
v::security
v::model
v::cloud_storage
v::features
Expand Down
8 changes: 4 additions & 4 deletions src/v/config/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,9 @@ v_cc_library(
DEPS
v::json
v::model
v::security
boost_filesystem
absl::node_hash_set
yaml-cpp
v::security_config
boost_filesystem
absl::node_hash_set
yaml-cpp
)
add_subdirectory(tests)
1 change: 1 addition & 0 deletions src/v/kafka/client/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ v_cc_library(
sasl_client.cc
DEPS
v::kafka_protocol
v::security
v::ssx
v::storage # batch builder
v::cluster # log cert reload helper
Expand Down
13 changes: 13 additions & 0 deletions src/v/security/CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -1,3 +1,14 @@
v_cc_library(
NAME security_config
SRCS
config_bsl.cc
config_rcl.cc
DEPS
v::json
Seastar::seastar
re2
)

v_cc_library(
NAME security
SRCS
Expand All @@ -12,6 +23,8 @@ v_cc_library(
krb5_configurator.cc
gssapi_principal_mapper.cc
DEPS
v::security_config
v::config
v::bytes
v::utils
v::rprandom
Expand Down
157 changes: 157 additions & 0 deletions src/v/security/config_bsl.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* Copyright 2022 Redpanda Data, Inc.
*
* Use of this software is governed by the Business Source License
* included in the file licenses/BSL.md
*
* As of the Change Date specified in that file, in accordance with
* the Business Source License, use of this software will be governed
* by the Apache License, Version 2.0
*/

#include "security/mtls.h"

#include <boost/algorithm/string/case_conv.hpp>

namespace security::tls {
namespace detail {

static constexpr const char* const rule_pattern{
R"((DEFAULT)|RULE:((\\.|[^\\/])*)\/((\\.|[^\\/])*)\/([LU]?).*?|(.*?))"};
static constexpr const char* const rule_pattern_splitter{
R"(\s*((DEFAULT)|RULE:((\\.|[^\\/])*)\/((\\.|[^\\/])*)\/([LU]?).*?|(.*?))\s*([,\n]\s*|$))"};

std::regex make_regex(std::string_view sv) {
return std::regex{
sv.begin(),
sv.length(),
std::regex_constants::ECMAScript | std::regex_constants::optimize};
}

bool regex_search(
std::string_view msg, std::cmatch& match, std::regex const& regex) {
return std::regex_search(
msg.begin(),
msg.end(),
match,
regex,
std::regex_constants::match_default);
}

bool regex_match(
std::string_view msg, std::cmatch& match, std::regex const& regex) {
return std::regex_match(
msg.begin(),
msg.end(),
match,
regex,
std::regex_constants::match_default);
}

constexpr std::string_view trim(std::string_view s) {
constexpr auto ws = " \t\n\r\f\v";
s.remove_prefix(std::min(s.find_first_not_of(ws), s.size()));
s.remove_suffix(std::min(s.size() - s.find_last_not_of(ws) - 1, s.size()));
return s;
}

constexpr std::optional<std::string_view> make_sv(const std::csub_match& sm) {
return sm.matched
? std::string_view{sm.first, static_cast<size_t>(sm.length())}
: std::optional<std::string_view>{std::nullopt};
}

std::vector<rule>
parse_rules(std::optional<std::vector<ss::sstring>> unparsed_rules) {
static const std::regex rule_splitter = make_regex(rule_pattern_splitter);
static const std::regex rule_parser = make_regex(rule_pattern);

std::string rules
= unparsed_rules.has_value() ? fmt::format(
"{}", fmt::join(unparsed_rules->begin(), unparsed_rules->end(), ","))
: "DEFAULT";

std::vector<rule> result;
std::cmatch rules_match;
while (!rules.empty() && regex_search(rules, rules_match, rule_splitter)) {
const auto& rule{rules_match[1]};

std::cmatch components_match;
if (!regex_search(*make_sv(rule), components_match, rule_parser)) {
throw std::runtime_error("Invalid rule: " + rule.str());
}
if (components_match.prefix().matched) {
throw std::runtime_error(
"Invalid rule - prefix: " + components_match.prefix().str());
}
if (components_match.suffix().matched) {
throw std::runtime_error(
"Invalid rule - suffix: " + components_match.suffix().str());
}

if (components_match[1].matched) {
result.emplace_back();
} else if (components_match[2].matched) {
const auto adjust_case = make_sv(components_match[6]);
result.emplace_back(
*make_sv(components_match[2]),
make_sv(components_match[4]),
rule::make_lower{adjust_case == "L"},
rule::make_upper{adjust_case == "U"});
}
rules = make_sv(rules_match.suffix()).value_or("");
}
return result;
}

} // namespace detail

rule::rule(
std::string_view pattern,
std::optional<std::string_view> replacement,
make_lower to_lower,
make_upper to_upper)
: _regex{detail::make_regex(pattern)}
, _pattern{pattern}
, _replacement{replacement}
, _is_default{false}
, _to_lower{to_lower}
, _to_upper{to_upper} {}

std::optional<ss::sstring> rule::apply(std::string_view dn) const {
if (_is_default) {
return ss::sstring{dn};
}

std::cmatch match;
if (!std::regex_match(dn.cbegin(), dn.cend(), match, _regex)) {
return {};
}

std::string result;
std::regex_replace(
std::back_inserter(result),
dn.begin(),
dn.end(),
_regex,
_replacement.value_or("").c_str());

if (_to_lower) {
boost::algorithm::to_lower(result, std::locale::classic());
} else if (_to_upper) {
boost::algorithm::to_upper(result, std::locale::classic());
}
return result;
}

std::optional<ss::sstring>
validate_rules(const std::optional<std::vector<ss::sstring>>& r) noexcept {
try {
security::tls::detail::parse_rules(r);
} catch (const std::exception& e) {
return e.what();
}
return std::nullopt;
}

} // namespace security::tls
144 changes: 144 additions & 0 deletions src/v/security/config_rcl.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,144 @@
/*
* Copyright 2023 Redpanda Data, Inc.
*
* Licensed as a Redpanda Enterprise file under the Redpanda Community
* License (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md
*/

#include "security/gssapi_principal_mapper.h"

#include <boost/algorithm/string/case_conv.hpp>
#include <re2/re2.h>

#include <charconv>

namespace security {

namespace detail {

static constexpr std::string_view rule_pattern
= R"(((DEFAULT)|((RULE:\[(\d*):([^\]]*)](\(([^)]*)\))?(s\/([^\/]*)\/([^\/]*)\/(g)?)?\/?(L|U)?))))";

/*
* some older versions of re2 don't have operator for implicit cast to
* string_view so add this helper to support older re2.
*/
static std::string_view spv(const re2::StringPiece& sp) {
return {sp.data(), sp.size()};
}

std::vector<gssapi_rule>
parse_rules(const std::vector<ss::sstring>& unparsed_rules) {
static thread_local const re2::RE2 rule_parser(
rule_pattern, re2::RE2::Quiet);

vassert(
rule_parser.ok(),
"Failed to build rule pattern regex: {}",
rule_parser.error());

if (unparsed_rules.empty()) {
return {gssapi_rule()};
}

std::vector<gssapi_rule> rv;
re2::StringPiece default_;
re2::StringPiece num_components_str;
re2::StringPiece format;
re2::StringPiece match_regex;
re2::StringPiece from_pattern;
re2::StringPiece to_pattern;
re2::StringPiece repeat;
re2::StringPiece upper_lower;
for (const auto& rule : unparsed_rules) {
const re2::StringPiece rule_piece(rule.data(), rule.size());
if (!re2::RE2::FullMatch(
rule_piece,
rule_parser,
nullptr,
&default_,
nullptr,
nullptr,
&num_components_str,
&format,
nullptr,
&match_regex,
nullptr,
&from_pattern,
&to_pattern,
&repeat,
&upper_lower)) {
throw std::runtime_error("GSSAPI: Invalid rule: " + rule);
}
if (!default_.empty()) {
rv.emplace_back();
} else {
int num_components = std::numeric_limits<int>::max();
auto conv_rc = std::from_chars(
num_components_str.begin(),
num_components_str.end(),
num_components);
if (conv_rc.ec != std::errc()) {
throw std::runtime_error(
"Invalid rule - Invalid value for number of components: "
+ num_components_str.as_string());
}
gssapi_rule::case_change_operation case_change
= gssapi_rule::case_change_operation::noop;

if (upper_lower == "L") {
case_change = gssapi_rule::case_change_operation::make_lower;
} else if (upper_lower == "U") {
case_change = gssapi_rule::case_change_operation::make_upper;
}
rv.emplace_back(
num_components,
spv(format),
spv(match_regex),
spv(from_pattern),
spv(to_pattern),
gssapi_rule::repeat{repeat == "g"},
case_change);
}
}

return rv;
}

} // namespace detail

gssapi_rule::gssapi_rule(
int number_of_components,
std::string_view format,
std::string_view match,
std::string_view from_pattern,
std::string_view to_pattern,
repeat repeat_,
case_change_operation case_change)
: _is_default(false)
, _number_of_components(number_of_components)
, _format(format)
, _match(match)
, _from_pattern(std::regex{
from_pattern.begin(),
from_pattern.length(),
std::regex_constants::ECMAScript | std::regex_constants::optimize})
, _from_pattern_str(from_pattern)
, _to_pattern(to_pattern)
, _repeat(repeat_)
, _case_change(case_change) {}

std::optional<ss::sstring>
validate_kerberos_mapping_rules(const std::vector<ss::sstring>& r) noexcept {
try {
detail::parse_rules(r);
} catch (const std::exception& e) {
return e.what();
}
return std::nullopt;
}

} // namespace security
Loading

0 comments on commit 1177f6f

Please sign in to comment.