diff --git a/proto/redpanda/core/admin/v2/BUILD b/proto/redpanda/core/admin/v2/BUILD index af5439aff503b..faf03ae45e048 100644 --- a/proto/redpanda/core/admin/v2/BUILD +++ b/proto/redpanda/core/admin/v2/BUILD @@ -106,6 +106,7 @@ proto_library( "//proto/redpanda/core/pbgen:rpc_proto", "@googleapis//google/api:field_behavior_proto", "@googleapis//google/api:resource_proto", + "@protobuf//:field_mask_proto", "@protobuf//:timestamp_proto", ], ) @@ -117,6 +118,7 @@ redpanda_proto_library( ], visibility = ["//visibility:public"], deps = [ + "//src/v/serde/protobuf:field_mask", "@abseil-cpp//absl/time:time", ], ) diff --git a/proto/redpanda/core/admin/v2/security.proto b/proto/redpanda/core/admin/v2/security.proto index ea498df26856b..fdce44c8079b4 100644 --- a/proto/redpanda/core/admin/v2/security.proto +++ b/proto/redpanda/core/admin/v2/security.proto @@ -21,11 +21,62 @@ import "proto/redpanda/core/pbgen/rpc.proto"; import "google/api/field_behavior.proto"; import "google/api/resource.proto"; import "google/protobuf/timestamp.proto"; +import "google/protobuf/field_mask.proto"; option (pbgen.cpp_namespace) = "proto::admin"; // The SecurityService provides security-related operations. service SecurityService { + // CreateScramCredential + // + // Create a SCRAM credential. + rpc CreateScramCredential(CreateScramCredentialRequest) + returns (CreateScramCredentialResponse) { + option (pbgen.rpc) = { + authz: SUPERUSER + }; + } + + // GetScramCredential + // + // Retrieve a SCRAM credential. + rpc GetScramCredential(GetScramCredentialRequest) + returns (GetScramCredentialResponse) { + option (pbgen.rpc) = { + authz: SUPERUSER + }; + } + + // ListScramCredentials + // + // List all SCRAM credentials. + rpc ListScramCredentials(ListScramCredentialsRequest) + returns (ListScramCredentialsResponse) { + option (pbgen.rpc) = { + authz: SUPERUSER + }; + } + + // UpdateScramCredential + // + // Update a SCRAM credential. + rpc UpdateScramCredential(UpdateScramCredentialRequest) + returns (UpdateScramCredentialResponse) { + option (pbgen.rpc) = { + authz: SUPERUSER + }; + } + + // DeleteScramCredential + // + // Delete a SCRAM credential. + rpc DeleteScramCredential(DeleteScramCredentialRequest) + returns (DeleteScramCredentialResponse) { + option (pbgen.rpc) = { + authz: SUPERUSER + }; + } + // CreateRole // // Create a new Role resource. @@ -134,6 +185,34 @@ service SecurityService { /* Resources */ // ============================================= +// The ScramCredential resource used for SCRAM authentication. +message ScramCredential { + option (google.api.resource) = { + type: "redpanda.core.admin.SecurityService/ScramCredential" + pattern: "scram_credentials/{scram_credential}" + }; + + // The name of the SCRAM credential. + string name = 1 [ + (google.api.field_behavior) = REQUIRED, + (google.api.field_behavior) = IMMUTABLE + ]; + + // The SCRAM mechanism. + enum ScramMechanism { + SCRAM_MECHANISM_UNSPECIFIED = 0; + SCRAM_MECHANISM_SCRAM_SHA_256 = 1; + SCRAM_MECHANISM_SCRAM_SHA_512 = 2; + } + + // The SCRAM mechanism used for this credential. + ScramMechanism mechanism = 2; + + // The password for the SCRAM credential. + string password = 3 + [debug_redact = true, (google.api.field_behavior) = INPUT_ONLY]; +} + // The Role resource represents a security role with associated members. message Role { option (google.api.resource) = { @@ -155,6 +234,83 @@ message Role { /* RPC Requests and Responses */ // ============================================= +// CreateScramCredentialRequest is the request for the CreateScramCredential +// RPC. +message CreateScramCredentialRequest { + // The SCRAM credential to create. + ScramCredential scram_credential = 1 + [(google.api.field_behavior) = REQUIRED]; +} + +// CreateScramCredentialResponse is the response from the CreateScramCredential +// RPC. +message CreateScramCredentialResponse { + // The created SCRAM credential. + ScramCredential scram_credential = 1; +} + +// GetScramCredentialRequest is the request for the GetScramCredential RPC. +message GetScramCredentialRequest { + // The name of the SCRAM credential to retrieve. + string name = 1 [ + (google.api.field_behavior) = REQUIRED, + (google.api.resource_reference) = { + type: "redpanda.core.admin.SecurityService/ScramCredential" + } + ]; +} + +// GetScramCredentialResponse is the response from the GetScramCredential RPC. +message GetScramCredentialResponse { + // The requested SCRAM credential. + ScramCredential scram_credential = 1; +} + +// ListScramCredentialsRequest is the request for the ListScramCredentials RPC. +message ListScramCredentialsRequest {} + +// ListScramCredentialsResponse is the response from the ListScramCredentials +// RPC. +message ListScramCredentialsResponse { + // The list of SCRAM credentials. + repeated ScramCredential scram_credentials = 1; +} + +// UpdateScramCredentialRequest is the request for the UpdateScramCredential +// RPC. +message UpdateScramCredentialRequest { + // The SCRAM credential to update. + ScramCredential scram_credential = 1 + [(google.api.field_behavior) = REQUIRED]; + + // The list of fields to update + // See [AIP-134](https://google.aip.dev/134) for how to use `field_mask` + google.protobuf.FieldMask update_mask = 2; +} + +// UpdateScramCredentialResponse is the response from the UpdateScramCredential +// RPC. +message UpdateScramCredentialResponse { + // The updated SCRAM credential. + ScramCredential scram_credential = 1; +} + +// DeleteScramCredentialRequest is the request for the DeleteScramCredential +// RPC. +message DeleteScramCredentialRequest { + // The name of the SCRAM credential to delete. + string name = 1 [ + (google.api.field_behavior) = REQUIRED, + (google.api.resource_reference) = { + type: "redpanda.core.admin.SecurityService/ScramCredential" + } + ]; +} + +// DeleteScramCredentialResponse is the response from the DeleteScramCredential +// RPC. +message DeleteScramCredentialResponse {} + // CreateRoleRequest is the request for the CreateRole RPC. message CreateRoleRequest { // The Role to create. diff --git a/src/v/redpanda/admin/services/security.cc b/src/v/redpanda/admin/services/security.cc index b95e674c190d6..5ad8349cc73aa 100644 --- a/src/v/redpanda/admin/services/security.cc +++ b/src/v/redpanda/admin/services/security.cc @@ -16,12 +16,15 @@ #include "kafka/server/server.h" #include "redpanda/admin/proxy/context.h" #include "redpanda/admin/services/utils.h" +#include "security/credential_store.h" #include "security/oidc_authenticator.h" #include "security/oidc_service.h" #include "security/request_auth.h" #include "security/role.h" #include "security/role_store.h" #include "security/scram_algorithm.h" +#include "security/types.h" +#include "serde/protobuf/rpc.h" namespace admin { @@ -36,6 +39,144 @@ ss::logger securitylog{"admin_api_server/security_service"}; } // namespace namespace internal { +bool match_scram_credential( + const proto::admin::scram_credential& pb_cred, + const security::scram_credential& cred) { + // Assume pb_cred has already been pre-validated via earlier + // validate_pb_scram_credential call + const auto& mechanism = pb_cred.get_mechanism(); + if ( + mechanism + == proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256) { + return security::scram_sha256::validate_password( + pb_cred.get_password(), + cred.stored_key(), + cred.salt(), + cred.iterations()); + } else if ( + mechanism + == proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512) { + return security::scram_sha512::validate_password( + pb_cred.get_password(), + cred.stored_key(), + cred.salt(), + cred.iterations()); + } else { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat("Unknown SCRAM mechanism: {}", mechanism)); + } +} + +void validate_scram_credential_name(const ss::sstring& cred_name) { + try { + validate_no_control(cred_name); + } catch (const control_character_present_exception& e) { + vlog( + securitylog.warn, + "SCRAM credential name contains invalid characters"); + throw serde::pb::rpc::invalid_argument_exception( + "SCRAM credential name contains invalid characters"); + } + + if (!security::validate_scram_username(cred_name)) { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat("Invalid SCRAM credential name {{{}}}", cred_name)); + } +} + +void validate_pb_scram_credential( + const proto::admin::scram_credential& pb_cred) { + const auto& name = pb_cred.get_name(); + + validate_scram_credential_name(name); + + const auto& password = pb_cred.get_password(); + + // TODO: Do we allow empty passwords? + + try { + validate_no_control(password); + } catch (const control_character_present_exception& e) { + vlog( + securitylog.warn, + "SCRAM credential password contains invalid characters"); + throw serde::pb::rpc::invalid_argument_exception( + "SCRAM credential password contains invalid characters"); + } + + if (crypto::is_scram_password_too_short(password)) { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat( + "Password length less than {} characters", + crypto::hmac_key_fips_min_bytes)); + } + + const auto& mechanism = pb_cred.get_mechanism(); + if ( + mechanism + != proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256 + && mechanism + != proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512) { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat("Unknown SCRAM mechanism: {}", mechanism)); + } +} + +security::scram_credential convert_to_security_scram_credential( + const proto::admin::scram_credential& pb_cred) { + // Assume pb_cred has already been pre-validated via earlier + // validate_pb_scram_credential call + const auto& password = pb_cred.get_password(); + const auto& mechanism = pb_cred.get_mechanism(); + + if ( + mechanism + == proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256) { + return security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + } else if ( + mechanism + == proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512) { + return security::scram_sha512::make_credentials( + password, security::scram_sha512::min_iterations); + } else { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat("Unknown SCRAM mechanism: {}", mechanism)); + } +} + +proto::admin::scram_credential convert_to_pb_scram_credential( + ss::sstring name, const security::scram_credential& cred) { + proto::admin::scram_credential pb_cred; + + // Determine mechanism based on length of stored key + if (cred.stored_key().size() == security::scram_sha256::key_size) { + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + } else if (cred.stored_key().size() == security::scram_sha512::key_size) { + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + } else { + vlog( + securitylog.error, + "Unknown SCRAM stored key size for user '{}': {}", + name, + cred.stored_key().size()); + throw serde::pb::rpc::internal_exception( + ssx::sformat("Unknown SCRAM stored key size for user '{}'", name)); + } + + pb_cred.set_name(std::move(name)); + return pb_cred; +} void validate_role_name(const ss::sstring& role_name) { try { @@ -151,6 +292,303 @@ security_service_impl::security_service_impl( , _kafka_server(kafka_server) , _md_cache(md_cache) {} +seastar::future +security_service_impl::create_scram_credential( + serde::pb::rpc::context ctx, + proto::admin::create_scram_credential_request req) { + vlog(securitylog.trace, "create_scram_credential: {}", req); + + const auto redirect_node = utils::redirect_to_leader( + _md_cache.local(), model::controller_ntp, _proxy_client.self_node_id()); + + if (redirect_node) { + vlog( + securitylog.debug, + "Redirecting to leader of {}: {}", + model::controller_ntp, + *redirect_node); + co_return co_await _proxy_client + .make_client_for_node( + *redirect_node) + .create_scram_credential(ctx, std::move(req)); + } + + auto& pb_cred = req.get_scram_credential(); + validate_pb_scram_credential(pb_cred); + + const security::credential_user name{pb_cred.get_name()}; + const security::scram_credential credential + = convert_to_security_scram_credential(pb_cred); + + // TODO: I feel like this isn't right to have. If the state of the + // credential store isn't caught up, we might incorrectly return success + // here. Commenting this out for now. + // auto& cred_store = _controller->get_credential_store().local(); + // auto user_opt = cred_store.get(name); + // if (user_opt.has_value() && user_opt.value() == credential) { + // vlog( + // securitylog.debug, + // "User {} already exists with matching credential", + // name); + // // Idempotency: if the user already exists with the same credential, + // // return success. + // proto::admin::create_scram_credential_response res; + // res.set_scram_credential(std::move(pb_cred)); + // co_return res; + // } + + auto err + = co_await _controller->get_security_frontend().local().create_user( + name, + credential, + model::timeout_clock::now() + security_operation_timeout); + + vlog( + securitylog.debug, "Creating user '{}' {}:{}", name, err, err.message()); + + if (err != cluster::errc::success && err != cluster::errc::user_exists) { + vlog( + securitylog.error, + "Failed to create SCRAM credential for user '{}': {}", + name, + err); + throw serde::pb::rpc::unknown_exception( + ssx::sformat( + "Failed to create SCRAM credential for user '{}'", name)); + } + + const auto& cred_store = _controller->get_credential_store().local(); + const auto& cred_opt = cred_store.get(name); + if (!cred_opt.has_value()) { + vlog( + securitylog.error, + "Unable to find created SCRAM credential for '{}'", + name); + throw serde::pb::rpc::internal_exception( + ssx::sformat( + "Unable to find created SCRAM credential for '{}'", name)); + } + + // Idempotency: if user is same as one that already exists, suppress the + // user_exists error and return success. Otherwise, throw an already exists + // error. + if ( + err == cluster::errc::user_exists + && !match_scram_credential(pb_cred, cred_opt.value())) { + vlog( + securitylog.debug, + "User '{}' exists but with different SCRAM credential", + name); + throw serde::pb::rpc::already_exists_exception("User already exists"); + } + + proto::admin::create_scram_credential_response res; + // Don't send the original protobuf scram credential, as it contains the + // password in plaintext. Instead, retrieve the created scram credential and + // convert that to protobuf form (which omits the password). + res.set_scram_credential( + convert_to_pb_scram_credential(name(), cred_opt.value())); + co_return res; +} + +seastar::future +security_service_impl::get_scram_credential( + serde::pb::rpc::context, proto::admin::get_scram_credential_request req) { + vlog(securitylog.trace, "get_scram_credential: {}", req); + + const auto& req_name = req.get_name(); + validate_scram_credential_name(req_name); + + const security::credential_user name{req_name}; + auto cred_opt = _controller->get_credential_store() + .local() + .get(name); + if (!cred_opt) { + vlog(securitylog.debug, "SCRAM credential '{}' does not exist", name); + throw serde::pb::rpc::not_found_exception( + ssx::sformat("SCRAM credential '{}' does not exist", name)); + } + + const auto& cred = cred_opt.value(); + + proto::admin::get_scram_credential_response res; + res.set_scram_credential(convert_to_pb_scram_credential(req_name, cred)); + co_return res; +} + +seastar::future +security_service_impl::list_scram_credentials( + serde::pb::rpc::context, proto::admin::list_scram_credentials_request req) { + vlog(securitylog.trace, "list_scram_credentials: {}", req); + + // TODO: implement filtering based on request parameters + + auto cred_views = _controller->get_credential_store().local().range( + security::credential_store::is_not_ephemeral); + + proto::admin::list_scram_credentials_response res; + auto& scram_credentials = res.get_scram_credentials(); + + for (const auto& cred_view : cred_views) { + const auto& cred_name = cred_view.first; + const auto& cred = cred_view.second; + vlog(securitylog.debug, "Found SCRAM credential: {}", cred_name); + scram_credentials.push_back( + ss::visit(cred, [&cred_name](const security::scram_credential& c) { + return convert_to_pb_scram_credential(cred_name, c); + })); + } + + co_return res; +} + +seastar::future +security_service_impl::update_scram_credential( + serde::pb::rpc::context ctx, + proto::admin::update_scram_credential_request req) { + vlog(securitylog.trace, "update_scram_credential: {}", req); + + const auto redirect_node = utils::redirect_to_leader( + _md_cache.local(), model::controller_ntp, _proxy_client.self_node_id()); + + if (redirect_node) { + vlog( + securitylog.debug, + "Redirecting to leader of {}: {}", + model::controller_ntp, + *redirect_node); + co_return co_await _proxy_client + .make_client_for_node( + *redirect_node) + .update_scram_credential(ctx, std::move(req)); + } + + auto& pb_cred_update = req.get_scram_credential(); + + auto& update_mask = req.get_update_mask(); + if (!update_mask.is_valid_for_message()) { + throw serde::pb::rpc::invalid_argument_exception( + ssx::sformat( + "Invalid update mask for scram_credential: {}", update_mask)); + } + + validate_scram_credential_name(pb_cred_update.get_name()); + const security::credential_user name{pb_cred_update.get_name()}; + + const auto& cred_store = _controller->get_credential_store().local(); + const auto& cred_opt = cred_store.get(name); + if (!cred_opt.has_value()) { + throw serde::pb::rpc::not_found_exception( + ssx::sformat("SCRAM credential '{}' does not exist", name)); + } + + auto pb_cred = convert_to_pb_scram_credential(name(), cred_opt.value()); + update_mask.merge_into(std::move(pb_cred_update), &pb_cred); + + // Require password to be provided in update (otherwise, it would set + // password to empty) + if (pb_cred.get_password().empty()) { + throw serde::pb::rpc::invalid_argument_exception( + "Password must be provided in update"); + } + + validate_pb_scram_credential(pb_cred); + + const auto security_cred = convert_to_security_scram_credential(pb_cred); + + auto err + = co_await _controller->get_security_frontend().local().update_user( + name, + security_cred, + model::timeout_clock::now() + security_operation_timeout); + + vlog( + securitylog.debug, "Updating SCRAM credential {}:{}", err, err.message()); + + if (err == cluster::errc::user_does_not_exist) { + vlog( + securitylog.warn, + "SCRAM credential '{}' should exist but was not found during update", + name); + throw serde::pb::rpc::not_found_exception( + ssx::sformat("SCRAM credential '{}' does not exist", name)); + } else if (err != cluster::errc::success) { + vlog( + securitylog.error, + "Failed to update SCRAM credential '{}': {}", + name, + err); + throw serde::pb::rpc::unknown_exception( + ssx::sformat("Failed to update SCRAM credential '{}'", name)); + } + + const auto& updated_cred = cred_store.get(name); + if (!updated_cred.has_value()) { + vlog( + securitylog.error, + "Unable to find updated SCRAM credential for '{}'", + name); + throw serde::pb::rpc::internal_exception( + ssx::sformat( + "Unable to find updated SCRAM credential for '{}'", name)); + } + + proto::admin::update_scram_credential_response res; + // Don't send the original protobuf scram credential, as it contains the + // password in plaintext. Instead, retrieve the created scram credential and + // convert that to protobuf form (which omits the password). + res.set_scram_credential( + convert_to_pb_scram_credential(name(), updated_cred.value())); + co_return res; +} + +seastar::future +security_service_impl::delete_scram_credential( + serde::pb::rpc::context ctx, + proto::admin::delete_scram_credential_request req) { + vlog(securitylog.trace, "delete_scram_credential: {}", req); + + const auto redirect_node = utils::redirect_to_leader( + _md_cache.local(), model::controller_ntp, _proxy_client.self_node_id()); + + if (redirect_node) { + vlog( + securitylog.debug, + "Redirecting to leader of {}: {}", + model::controller_ntp, + *redirect_node); + co_return co_await _proxy_client + .make_client_for_node( + *redirect_node) + .delete_scram_credential(ctx, std::move(req)); + } + + const auto& req_name = req.get_name(); + validate_scram_credential_name(req_name); + + const security::credential_user name{req_name}; + + auto err + = co_await _controller->get_security_frontend().local().delete_user( + name, model::timeout_clock::now() + security_operation_timeout); + vlog( + securitylog.debug, + "Deleting SCRAM credential '{}' {}:{}", + name, + err, + err.message()); + + if ( + err != cluster::errc::success + && err != cluster::errc::user_does_not_exist) { + // Idempotency: removing a non-existent SCRAM credential is successful. + throw serde::pb::rpc::unknown_exception( + ssx::sformat("Failed to delete SCRAM credential '{}'", name)); + } + + co_return proto::admin::delete_scram_credential_response{}; +} + seastar::future security_service_impl::create_role( serde::pb::rpc::context ctx, proto::admin::create_role_request req) { diff --git a/src/v/redpanda/admin/services/security.h b/src/v/redpanda/admin/services/security.h index b116409721f71..7643646ae4a97 100644 --- a/src/v/redpanda/admin/services/security.h +++ b/src/v/redpanda/admin/services/security.h @@ -24,7 +24,20 @@ namespace admin { // security_service_impl and its tests. namespace internal { -void validate_role_name(const ss::sstring& role_name); +bool match_scram_credential( + const proto::admin::scram_credential&, const security::scram_credential&); + +void validate_scram_credential_name(const ss::sstring&); + +void validate_pb_scram_credential(const proto::admin::scram_credential&); + +security::scram_credential +convert_to_security_scram_credential(const proto::admin::scram_credential&); + +proto::admin::scram_credential +convert_to_pb_scram_credential(ss::sstring, const security::scram_credential&); + +void validate_role_name(const ss::sstring&); void validate_pb_role_member(const proto::admin::role_member& pb_member); @@ -49,6 +62,31 @@ class security_service_impl : public proto::admin::security_service { ss::sharded& kafka_server, ss::sharded& md_cache); + seastar::future + create_scram_credential( + serde::pb::rpc::context, + proto::admin::create_scram_credential_request) override; + + seastar::future + get_scram_credential( + serde::pb::rpc::context, + proto::admin::get_scram_credential_request) override; + + seastar::future + list_scram_credentials( + serde::pb::rpc::context, + proto::admin::list_scram_credentials_request) override; + + seastar::future + update_scram_credential( + serde::pb::rpc::context, + proto::admin::update_scram_credential_request) override; + + seastar::future + delete_scram_credential( + serde::pb::rpc::context, + proto::admin::delete_scram_credential_request) override; + seastar::future create_role( serde::pb::rpc::context, proto::admin::create_role_request) override; diff --git a/src/v/redpanda/admin/services/tests/security_test.cc b/src/v/redpanda/admin/services/tests/security_test.cc index 625fd447a45e2..5363fcdb73a4b 100644 --- a/src/v/redpanda/admin/services/tests/security_test.cc +++ b/src/v/redpanda/admin/services/tests/security_test.cc @@ -12,6 +12,7 @@ #include "proto/redpanda/core/admin/v2/security.proto.h" #include "redpanda/admin/services/security.h" #include "security/role.h" +#include "security/scram_algorithm.h" #include "serde/protobuf/rpc.h" #include @@ -23,6 +24,369 @@ class SecurityServiceTest : public ::testing::Test {}; // Bring internal namespace into scope for tests using namespace internal; +// ============================================= +// Tests for match_scram_credential +// ============================================= + +TEST_F(SecurityServiceTest, MatchScramCredentialSha256Valid) { + ss::sstring password = "test_password"; + + // Create SCRAM credential using the algorithm + auto cred = security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + + // Create protobuf SCRAM credential with the same password + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password(std::move(password)); + + // Should match + EXPECT_TRUE(match_scram_credential(pb_cred, cred)); +} + +TEST_F(SecurityServiceTest, MatchScramCredentialSha256Invalid) { + ss::sstring password = "test_password"; + ss::sstring wrong_password = "wrong_password"; + + // Create SCRAM credential with the correct password + auto cred = security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + + // Create protobuf SCRAM credential with a wrong password + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password(std::move(wrong_password)); + + // Should not match + EXPECT_FALSE(match_scram_credential(pb_cred, cred)); +} + +TEST_F(SecurityServiceTest, MatchScramCredentialSha512Valid) { + ss::sstring password = "test_password"; + + // Create SCRAM credential using the algorithm + auto cred = security::scram_sha512::make_credentials( + password, security::scram_sha512::min_iterations); + + // Create protobuf SCRAM credential with the same password + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + pb_cred.set_password(std::move(password)); + + // Should match + EXPECT_TRUE(match_scram_credential(pb_cred, cred)); +} + +TEST_F(SecurityServiceTest, MatchScramCredentialSha512Invalid) { + ss::sstring password = "test_password"; + ss::sstring wrong_password = "wrong_password"; + + // Create SCRAM credential with the correct password + auto cred = security::scram_sha512::make_credentials( + password, security::scram_sha512::min_iterations); + + // Create protobuf SCRAM credential with a wrong password + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + pb_cred.set_password(std::move(wrong_password)); + + // Should not match + EXPECT_FALSE(match_scram_credential(pb_cred, cred)); +} + +TEST_F(SecurityServiceTest, MatchScramCredentialUnknownMechanism) { + ss::sstring password = "test_password"; + + // Create SCRAM credential + auto cred = security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + + // Create protobuf SCRAM credential with an unknown mechanism + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_unspecified); + pb_cred.set_password(std::move(password)); + + // Should throw invalid_argument_exception + EXPECT_THROW( + match_scram_credential(pb_cred, cred), + serde::pb::rpc::invalid_argument_exception); +} + +TEST_F(SecurityServiceTest, MatchScramCredentialMismatchedMechanism) { + ss::sstring password = "test_password"; + + // Create SHA-256 SCRAM credential + auto cred = security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + + // Try to validate with SHA-512 mechanism + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + pb_cred.set_password(std::move(password)); + + // Should not match because the mechanisms are different + EXPECT_FALSE(match_scram_credential(pb_cred, cred)); +} + +// ============================================= +// Tests for validate_scram_credential_name +// ============================================= + +TEST_F(SecurityServiceTest, ValidateScramCredentialNameValid) { + // Valid SCRAM credential names should not throw + EXPECT_NO_THROW(validate_scram_credential_name("admin")); + EXPECT_NO_THROW(validate_scram_credential_name("user1")); + EXPECT_NO_THROW(validate_scram_credential_name("my-cred")); + EXPECT_NO_THROW(validate_scram_credential_name("my_cred")); + EXPECT_NO_THROW(validate_scram_credential_name("user123")); +} + +// Parameterized tests for invalid SCRAM credential names +struct InvalidScramCredentialNameCase { + ss::sstring name; + ss::sstring test_suffix; +}; + +class InvalidScramCredentialNameTest + : public ::testing::TestWithParam {}; + +TEST_P(InvalidScramCredentialNameTest, RejectsInvalidName) { + EXPECT_THROW( + validate_scram_credential_name(GetParam().name), + serde::pb::rpc::invalid_argument_exception); +} + +INSTANTIATE_TEST_SUITE_P( + InvalidNames, + InvalidScramCredentialNameTest, + ::testing::Values( + InvalidScramCredentialNameCase{"user\nname", "newline"}, + InvalidScramCredentialNameCase{"user\tname", "tab"}, + InvalidScramCredentialNameCase{"user\rname", "carriage_return"}, + InvalidScramCredentialNameCase{"\x01user", "control_char"}, + InvalidScramCredentialNameCase{"user,name", "comma"}, + InvalidScramCredentialNameCase{"user=name", "equals"}, + InvalidScramCredentialNameCase{"", "empty"}, + InvalidScramCredentialNameCase{std::string("user\0name", 9), "null_char"}), + [](const ::testing::TestParamInfo& info) { + return info.param.test_suffix; + }); + +// ============================================= +// Tests for validate_pb_scram_credential +// ============================================= + +TEST_F(SecurityServiceTest, ValidatePbScramCredentialValidSha256) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("valid_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password("a_valid_password_that_is_long_enough"); + + // Should not throw + EXPECT_NO_THROW(validate_pb_scram_credential(pb_cred)); +} + +TEST_F(SecurityServiceTest, ValidatePbScramCredentialValidSha512) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("valid_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + pb_cred.set_password("a_valid_password_that_is_long_enough"); + + // Should not throw + EXPECT_NO_THROW(validate_pb_scram_credential(pb_cred)); +} + +TEST_F(SecurityServiceTest, ValidatePbScramCredentialInvalidName) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("user\nname"); // Newline is invalid + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password("a_valid_password_that_is_long_enough"); + + // Should throw due to invalid name + EXPECT_THROW( + validate_pb_scram_credential(pb_cred), + serde::pb::rpc::invalid_argument_exception); +} + +TEST_F(SecurityServiceTest, ValidatePbScramCredentialPasswordWithControlChar) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("valid_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password("password_with\ncontrol"); // Newline in password + + // Should throw due to control character in password + EXPECT_THROW( + validate_pb_scram_credential(pb_cred), + serde::pb::rpc::invalid_argument_exception); +} + +TEST_F(SecurityServiceTest, ValidatePbScramCredentialUnspecifiedMechanism) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("valid_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_unspecified); + pb_cred.set_password("a_valid_password_that_is_long_enough"); + + // Should throw due to unspecified mechanism + EXPECT_THROW( + validate_pb_scram_credential(pb_cred), + serde::pb::rpc::invalid_argument_exception); +} + +// ============================================= +// Tests for convert_to_security_scram_credential +// ============================================= + +TEST_F(SecurityServiceTest, ConvertToSecurityScramCredentialSha256) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + pb_cred.set_password("test_password"); + + auto security_cred = convert_to_security_scram_credential(pb_cred); + + // Verify the credential was created with correct properties + EXPECT_EQ( + security_cred.iterations(), security::scram_sha256::min_iterations); + EXPECT_FALSE(security_cred.salt().empty()); + EXPECT_FALSE(security_cred.stored_key().empty()); + EXPECT_FALSE(security_cred.server_key().empty()); + + EXPECT_TRUE(match_scram_credential(pb_cred, security_cred)); +} + +TEST_F(SecurityServiceTest, ConvertToSecurityScramCredentialSha512) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + pb_cred.set_password("test_password"); + + auto security_cred = convert_to_security_scram_credential(pb_cred); + + // Verify the credential was created with correct properties + EXPECT_EQ( + security_cred.iterations(), security::scram_sha512::min_iterations); + EXPECT_FALSE(security_cred.salt().empty()); + EXPECT_FALSE(security_cred.stored_key().empty()); + EXPECT_FALSE(security_cred.server_key().empty()); + + EXPECT_TRUE(match_scram_credential(pb_cred, security_cred)); +} + +TEST_F(SecurityServiceTest, ConvertToSecurityScramCredentialUnknownMechanism) { + proto::admin::scram_credential pb_cred; + pb_cred.set_name("test_user"); + pb_cred.set_mechanism( + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_unspecified); + pb_cred.set_password("test_password"); + + // Should throw due to unknown mechanism + EXPECT_THROW( + convert_to_security_scram_credential(pb_cred), + serde::pb::rpc::invalid_argument_exception); +} + +// ============================================= +// Tests for convert_to_pb_scram_credential +// ============================================= + +TEST_F(SecurityServiceTest, ConvertToPbScramCredentialSha256) { + ss::sstring password = "test_password"; + ss::sstring name = "test_user"; + + // Create a SHA-256 SCRAM credential + auto security_cred = security::scram_sha256::make_credentials( + password, security::scram_sha256::min_iterations); + + // Convert to protobuf + auto pb_cred = convert_to_pb_scram_credential(name, security_cred); + + // Verify the mechanism is set correctly + EXPECT_EQ( + pb_cred.get_mechanism(), + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_256); + EXPECT_EQ(pb_cred.get_name(), name); + + // The password field is not able to be populated during conversion. + // Therefore, the match should fail. + EXPECT_TRUE(pb_cred.get_password().empty()); + EXPECT_FALSE(match_scram_credential(pb_cred, security_cred)); +} + +TEST_F(SecurityServiceTest, ConvertToPbScramCredentialSha512) { + ss::sstring password = "test_password"; + ss::sstring name = "test_user"; + + // Create a SHA-512 SCRAM credential + auto security_cred = security::scram_sha512::make_credentials( + password, security::scram_sha512::min_iterations); + + // Convert to protobuf + auto pb_cred = convert_to_pb_scram_credential(name, security_cred); + + // Verify the mechanism is set correctly + EXPECT_EQ( + pb_cred.get_mechanism(), + proto::admin::scram_credential_scram_mechanism:: + scram_mechanism_scram_sha_512); + EXPECT_EQ(pb_cred.get_name(), name); + + // The password field is not able to be populated during conversion. + // Therefore, the match should fail. + EXPECT_TRUE(pb_cred.get_password().empty()); + EXPECT_FALSE(match_scram_credential(pb_cred, security_cred)); +} + +TEST_F(SecurityServiceTest, ConvertToPbScramCredentialUnknownKeySize) { + ss::sstring name = "test_user"; + + // Create a credential with an invalid stored key size + // Using empty keys which will have size 0 (not matching SHA-256 or SHA-512) + security::scram_credential invalid_cred{ + bytes{}, // salt + bytes{}, // server_key + bytes{}, // stored_key - empty, so size = 0 + security::scram_sha256::min_iterations}; + + // Should throw internal_exception due to unknown key size + EXPECT_THROW( + convert_to_pb_scram_credential(name, invalid_cred), + serde::pb::rpc::internal_exception); +} + // ============================================= // Tests for validate_role_name // ============================================= diff --git a/src/v/security/fwd.h b/src/v/security/fwd.h index 68db271090fce..8e9236c1bf402 100644 --- a/src/v/security/fwd.h +++ b/src/v/security/fwd.h @@ -22,6 +22,7 @@ class role; class role_member; class role_member_view; class role_store; +class scram_credential; namespace oidc { diff --git a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.py b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.py index f84c1b3fb8f7b..1dba2f64887cb 100644 --- a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.py +++ b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.py @@ -11,17 +11,32 @@ from ......google.api import field_behavior_pb2 as google_dot_api_dot_field__behavior__pb2 from ......google.api import resource_pb2 as google_dot_api_dot_resource__pb2 from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n+proto/redpanda/core/admin/v2/security.proto\x12\x16redpanda.core.admin.v2\x1a\'proto/redpanda/core/pbgen/options.proto\x1a#proto/redpanda/core/pbgen/rpc.proto\x1a\x1fgoogle/api/field_behavior.proto\x1a\x19google/api/resource.proto\x1a\x1fgoogle/protobuf/timestamp.proto"\x8e\x01\n\x04Role\x12\x14\n\x04name\x18\x01 \x01(\tB\x06\xe0A\x02\xe0A\x05\x123\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMember:;\xeaA8\n(redpanda.core.admin.SecurityService/Role\x12\x0croles/{role}"D\n\x11CreateRoleRequest\x12/\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.RoleB\x03\xe0A\x02"@\n\x12CreateRoleResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"P\n\x0eGetRoleRequest\x12>\n\x04name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role"=\n\x0fGetRoleResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"\x12\n\x10ListRolesRequest"@\n\x11ListRolesResponse\x12+\n\x05roles\x18\x01 \x03(\x0b2\x1c.redpanda.core.admin.v2.Role"\x96\x01\n\x15AddRoleMembersRequest\x12C\n\trole_name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x128\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMemberB\x03\xe0A\x02"D\n\x16AddRoleMembersResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"\x99\x01\n\x18RemoveRoleMembersRequest\x12C\n\trole_name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x128\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMemberB\x03\xe0A\x02"G\n\x19RemoveRoleMembersResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"h\n\x11DeleteRoleRequest\x12>\n\x04name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x12\x13\n\x0bdelete_acls\x18\x02 \x01(\x08"\x14\n\x12DeleteRoleResponse"\x1d\n\x1bListCurrentUserRolesRequest"_\n\x1cListCurrentUserRolesResponse\x12?\n\x05roles\x18\x01 \x03(\tB0\xe0A\x03\xfaA*\n(redpanda.core.admin.SecurityService/Role"\x1c\n\x1aResolveOidcIdentityRequest"\\\n\x1bResolveOidcIdentityResponse\x12\x11\n\tprincipal\x18\x01 \x01(\t\x12*\n\x06expire\x18\x02 \x01(\x0b2\x1a.google.protobuf.Timestamp"\x18\n\x16RefreshOidcKeysRequest"\x19\n\x17RefreshOidcKeysResponse"\x1b\n\x19RevokeOidcSessionsRequest"\x1c\n\x1aRevokeOidcSessionsResponse"\x18\n\x08RoleUser\x12\x0c\n\x04name\x18\x01 \x01(\t"H\n\nRoleMember\x120\n\x04user\x18\x01 \x01(\x0b2 .redpanda.core.admin.v2.RoleUserH\x00B\x08\n\x06member2\xcc\t\n\x0fSecurityService\x12k\n\nCreateRole\x12).redpanda.core.admin.v2.CreateRoleRequest\x1a*.redpanda.core.admin.v2.CreateRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12b\n\x07GetRole\x12&.redpanda.core.admin.v2.GetRoleRequest\x1a\'.redpanda.core.admin.v2.GetRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12h\n\tListRoles\x12(.redpanda.core.admin.v2.ListRolesRequest\x1a).redpanda.core.admin.v2.ListRolesResponse"\x06\xea\x92\x19\x02\x10\x03\x12w\n\x0eAddRoleMembers\x12-.redpanda.core.admin.v2.AddRoleMembersRequest\x1a..redpanda.core.admin.v2.AddRoleMembersResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x80\x01\n\x11RemoveRoleMembers\x120.redpanda.core.admin.v2.RemoveRoleMembersRequest\x1a1.redpanda.core.admin.v2.RemoveRoleMembersResponse"\x06\xea\x92\x19\x02\x10\x03\x12k\n\nDeleteRole\x12).redpanda.core.admin.v2.DeleteRoleRequest\x1a*.redpanda.core.admin.v2.DeleteRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x89\x01\n\x14ListCurrentUserRoles\x123.redpanda.core.admin.v2.ListCurrentUserRolesRequest\x1a4.redpanda.core.admin.v2.ListCurrentUserRolesResponse"\x06\xea\x92\x19\x02\x10\x02\x12\x86\x01\n\x13ResolveOidcIdentity\x122.redpanda.core.admin.v2.ResolveOidcIdentityRequest\x1a3.redpanda.core.admin.v2.ResolveOidcIdentityResponse"\x06\xea\x92\x19\x02\x10\x02\x12z\n\x0fRefreshOidcKeys\x12..redpanda.core.admin.v2.RefreshOidcKeysRequest\x1a/.redpanda.core.admin.v2.RefreshOidcKeysResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x83\x01\n\x12RevokeOidcSessions\x121.redpanda.core.admin.v2.RevokeOidcSessionsRequest\x1a2.redpanda.core.admin.v2.RevokeOidcSessionsResponse"\x06\xea\x92\x19\x02\x10\x03B\x10\xea\x92\x19\x0cproto::adminb\x06proto3') +from google.protobuf import field_mask_pb2 as google_dot_protobuf_dot_field__mask__pb2 +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n+proto/redpanda/core/admin/v2/security.proto\x12\x16redpanda.core.admin.v2\x1a\'proto/redpanda/core/pbgen/options.proto\x1a#proto/redpanda/core/pbgen/rpc.proto\x1a\x1fgoogle/api/field_behavior.proto\x1a\x19google/api/resource.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a google/protobuf/field_mask.proto"\xe5\x02\n\x0fScramCredential\x12\x14\n\x04name\x18\x01 \x01(\tB\x06\xe0A\x02\xe0A\x05\x12I\n\tmechanism\x18\x02 \x01(\x0e26.redpanda.core.admin.v2.ScramCredential.ScramMechanism\x12\x18\n\x08password\x18\x03 \x01(\tB\x06\x80\x01\x01\xe0A\x04"w\n\x0eScramMechanism\x12\x1f\n\x1bSCRAM_MECHANISM_UNSPECIFIED\x10\x00\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_256\x10\x01\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_512\x10\x02:^\xeaA[\n3redpanda.core.admin.SecurityService/ScramCredential\x12$scram_credentials/{scram_credential}"\x8e\x01\n\x04Role\x12\x14\n\x04name\x18\x01 \x01(\tB\x06\xe0A\x02\xe0A\x05\x123\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMember:;\xeaA8\n(redpanda.core.admin.SecurityService/Role\x12\x0croles/{role}"f\n\x1cCreateScramCredentialRequest\x12F\n\x10scram_credential\x18\x01 \x01(\x0b2\'.redpanda.core.admin.v2.ScramCredentialB\x03\xe0A\x02"b\n\x1dCreateScramCredentialResponse\x12A\n\x10scram_credential\x18\x01 \x01(\x0b2\'.redpanda.core.admin.v2.ScramCredential"f\n\x19GetScramCredentialRequest\x12I\n\x04name\x18\x01 \x01(\tB;\xe0A\x02\xfaA5\n3redpanda.core.admin.SecurityService/ScramCredential"_\n\x1aGetScramCredentialResponse\x12A\n\x10scram_credential\x18\x01 \x01(\x0b2\'.redpanda.core.admin.v2.ScramCredential"\x1d\n\x1bListScramCredentialsRequest"b\n\x1cListScramCredentialsResponse\x12B\n\x11scram_credentials\x18\x01 \x03(\x0b2\'.redpanda.core.admin.v2.ScramCredential"\x97\x01\n\x1cUpdateScramCredentialRequest\x12F\n\x10scram_credential\x18\x01 \x01(\x0b2\'.redpanda.core.admin.v2.ScramCredentialB\x03\xe0A\x02\x12/\n\x0bupdate_mask\x18\x02 \x01(\x0b2\x1a.google.protobuf.FieldMask"b\n\x1dUpdateScramCredentialResponse\x12A\n\x10scram_credential\x18\x01 \x01(\x0b2\'.redpanda.core.admin.v2.ScramCredential"i\n\x1cDeleteScramCredentialRequest\x12I\n\x04name\x18\x01 \x01(\tB;\xe0A\x02\xfaA5\n3redpanda.core.admin.SecurityService/ScramCredential"\x1f\n\x1dDeleteScramCredentialResponse"D\n\x11CreateRoleRequest\x12/\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.RoleB\x03\xe0A\x02"@\n\x12CreateRoleResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"P\n\x0eGetRoleRequest\x12>\n\x04name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role"=\n\x0fGetRoleResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"\x12\n\x10ListRolesRequest"@\n\x11ListRolesResponse\x12+\n\x05roles\x18\x01 \x03(\x0b2\x1c.redpanda.core.admin.v2.Role"\x96\x01\n\x15AddRoleMembersRequest\x12C\n\trole_name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x128\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMemberB\x03\xe0A\x02"D\n\x16AddRoleMembersResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"\x99\x01\n\x18RemoveRoleMembersRequest\x12C\n\trole_name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x128\n\x07members\x18\x02 \x03(\x0b2".redpanda.core.admin.v2.RoleMemberB\x03\xe0A\x02"G\n\x19RemoveRoleMembersResponse\x12*\n\x04role\x18\x01 \x01(\x0b2\x1c.redpanda.core.admin.v2.Role"h\n\x11DeleteRoleRequest\x12>\n\x04name\x18\x01 \x01(\tB0\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role\x12\x13\n\x0bdelete_acls\x18\x02 \x01(\x08"\x14\n\x12DeleteRoleResponse"\x1d\n\x1bListCurrentUserRolesRequest"_\n\x1cListCurrentUserRolesResponse\x12?\n\x05roles\x18\x01 \x03(\tB0\xe0A\x03\xfaA*\n(redpanda.core.admin.SecurityService/Role"\x1c\n\x1aResolveOidcIdentityRequest"\\\n\x1bResolveOidcIdentityResponse\x12\x11\n\tprincipal\x18\x01 \x01(\t\x12*\n\x06expire\x18\x02 \x01(\x0b2\x1a.google.protobuf.Timestamp"\x18\n\x16RefreshOidcKeysRequest"\x19\n\x17RefreshOidcKeysResponse"\x1b\n\x19RevokeOidcSessionsRequest"\x1c\n\x1aRevokeOidcSessionsResponse"\x18\n\x08RoleUser\x12\x0c\n\x04name\x18\x01 \x01(\t"H\n\nRoleMember\x120\n\x04user\x18\x01 \x01(\x0b2 .redpanda.core.admin.v2.RoleUserH\x00B\x08\n\x06member2\x8b\x0f\n\x0fSecurityService\x12\x8c\x01\n\x15CreateScramCredential\x124.redpanda.core.admin.v2.CreateScramCredentialRequest\x1a5.redpanda.core.admin.v2.CreateScramCredentialResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x83\x01\n\x12GetScramCredential\x121.redpanda.core.admin.v2.GetScramCredentialRequest\x1a2.redpanda.core.admin.v2.GetScramCredentialResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x89\x01\n\x14ListScramCredentials\x123.redpanda.core.admin.v2.ListScramCredentialsRequest\x1a4.redpanda.core.admin.v2.ListScramCredentialsResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x8c\x01\n\x15UpdateScramCredential\x124.redpanda.core.admin.v2.UpdateScramCredentialRequest\x1a5.redpanda.core.admin.v2.UpdateScramCredentialResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x8c\x01\n\x15DeleteScramCredential\x124.redpanda.core.admin.v2.DeleteScramCredentialRequest\x1a5.redpanda.core.admin.v2.DeleteScramCredentialResponse"\x06\xea\x92\x19\x02\x10\x03\x12k\n\nCreateRole\x12).redpanda.core.admin.v2.CreateRoleRequest\x1a*.redpanda.core.admin.v2.CreateRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12b\n\x07GetRole\x12&.redpanda.core.admin.v2.GetRoleRequest\x1a\'.redpanda.core.admin.v2.GetRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12h\n\tListRoles\x12(.redpanda.core.admin.v2.ListRolesRequest\x1a).redpanda.core.admin.v2.ListRolesResponse"\x06\xea\x92\x19\x02\x10\x03\x12w\n\x0eAddRoleMembers\x12-.redpanda.core.admin.v2.AddRoleMembersRequest\x1a..redpanda.core.admin.v2.AddRoleMembersResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x80\x01\n\x11RemoveRoleMembers\x120.redpanda.core.admin.v2.RemoveRoleMembersRequest\x1a1.redpanda.core.admin.v2.RemoveRoleMembersResponse"\x06\xea\x92\x19\x02\x10\x03\x12k\n\nDeleteRole\x12).redpanda.core.admin.v2.DeleteRoleRequest\x1a*.redpanda.core.admin.v2.DeleteRoleResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x89\x01\n\x14ListCurrentUserRoles\x123.redpanda.core.admin.v2.ListCurrentUserRolesRequest\x1a4.redpanda.core.admin.v2.ListCurrentUserRolesResponse"\x06\xea\x92\x19\x02\x10\x02\x12\x86\x01\n\x13ResolveOidcIdentity\x122.redpanda.core.admin.v2.ResolveOidcIdentityRequest\x1a3.redpanda.core.admin.v2.ResolveOidcIdentityResponse"\x06\xea\x92\x19\x02\x10\x02\x12z\n\x0fRefreshOidcKeys\x12..redpanda.core.admin.v2.RefreshOidcKeysRequest\x1a/.redpanda.core.admin.v2.RefreshOidcKeysResponse"\x06\xea\x92\x19\x02\x10\x03\x12\x83\x01\n\x12RevokeOidcSessions\x121.redpanda.core.admin.v2.RevokeOidcSessionsRequest\x1a2.redpanda.core.admin.v2.RevokeOidcSessionsResponse"\x06\xea\x92\x19\x02\x10\x03B\x10\xea\x92\x19\x0cproto::adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'proto.redpanda.core.admin.v2.security_pb2', _globals) if not _descriptor._USE_C_DESCRIPTORS: _globals['DESCRIPTOR']._loaded_options = None _globals['DESCRIPTOR']._serialized_options = b'\xea\x92\x19\x0cproto::admin' + _globals['_SCRAMCREDENTIAL'].fields_by_name['name']._loaded_options = None + _globals['_SCRAMCREDENTIAL'].fields_by_name['name']._serialized_options = b'\xe0A\x02\xe0A\x05' + _globals['_SCRAMCREDENTIAL'].fields_by_name['password']._loaded_options = None + _globals['_SCRAMCREDENTIAL'].fields_by_name['password']._serialized_options = b'\x80\x01\x01\xe0A\x04' + _globals['_SCRAMCREDENTIAL']._loaded_options = None + _globals['_SCRAMCREDENTIAL']._serialized_options = b'\xeaA[\n3redpanda.core.admin.SecurityService/ScramCredential\x12$scram_credentials/{scram_credential}' _globals['_ROLE'].fields_by_name['name']._loaded_options = None _globals['_ROLE'].fields_by_name['name']._serialized_options = b'\xe0A\x02\xe0A\x05' _globals['_ROLE']._loaded_options = None _globals['_ROLE']._serialized_options = b'\xeaA8\n(redpanda.core.admin.SecurityService/Role\x12\x0croles/{role}' + _globals['_CREATESCRAMCREDENTIALREQUEST'].fields_by_name['scram_credential']._loaded_options = None + _globals['_CREATESCRAMCREDENTIALREQUEST'].fields_by_name['scram_credential']._serialized_options = b'\xe0A\x02' + _globals['_GETSCRAMCREDENTIALREQUEST'].fields_by_name['name']._loaded_options = None + _globals['_GETSCRAMCREDENTIALREQUEST'].fields_by_name['name']._serialized_options = b'\xe0A\x02\xfaA5\n3redpanda.core.admin.SecurityService/ScramCredential' + _globals['_UPDATESCRAMCREDENTIALREQUEST'].fields_by_name['scram_credential']._loaded_options = None + _globals['_UPDATESCRAMCREDENTIALREQUEST'].fields_by_name['scram_credential']._serialized_options = b'\xe0A\x02' + _globals['_DELETESCRAMCREDENTIALREQUEST'].fields_by_name['name']._loaded_options = None + _globals['_DELETESCRAMCREDENTIALREQUEST'].fields_by_name['name']._serialized_options = b'\xe0A\x02\xfaA5\n3redpanda.core.admin.SecurityService/ScramCredential' _globals['_CREATEROLEREQUEST'].fields_by_name['role']._loaded_options = None _globals['_CREATEROLEREQUEST'].fields_by_name['role']._serialized_options = b'\xe0A\x02' _globals['_GETROLEREQUEST'].fields_by_name['name']._loaded_options = None @@ -38,6 +53,16 @@ _globals['_DELETEROLEREQUEST'].fields_by_name['name']._serialized_options = b'\xe0A\x02\xfaA*\n(redpanda.core.admin.SecurityService/Role' _globals['_LISTCURRENTUSERROLESRESPONSE'].fields_by_name['roles']._loaded_options = None _globals['_LISTCURRENTUSERROLESRESPONSE'].fields_by_name['roles']._serialized_options = b'\xe0A\x03\xfaA*\n(redpanda.core.admin.SecurityService/Role' + _globals['_SECURITYSERVICE'].methods_by_name['CreateScramCredential']._loaded_options = None + _globals['_SECURITYSERVICE'].methods_by_name['CreateScramCredential']._serialized_options = b'\xea\x92\x19\x02\x10\x03' + _globals['_SECURITYSERVICE'].methods_by_name['GetScramCredential']._loaded_options = None + _globals['_SECURITYSERVICE'].methods_by_name['GetScramCredential']._serialized_options = b'\xea\x92\x19\x02\x10\x03' + _globals['_SECURITYSERVICE'].methods_by_name['ListScramCredentials']._loaded_options = None + _globals['_SECURITYSERVICE'].methods_by_name['ListScramCredentials']._serialized_options = b'\xea\x92\x19\x02\x10\x03' + _globals['_SECURITYSERVICE'].methods_by_name['UpdateScramCredential']._loaded_options = None + _globals['_SECURITYSERVICE'].methods_by_name['UpdateScramCredential']._serialized_options = b'\xea\x92\x19\x02\x10\x03' + _globals['_SECURITYSERVICE'].methods_by_name['DeleteScramCredential']._loaded_options = None + _globals['_SECURITYSERVICE'].methods_by_name['DeleteScramCredential']._serialized_options = b'\xea\x92\x19\x02\x10\x03' _globals['_SECURITYSERVICE'].methods_by_name['CreateRole']._loaded_options = None _globals['_SECURITYSERVICE'].methods_by_name['CreateRole']._serialized_options = b'\xea\x92\x19\x02\x10\x03' _globals['_SECURITYSERVICE'].methods_by_name['GetRole']._loaded_options = None @@ -58,51 +83,75 @@ _globals['_SECURITYSERVICE'].methods_by_name['RefreshOidcKeys']._serialized_options = b'\xea\x92\x19\x02\x10\x03' _globals['_SECURITYSERVICE'].methods_by_name['RevokeOidcSessions']._loaded_options = None _globals['_SECURITYSERVICE'].methods_by_name['RevokeOidcSessions']._serialized_options = b'\xea\x92\x19\x02\x10\x03' - _globals['_ROLE']._serialized_start = 243 - _globals['_ROLE']._serialized_end = 385 - _globals['_CREATEROLEREQUEST']._serialized_start = 387 - _globals['_CREATEROLEREQUEST']._serialized_end = 455 - _globals['_CREATEROLERESPONSE']._serialized_start = 457 - _globals['_CREATEROLERESPONSE']._serialized_end = 521 - _globals['_GETROLEREQUEST']._serialized_start = 523 - _globals['_GETROLEREQUEST']._serialized_end = 603 - _globals['_GETROLERESPONSE']._serialized_start = 605 - _globals['_GETROLERESPONSE']._serialized_end = 666 - _globals['_LISTROLESREQUEST']._serialized_start = 668 - _globals['_LISTROLESREQUEST']._serialized_end = 686 - _globals['_LISTROLESRESPONSE']._serialized_start = 688 - _globals['_LISTROLESRESPONSE']._serialized_end = 752 - _globals['_ADDROLEMEMBERSREQUEST']._serialized_start = 755 - _globals['_ADDROLEMEMBERSREQUEST']._serialized_end = 905 - _globals['_ADDROLEMEMBERSRESPONSE']._serialized_start = 907 - _globals['_ADDROLEMEMBERSRESPONSE']._serialized_end = 975 - _globals['_REMOVEROLEMEMBERSREQUEST']._serialized_start = 978 - _globals['_REMOVEROLEMEMBERSREQUEST']._serialized_end = 1131 - _globals['_REMOVEROLEMEMBERSRESPONSE']._serialized_start = 1133 - _globals['_REMOVEROLEMEMBERSRESPONSE']._serialized_end = 1204 - _globals['_DELETEROLEREQUEST']._serialized_start = 1206 - _globals['_DELETEROLEREQUEST']._serialized_end = 1310 - _globals['_DELETEROLERESPONSE']._serialized_start = 1312 - _globals['_DELETEROLERESPONSE']._serialized_end = 1332 - _globals['_LISTCURRENTUSERROLESREQUEST']._serialized_start = 1334 - _globals['_LISTCURRENTUSERROLESREQUEST']._serialized_end = 1363 - _globals['_LISTCURRENTUSERROLESRESPONSE']._serialized_start = 1365 - _globals['_LISTCURRENTUSERROLESRESPONSE']._serialized_end = 1460 - _globals['_RESOLVEOIDCIDENTITYREQUEST']._serialized_start = 1462 - _globals['_RESOLVEOIDCIDENTITYREQUEST']._serialized_end = 1490 - _globals['_RESOLVEOIDCIDENTITYRESPONSE']._serialized_start = 1492 - _globals['_RESOLVEOIDCIDENTITYRESPONSE']._serialized_end = 1584 - _globals['_REFRESHOIDCKEYSREQUEST']._serialized_start = 1586 - _globals['_REFRESHOIDCKEYSREQUEST']._serialized_end = 1610 - _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_start = 1612 - _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_end = 1637 - _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_start = 1639 - _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_end = 1666 - _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_start = 1668 - _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_end = 1696 - _globals['_ROLEUSER']._serialized_start = 1698 - _globals['_ROLEUSER']._serialized_end = 1722 - _globals['_ROLEMEMBER']._serialized_start = 1724 - _globals['_ROLEMEMBER']._serialized_end = 1796 - _globals['_SECURITYSERVICE']._serialized_start = 1799 - _globals['_SECURITYSERVICE']._serialized_end = 3027 \ No newline at end of file + _globals['_SCRAMCREDENTIAL']._serialized_start = 277 + _globals['_SCRAMCREDENTIAL']._serialized_end = 634 + _globals['_SCRAMCREDENTIAL_SCRAMMECHANISM']._serialized_start = 419 + _globals['_SCRAMCREDENTIAL_SCRAMMECHANISM']._serialized_end = 538 + _globals['_ROLE']._serialized_start = 637 + _globals['_ROLE']._serialized_end = 779 + _globals['_CREATESCRAMCREDENTIALREQUEST']._serialized_start = 781 + _globals['_CREATESCRAMCREDENTIALREQUEST']._serialized_end = 883 + _globals['_CREATESCRAMCREDENTIALRESPONSE']._serialized_start = 885 + _globals['_CREATESCRAMCREDENTIALRESPONSE']._serialized_end = 983 + _globals['_GETSCRAMCREDENTIALREQUEST']._serialized_start = 985 + _globals['_GETSCRAMCREDENTIALREQUEST']._serialized_end = 1087 + _globals['_GETSCRAMCREDENTIALRESPONSE']._serialized_start = 1089 + _globals['_GETSCRAMCREDENTIALRESPONSE']._serialized_end = 1184 + _globals['_LISTSCRAMCREDENTIALSREQUEST']._serialized_start = 1186 + _globals['_LISTSCRAMCREDENTIALSREQUEST']._serialized_end = 1215 + _globals['_LISTSCRAMCREDENTIALSRESPONSE']._serialized_start = 1217 + _globals['_LISTSCRAMCREDENTIALSRESPONSE']._serialized_end = 1315 + _globals['_UPDATESCRAMCREDENTIALREQUEST']._serialized_start = 1318 + _globals['_UPDATESCRAMCREDENTIALREQUEST']._serialized_end = 1469 + _globals['_UPDATESCRAMCREDENTIALRESPONSE']._serialized_start = 1471 + _globals['_UPDATESCRAMCREDENTIALRESPONSE']._serialized_end = 1569 + _globals['_DELETESCRAMCREDENTIALREQUEST']._serialized_start = 1571 + _globals['_DELETESCRAMCREDENTIALREQUEST']._serialized_end = 1676 + _globals['_DELETESCRAMCREDENTIALRESPONSE']._serialized_start = 1678 + _globals['_DELETESCRAMCREDENTIALRESPONSE']._serialized_end = 1709 + _globals['_CREATEROLEREQUEST']._serialized_start = 1711 + _globals['_CREATEROLEREQUEST']._serialized_end = 1779 + _globals['_CREATEROLERESPONSE']._serialized_start = 1781 + _globals['_CREATEROLERESPONSE']._serialized_end = 1845 + _globals['_GETROLEREQUEST']._serialized_start = 1847 + _globals['_GETROLEREQUEST']._serialized_end = 1927 + _globals['_GETROLERESPONSE']._serialized_start = 1929 + _globals['_GETROLERESPONSE']._serialized_end = 1990 + _globals['_LISTROLESREQUEST']._serialized_start = 1992 + _globals['_LISTROLESREQUEST']._serialized_end = 2010 + _globals['_LISTROLESRESPONSE']._serialized_start = 2012 + _globals['_LISTROLESRESPONSE']._serialized_end = 2076 + _globals['_ADDROLEMEMBERSREQUEST']._serialized_start = 2079 + _globals['_ADDROLEMEMBERSREQUEST']._serialized_end = 2229 + _globals['_ADDROLEMEMBERSRESPONSE']._serialized_start = 2231 + _globals['_ADDROLEMEMBERSRESPONSE']._serialized_end = 2299 + _globals['_REMOVEROLEMEMBERSREQUEST']._serialized_start = 2302 + _globals['_REMOVEROLEMEMBERSREQUEST']._serialized_end = 2455 + _globals['_REMOVEROLEMEMBERSRESPONSE']._serialized_start = 2457 + _globals['_REMOVEROLEMEMBERSRESPONSE']._serialized_end = 2528 + _globals['_DELETEROLEREQUEST']._serialized_start = 2530 + _globals['_DELETEROLEREQUEST']._serialized_end = 2634 + _globals['_DELETEROLERESPONSE']._serialized_start = 2636 + _globals['_DELETEROLERESPONSE']._serialized_end = 2656 + _globals['_LISTCURRENTUSERROLESREQUEST']._serialized_start = 2658 + _globals['_LISTCURRENTUSERROLESREQUEST']._serialized_end = 2687 + _globals['_LISTCURRENTUSERROLESRESPONSE']._serialized_start = 2689 + _globals['_LISTCURRENTUSERROLESRESPONSE']._serialized_end = 2784 + _globals['_RESOLVEOIDCIDENTITYREQUEST']._serialized_start = 2786 + _globals['_RESOLVEOIDCIDENTITYREQUEST']._serialized_end = 2814 + _globals['_RESOLVEOIDCIDENTITYRESPONSE']._serialized_start = 2816 + _globals['_RESOLVEOIDCIDENTITYRESPONSE']._serialized_end = 2908 + _globals['_REFRESHOIDCKEYSREQUEST']._serialized_start = 2910 + _globals['_REFRESHOIDCKEYSREQUEST']._serialized_end = 2934 + _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_start = 2936 + _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_end = 2961 + _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_start = 2963 + _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_end = 2990 + _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_start = 2992 + _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_end = 3020 + _globals['_ROLEUSER']._serialized_start = 3022 + _globals['_ROLEUSER']._serialized_end = 3046 + _globals['_ROLEMEMBER']._serialized_start = 3048 + _globals['_ROLEMEMBER']._serialized_end = 3120 + _globals['_SECURITYSERVICE']._serialized_start = 3123 + _globals['_SECURITYSERVICE']._serialized_end = 5054 \ No newline at end of file diff --git a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.pyi b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.pyi index 873dc42af9842..b5258702202b4 100644 --- a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.pyi +++ b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2.pyi @@ -18,7 +18,9 @@ limitations under the License. import builtins import collections.abc import google.protobuf.descriptor +import google.protobuf.field_mask_pb2 import google.protobuf.internal.containers +import google.protobuf.internal.enum_type_wrapper import google.protobuf.message import google.protobuf.timestamp_pb2 import sys @@ -30,12 +32,49 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor @typing.final -class Role(google.protobuf.message.Message): +class ScramCredential(google.protobuf.message.Message): """============================================= - The Role resource represents a security role with associated members. + The ScramCredential resource used for SCRAM authentication. """ DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _ScramMechanism: + ValueType = typing.NewType('ValueType', builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _ScramMechanismEnumTypeWrapper(google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ScramCredential._ScramMechanism.ValueType], builtins.type): + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + SCRAM_MECHANISM_UNSPECIFIED: ScramCredential._ScramMechanism.ValueType + SCRAM_MECHANISM_SCRAM_SHA_256: ScramCredential._ScramMechanism.ValueType + SCRAM_MECHANISM_SCRAM_SHA_512: ScramCredential._ScramMechanism.ValueType + + class ScramMechanism(_ScramMechanism, metaclass=_ScramMechanismEnumTypeWrapper): + """The SCRAM mechanism.""" + SCRAM_MECHANISM_UNSPECIFIED: ScramCredential.ScramMechanism.ValueType + SCRAM_MECHANISM_SCRAM_SHA_256: ScramCredential.ScramMechanism.ValueType + SCRAM_MECHANISM_SCRAM_SHA_512: ScramCredential.ScramMechanism.ValueType + NAME_FIELD_NUMBER: builtins.int + MECHANISM_FIELD_NUMBER: builtins.int + PASSWORD_FIELD_NUMBER: builtins.int + name: builtins.str + 'The name of the SCRAM credential.' + mechanism: Global___ScramCredential.ScramMechanism.ValueType + 'The SCRAM mechanism used for this credential.' + password: builtins.str + 'The password for the SCRAM credential.' + + def __init__(self, *, name: builtins.str=..., mechanism: Global___ScramCredential.ScramMechanism.ValueType=..., password: builtins.str=...) -> None: + ... + + def ClearField(self, field_name: typing.Literal['mechanism', b'mechanism', 'name', b'name', 'password', b'password']) -> None: + ... +Global___ScramCredential: typing_extensions.TypeAlias = ScramCredential + +@typing.final +class Role(google.protobuf.message.Message): + """The Role resource represents a security role with associated members.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor NAME_FIELD_NUMBER: builtins.int MEMBERS_FIELD_NUMBER: builtins.int name: builtins.str @@ -53,12 +92,197 @@ class Role(google.protobuf.message.Message): Global___Role: typing_extensions.TypeAlias = Role @typing.final -class CreateRoleRequest(google.protobuf.message.Message): +class CreateScramCredentialRequest(google.protobuf.message.Message): """============================================= - CreateRoleRequest is the request for the CreateRole RPC. + CreateScramCredentialRequest is the request for the CreateScramCredential + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIAL_FIELD_NUMBER: builtins.int + + @property + def scram_credential(self) -> Global___ScramCredential: + """The SCRAM credential to create.""" + + def __init__(self, *, scram_credential: Global___ScramCredential | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> None: + ... +Global___CreateScramCredentialRequest: typing_extensions.TypeAlias = CreateScramCredentialRequest + +@typing.final +class CreateScramCredentialResponse(google.protobuf.message.Message): + """CreateScramCredentialResponse is the response from the CreateScramCredential + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIAL_FIELD_NUMBER: builtins.int + + @property + def scram_credential(self) -> Global___ScramCredential: + """The created SCRAM credential.""" + + def __init__(self, *, scram_credential: Global___ScramCredential | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> None: + ... +Global___CreateScramCredentialResponse: typing_extensions.TypeAlias = CreateScramCredentialResponse + +@typing.final +class GetScramCredentialRequest(google.protobuf.message.Message): + """GetScramCredentialRequest is the request for the GetScramCredential RPC.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor + NAME_FIELD_NUMBER: builtins.int + name: builtins.str + 'The name of the SCRAM credential to retrieve.' + + def __init__(self, *, name: builtins.str=...) -> None: + ... + + def ClearField(self, field_name: typing.Literal['name', b'name']) -> None: + ... +Global___GetScramCredentialRequest: typing_extensions.TypeAlias = GetScramCredentialRequest + +@typing.final +class GetScramCredentialResponse(google.protobuf.message.Message): + """GetScramCredentialResponse is the response from the GetScramCredential RPC.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIAL_FIELD_NUMBER: builtins.int + + @property + def scram_credential(self) -> Global___ScramCredential: + """The requested SCRAM credential.""" + + def __init__(self, *, scram_credential: Global___ScramCredential | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> None: + ... +Global___GetScramCredentialResponse: typing_extensions.TypeAlias = GetScramCredentialResponse + +@typing.final +class ListScramCredentialsRequest(google.protobuf.message.Message): + """ListScramCredentialsRequest is the request for the ListScramCredentials RPC.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__(self) -> None: + ... +Global___ListScramCredentialsRequest: typing_extensions.TypeAlias = ListScramCredentialsRequest + +@typing.final +class ListScramCredentialsResponse(google.protobuf.message.Message): + """ListScramCredentialsResponse is the response from the ListScramCredentials + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIALS_FIELD_NUMBER: builtins.int + + @property + def scram_credentials(self) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[Global___ScramCredential]: + """The list of SCRAM credentials.""" + + def __init__(self, *, scram_credentials: collections.abc.Iterable[Global___ScramCredential] | None=...) -> None: + ... + + def ClearField(self, field_name: typing.Literal['scram_credentials', b'scram_credentials']) -> None: + ... +Global___ListScramCredentialsResponse: typing_extensions.TypeAlias = ListScramCredentialsResponse + +@typing.final +class UpdateScramCredentialRequest(google.protobuf.message.Message): + """UpdateScramCredentialRequest is the request for the UpdateScramCredential + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIAL_FIELD_NUMBER: builtins.int + UPDATE_MASK_FIELD_NUMBER: builtins.int + + @property + def scram_credential(self) -> Global___ScramCredential: + """The SCRAM credential to update.""" + + @property + def update_mask(self) -> google.protobuf.field_mask_pb2.FieldMask: + """The list of fields to update + See [AIP-134](https://google.aip.dev/134) for how to use `field_mask` + """ + + def __init__(self, *, scram_credential: Global___ScramCredential | None=..., update_mask: google.protobuf.field_mask_pb2.FieldMask | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['scram_credential', b'scram_credential', 'update_mask', b'update_mask']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['scram_credential', b'scram_credential', 'update_mask', b'update_mask']) -> None: + ... +Global___UpdateScramCredentialRequest: typing_extensions.TypeAlias = UpdateScramCredentialRequest + +@typing.final +class UpdateScramCredentialResponse(google.protobuf.message.Message): + """UpdateScramCredentialResponse is the response from the UpdateScramCredential + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + SCRAM_CREDENTIAL_FIELD_NUMBER: builtins.int + + @property + def scram_credential(self) -> Global___ScramCredential: + """The updated SCRAM credential.""" + + def __init__(self, *, scram_credential: Global___ScramCredential | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['scram_credential', b'scram_credential']) -> None: + ... +Global___UpdateScramCredentialResponse: typing_extensions.TypeAlias = UpdateScramCredentialResponse + +@typing.final +class DeleteScramCredentialRequest(google.protobuf.message.Message): + """DeleteScramCredentialRequest is the request for the DeleteScramCredential + RPC. + """ + DESCRIPTOR: google.protobuf.descriptor.Descriptor + NAME_FIELD_NUMBER: builtins.int + name: builtins.str + 'The name of the SCRAM credential to delete.' + + def __init__(self, *, name: builtins.str=...) -> None: + ... + + def ClearField(self, field_name: typing.Literal['name', b'name']) -> None: + ... +Global___DeleteScramCredentialRequest: typing_extensions.TypeAlias = DeleteScramCredentialRequest + +@typing.final +class DeleteScramCredentialResponse(google.protobuf.message.Message): + """DeleteScramCredentialResponse is the response from the DeleteScramCredential + RPC. """ DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__(self) -> None: + ... +Global___DeleteScramCredentialResponse: typing_extensions.TypeAlias = DeleteScramCredentialResponse + +@typing.final +class CreateRoleRequest(google.protobuf.message.Message): + """CreateRoleRequest is the request for the CreateRole RPC.""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor ROLE_FIELD_NUMBER: builtins.int @property diff --git a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2_connect.py b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2_connect.py index c005e7b55c2be..5cbcaffac59c1 100644 --- a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2_connect.py +++ b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/security_pb2_connect.py @@ -34,6 +34,81 @@ def __init__(self, base_url: str, http_client: urllib3.PoolManager | None=None, self.base_url = base_url self._connect_client = ConnectClient(http_client, protocol) + def call_create_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse]: + """Low-level method to call CreateScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/CreateScramCredential' + return self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse, extra_headers, timeout_seconds) + + def create_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse: + response = self.call_create_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + def call_get_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse]: + """Low-level method to call GetScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/GetScramCredential' + return self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse, extra_headers, timeout_seconds) + + def get_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse: + response = self.call_get_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + def call_list_scram_credentials(self, req: proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse]: + """Low-level method to call ListScramCredentials, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/ListScramCredentials' + return self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse, extra_headers, timeout_seconds) + + def list_scram_credentials(self, req: proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse: + response = self.call_list_scram_credentials(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + def call_update_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse]: + """Low-level method to call UpdateScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/UpdateScramCredential' + return self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse, extra_headers, timeout_seconds) + + def update_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse: + response = self.call_update_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + def call_delete_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse]: + """Low-level method to call DeleteScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/DeleteScramCredential' + return self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse, extra_headers, timeout_seconds) + + def delete_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse: + response = self.call_delete_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + def call_create_role(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateRoleRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.CreateRoleResponse]: """Low-level method to call CreateRole, granting access to errors and metadata""" url = self.base_url + '/redpanda.core.admin.v2.SecurityService/CreateRole' @@ -190,6 +265,81 @@ def __init__(self, base_url: str, http_client: aiohttp.ClientSession, protocol: self.base_url = base_url self._connect_client = AsyncConnectClient(http_client, protocol) + async def call_create_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse]: + """Low-level method to call CreateScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/CreateScramCredential' + return await self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse, extra_headers, timeout_seconds) + + async def create_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse: + response = await self.call_create_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + async def call_get_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse]: + """Low-level method to call GetScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/GetScramCredential' + return await self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse, extra_headers, timeout_seconds) + + async def get_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse: + response = await self.call_get_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + async def call_list_scram_credentials(self, req: proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse]: + """Low-level method to call ListScramCredentials, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/ListScramCredentials' + return await self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse, extra_headers, timeout_seconds) + + async def list_scram_credentials(self, req: proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse: + response = await self.call_list_scram_credentials(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + async def call_update_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse]: + """Low-level method to call UpdateScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/UpdateScramCredential' + return await self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse, extra_headers, timeout_seconds) + + async def update_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse: + response = await self.call_update_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + + async def call_delete_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse]: + """Low-level method to call DeleteScramCredential, granting access to errors and metadata""" + url = self.base_url + '/redpanda.core.admin.v2.SecurityService/DeleteScramCredential' + return await self._connect_client.call_unary(url, req, proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse, extra_headers, timeout_seconds) + + async def delete_scram_credential(self, req: proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse: + response = await self.call_delete_scram_credential(req, extra_headers, timeout_seconds) + err = response.error() + if err is not None: + raise err + msg = response.message() + if msg is None: + raise ConnectProtocolError('missing response message') + return msg + async def call_create_role(self, req: proto.redpanda.core.admin.v2.security_pb2.CreateRoleRequest, extra_headers: HeaderInput | None=None, timeout_seconds: float | None=None) -> UnaryOutput[proto.redpanda.core.admin.v2.security_pb2.CreateRoleResponse]: """Low-level method to call CreateRole, granting access to errors and metadata""" url = self.base_url + '/redpanda.core.admin.v2.SecurityService/CreateRole' @@ -343,6 +493,21 @@ async def revoke_oidc_sessions(self, req: proto.redpanda.core.admin.v2.security_ @typing.runtime_checkable class SecurityServiceProtocol(typing.Protocol): + def create_scram_credential(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialResponse]: + ... + + def get_scram_credential(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialResponse]: + ... + + def list_scram_credentials(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsResponse]: + ... + + def update_scram_credential(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialResponse]: + ... + + def delete_scram_credential(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialResponse]: + ... + def create_role(self, req: ClientRequest[proto.redpanda.core.admin.v2.security_pb2.CreateRoleRequest]) -> ServerResponse[proto.redpanda.core.admin.v2.security_pb2.CreateRoleResponse]: ... @@ -376,6 +541,11 @@ def revoke_oidc_sessions(self, req: ClientRequest[proto.redpanda.core.admin.v2.s def wsgi_security_service(implementation: SecurityServiceProtocol) -> WSGIApplication: app = ConnectWSGI() + app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/CreateScramCredential', implementation.create_scram_credential, proto.redpanda.core.admin.v2.security_pb2.CreateScramCredentialRequest) + app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/GetScramCredential', implementation.get_scram_credential, proto.redpanda.core.admin.v2.security_pb2.GetScramCredentialRequest) + app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/ListScramCredentials', implementation.list_scram_credentials, proto.redpanda.core.admin.v2.security_pb2.ListScramCredentialsRequest) + app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/UpdateScramCredential', implementation.update_scram_credential, proto.redpanda.core.admin.v2.security_pb2.UpdateScramCredentialRequest) + app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/DeleteScramCredential', implementation.delete_scram_credential, proto.redpanda.core.admin.v2.security_pb2.DeleteScramCredentialRequest) app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/CreateRole', implementation.create_role, proto.redpanda.core.admin.v2.security_pb2.CreateRoleRequest) app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/GetRole', implementation.get_role, proto.redpanda.core.admin.v2.security_pb2.GetRoleRequest) app.register_unary_rpc('/redpanda.core.admin.v2.SecurityService/ListRoles', implementation.list_roles, proto.redpanda.core.admin.v2.security_pb2.ListRolesRequest) diff --git a/tests/rptest/tests/scram_test.py b/tests/rptest/tests/scram_test.py index 6735872104b60..45904860dac9b 100644 --- a/tests/rptest/tests/scram_test.py +++ b/tests/rptest/tests/scram_test.py @@ -20,6 +20,7 @@ import requests from confluent_kafka import KafkaError, KafkaException +from connectrpc.errors import ConnectError, ConnectErrorCode from ducktape.cluster.cluster import ClusterNode from ducktape.errors import TimeoutError from ducktape.mark import matrix, parametrize @@ -29,6 +30,10 @@ from rptest.util import expect_exception from rptest.utils.mode_checks import in_fips_environment +from rptest.clients.admin.proto.redpanda.core.admin.v2 import ( + security_pb2, +) +from rptest.clients.admin.v2 import Admin as AdminV2 from rptest.clients.kafka_cli_tools import KafkaCliTools, KafkaCliToolsError from rptest.clients.kcl import RawKCL from rptest.clients.python_librdkafka import PythonLibrdkafka @@ -54,10 +59,35 @@ generate_string_with_control_character, ) +SCRAM_MECHANISM_MAP = { + "SCRAM_MECHANISM_UNSPECIFIED": security_pb2.ScramCredential.SCRAM_MECHANISM_UNSPECIFIED, + "SCRAM-SHA-256": security_pb2.ScramCredential.SCRAM_MECHANISM_SCRAM_SHA_256, + "SCRAM-SHA-512": security_pb2.ScramCredential.SCRAM_MECHANISM_SCRAM_SHA_512, +} + + +def scram_mechanism_from_string( + name: str, +) -> security_pb2.ScramCredential.ScramMechanism: + try: + return SCRAM_MECHANISM_MAP[name] + except KeyError: + return security_pb2.ScramCredential.SCRAM_MECHANISM_UNSPECIFIED + class BaseScramTest(RedpandaTest): def __init__(self, test_context, **kwargs): super(BaseScramTest, self).__init__(test_context, **kwargs) + self.admin = AdminV2( + self.redpanda, + auth=( + self.redpanda.SUPERUSER_CREDENTIALS.username, + self.redpanda.SUPERUSER_CREDENTIALS.password, + ), + ) + + def gen_random_password(self, length): + return "".join(random.choice(string.ascii_letters) for _ in range(length)) def update_user( self, @@ -67,13 +97,10 @@ def update_user( expected_status_code=200, err_msg=None, ): - def gen(length): - return "".join(random.choice(string.ascii_letters) for _ in range(length)) - if quote: username = urllib.parse.quote(username, safe="") if password is None: - password = gen(20) + password = self.gen_random_password(20) controller = self.redpanda.nodes[0] url = f"http://{controller.account.hostname}:9644/v1/security/users/{username}" @@ -95,6 +122,38 @@ def gen(length): return password + def update_scram_credential_v2( + self, + name: str, + quote: bool = False, + password: str | None = None, + mechanism: security_pb2.ScramCredential.ScramMechanism = security_pb2.ScramCredential.SCRAM_MECHANISM_SCRAM_SHA_256, + expected_error: ConnectErrorCode | None = None, + ) -> str: + if quote: + name = urllib.parse.quote(name, safe="") + if password is None: + password = self.gen_random_password(20) + + req = security_pb2.UpdateScramCredentialRequest( + scram_credential=security_pb2.ScramCredential( + name=name, + password=password, + mechanism=mechanism, + ) + ) + + if expected_error is not None: + with expect_exception( + ConnectError, + lambda e: e.code == expected_error, + ): + _ = self.admin.security().update_scram_credential(req) + else: + _ = self.admin.security().update_scram_credential(req) + + return password + def delete_user(self, username, quote: bool = True): if quote: username = urllib.parse.quote(username, safe="") @@ -105,6 +164,14 @@ def delete_user(self, username, quote: bool = True): f"Status code: {res.status_code} for DELETE user {username}" ) + def delete_scram_credential_v2(self, name: str, quote: bool = False) -> None: + if quote: + name = urllib.parse.quote(name, safe="") + + _ = self.admin.security().delete_scram_credential( + security_pb2.DeleteScramCredentialRequest(name=name) + ) + def list_users(self): controller = self.redpanda.nodes[0] url = f"http://{controller.account.hostname}:9644/v1/security/users" @@ -112,14 +179,17 @@ def list_users(self): assert res.status_code == 200 return res.json() + def list_scram_credentials_v2(self) -> list[str]: + res = self.admin.security().list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ) + return [cred.name for cred in res.scram_credentials] + def create_user( self, username, algorithm, password=None, expected_status_code=200, err_msg=None ): - def gen(length): - return "".join(random.choice(string.ascii_letters) for _ in range(length)) - if password is None: - password = gen(15) + password = self.gen_random_password(15) controller = self.redpanda.nodes[0] url = f"http://{controller.account.hostname}:9644/v1/security/users" @@ -142,6 +212,35 @@ def gen(length): return password + def create_scram_credential_v2( + self, + name: str, + mechanism: security_pb2.ScramCredential.ScramMechanism, + password: str | None = None, + expected_error: ConnectErrorCode | None = None, + ) -> str: + if password is None: + password = self.gen_random_password(15) + + req = security_pb2.CreateScramCredentialRequest( + scram_credential=security_pb2.ScramCredential( + name=name, + password=password, + mechanism=mechanism, + ) + ) + + if expected_error is not None: + with expect_exception( + ConnectError, + lambda e: e.code == expected_error, + ): + _ = self.admin.security().create_scram_credential(req) + else: + _ = self.admin.security().create_scram_credential(req) + + return password + def make_superuser_client(self, password_override=None, algorithm_override=None): username, password, algorithm = self.redpanda.SUPERUSER_CREDENTIALS password = password_override or password @@ -253,7 +352,8 @@ def test_redirects(self, alternate_listener): assert resp.status_code == 200 @cluster(num_nodes=3) - def test_scram(self): + @matrix(use_v2_api=[False, True]) + def test_scram(self, use_v2_api): topic = TopicSpec() client = self.make_superuser_client() @@ -283,7 +383,10 @@ def test_scram(self): assert topic.name in topics username = self.redpanda.SUPERUSER_CREDENTIALS.username - self.delete_user(username) + if use_v2_api: + self.delete_scram_credential_v2(username) + else: + self.delete_user(username) try: # now listing should fail because the user has been deleted. add @@ -300,8 +403,16 @@ def test_scram(self): pass # recreate user - algorithm = self.redpanda.SUPERUSER_CREDENTIALS.algorithm - password = self.create_user(username, algorithm) + if use_v2_api: + password = self.create_scram_credential_v2( + username, + scram_mechanism_from_string( + self.redpanda.SUPERUSER_CREDENTIALS.algorithm + ), + ) + else: + algorithm = self.redpanda.SUPERUSER_CREDENTIALS.algorithm + password = self.create_user(username, algorithm) # works ok again client = self.make_superuser_client(password_override=password) @@ -310,7 +421,10 @@ def test_scram(self): assert topic.name in topics # update password - new_password = self.update_user(username) + if use_v2_api: + new_password = self.update_scram_credential_v2(username) + else: + new_password = self.update_user(username) try: # now listing should fail because the password is different @@ -331,22 +445,32 @@ def test_scram(self): print(topics) assert topic.name in topics - users = self.list_users() + if use_v2_api: + users = self.list_scram_credentials_v2() + else: + users = self.list_users() assert username in users @cluster(num_nodes=3) - @matrix(scram_algo=["SCRAM-SHA-256", "SCRAM-SHA-512"]) - def test_scram_kafka_api_describe(self, scram_algo): + @matrix(scram_algo=["SCRAM-SHA-256", "SCRAM-SHA-512"], use_v2_api=[False, True]) + def test_scram_kafka_api_describe(self, scram_algo, use_v2_api): """ This test validates the KIP-554 implementation of Redpanda """ test_username = "test-user" test_password = "test-password0" test_algorithm = scram_algo + if use_v2_api: + test_mechanism = scram_mechanism_from_string(scram_algo) - self.create_user( - username=test_username, algorithm=test_algorithm, password=test_password - ) + if use_v2_api: + self.create_scram_credential_v2( + name=test_username, mechanism=test_mechanism, password=test_password + ) + else: + self.create_user( + username=test_username, algorithm=test_algorithm, password=test_password + ) kcli = KafkaCliTools(self.redpanda) (username, algo, iterations) = kcli.describe_user_scram(user=test_username) @@ -355,7 +479,8 @@ def test_scram_kafka_api_describe(self, scram_algo): assert iterations == 4096, f"Expected 4096, got {iterations}" @cluster(num_nodes=3) - def test_scram_kafka_api_create_user(self): + @matrix(use_v2_api=[False, True]) + def test_scram_kafka_api_create_user(self, use_v2_api): test_username = "test-user" test_password = "test-password0" test_algorithm = "SCRAM-SHA-256" @@ -380,7 +505,10 @@ def test_scram_kafka_api_create_user(self): iteration_count=iteration_count, ) - users = self.list_users() + if use_v2_api: + users = self.list_scram_credentials_v2() + else: + users = self.list_users() assert test_username in users, f"Expected {test_username} to be in {users}" # Validate that we can use RPK to list topics with the new user @@ -392,16 +520,26 @@ def test_scram_kafka_api_create_user(self): ).list_topics() @cluster(num_nodes=3) - def test_scram_kafka_api_modify_user(self): + @matrix(use_v2_api=[False, True]) + def test_scram_kafka_api_modify_user(self, use_v2_api): test_username = "test-user" orig_password = "test-password0" new_password = "new-password" orig_algorithm = "SCRAM-SHA-256" new_algorithm = "SCRAM-SHA-512" - - self.create_user( - username=test_username, algorithm=orig_algorithm, password=orig_password - ) + if use_v2_api: + orig_mechanism = scram_mechanism_from_string(orig_algorithm) + + if use_v2_api: + self.create_scram_credential_v2( + test_username, + orig_mechanism, + orig_password, + ) + else: + self.create_user( + username=test_username, algorithm=orig_algorithm, password=orig_password + ) kcli = KafkaCliTools(self.redpanda) kcli.create_alter_scram_user( @@ -417,14 +555,22 @@ def test_scram_kafka_api_modify_user(self): ).list_topics() @cluster(num_nodes=3) - def test_scram_kafka_api_delete_user(self): + @matrix(use_v2_api=[False, True]) + def test_scram_kafka_api_delete_user(self, use_v2_api): test_username = "test-user" test_password = "test-password0" test_algorithm = "SCRAM-SHA-256" + if use_v2_api: + test_mechanism = scram_mechanism_from_string(test_algorithm) - self.create_user( - username=test_username, algorithm=test_algorithm, password=test_password - ) + if use_v2_api: + self.create_scram_credential_v2( + test_username, test_mechanism, test_password + ) + else: + self.create_user( + username=test_username, algorithm=test_algorithm, password=test_password + ) kcli = KafkaCliTools(self.redpanda) kcli.delete_scram_user(user=test_username, algorithm=test_algorithm) @@ -611,8 +757,9 @@ def _make_topic( client_type=list(ClientType), scram_type=list(ScramType), sasl_plain_config=list(SaslPlainConfig), + use_v2_api=[False, True], ) - def test_plain_authn(self, client_type, scram_type, sasl_plain_config): + def test_plain_authn(self, client_type, scram_type, sasl_plain_config, use_v2_api): """ This test validates that SASL/PLAIN works with common kafka client libraries: @@ -634,9 +781,14 @@ def test_plain_authn(self, client_type, scram_type, sasl_plain_config): ).sasl_allow_principal( principal=username, operations=["all"], resource="topic", resource_name="*" ) - self.create_user( - username=username, algorithm=str(scram_type), password=password - ) + if use_v2_api: + self.create_scram_credential_v2( + username, scram_mechanism_from_string(str(scram_type)), password + ) + else: + self.create_user( + username=username, algorithm=str(scram_type), password=password + ) self._config_plain_authn(sasl_plain_config) @@ -653,7 +805,8 @@ def test_plain_authn(self, client_type, scram_type, sasl_plain_config): self._make_topic(client, sasl_plain_enabled) @cluster(num_nodes=3) - def test_plain_authn_short_password(self): + @matrix(use_v2_api=[False, True]) + def test_plain_authn_short_password(self, use_v2_api): """ This test validates that SASL/PLAIN in fips mode fails gracefully when the user provides a short password. @@ -661,6 +814,7 @@ def test_plain_authn_short_password(self): username = "test-user" good_password = "sufficiently_long_password" bad_password = "short-pwd" + algorithm = str(self.ScramType.SCRAM_SHA_256) RpkTool( self.redpanda, username=self.redpanda.SUPERUSER_CREDENTIALS.username, @@ -669,12 +823,18 @@ def test_plain_authn_short_password(self): ).sasl_allow_principal( principal=username, operations=["all"], resource="topic", resource_name="*" ) - self.create_user( - username=username, - algorithm=str(self.ScramType.SCRAM_SHA_256), - password=good_password, - ) - + if use_v2_api: + self.create_scram_credential_v2( + username, + scram_mechanism_from_string(algorithm), + good_password, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + password=good_password, + ) self._config_plain_authn(self.SaslPlainConfig.ON) # We create the user with a good password to not have to worry about short password error @@ -895,6 +1055,26 @@ def _check_http_status_everywhere(self, expect_status, callable): return True + def _check_connect_err_everywhere(self, expected_code: ConnectErrorCode, callable): + """ + Check that the callback results in an HTTP error with the + given status code from all nodes in the cluster. This enables + checking that auth state has propagated as expected. + + :returns: true if all nodes throw an error with the expected status code + """ + + for n in self.redpanda.nodes: + try: + callable(n) + except ConnectError as e: + if e.code != expected_code: + return False + else: + return False + + return True + @cluster(num_nodes=3) @parametrize(mechanism="SCRAM-SHA-512") @parametrize(mechanism="SCRAM-SHA-256") @@ -938,6 +1118,82 @@ def test_bootstrap_user(self, mechanism): self.redpanda.restart_nodes(self.redpanda.nodes) admin.list_users() + @cluster(num_nodes=3) + @parametrize(mechanism="SCRAM-SHA-512") + @parametrize(mechanism="SCRAM-SHA-256") + def test_bootstrap_user_v2(self, mechanism): + # Anonymous access should be refused + admin = AdminV2(self.redpanda) + new_password = "newpassword0123456789" + + with expect_exception( + ConnectError, + lambda e: e.code == ConnectErrorCode.PERMISSION_DENIED, + ): + admin.security().list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ) + + # Access using the bootstrap credentials should succeed + admin = AdminV2( + self.redpanda, auth=(self.BOOTSTRAP_USERNAME, self.BOOTSTRAP_PASSWORD) + ) + + users = [ + cred.name + for cred in admin.security() + .list_scram_credentials(security_pb2.ListScramCredentialsRequest()) + .scram_credentials + ] + assert self.BOOTSTRAP_USERNAME in users + + # Modify the bootstrap user's credential + admin.security().update_scram_credential( + security_pb2.UpdateScramCredentialRequest( + scram_credential=security_pb2.ScramCredential( + name=self.BOOTSTRAP_USERNAME, + mechanism=scram_mechanism_from_string(mechanism), + password=new_password, + ) + ) + ) + + # Getting UNAUTHENTICATED with old credentials everywhere will show that the + # credential update has propagated to all nodes + wait_until( + lambda: self._check_connect_err_everywhere( + ConnectErrorCode.UNAUTHENTICATED, + lambda n: admin.security(node=n).list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ), + ), + timeout_sec=10, + backoff_sec=0.5, + ) + + # Using old password should fail + with expect_exception( + ConnectError, + lambda e: e.code == ConnectErrorCode.UNAUTHENTICATED, + ): + admin.security().list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ) + + # Using new credential should succeed + admin = AdminV2(self.redpanda, auth=(self.BOOTSTRAP_USERNAME, new_password)) + admin.security().list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ) + + # Modified credential should survive a restart: this verifies that + # the RP_BOOTSTRAP_USER setting does not fight with changes made + # by other means. + self.redpanda.restart_nodes(self.redpanda.nodes) + admin.security().list_scram_credentials( + security_pb2.ListScramCredentialsRequest() + ) + @cluster( num_nodes=1, log_allow_list=[re.compile(r"std::invalid_argument.*Invalid SCRAM mechanism")], @@ -967,101 +1223,179 @@ def __init__(self, test_context): ) @cluster(num_nodes=3) - def test_invalid_user_name(self): + @matrix(use_v2_api=[False, True]) + def test_invalid_user_name(self, use_v2_api): """ Validates that usernames that contain control characters and usernames which do not match the SCRAM regex are properly rejected """ username = generate_string_with_control_character(15) + algorithm = "SCRAM-SHA-256" - self.create_user( - username=username, - algorithm="SCRAM-SHA-256", - expected_status_code=400, - err_msg="Parameter 'username' contained invalid control characters", - ) - - # Two ordinals (corresponding to ',' and '=') are explicitly excluded from SASL usernames - for ordinal in [0x2C, 0x3D]: - username = f"john{chr(ordinal)}doe" + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + expected_error=ConnectErrorCode.INVALID_ARGUMENT, + ) + else: self.create_user( username=username, - algorithm="SCRAM-SHA-256", + algorithm=algorithm, expected_status_code=400, - err_msg=f"Invalid SCRAM username {'{' + username + '}'}", + err_msg="Parameter 'username' contained invalid control characters", ) + # Two ordinals (corresponding to ',' and '=') are explicitly excluded from SASL usernames + for ordinal in [0x2C, 0x3D]: + username = f"john{chr(ordinal)}doe" + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + expected_error=ConnectErrorCode.INVALID_ARGUMENT, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + expected_status_code=400, + err_msg=f"Invalid SCRAM username {'{' + username + '}'}", + ) + @cluster(num_nodes=3) - def test_invalid_alg(self): + @matrix(use_v2_api=[False, True]) + def test_invalid_alg(self, use_v2_api): """ Validates that algorithms that contain control characters are properly rejected """ algorithm = generate_string_with_control_character(10) - - self.create_user( - username="test", - algorithm=algorithm, - expected_status_code=400, - err_msg="Parameter 'algorithm' contained invalid control characters", - ) + username = "test" + + if use_v2_api: + # TODO: Is this actually meaningful to test with v2 API? + mechanism = scram_mechanism_from_string(algorithm) + self.create_scram_credential_v2( + name=username, + mechanism=mechanism, + expected_error=ConnectErrorCode.INVALID_ARGUMENT, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + expected_status_code=400, + err_msg="Parameter 'algorithm' contained invalid control characters", + ) @cluster(num_nodes=3) - def test_invalid_password(self): + @matrix(use_v2_api=[False, True]) + def test_invalid_password(self, use_v2_api): """ Validates that passwords that contain control characters are properly rejected """ + username = "test" password = generate_string_with_control_character(15) - self.create_user( - username="test", - algorithm="SCRAM-SHA-256", - password=password, - expected_status_code=400, - err_msg="Parameter 'password' contained invalid control characters", - ) + algorithm = "SCRAM-SHA-256" + + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=password, + expected_error=ConnectErrorCode.INVALID_ARGUMENT, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + password=password, + expected_status_code=400, + err_msg="Parameter 'password' contained invalid control characters", + ) @cluster(num_nodes=3) - def test_short_password(self): + @matrix(use_v2_api=[False, True]) + def test_short_password(self, use_v2_api): """ Validate that in fips mode, short scram passwords (<14 chars) are rejected with a clean error. In non-fips mode, they should be accepted. """ - password = "short_pwd" + username = "test-user" + algorithm = "SCRAM-SHA-256" + short_password = "short_pwd" + if in_fips_environment(): expected_status_code = 400 + expected_v2_error = ConnectErrorCode.INVALID_ARGUMENT err_msg = "Password length less than 14 characters" else: expected_status_code = 200 + expected_v2_error = None err_msg = None # Validate failure in fips mode and success in non-fips - self.create_user( - username="test-user", - algorithm="SCRAM-SHA-256", - password=password, - expected_status_code=expected_status_code, - err_msg=err_msg, - ) + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=short_password, + expected_error=expected_v2_error, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + password=short_password, + expected_status_code=expected_status_code, + err_msg=err_msg, + ) # Validate success in both - password is long enough - self.create_user( - username="test-user-2", - algorithm="SCRAM-SHA-256", - password="sufficiently_long_password", - ) + username = "test-user-2" + long_password = "sufficiently_long_password" + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=long_password, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + password=long_password, + ) # Validate failure in fips mode and success in non-fips - self.update_user( - username="test-user-2", - password=password, - expected_status_code=expected_status_code, - err_msg=err_msg, - ) + if use_v2_api: + self.update_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=short_password, + expected_error=expected_v2_error, + ) + else: + self.update_user( + username=username, + password=short_password, + expected_status_code=expected_status_code, + err_msg=err_msg, + ) # Validate success in both - password is long enough - self.update_user( - username="test-user-2", - password="other_sufficiently_long_password", - ) + other_long_password = "other_sufficiently_long_password" + if use_v2_api: + self.update_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=other_long_password, + ) + else: + self.update_user( + username=username, + password=other_long_password, + ) class EscapedNewUserStrings(BaseScramTest): @@ -1096,7 +1430,8 @@ class EscapedNewUserStrings(BaseScramTest): ] @cluster(num_nodes=3) - def test_update_delete_user(self): + @matrix(use_v2_api=[False, True]) + def test_update_delete_user(self, use_v2_api): """ Verifies that users whose names contain characters which require URL escaping can be subsequently deleted. i.e. that the username included with a delete request is properly unescaped by the admin server. @@ -1108,20 +1443,35 @@ def test_update_delete_user(self): self.logger.debug("Create some users with names that will require URL escaping") + password = "passwd01234567" + algorithm = "SCRAM-SHA-256" + for ch in self.NEED_ESCAPE: username = f"john{ch}doe" - self.create_user( - username=username, - algorithm="SCRAM-SHA-256", - password="passwd01234567", - expected_status_code=200, - ) + if use_v2_api: + self.create_scram_credential_v2( + name=username, + mechanism=scram_mechanism_from_string(algorithm), + password=password, + expected_error=None, + ) + else: + self.create_user( + username=username, + algorithm=algorithm, + password=password, + expected_status_code=200, + ) users.append(username) admin = Admin(self.redpanda) def _users_match(expected: list[str]): - live_users = admin.list_users() + if use_v2_api: + live_users = self.list_scram_credentials_v2() + else: + live_users = admin.list_users() + live_users.remove(su_username) return len(expected) == len(live_users) and set(expected) == set(live_users) @@ -1131,13 +1481,20 @@ def _users_match(expected: list[str]): "We should be able to update and delete these users without issue" ) for username in users: - self.update_user(username=username) - self.delete_user(username=username) + if use_v2_api: + self.update_scram_credential_v2(name=username, quote=False) + self.delete_scram_credential_v2(name=username, quote=False) + else: + self.update_user(username=username) + self.delete_user(username=username) try: wait_until(lambda: _users_match([]), timeout_sec=5, backoff_sec=0.5) except TimeoutError: - live_users = admin.list_users() + if use_v2_api: + live_users = self.list_scram_credentials_v2() + else: + live_users = admin.list_users() live_users.remove(su_username) assert len(live_users) == 0, ( f"Expected no users, got {len(live_users)}: {live_users}"