diff --git a/src/v/cluster/CMakeLists.txt b/src/v/cluster/CMakeLists.txt index 928a1bdafc52d..8e70bb7254a9f 100644 --- a/src/v/cluster/CMakeLists.txt +++ b/src/v/cluster/CMakeLists.txt @@ -197,6 +197,7 @@ v_cc_library( v::raft Roaring::roaring absl::flat_hash_map + v::security v::model v::cloud_storage v::features diff --git a/src/v/config/CMakeLists.txt b/src/v/config/CMakeLists.txt index f4864caff44b7..808b60df8642c 100644 --- a/src/v/config/CMakeLists.txt +++ b/src/v/config/CMakeLists.txt @@ -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) diff --git a/src/v/kafka/client/CMakeLists.txt b/src/v/kafka/client/CMakeLists.txt index 8b9b012eabf9a..8acd0e99100e6 100644 --- a/src/v/kafka/client/CMakeLists.txt +++ b/src/v/kafka/client/CMakeLists.txt @@ -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 diff --git a/src/v/security/CMakeLists.txt b/src/v/security/CMakeLists.txt index 866fae5776d9c..a80ca0eb1cfea 100644 --- a/src/v/security/CMakeLists.txt +++ b/src/v/security/CMakeLists.txt @@ -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 @@ -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 diff --git a/src/v/security/config_bsl.cc b/src/v/security/config_bsl.cc new file mode 100644 index 0000000000000..0499f567c8d21 --- /dev/null +++ b/src/v/security/config_bsl.cc @@ -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 + +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 make_sv(const std::csub_match& sm) { + return sm.matched + ? std::string_view{sm.first, static_cast(sm.length())} + : std::optional{std::nullopt}; +} + +std::vector +parse_rules(std::optional> 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 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 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 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 +validate_rules(const std::optional>& r) noexcept { + try { + security::tls::detail::parse_rules(r); + } catch (const std::exception& e) { + return e.what(); + } + return std::nullopt; +} + +} // namespace security::tls diff --git a/src/v/security/config_rcl.cc b/src/v/security/config_rcl.cc new file mode 100644 index 0000000000000..035dad7c3e1a1 --- /dev/null +++ b/src/v/security/config_rcl.cc @@ -0,0 +1,143 @@ +/* + * 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 +#include + +#include + +namespace security { + +namespace gssapi::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 +parse_rules(const std::vector& 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 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::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 gssapi::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 +validate_kerberos_mapping_rules(const std::vector& r) noexcept { + try { + gssapi::detail::parse_rules(r); + } catch (const std::exception& e) { + return e.what(); + } + return std::nullopt; +} + +} // namespace security diff --git a/src/v/security/gssapi_principal_mapper.cc b/src/v/security/gssapi_principal_mapper.cc index 3a8a553cef4ed..83ce1c87864b2 100644 --- a/src/v/security/gssapi_principal_mapper.cc +++ b/src/v/security/gssapi_principal_mapper.cc @@ -17,6 +17,7 @@ #include #include #include +#include #include @@ -29,92 +30,17 @@ static std::string_view spv(const re2::StringPiece& sp) { } namespace security { +namespace gssapi { static constexpr std::string_view gssapi_name_pattern = R"(([^/@]*)(/([^/@]*))?@([^/@]*))"; static constexpr std::string_view non_simple_pattern = R"([/@])"; static constexpr std::string_view parameter_pattern = R"(([^$]*)(\$(\d*))?)"; -static constexpr std::string_view rule_pattern - = R"(((DEFAULT)|((RULE:\[(\d*):([^\]]*)](\(([^)]*)\))?(s\/([^\/]*)\/([^\/]*)\/(g)?)?\/?(L|U)?))))"; namespace detail { std::vector -parse_rules(const std::vector& 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 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::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; +parse_rules(const std::vector& unparsed_rules); } -} // namespace detail +} // namespace gssapi gssapi_name::gssapi_name( std::string_view primary, std::string_view host_name, std::string_view realm) @@ -128,7 +54,7 @@ gssapi_name::gssapi_name( std::optional gssapi_name::parse(std::string_view principal_name) { static thread_local const re2::RE2 gssapi_name_regex( - gssapi_name_pattern, re2::RE2::Quiet); + gssapi::gssapi_name_pattern, re2::RE2::Quiet); vassert( gssapi_name_regex.ok(), "Invalid name pattern: {}", @@ -164,32 +90,10 @@ const ss::sstring& gssapi_name::host_name() const noexcept { } const ss::sstring& gssapi_name::realm() const noexcept { return _realm; } - -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 gssapi_rule::apply( std::string_view default_realm, std::vector params) const { static thread_local const re2::RE2 non_simple_regex( - non_simple_pattern, re2::RE2::Quiet); + gssapi::non_simple_pattern, re2::RE2::Quiet); const re2::StringPiece match_piece(_match.data(), _match.size()); const re2::RE2 match_regex(match_piece, re2::RE2::Quiet); vassert( @@ -307,7 +211,7 @@ std::optional gssapi_rule::apply( std::optional gssapi_rule::replace_parameters( std::string_view format, std::vector params) { static thread_local const re2::RE2 parameter_parser( - parameter_pattern, re2::RE2::Quiet); + gssapi::parameter_pattern, re2::RE2::Quiet); vassert( parameter_parser.ok(), "Invalid parameter pattern: {}", @@ -383,9 +287,10 @@ ss::sstring gssapi_rule::replace_substitution( gssapi_principal_mapper::gssapi_principal_mapper( config::binding> principal_to_local_rules_cb) : _principal_to_local_rules_binding(std::move(principal_to_local_rules_cb)) - , _rules{detail::parse_rules(_principal_to_local_rules_binding())} { + , _rules{gssapi::detail::parse_rules(_principal_to_local_rules_binding())} { _principal_to_local_rules_binding.watch([this]() { - _rules = detail::parse_rules(_principal_to_local_rules_binding()); + _rules = gssapi::detail::parse_rules( + _principal_to_local_rules_binding()); }); } @@ -433,15 +338,6 @@ std::ostream& operator<<(std::ostream& os, const gssapi_principal_mapper& m) { return os; } -std::optional -validate_kerberos_mapping_rules(const std::vector& r) noexcept { - try { - detail::parse_rules(r); - } catch (const std::exception& e) { - return e.what(); - } - return std::nullopt; -} } // namespace security // explicit instantiations so as to avoid bringing in in the diff --git a/src/v/security/gssapi_principal_mapper.h b/src/v/security/gssapi_principal_mapper.h index aadb4773f40b8..fec919e07a52f 100644 --- a/src/v/security/gssapi_principal_mapper.h +++ b/src/v/security/gssapi_principal_mapper.h @@ -17,7 +17,6 @@ #include #include -#include #include diff --git a/src/v/security/mtls.cc b/src/v/security/mtls.cc index 3ab523d7adb10..a255e4bd0f4af 100644 --- a/src/v/security/mtls.cc +++ b/src/v/security/mtls.cc @@ -11,7 +11,6 @@ #include "security/mtls.h" -#include #include #include #include @@ -23,143 +22,13 @@ 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 make_sv(const std::csub_match& sm) { - return sm.matched - ? std::string_view{sm.first, static_cast(sm.length())} - : std::optional{std::nullopt}; -} - std::vector -parse_rules(std::optional> 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 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; -} +parse_rules(std::optional> unparsed_rules); } // namespace detail -rule::rule( - std::string_view pattern, - std::optional 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 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 -validate_rules(const std::optional>& r) noexcept { - try { - security::tls::detail::parse_rules(r); - } catch (const std::exception& e) { - return e.what(); - } - return std::nullopt; -} +validate_rules(const std::optional>& r) noexcept; std::ostream& operator<<(std::ostream& os, const rule& r) { fmt::print(os, "{}", r); diff --git a/src/v/security/tests/CMakeLists.txt b/src/v/security/tests/CMakeLists.txt index 042654cf548ca..a55d61c19402f 100644 --- a/src/v/security/tests/CMakeLists.txt +++ b/src/v/security/tests/CMakeLists.txt @@ -9,7 +9,7 @@ rp_test( license_test.cc gssapi_principal_mapper_test.cc DEFINITIONS BOOST_TEST_DYN_LINK - LIBRARIES Boost::unit_test_framework v::kafka_protocol v::storage + LIBRARIES Boost::unit_test_framework v::kafka_protocol v::storage v::security LABELS kafka )