From 8885778cca24c6d133d5bbd2abfc6ab07c0d6e98 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 20:37:36 +0000 Subject: [PATCH 01/12] security: Extract validate_scram_credential Pure refactor, no change in behaviour. Signed-off-by: Ben Pope --- src/v/security/request_auth.cc | 25 ++++--------------------- src/v/security/scram_authenticator.cc | 14 ++++++++++++++ src/v/security/scram_authenticator.h | 4 ++++ 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/src/v/security/request_auth.cc b/src/v/security/request_auth.cc index eaf12e5eb8e66..63743bad2ac4f 100644 --- a/src/v/security/request_auth.cc +++ b/src/v/security/request_auth.cc @@ -17,7 +17,6 @@ #include "seastar/http/exception.hh" #include "security/credential_store.h" #include "security/oidc_authenticator.h" -#include "security/scram_algorithm.h" #include "security/scram_authenticator.h" #include "security/types.h" @@ -124,25 +123,9 @@ request_auth_result request_authenticator::do_authenticate( throw unauthorized_user_exception( std::move(username), "Unauthorized"); } else { - const auto& cred = cred_opt.value(); - ss::sstring sasl_mechanism; - bool is_valid{false}; - if (security::scram_sha256::validate_password( - password, - cred.stored_key(), - cred.salt(), - cred.iterations())) { - is_valid = true; - sasl_mechanism = security::scram_sha256_authenticator::name; - } else if (security::scram_sha512::validate_password( - password, - cred.stored_key(), - cred.salt(), - cred.iterations())) { - is_valid = true; - sasl_mechanism = security::scram_sha512_authenticator::name; - } - if (!is_valid) { + auto sasl_mechanism = validate_scram_credential( + *cred_opt, password); + if (!sasl_mechanism.has_value()) { // User found, password doesn't match vlog( logger.warn, @@ -159,7 +142,7 @@ request_auth_result request_authenticator::do_authenticate( return request_auth_result( std::move(username), std::move(password), - std::move(sasl_mechanism), + ss::sstring{*sasl_mechanism}, request_auth_result::superuser(superuser)); } } diff --git a/src/v/security/scram_authenticator.cc b/src/v/security/scram_authenticator.cc index 9dc87af60fa3c..8333342499277 100644 --- a/src/v/security/scram_authenticator.cc +++ b/src/v/security/scram_authenticator.cc @@ -137,4 +137,18 @@ scram_authenticator::authenticate(bytes auth_bytes) { template class scram_authenticator; template class scram_authenticator; +std::optional validate_scram_credential( + const scram_credential& cred, const credential_password& password) { + std::optional sasl_mechanism; + if (security::scram_sha256::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { + sasl_mechanism = security::scram_sha256_authenticator::name; + } else if (security::scram_sha512::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { + sasl_mechanism = security::scram_sha512_authenticator::name; + } + + return sasl_mechanism; +} + } // namespace security diff --git a/src/v/security/scram_authenticator.h b/src/v/security/scram_authenticator.h index a3509cddb6f5c..31858390e6f39 100644 --- a/src/v/security/scram_authenticator.h +++ b/src/v/security/scram_authenticator.h @@ -13,6 +13,7 @@ #include "security/fwd.h" #include "security/sasl_authentication.h" #include "security/scram_algorithm.h" +#include "security/types.h" namespace security { @@ -80,4 +81,7 @@ struct scram_sha512_authenticator { static constexpr const char* name = "SCRAM-SHA-512"; }; +std::optional validate_scram_credential( + const scram_credential& cred, const credential_password& password); + } // namespace security From 8ace98e9e916bb1902ef4d8489286c92407a92b9 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 20:40:42 +0000 Subject: [PATCH 02/12] security: Improve validate_scram_credential Optimise the password validation when SCRAM-SHA-512 is in use, by avoiding the validation against SCRAM-SHA-256 that will fail. Signed-off-by: Ben Pope --- src/v/security/scram_algorithm.h | 2 ++ src/v/security/scram_authenticator.cc | 12 ++++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/v/security/scram_algorithm.h b/src/v/security/scram_algorithm.h index 25c4dfc3f9c2d..bfa631d5f65ba 100644 --- a/src/v/security/scram_algorithm.h +++ b/src/v/security/scram_algorithm.h @@ -196,6 +196,8 @@ class scram_algorithm { static constexpr int min_iterations = MinIterations; static_assert(min_iterations > 0, "Minimum iterations must be positive"); + static constexpr auto key_size = HashType::digest_size; + static bytes client_signature( bytes_view stored_key, const client_first_message& client_first, diff --git a/src/v/security/scram_authenticator.cc b/src/v/security/scram_authenticator.cc index 8333342499277..7247d54f8f20b 100644 --- a/src/v/security/scram_authenticator.cc +++ b/src/v/security/scram_authenticator.cc @@ -140,11 +140,15 @@ template class scram_authenticator; std::optional validate_scram_credential( const scram_credential& cred, const credential_password& password) { std::optional sasl_mechanism; - if (security::scram_sha256::validate_password( - password, cred.stored_key(), cred.salt(), cred.iterations())) { + if ( + cred.stored_key().size() == security::scram_sha256::key_size + && security::scram_sha256::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { sasl_mechanism = security::scram_sha256_authenticator::name; - } else if (security::scram_sha512::validate_password( - password, cred.stored_key(), cred.salt(), cred.iterations())) { + } else if ( + cred.stored_key().size() == security::scram_sha512::key_size + && security::scram_sha512::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { sasl_mechanism = security::scram_sha512_authenticator::name; } From 6cab505cc903cd20fa4edd18743418c29d155f4b Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 19:53:26 +0000 Subject: [PATCH 03/12] security: Introduce plain_authenticator SUCCESS-02 Signed-off-by: Ben Pope --- src/v/security/BUILD | 2 + src/v/security/CMakeLists.txt | 1 + src/v/security/plain_authenticator.cc | 108 ++++++++++++++++++++++++++ src/v/security/plain_authenticator.h | 62 +++++++++++++++ 4 files changed, 173 insertions(+) create mode 100644 src/v/security/plain_authenticator.cc create mode 100644 src/v/security/plain_authenticator.h diff --git a/src/v/security/BUILD b/src/v/security/BUILD index 71a5bc1242cb2..74f58a80d099f 100644 --- a/src/v/security/BUILD +++ b/src/v/security/BUILD @@ -91,6 +91,7 @@ redpanda_cc_library( "oidc_authenticator.cc", "oidc_principal_mapping_applicator.cc", "oidc_service.cc", + "plain_authenticator.cc", "role.cc", "scram_algorithm.cc", "scram_authenticator.cc", @@ -118,6 +119,7 @@ redpanda_cc_library( "oidc_error.h", "oidc_principal_mapping_applicator.h", "oidc_service.h", + "plain_authenticator.h", "role.h", "role_store.h", "sasl_authentication.h", diff --git a/src/v/security/CMakeLists.txt b/src/v/security/CMakeLists.txt index 4471df8ca2819..2add10498168f 100644 --- a/src/v/security/CMakeLists.txt +++ b/src/v/security/CMakeLists.txt @@ -29,6 +29,7 @@ v_cc_library( oidc_authenticator.cc oidc_principal_mapping_applicator.cc oidc_service.cc + plain_authenticator.cc request_auth.cc role.cc scram_algorithm.cc diff --git a/src/v/security/plain_authenticator.cc b/src/v/security/plain_authenticator.cc new file mode 100644 index 0000000000000..21d048954ce7c --- /dev/null +++ b/src/v/security/plain_authenticator.cc @@ -0,0 +1,108 @@ +/* + * Copyright 2024 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/plain_authenticator.h" + +#include "base/vlog.h" +#include "security/acl.h" +#include "security/credential_store.h" +#include "security/errc.h" +#include "security/logger.h" +#include "security/scram_authenticator.h" +#include "security/types.h" +#include "strings/utf8.h" + +#include + +namespace security { + +ss::future> plain_authenticator::authenticate(bytes auth_bytes) { + constexpr size_t max_length{255}; + constexpr std::string_view sep{"\0", 1}; + + auto make_failed = ss::defer([this] { _state = state::failed; }); + + if (_state != state::init) { + vlog( + seclog.warn, + "invalid plain state: {}", + _state == state::failed ? "failed" : "complete"); + co_return errc::invalid_credentials; + } + + auto auth_str = std::string_view( + reinterpret_cast(auth_bytes.data()), auth_bytes.size()); + + if (!is_valid_utf8(auth_str)) { + vlog(seclog.warn, "invalid utf8"); + co_return errc::invalid_credentials; + } + + // [authorization identity] not supported + if (!auth_str.starts_with(sep)) { + vlog(seclog.warn, "[authorization identity] not supported"); + co_return errc::invalid_credentials; + } + auth_str = auth_str.substr(sep.length()); + auto it = auth_str.find(sep); + if (std::string_view::npos == it) { + vlog(seclog.warn, "seperator not found"); + co_return errc::invalid_credentials; + } + + credential_user username{auth_str.substr(0, it)}; + credential_password password{auth_str.substr(it + sep.length())}; + + if (username().empty()) { + vlog(seclog.warn, "username not found"); + co_return errc::invalid_credentials; + } + + if (username().length() > max_length) { + vlog(seclog.warn, "username too long"); + co_return errc::invalid_credentials; + } + + if (password().empty()) { + vlog(seclog.warn, "password not found"); + co_return errc::invalid_credentials; + } + + if (password().length() > max_length) { + vlog(seclog.warn, "password too long"); + co_return errc::invalid_credentials; + } + + _audit_user.name = username; + auto cred = _credentials.get(username); + if (!cred.has_value()) { + vlog(seclog.warn, "credential not found"); + co_return errc::invalid_credentials; + } + + if (!validate_scram_credential(*cred, password).has_value()) { + vlog(seclog.warn, "scram authentication failed"); + co_return errc::invalid_credentials; + } + + vlog(seclog.trace, "Authenticated user {}", username); + + make_failed.cancel(); + + _principal = cred->principal().value_or( + acl_principal{principal_type::user, username()}); + _audit_user.name = _principal.name(); + _audit_user.type_id = audit::user::type::user; + + _state = state::complete; + co_return bytes{}; +} + +} // namespace security diff --git a/src/v/security/plain_authenticator.h b/src/v/security/plain_authenticator.h new file mode 100644 index 0000000000000..468a3b8ce3e97 --- /dev/null +++ b/src/v/security/plain_authenticator.h @@ -0,0 +1,62 @@ +/* + * Copyright 2024 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 + */ +#pragma once +#include "security/acl.h" +#include "security/fwd.h" +#include "security/sasl_authentication.h" + +namespace security { + +/** + * @class plain_authenticator + * @brief A class that implements SASL/PLAIN authentication mechanism. + * + * This class is responsible for handling the SASL/PLAIN authentication process. + * It authenticates the username and password provided by the client against + * SCRAM users in the credential store. + */ +class plain_authenticator final : public sasl_mechanism { +public: + static constexpr const char* name = "PLAIN"; + + explicit plain_authenticator(credential_store& credentials) + : _credentials(credentials) {} + + ss::future> authenticate(bytes auth_bytes) override; + + bool complete() const override { return _state == state::complete; } + bool failed() const override { return _state == state::failed; } + + const acl_principal& principal() const override { + vassert( + _state == state::complete, + "Authentication id is not valid until auth process complete"); + return _principal; + } + + const audit::user& audit_user() const override { return _audit_user; } + + const char* mechanism_name() const override { return "SASL-PLAIN"; } + +private: + enum class state { + init, + complete, + failed, + }; + + state _state{state::init}; + credential_store& _credentials; + acl_principal _principal; + security::audit::user _audit_user; +}; + +} // namespace security From 08d63c9224d65c05d8d7f4a94caf137d418c165c Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:30:01 +0000 Subject: [PATCH 04/12] config: Add configuration for SASL/PLAIN Signed-off-by: Ben Pope --- src/v/config/configuration.cc | 2 +- src/v/config/validators.cc | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index d9a064eecb5a4..d0ed6d851da9d 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -1500,7 +1500,7 @@ configuration::configuration() std::vector{"GSSAPI", "OAUTHBEARER"}, "sasl_mechanisms", "A list of supported SASL mechanisms. Accepted values: `SCRAM`, " - "`GSSAPI`, `OAUTHBEARER`.", + "`GSSAPI`, `OAUTHBEARER`, `PLAIN`.", meta{ .needs_restart = needs_restart::no, .visibility = visibility::user, diff --git a/src/v/config/validators.cc b/src/v/config/validators.cc index d5576ac3b0b7a..3365e132a6d76 100644 --- a/src/v/config/validators.cc +++ b/src/v/config/validators.cc @@ -115,7 +115,7 @@ std::optional validate_client_groups_byte_rate_quota( std::optional validate_sasl_mechanisms(const std::vector& mechanisms) { constexpr auto supported = std::to_array( - {"GSSAPI", "SCRAM", "OAUTHBEARER"}); + {"GSSAPI", "SCRAM", "OAUTHBEARER", "PLAIN"}); // Validate results for (const auto& m : mechanisms) { @@ -124,6 +124,12 @@ validate_sasl_mechanisms(const std::vector& mechanisms) { return ssx::sformat("'{}' is not a supported SASL mechanism", m); } } + + if (mechanisms.size() == 1 && mechanisms[0] == "PLAIN") { + return "When PLAIN is enabled, at least one other mechanism must be " + "enabled"; + } + return std::nullopt; } From 4beb86aa0d202ed8749be82f47b09ebe662c4274 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:30:32 +0000 Subject: [PATCH 05/12] kafka: Wire up SASL/PLAIN Signed-off-by: Ben Pope --- src/v/kafka/server/server.cc | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/v/kafka/server/server.cc b/src/v/kafka/server/server.cc index ba04ef527d656..9781434facd58 100644 --- a/src/v/kafka/server/server.cc +++ b/src/v/kafka/server/server.cc @@ -65,6 +65,7 @@ #include "security/gssapi_authenticator.h" #include "security/mtls.h" #include "security/oidc_authenticator.h" +#include "security/plain_authenticator.h" #include "security/scram_algorithm.h" #include "security/scram_authenticator.h" #include "ssx/future-util.h" @@ -720,6 +721,16 @@ ss::future sasl_handshake_handler::handle( } } + if (supports("PLAIN")) { + supported_sasl_mechanisms.emplace_back( + security::plain_authenticator::name); + if (request.data.mechanism == security::plain_authenticator::name) { + ctx.sasl()->set_mechanism( + std::make_unique( + ctx.credentials())); + } + } + if (supports("GSSAPI")) { supported_sasl_mechanisms.emplace_back( security::gssapi_authenticator::name); From ca1b2ec65e1bf7d5283cb16721d4b58dff98210a Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:37:14 +0000 Subject: [PATCH 06/12] feature_manager: Refactor any_of from absl to std::ranges No change in behaviour Signed-off-by: Ben Pope --- src/v/cluster/feature_manager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 97457501c73e6..3e6dc81ee8dd4 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -34,9 +34,9 @@ #include -#include #include +#include #include namespace cluster { @@ -208,7 +208,7 @@ feature_manager::report_enterprise_features() const { const auto& cfg = config::shard_local_cfg(); const auto& node_cfg = config::node(); auto has_gssapi = [&cfg]() { - return absl::c_any_of( + return std::ranges::any_of( cfg.sasl_mechanisms(), [](const auto& m) { return m == "GSSAPI"; }); }; auto has_oidc = []() { From 588ec77af989fa04233fb3aea64e01527c52bc39 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:20:03 +0000 Subject: [PATCH 07/12] feature_manager: Log bad SASL/PLAIN config REQ-02 Signed-off-by: Ben Pope --- src/v/cluster/feature_manager.cc | 46 ++++++++++++++++---- src/v/cluster/feature_manager.h | 4 +- tests/rptest/tests/rbac_test.py | 18 +++----- tests/rptest/tests/redpanda_kerberos_test.py | 18 +++----- tests/rptest/tests/redpanda_oauth_test.py | 20 ++++----- tests/rptest/tests/redpanda_startup_test.py | 12 ++--- tests/rptest/tests/schema_registry_test.py | 18 +++----- tests/rptest/utils/log_utils.py | 25 +++++++++++ 8 files changed, 98 insertions(+), 63 deletions(-) create mode 100644 tests/rptest/utils/log_utils.py diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 3e6dc81ee8dd4..5df81b80ad2ff 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -20,7 +20,9 @@ #include "cluster/logger.h" #include "cluster/members_table.h" #include "config/configuration.h" +#include "config/endpoint_tls_config.h" #include "config/node_config.h" +#include "config/tls_config.h" #include "config/types.h" #include "config/validators.h" #include "features/enterprise_feature_messages.h" @@ -184,7 +186,7 @@ feature_manager::start(std::vector&& cluster_founder_nodes) { ssx::background = ssx::spawn_with_gate_then(_gate, [this] { return ss::do_until( [this] { return _as.local().abort_requested(); }, - [this] { return maybe_log_license_check_info(); }); + [this] { return maybe_log_periodic_reminders(); }); }); for (const model::node_id n : cluster_founder_nodes) { @@ -274,32 +276,38 @@ feature_manager::report_enterprise_features() const { return report; } -ss::future<> feature_manager::maybe_log_license_check_info() { - auto license_check_retry = std::chrono::seconds(60 * 5); +ss::future<> feature_manager::maybe_log_periodic_reminders() { + auto reminder_period = std::chrono::seconds(60 * 5); auto interval_override = std::getenv( "__REDPANDA_LICENSE_CHECK_INTERVAL_SEC"); if (interval_override != nullptr) { try { - license_check_retry = std::min( - std::chrono::seconds{license_check_retry}, + reminder_period = std::min( + std::chrono::seconds{reminder_period}, std::chrono::seconds{std::stoi(interval_override)}); vlog( clusterlog.info, - "Overriding default license log annoy interval to: {}s", - license_check_retry.count()); + "Overriding default reminder period interval to: {}s", + reminder_period.count()); } catch (...) { vlog( clusterlog.error, - "Invalid license check interval override '{}'", + "Invalid reminder period interval override '{}'", interval_override); } } try { - co_await ss::sleep_abortable(license_check_retry, _as.local()); + co_await ss::sleep_abortable(reminder_period, _as.local()); } catch (const ss::sleep_aborted&) { // Shutting down - next iteration will drop out co_return; } + + maybe_log_license_nag(); + maybe_log_security_nag(); +} + +void feature_manager::maybe_log_license_nag() { auto enterprise_features = report_enterprise_features(); if (enterprise_features.any()) { if (_feature_table.local().should_sanction()) { @@ -312,6 +320,26 @@ ss::future<> feature_manager::maybe_log_license_check_info() { } } +void feature_manager::maybe_log_security_nag() { + if (std::ranges::any_of( + config::shard_local_cfg().sasl_mechanisms(), + [](const auto& m) { return m == "PLAIN"; })) { + const bool any_tls_disabled + = std::ranges::any_of( + config::node_config().kafka_api_tls.value(), + [](const config::endpoint_tls_config& cfg) { + return !cfg.config.is_enabled(); + }) + || config::node_config().kafka_api_tls.value().empty(); + + vlogl( + clusterlog, + any_tls_disabled ? ss::log_level::error : ss::log_level::warn, + "SASL/PLAIN is enabled. This is insecure and not recommended for " + "production."); + } +} + bool feature_manager::need_to_verify_enterprise_license() { return features::is_major_version_upgrade( _feature_table.local().get_active_version(), diff --git a/src/v/cluster/feature_manager.h b/src/v/cluster/feature_manager.h index c2a4deea6e4ef..d6ebfe94912e0 100644 --- a/src/v/cluster/feature_manager.h +++ b/src/v/cluster/feature_manager.h @@ -142,7 +142,9 @@ class feature_manager { && _am_controller_leader; } - ss::future<> maybe_log_license_check_info(); + ss::future<> maybe_log_periodic_reminders(); + void maybe_log_license_nag(); + void maybe_log_security_nag(); bool need_to_verify_enterprise_license(); // Compose a command struct, replicate it via raft and wait for apply. diff --git a/tests/rptest/tests/rbac_test.py b/tests/rptest/tests/rbac_test.py index 28c6f35d7740e..66f6bb462310a 100644 --- a/tests/rptest/tests/rbac_test.py +++ b/tests/rptest/tests/rbac_test.py @@ -30,6 +30,7 @@ from rptest.tests.admin_api_auth_test import create_user_and_wait from rptest.tests.metrics_reporter_test import MetricsReporterServer from rptest.util import expect_exception, expect_http_error, wait_until_result +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode ALICE = SaslCredentials("alice", "itsMeH0nest", "SCRAM-SHA-256") @@ -633,17 +634,12 @@ def __init__(self, test_ctx, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}' }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=1) @skip_fips_mode # See NOTE below def test_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -658,9 +654,9 @@ def test_license_nag(self): {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': '1'}) self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_kerberos_test.py b/tests/rptest/tests/redpanda_kerberos_test.py index ee56a0864ca44..a764651201eaa 100644 --- a/tests/rptest/tests/redpanda_kerberos_test.py +++ b/tests/rptest/tests/redpanda_kerberos_test.py @@ -23,6 +23,7 @@ from rptest.services.kerberos import KrbKdc, KrbClient, RedpandaKerberosNode, AuthenticationError, KRB5_CONF_PATH, render_krb5_config, ActiveDirectoryKdc from rptest.services.redpanda import LoggingConfig, RedpandaService, SecurityConfig from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode from rptest.utils.rpenv import IsCIOrNotEmpty @@ -170,17 +171,12 @@ def __init__(self, test_context, num_nodes=3, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below def test_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -197,9 +193,9 @@ def test_license_nag(self): self.redpanda.stop() self.redpanda.start(clean_nodes=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_oauth_test.py b/tests/rptest/tests/redpanda_oauth_test.py index 41bcf8c0eaa50..b28a983044b0b 100644 --- a/tests/rptest/tests/redpanda_oauth_test.py +++ b/tests/rptest/tests/redpanda_oauth_test.py @@ -20,9 +20,10 @@ from rptest.services.cluster import cluster from rptest.services.tls import TLSCertManager from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC +from rptest.tests.tls_metrics_test import FaketimeTLSProvider from rptest.util import expect_exception +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode -from rptest.tests.tls_metrics_test import FaketimeTLSProvider import requests import time @@ -612,19 +613,14 @@ def __init__(self, test_context, num_nodes=3, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below @parametrize(authn_config={"sasl_mechanisms": ["OAUTHBEARER", "SCRAM"]}) @parametrize(authn_config={"http_authentication": ["OIDC", "BASIC"]}) def test_license_nag(self, authn_config): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -640,9 +636,9 @@ def test_license_nag(self, authn_config): self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_startup_test.py b/tests/rptest/tests/redpanda_startup_test.py index 8256adccff9d3..818f16403b6f2 100644 --- a/tests/rptest/tests/redpanda_startup_test.py +++ b/tests/rptest/tests/redpanda_startup_test.py @@ -17,6 +17,7 @@ from rptest.services.cluster import cluster from rptest.services.redpanda import MetricsEndpoint, MetricSamples, RedpandaServiceBase from rptest.tests.redpanda_test import RedpandaTest +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import in_fips_environment @@ -249,17 +250,12 @@ def __init__(self, test_context): True }) - def _license_nag_is_set(self) -> bool: - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @ignore # https://redpandadata.atlassian.net/browse/CORE-4283 @cluster(num_nodes=3) def test_fips_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag interval") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) diff --git a/tests/rptest/tests/schema_registry_test.py b/tests/rptest/tests/schema_registry_test.py index 53c57544a1fee..31e1b04a07302 100644 --- a/tests/rptest/tests/schema_registry_test.py +++ b/tests/rptest/tests/schema_registry_test.py @@ -39,6 +39,7 @@ from rptest.tests.pandaproxy_test import User, PandaProxyTLSProvider from rptest.tests.redpanda_test import RedpandaTest from rptest.util import expect_exception, inject_remote_script, search_logs_with_timeout +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode @@ -3843,19 +3844,14 @@ def __init__(self, *args, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below @parametrize(mode=SchemaIdValidationMode.REDPANDA) @parametrize(mode=SchemaIdValidationMode.COMPAT) def test_license_nag(self, mode): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -3871,9 +3867,9 @@ def test_license_nag(self, mode): self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/utils/log_utils.py b/tests/rptest/utils/log_utils.py new file mode 100644 index 0000000000000..933ae59898332 --- /dev/null +++ b/tests/rptest/utils/log_utils.py @@ -0,0 +1,25 @@ +from ducktape.utils.util import wait_until + +from rptest.services.redpanda import RedpandaService + + +def wait_until_nag_is_set(redpanda: RedpandaService, + check_interval_sec: int, + timeout_sec: int = 30): + """ + Waits until the log message indicating override of the periodic reminder + interval has been set + + Parameters: + redpanda (RedpandaService): Redpanda service instance to query the logs from + check_interval_sec (int): The interval in seconds that should be logged by Redpanda + timeotu_sec (int): The maximum time to wait for the log message to appear + """ + def nag_check() -> bool: + return redpanda.search_log_all( + f"Overriding default reminder period interval to: {check_interval_sec}s" + ) + + wait_until(nag_check, + timeout_sec=timeout_sec, + err_msg="Failed to set periodic reminder interval") From ef9dcc57d81ab8afe93c6e0205112b8c3307a95d Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 16 Dec 2024 15:10:01 -0500 Subject: [PATCH 08/12] fm/dt: Renamed __REDPANDA_LICENSE_CHECK_INTERVAL_SEC Renamed `__REDPANDA_LICENSE_CHECK_INTERVAL_SEC` with `__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC` to better reflect what messages this environmental variable is controlling. Signed-off-by: Michael Boquard --- src/v/cluster/feature_manager.cc | 2 +- tests/rptest/tests/rbac_test.py | 2 +- tests/rptest/tests/rbac_upgrade_test.py | 2 +- tests/rptest/tests/redpanda_kerberos_test.py | 2 +- tests/rptest/tests/redpanda_oauth_test.py | 2 +- tests/rptest/tests/redpanda_startup_test.py | 2 +- tests/rptest/tests/schema_registry_test.py | 2 +- tests/rptest/tests/workload_license.py | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 5df81b80ad2ff..2cd2aa05af09f 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -279,7 +279,7 @@ feature_manager::report_enterprise_features() const { ss::future<> feature_manager::maybe_log_periodic_reminders() { auto reminder_period = std::chrono::seconds(60 * 5); auto interval_override = std::getenv( - "__REDPANDA_LICENSE_CHECK_INTERVAL_SEC"); + "__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC"); if (interval_override != nullptr) { try { reminder_period = std::min( diff --git a/tests/rptest/tests/rbac_test.py b/tests/rptest/tests/rbac_test.py index 66f6bb462310a..3f472e3112480 100644 --- a/tests/rptest/tests/rbac_test.py +++ b/tests/rptest/tests/rbac_test.py @@ -630,7 +630,7 @@ class RBACLicenseTest(RBACTestBase): def __init__(self, test_ctx, **kwargs): super().__init__(test_ctx, **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}' }) diff --git a/tests/rptest/tests/rbac_upgrade_test.py b/tests/rptest/tests/rbac_upgrade_test.py index 8bc41b63d150d..c1a6f2d5f3cae 100644 --- a/tests/rptest/tests/rbac_upgrade_test.py +++ b/tests/rptest/tests/rbac_upgrade_test.py @@ -28,7 +28,7 @@ class UpgradeMigrationCreatingDefaultRole(RedpandaTest): def __init__(self, test_ctx, **kwargs): super().__init__(test_ctx, **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', '__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True diff --git a/tests/rptest/tests/redpanda_kerberos_test.py b/tests/rptest/tests/redpanda_kerberos_test.py index a764651201eaa..8532db5b4d976 100644 --- a/tests/rptest/tests/redpanda_kerberos_test.py +++ b/tests/rptest/tests/redpanda_kerberos_test.py @@ -167,7 +167,7 @@ def __init__(self, test_context, num_nodes=3, **kwargs): sasl_mechanisms=["SCRAM"], **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/redpanda_oauth_test.py b/tests/rptest/tests/redpanda_oauth_test.py index b28a983044b0b..70498c42c3093 100644 --- a/tests/rptest/tests/redpanda_oauth_test.py +++ b/tests/rptest/tests/redpanda_oauth_test.py @@ -609,7 +609,7 @@ def __init__(self, test_context, num_nodes=3, **kwargs): http_authentication=["BASIC"], **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/redpanda_startup_test.py b/tests/rptest/tests/redpanda_startup_test.py index 818f16403b6f2..a020b6337a652 100644 --- a/tests/rptest/tests/redpanda_startup_test.py +++ b/tests/rptest/tests/redpanda_startup_test.py @@ -244,7 +244,7 @@ def __init__(self, test_context): fips_mode=RedpandaServiceBase.FIPSMode.disabled) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', '__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True diff --git a/tests/rptest/tests/schema_registry_test.py b/tests/rptest/tests/schema_registry_test.py index 31e1b04a07302..75de78d1a1b24 100644 --- a/tests/rptest/tests/schema_registry_test.py +++ b/tests/rptest/tests/schema_registry_test.py @@ -3840,7 +3840,7 @@ def __init__(self, *args, **kwargs): schema_registry_config=SchemaRegistryConfig(), **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/workload_license.py b/tests/rptest/tests/workload_license.py index 41c2850305c26..47ffd47639be7 100644 --- a/tests/rptest/tests/workload_license.py +++ b/tests/rptest/tests/workload_license.py @@ -43,7 +43,7 @@ def begin(self): return self.ctx.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{LicenseWorkload.LICENSE_CHECK_INTERVAL_SEC}' }) @@ -107,7 +107,7 @@ def on_cluster_upgraded(self, version: tuple[int, int, int]) -> int: # Install license assert admin.put_license(self.license).status_code == 200 self.ctx.redpanda.unset_environment( - ['__REDPANDA_LICENSE_CHECK_INTERVAL_SEC']) + ['__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC']) self.license_installed = True return PWorkload.DONE From 176570ea3f19898a635d980f365df3f52c9aa354 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 09:22:09 -0500 Subject: [PATCH 09/12] dt: Added 'PLAIN' to list of simple SASL mechanisms This change allows for the selection of SASL/PLAIN as an authentication method using the Python RdKafka library. Signed-off-by: Michael Boquard --- tests/rptest/services/redpanda_types.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/rptest/services/redpanda_types.py b/tests/rptest/services/redpanda_types.py index 202f7184834e3..ca484b4beb530 100644 --- a/tests/rptest/services/redpanda_types.py +++ b/tests/rptest/services/redpanda_types.py @@ -59,7 +59,7 @@ def __str__(self): return self.name -SIMPLE_SASL_MECHANISMS = ['SCRAM-SHA-256', 'SCRAM-SHA-512'] +SIMPLE_SASL_MECHANISMS = ['PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512'] COMPLEX_SASL_MECHANISMS = ['GSSAPI', 'OAUTHBEARER'] From c4afa35fc8bf3c4f350905b1d2ceff14cbb9dafb Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 16:23:48 -0500 Subject: [PATCH 10/12] dt: Permitted selection of SASL algorithm with kafka cli Signed-off-by: Michael Boquard --- tests/rptest/clients/kafka_cli_tools.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/rptest/clients/kafka_cli_tools.py b/tests/rptest/clients/kafka_cli_tools.py index 3f6ae9884f804..01bc0d13e5a33 100644 --- a/tests/rptest/clients/kafka_cli_tools.py +++ b/tests/rptest/clients/kafka_cli_tools.py @@ -72,6 +72,7 @@ def __init__(self, version: str | None = None, user: str | None = None, passwd: str | None = None, + algorithm: str | None = 'SCRAM-SHA-256', protocol: str = 'SASL_PLAINTEXT', oauth_cfg: OAuthConfig | None = None): self._redpanda = redpanda @@ -94,7 +95,7 @@ def __init__(self, if user: security = security.override(user, passwd, - 'SCRAM-SHA-256', + algorithm, tls_enabled=None) if sasl := security.simple_credentials(): From 7e072e3ac6a860f331f3be7a9d548d2253faefea Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 09:22:52 -0500 Subject: [PATCH 11/12] dt: Added test to exercise SASL/PLAIN using various clients Added SASL/PLAIN verification tests for: * KCL * RPK * Python RdKafka * Kafka CLI tools Validated that both SCRAM-256 and SCRAM-512 users worked. Fixes: CORE-8458 Fixes: CORE-8459 Fixes: CORE-8460 Tested that a log message appears when SASL/PLAIN is enabled and that PLAIN cannot be the only SASL mechanism in the mechanisms list. Signed-off-by: Michael Boquard --- tests/rptest/tests/scram_test.py | 278 ++++++++++++++++++++++++++++++- 1 file changed, 274 insertions(+), 4 deletions(-) diff --git a/tests/rptest/tests/scram_test.py b/tests/rptest/tests/scram_test.py index 5ea97d254c5dc..5563bfff8fed9 100644 --- a/tests/rptest/tests/scram_test.py +++ b/tests/rptest/tests/scram_test.py @@ -6,28 +6,39 @@ # 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 +from enum import IntEnum +import json import random import socket import string import requests from requests.exceptions import HTTPError +import socket import time import urllib.parse import re -from ducktape.mark import parametrize +from confluent_kafka import KafkaException, KafkaError + +from ducktape.cluster.cluster import ClusterNode +from ducktape.mark import parametrize, matrix from ducktape.utils.util import wait_until from ducktape.errors import TimeoutError +from ducktape.services.service import Service from rptest.services.cluster import cluster from rptest.tests.redpanda_test import RedpandaTest +from rptest.clients.kcl import RawKCL +from rptest.clients.kafka_cli_tools import KafkaCliTools, KafkaCliToolsError from rptest.clients.types import TopicSpec -from rptest.clients.rpk import RpkTool +from rptest.clients.rpk import RpkTool, RpkException from rptest.clients.python_librdkafka import PythonLibrdkafka from rptest.services.admin import Admin -from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig +from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig, TLSProvider +from rptest.services.tls import Certificate, CertificateAuthority, TLSCertManager from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC from rptest.util import expect_http_error +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.utf8 import CONTROL_CHARS, CONTROL_CHARS_MAP, generate_string_with_control_character @@ -102,9 +113,12 @@ def gen(length): return password - def make_superuser_client(self, password_override=None): + def make_superuser_client(self, + password_override=None, + algorithm_override=None): username, password, algorithm = self.redpanda.SUPERUSER_CREDENTIALS password = password_override or password + algorithm = algorithm_override or algorithm return PythonLibrdkafka(self.redpanda, username=username, password=password, @@ -294,6 +308,262 @@ def test_scram(self): assert username in users +class SaslPlainTest(BaseScramTest): + """ + These tests validate the functionality of the SASL/PLAIN + authentication mechanism. + """ + class ClientType(IntEnum): + KCL = 1 + RPK = 2 + PYTHON_RDKAFKA = 3 + KCLI = 4 + + class ScramType(IntEnum): + SCRAM_SHA_256 = 1 + SCRAM_SHA_512 = 2 + + def __str__(self): + return self.name.replace("_", "-") + + def __init__(self, test_context): + security = SecurityConfig() + security.enable_sasl = True + super(SaslPlainTest, + self).__init__(test_context, + num_brokers=3, + security=security, + extra_node_conf={'developer_mode': True}) + + def _enable_plain_authn(self): + self.logger.debug("Enabling SASL PLAIN and disabling SCRAM") + admin = Admin(self.redpanda) + # PLAIN cannot be on by itself, so we will enable OAUTHBEARER as well + # but keep SCRAM disabled to ensure we are validating the PLAIN authentication + # mechanism + admin.patch_cluster_config( + upsert={'sasl_mechanisms': ['PLAIN', 'OAUTHBEARER']}) + + def _make_client( + self, + client_type: ClientType, + username_override: str | None = None, + password_override: str | None = None, + algorithm_override: str | None = None + ) -> PythonLibrdkafka | RawKCL | RpkTool | KafkaCliTools: + username, password, algorithm = self.redpanda.SUPERUSER_CREDENTIALS + username = username_override or username + password = password_override or password + algorithm = algorithm_override or algorithm + + if client_type == self.ClientType.PYTHON_RDKAFKA: + return PythonLibrdkafka(self.redpanda, + username=username, + password=password, + algorithm=algorithm) + elif client_type == self.ClientType.KCL: + return RawKCL(self.redpanda, + username=username, + password=password, + sasl_mechanism=algorithm) + elif client_type == self.ClientType.RPK: + return RpkTool(self.redpanda, + username=username, + password=password, + sasl_mechanism=algorithm) + elif client_type == self.ClientType.KCLI: + return KafkaCliTools(self.redpanda, + user=username, + passwd=password, + algorithm=algorithm) + else: + assert False, f'Unknown client type: {client_type}' + + def _make_topic(self, client: PythonLibrdkafka | RawKCL | RpkTool + | KafkaCliTools, expect_success: bool) -> TopicSpec: + topic_name = "test-topic" + topic = TopicSpec(name=topic_name) + try: + if isinstance(client, PythonLibrdkafka): + client.create_topic(topic) + elif isinstance(client, RawKCL): + resp = client.create_topics(6, [{"name": topic_name}]) + self.logger.info(f"RESP: {resp}") + if expect_success: + assert len( + resp + ) != 0, "Should have received response with SASL/PLAIN enabled" + assert resp[0][ + 'ErrorCode'] == 0, f"Expected error code 0, got {resp[0]['ErrorCode']}" + return + else: + assert len( + resp + ) == 0, "Should not have received response with SASL/PLAIN disabled" + return + elif isinstance(client, RpkTool): + client.create_topic(topic=topic_name) + elif isinstance(client, KafkaCliTools): + client.create_topic(topic) + else: + assert False, f'Unknown client type: {client} ({type(client)})' + assert expect_success, "Should have failed with SASL/PLAIN disabled" + except RpkException as e: + assert isinstance( + client, RpkTool + ), f"Should not have received an RPK exception from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert "UNSUPPORTED_SASL_MECHANISM" in str( + e), f"Expected UNSUPPORTED_SASL_MECHANISM, got {e}" + except KafkaException as e: + assert isinstance( + client, PythonLibrdkafka + ), f"Should not have received a KafkaException from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert e.args[0].code( + ) == KafkaError._TIMED_OUT, f"Expected KafkaError._TIMED_OUT, got {e.args[0].code()}" + except KafkaCliToolsError as e: + assert isinstance( + client, KafkaCliTools + ), f"Should not have received a KafkaCliToolsError from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert "UnsupportedSaslMechanismException" in str( + e + ), f"Expected to see UnsupportedSaslMechanismException, got {e}" + + @cluster(num_nodes=3) + @matrix(client_type=list(ClientType), + scram_type=list(ScramType), + sasl_plain_enabled=[True, False]) + def test_plain_authn(self, client_type, scram_type, sasl_plain_enabled): + """ + This test validates that SASL/PLAIN works with common kafka client + libraries: + - Python librdkafka + - Raw KCL + - RPK + - Kafka CLI tools + + This test will validate that SASL/PLAIN works with both SCRAM-SHA-256 + and SCRAM-SHA-512 users. + """ + username = "test-user" + password = "test-password" + RpkTool(self.redpanda, + username=self.redpanda.SUPERUSER_CREDENTIALS.username, + password=self.redpanda.SUPERUSER_CREDENTIALS.password, + sasl_mechanism=self.redpanda.SUPERUSER_CREDENTIALS.algorithm + ).sasl_allow_principal(principal=username, + operations=["all"], + resource="topic", + resource_name="*") + self.create_user(username=username, + algorithm=str(scram_type), + password=password) + + if sasl_plain_enabled: + self._enable_plain_authn() + + client = self._make_client(client_type, + username_override=username, + password_override=password, + algorithm_override="PLAIN") + self._make_topic(client, sasl_plain_enabled) + + +class SaslPlainTLSProvider(TLSProvider): + def __init__(self, tls: TLSCertManager): + self._tls = tls + + @property + def ca(self) -> CertificateAuthority: + return self._tls.ca + + def create_broker_cert(self, service: Service, + node: ClusterNode) -> Certificate: + assert node in service.nodes + return self._tls.create_cert(node.name) + + def create_service_client_cert(self, _: Service, name: str) -> Certificate: + return self._tls.create_cert(socket.gethostname(), name=name) + + +class SaslPlainConfigTest(BaseScramTest): + """ + These tests verify the behavior of Redpanda in different + configurations with SASL/PLAIN enabled + """ + + LICENSE_CHECK_INTERVAL_SEC = 1 + + def __init__(self, test_context): + self.security = SecurityConfig() + self.security.enable_sasl = True + super(SaslPlainConfigTest, self).__init__(test_context, + num_brokers=3, + security=self.security) + self.redpanda.set_environment({ + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': + f'{self.LICENSE_CHECK_INTERVAL_SEC}' + }) + self.tls = TLSCertManager(self.logger) + + def setUp(self): + pass + + def _start_cluster(self, enable_tls: bool): + if enable_tls: + self.security.tls_provider = SaslPlainTLSProvider(tls=self.tls) + self.redpanda.set_security_settings(self.security) + super().setUp() + + @cluster(num_nodes=3) + def test_cannot_enable_only_plain(self): + """ + This test verifies that a user cannot select PLAIN as the only + sasl_mechanism + """ + self._start_cluster(enable_tls=False) + admin = Admin(self.redpanda) + try: + admin.patch_cluster_config(upsert={'sasl_mechanisms': ['PLAIN']}) + assert False, "Should not be able to enable only PLAIN" + except HTTPError as e: + assert e.response.status_code == 400, f"Expected 400, got {e.response.status_code}" + response = json.loads(e.response.text) + assert 'sasl_mechanisms' in response, f'Response missing "sasl_mechanisms": {response}' + assert "When PLAIN is enabled, at least one other mechanism must be enabled" == response[ + 'sasl_mechanisms'], f"Invalid message in response: {response['sasl_mechanisms']}" + + @cluster(num_nodes=3, log_allow_list=[re.compile('SASL/PLAIN is enabled')]) + @parametrize(enable_tls=True) + @parametrize(enable_tls=False) + def test_sasl_plain_log(self, enable_tls: bool): + """ + This test verifies that a log message is emitted when SASL/PLAIN is enabled + """ + self._start_cluster(enable_tls=enable_tls) + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) + admin = Admin(self.redpanda) + admin.patch_cluster_config( + upsert={'sasl_mechanisms': ['SCRAM', 'PLAIN']}) + + self.logger.debug("Waiting for SASL/PLAIN message") + + def has_sasl_plain_log(): + # There is always at least one Kafka API with TLS disabled meaning + # this will always log at the error level + return self.redpanda.search_log_all( + r"^ERROR.*SASL/PLAIN is enabled\. This is insecure and not recommended for production\.$" + ) + + wait_until(has_sasl_plain_log, + timeout_sec=self.LICENSE_CHECK_INTERVAL_SEC * 2, + err_msg="Failed to find SASL/PLAIN log message") + + class ScramLiveUpdateTest(RedpandaTest): def __init__(self, test_context): super(ScramLiveUpdateTest, self).__init__(test_context, num_brokers=1) From b2d8fa9fe7c984e1b9cb53026427761b72466900 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Tue, 10 Dec 2024 21:04:04 -0500 Subject: [PATCH 12/12] security/test: Added unit tests for plain authenticator Signed-off-by: Michael Boquard --- src/v/security/tests/BUILD | 16 +- src/v/security/tests/CMakeLists.txt | 13 + .../tests/plain_authenticator_test.cc | 225 ++++++++++++++++++ 3 files changed, 253 insertions(+), 1 deletion(-) create mode 100644 src/v/security/tests/plain_authenticator_test.cc diff --git a/src/v/security/tests/BUILD b/src/v/security/tests/BUILD index 895f830a48a2d..2fabe6c060704 100644 --- a/src/v/security/tests/BUILD +++ b/src/v/security/tests/BUILD @@ -1,4 +1,4 @@ -load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest", "redpanda_test_cc_library") +load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest", "redpanda_cc_gtest", "redpanda_test_cc_library") redpanda_test_cc_library( name = "utils", @@ -30,6 +30,20 @@ redpanda_cc_btest( ], ) +redpanda_cc_gtest( + name = "plain_authenticator_test", + timeout = "short", + srcs = [ + "plain_authenticator_test.cc", + ], + deps = [ + "//src/v/security", + "//src/v/test_utils:gtest", + "@googletest//:gtest", + "@seastar", + ], +) + redpanda_cc_btest( name = "credential_store_test", timeout = "short", diff --git a/src/v/security/tests/CMakeLists.txt b/src/v/security/tests/CMakeLists.txt index e1e28568d7d8d..ecf41dee1e84e 100644 --- a/src/v/security/tests/CMakeLists.txt +++ b/src/v/security/tests/CMakeLists.txt @@ -17,6 +17,19 @@ rp_test( LABELS kafka ) +rp_test( + UNIT_TEST + GTEST + BINARY_NAME test_authenticators + SOURCES + plain_authenticator_test.cc + LIBRARIES + v::security + v::gtest_main + LABELS kafka + ARGS "-- -c 1" +) + rp_test( UNIT_TEST BINARY_NAME test_kafka_security_single_thread diff --git a/src/v/security/tests/plain_authenticator_test.cc b/src/v/security/tests/plain_authenticator_test.cc new file mode 100644 index 0000000000000..9acee7a7b650d --- /dev/null +++ b/src/v/security/tests/plain_authenticator_test.cc @@ -0,0 +1,225 @@ +// Copyright 2024 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/acl.h" +#include "security/credential_store.h" +#include "security/errc.h" +#include "security/plain_authenticator.h" +#include "security/scram_algorithm.h" +#include "security/scram_credential.h" +#include "test_utils/test.h" + +class sasl_plain_test_fixture : public seastar_test { +public: + ss::future<> SetUpAsync() override { + authn = std::make_unique(store); + ASSERT_FALSE_CORO(authn->complete()); + ASSERT_FALSE_CORO(authn->failed()); + } + + ss::future<> TearDownAsync() override { + authn.reset(); + co_return; + } + + security::credential_store store; + std::unique_ptr authn; + + enum class scram_type_t { + scram_sha256, + scram_sha512, + }; + + security::scram_credential make_scram_credential( + scram_type_t scram_type, + const ss::sstring& username, + const ss::sstring& password) { + auto iterations = [scram_type]() { + if (scram_type == scram_type_t::scram_sha256) { + return security::scram_sha256::min_iterations; + } + return security::scram_sha512::min_iterations; + }(); + + security::acl_principal principal( + security::principal_type::user, username); + + return scram_type == scram_type_t::scram_sha256 + ? security::scram_sha256::make_credentials( + std::move(principal), password, iterations) + : security::scram_sha512::make_credentials( + std::move(principal), password, iterations); + } + + bytes create_authn_message( + const ss::sstring& username, const ss::sstring& password) { + bytes rv; + rv.push_back('\0'); + std::ranges::move(bytes::from_string(username), std::back_inserter(rv)); + rv.push_back('\0'); + std::ranges::move(bytes::from_string(password), std::back_inserter(rv)); + return rv; + } +}; + +TEST_F_CORO(sasl_plain_test_fixture, test_success_scram256) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + EXPECT_EQ(authn->principal().name(), username); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_success_scram512) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha512, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + EXPECT_EQ(authn->principal().name(), username); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_invalid_password) { + ss::sstring username("user"); + ss::sstring password("password"); + ss::sstring wrong_password("wrong_password"); + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, wrong_password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_invalid_user) { + ss::sstring username("user"); + ss::sstring password("password"); + ss::sstring wrong_username("wrong_user"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(wrong_username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_resuse_authn) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + { + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + } + + { + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); + } +} + +TEST_F_CORO(sasl_plain_test_fixture, test_authz_id) { + ss::sstring username("user"); + ss::sstring password("password"); + auto authn_msg = bytes::from_string("authz"); + auto authn_msg_tmp = create_authn_message(username, password); + std::ranges::move(authn_msg_tmp, std::back_inserter(authn_msg)); + + auto res = co_await authn->authenticate(authn_msg); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, no_username) { + ss::sstring password("password"); + auto res = co_await authn->authenticate(create_authn_message("", password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, no_password) { + ss::sstring username("user"); + auto res = co_await authn->authenticate(create_authn_message(username, "")); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, massive_username) { + ss::sstring username(1024, 'a'); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, massive_password) { + ss::sstring username("user"); + ss::sstring password(1024, 'a'); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +}