diff --git a/proto/redpanda/core/admin/v2/security.proto b/proto/redpanda/core/admin/v2/security.proto index ea498df26856b..1243db245474f 100644 --- a/proto/redpanda/core/admin/v2/security.proto +++ b/proto/redpanda/core/admin/v2/security.proto @@ -278,6 +278,9 @@ message ResolveOidcIdentityResponse { // The timestamp of the token's expiry. google.protobuf.Timestamp expire = 2; + + // The groups resolved from the OIDC token. + repeated string groups = 3; } // RefreshOidcKeysRequest is the request for the RefreshOidcKeys RPC. diff --git a/src/v/kafka/server/connection_context.cc b/src/v/kafka/server/connection_context.cc index d382c355d7bb1..e7b4a3434b445 100644 --- a/src/v/kafka/server/connection_context.cc +++ b/src/v/kafka/server/connection_context.cc @@ -271,7 +271,12 @@ security::auth_result connection_context::authorized( } return authorized_user( - get_principal(), operation, name, quiet, superuser_required); + get_principal(), + operation, + name, + quiet, + superuser_required, + get_groups()); } template security::auth_result connection_context::authorized( @@ -306,7 +311,8 @@ security::auth_result connection_context::authorized_user( security::acl_operation operation, const T& name, authz_quiet quiet, - superuser_required superuser_required) { + superuser_required superuser_required, + const chunked_vector& groups) { auto authorized = _server.authorizer().authorized( name, operation, @@ -314,7 +320,8 @@ security::auth_result connection_context::authorized_user( security::acl_host(_client_addr), security::superuser_required{ superuser_required ? security::superuser_required::yes - : security::superuser_required::no}); + : security::superuser_required::no}, + groups); if (!authorized) { if (_sasl) { @@ -369,7 +376,8 @@ connection_context::authorized_user( security::acl_operation operation, const model::topic& name, authz_quiet quiet, - superuser_required); + superuser_required, + const chunked_vector& groups); template security::auth_result connection_context::authorized_user( @@ -377,7 +385,8 @@ connection_context::authorized_user( security::acl_operation operation, const kafka::group_id& name, authz_quiet quiet, - superuser_required); + superuser_required, + const chunked_vector& groups); template security::auth_result connection_context::authorized_user( @@ -385,7 +394,8 @@ connection_context::authorized_user( security::acl_operation operation, const kafka::transactional_id& name, authz_quiet quiet, - superuser_required); + superuser_required, + const chunked_vector& groups); template security::auth_result connection_context::authorized_user( @@ -393,7 +403,8 @@ connection_context::authorized_user( security::acl_operation operation, const security::acl_cluster_name& name, authz_quiet quiet, - superuser_required); + superuser_required, + const chunked_vector& groups); ss::future<> connection_context::revoke_credentials(std::string_view name) { if ( @@ -606,6 +617,15 @@ ss::future<> connection_context::handle_auth_v0(const size_t size) { co_await conn->write(std::move(msg)); } +const chunked_vector& +connection_context::get_groups() const { + if (_sasl && _sasl->has_mechanism()) { + return _sasl->mechanism().groups(); + } + static const chunked_vector empty; + return empty; +} + bool connection_context::is_finished_parsing() const { return conn->input().eof() || abort_requested(); } diff --git a/src/v/kafka/server/connection_context.h b/src/v/kafka/server/connection_context.h index f99420a66dc32..a8a74a4c80d99 100644 --- a/src/v/kafka/server/connection_context.h +++ b/src/v/kafka/server/connection_context.h @@ -305,7 +305,8 @@ class connection_context final security::acl_operation operation, const T& name, authz_quiet quiet, - superuser_required superuser_required); + superuser_required superuser_required, + const chunked_vector& groups); security::acl_principal get_principal() const { if (_mtls_state) { @@ -317,6 +318,8 @@ class connection_context final return security::acl_principal{security::principal_type::user, {}}; } + const chunked_vector& get_groups() const; + bool is_finished_parsing() const; // Reserve units from memory from the memory semaphore in proportion diff --git a/src/v/pandaproxy/schema_registry/authorization.cc b/src/v/pandaproxy/schema_registry/authorization.cc index 8364019b97f19..81b8f5d33215e 100644 --- a/src/v/pandaproxy/schema_registry/authorization.cc +++ b/src/v/pandaproxy/schema_registry/authorization.cc @@ -132,7 +132,8 @@ void handle_authz( op, params.principal, params.host, - security::superuser_required::no); + security::superuser_required::no, + auth_result.get_groups()); }, [&](const detail::no_auth auto&) { return security::auth_result::authz_disabled( @@ -187,7 +188,8 @@ void handle_get_schemas_ids_id_authz( op, params.principal, params.host, - security::superuser_required::no); + security::superuser_required::no, + auth_result.value().get_groups()); if (res.is_authorized()) { authorizing_result = std::move(res); @@ -237,7 +239,8 @@ void handle_get_subjects_authz( op, params.principal, params.host, - security::superuser_required::no); + security::superuser_required::no, + auth_result.value().get_groups()); if (res.is_authorized()) { passing_results.emplace_back(subject(), subject_resource_type); return false; // keep diff --git a/src/v/redpanda/admin/services/security.cc b/src/v/redpanda/admin/services/security.cc index b95e674c190d6..3d3c05724a457 100644 --- a/src/v/redpanda/admin/services/security.cc +++ b/src/v/redpanda/admin/services/security.cc @@ -23,6 +23,8 @@ #include "security/role_store.h" #include "security/scram_algorithm.h" +#include + namespace admin { namespace { @@ -561,6 +563,11 @@ security_service_impl::resolve_oidc_identity( std::chrono::system_clock::time_point{ res.assume_value().expiry.time_since_epoch()})); + resp.set_groups( + {std::from_range, + auth_result->get_groups() + | std::views::transform(&security::acl_principal::name)}); + co_return resp; } diff --git a/src/v/security/audit/schemas/tests/ocsf_schemas_test.cc b/src/v/security/audit/schemas/tests/ocsf_schemas_test.cc index 4fda487e2198f..7586430417c34 100644 --- a/src/v/security/audit/schemas/tests/ocsf_schemas_test.cc +++ b/src/v/security/audit/schemas/tests/ocsf_schemas_test.cc @@ -562,7 +562,8 @@ BOOST_AUTO_TEST_CASE(make_api_activity_event_authorized) { security::credential_user{username}, security::credential_password{"password"}, "sasl", - request_auth_result::superuser::no}; + request_auth_result::superuser::no, + {}}; auto api_activity = sa::api_activity::construct( req, auth_result, "http", true, std::nullopt); @@ -681,7 +682,8 @@ BOOST_AUTO_TEST_CASE(make_authentication_event_success) { security::credential_user{username}, security::credential_password{"password"}, "sasl", - request_auth_result::superuser::no}; + request_auth_result::superuser::no, + {}}; auto authn = sa::authentication::construct(sa::authentication_event_options { .auth_protocol = "sasl", diff --git a/src/v/security/authorizer.cc b/src/v/security/authorizer.cc index c7bb75c2579b2..825d95c167135 100644 --- a/src/v/security/authorizer.cc +++ b/src/v/security/authorizer.cc @@ -166,9 +166,10 @@ auth_result authorizer::authorized( acl_operation operation, const acl_principal& principal, const acl_host& host, - superuser_required superuser_required) const { + superuser_required superuser_required, + const chunked_vector& groups) const { auth_result r = do_authorized( - resource_name, operation, principal, host, superuser_required); + resource_name, operation, principal, host, superuser_required, groups); _probe->record_authz_result( r.is_authorized() ? authz_result::allow : r.empty_matches ? authz_result::empty @@ -182,7 +183,8 @@ auth_result authorizer::do_authorized( acl_operation operation, const acl_principal& principal, const acl_host& host, - superuser_required superuser_required) const { + superuser_required superuser_required, + const chunked_vector& groups) const { auto type = get_resource_type(); auto acls = store().find(type, resource_name()); @@ -205,99 +207,89 @@ auth_result authorizer::do_authorized( bool(_allow_empty_matches)); } + chunked_vector effective_principals; + + const auto append_roles = [&effective_principals, + this](const acl_principal& p) { + std::ranges::copy( + _role_store->roles_for_member(role_member_view::from_principal(p)) + | std::views::transform( + [](const auto& r) { return role::to_principal_view(r); }), + std::back_inserter(effective_principals)); + }; + + effective_principals.emplace_back(principal); + + // Only users can be a member of roles, not ephemeral_users + if (principal.type() == principal_type::user) { + append_roles(principal); + } + + std::ranges::for_each(groups, [&effective_principals](const auto& g) { + effective_principals.emplace_back(g); + // TODO(gbac) Call append_roles for groups + }); + auto check_access = [this, &acls, &operation, &host, &resource_name]( acl_permission perm, - const security::acl_principal& user, - std::optional role - = std::nullopt) -> std::optional { - vassert( - !role - || (*role != nullptr && (*role)->type() == principal_type::role), - "Role principal should be non-null and have 'role' type if " - "present"); - const acl_principal_base& to_check = *role.value_or(&user); + const acl_principal& user, + acl_principal_view check_principal) -> std::optional { bool is_allow = perm == acl_permission::allow; std::optional entry; if (is_allow) { entry = acl_any_implied_ops_allowed( - acls, to_check, host, operation); + acls, check_principal, host, operation); } else { - entry = acls.find(operation, to_check, host, perm); + entry = acls.find(operation, check_principal, host, perm); } + if (!entry) { return std::nullopt; } - switch (to_check.type()) { + + switch (check_principal.type()) { case principal_type::user: case principal_type::ephemeral_user: return auth_result::acl_match( user, host, operation, resource_name, is_allow, *entry); case principal_type::role: - // TODO(GBAC) - CORE-14896 - case principal_type::group: return auth_result::role_acl_match( user, - security::role_name{to_check.name_view()}, + security::role_name{check_principal.name_view()}, + host, + operation, + resource_name, + is_allow, + *entry); + case principal_type::group: + return auth_result::group_acl_match( + user, + acl_principal{ + check_principal.type(), + ss::sstring{check_principal.name_view()}}, host, operation, resource_name, is_allow, *entry); } - __builtin_unreachable(); + std::unreachable(); }; - auto check_role_access = - [this, &principal, &check_access]( - acl_permission perm, - const acl_principal& user) -> std::optional { - switch (principal.type()) { - case security::principal_type::user: { - auto result - = _role_store->roles_for_member( - security::role_member_view::from_principal(principal)) - | std::views::transform( - [](const auto& e) { return role::to_principal_view(e); }) - | std::views::transform( - [&user, &check_access, perm](const auto& e) { - return check_access(perm, user, &e); - }) - | std::views::filter([](const std::optional& r) { - return r.has_value(); - }) - | std::views::take(1); - return (result.empty() ? std::nullopt : result.front()); + for (const auto& p : effective_principals) { + if (auto r = check_access(acl_permission::deny, principal, p); + r.has_value()) { + return *r; } - case security::principal_type::ephemeral_user: - case security::principal_type::role: - // TODO(GBAC) - CORE-14895 - case security::principal_type::group: - return std::nullopt; - } - __builtin_unreachable(); - }; - - if (auto result = check_access(acl_permission::deny, principal); - result.has_value()) { - return std::move(result).value(); - } - - if (auto result = check_role_access(acl_permission::deny, principal); - result.has_value()) { - return std::move(result).value(); } - if (auto result = check_access(acl_permission::allow, principal); - result.has_value()) { - return std::move(result).value(); - } - - if (auto result = check_role_access(acl_permission::allow, principal); - result.has_value()) { - return std::move(result).value(); + for (const auto& p : effective_principals) { + if (auto r = check_access(acl_permission::allow, principal, p); + r.has_value()) { + return *r; + } } - return auth_result::opt_acl_match( principal, host, operation, resource_name, std::nullopt); } @@ -307,42 +299,48 @@ template auth_result authorizer::authorized( acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector&) const; template auth_result authorizer::authorized( const kafka::group_id&, acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector& groups) const; template auth_result authorizer::authorized( const security::acl_cluster_name&, acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector&) const; template auth_result authorizer::authorized( const kafka::transactional_id&, acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector&) const; template auth_result authorizer::authorized( const pandaproxy::schema_registry::subject&, acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector&) const; template auth_result authorizer::authorized( const pandaproxy::schema_registry::registry_resource&, acl_operation, const acl_principal&, const acl_host&, - superuser_required) const; + superuser_required, + const chunked_vector&) const; std::optional authorizer::acl_any_implied_ops_allowed( const acl_matches& acls, diff --git a/src/v/security/authorizer.h b/src/v/security/authorizer.h index 6113e975543f4..c96dc8b360950 100644 --- a/src/v/security/authorizer.h +++ b/src/v/security/authorizer.h @@ -64,6 +64,9 @@ struct auth_result { // If found, the role that was matched to provide authZ decision std::optional role; + // If found, the group that was matched to provide authz decision + std::optional group; + friend std::ostream& operator<<(std::ostream& os, const auth_result& a); explicit operator bool() const noexcept { return is_authorized(); } @@ -177,6 +180,27 @@ struct auth_result { .role = role}; } + template + static auth_result group_acl_match( + const security::acl_principal& principal, + const security::acl_principal& group, + security::acl_host host, + security::acl_operation operation, + const T& resource, + bool authorized, + const security::acl_match& match) { + return { + .authorized = authorized, + .resource_pattern = match.resource, + .acl = match.acl, + .principal = principal, + .host = host, + .resource_type = get_resource_type(), + .resource_name = resource(), + .operation = operation, + .group = group}; + } + template static auth_result opt_acl_match( const security::acl_principal& principal, @@ -254,7 +278,8 @@ class authorizer final { acl_operation operation, const acl_principal& principal, const acl_host& host, - superuser_required superuser_required) const; + superuser_required superuser_required, + const chunked_vector& groups) const; ss::future> all_bindings() const; ss::future<> reset_bindings(const chunked_vector& bindings); @@ -269,7 +294,8 @@ class authorizer final { acl_operation operation, const acl_principal& principal, const acl_host& host, - superuser_required superuser_required) const; + superuser_required superuser_required, + const chunked_vector& groups) const; /* * Compute whether the specified operation is allowed based on the implied diff --git a/src/v/security/oidc_authenticator.h b/src/v/security/oidc_authenticator.h index 5b28142d69232..cd6eaacd7e01d 100644 --- a/src/v/security/oidc_authenticator.h +++ b/src/v/security/oidc_authenticator.h @@ -90,6 +90,10 @@ class sasl_authenticator final : public sasl_mechanism { const char* mechanism_name() const override { return name; } + const chunked_vector& groups() const override { + return _auth_data.groups; + } + private: friend std::ostream& operator<<(std::ostream& os, const sasl_authenticator::state s); diff --git a/src/v/security/request_auth.cc b/src/v/security/request_auth.cc index 1807bea657f9c..6267d707d994e 100644 --- a/src/v/security/request_auth.cc +++ b/src/v/security/request_auth.cc @@ -154,7 +154,8 @@ request_auth_result request_authenticator::do_authenticate( std::move(username), std::move(password), ss::sstring{*sasl_mechanism}, - request_auth_result::superuser(superuser)); + request_auth_result::superuser(superuser), + {}); } } } else if (supports("OIDC") && auth_hdr.starts_with(authz_bearer_prefix)) { @@ -176,11 +177,13 @@ request_auth_result request_authenticator::do_authenticate( auto found = std::find(superusers.begin(), superusers.end(), principal); bool superuser = (found != superusers.end()) || (!require_auth); vlog(logger.trace, "Authenticated principal {}", principal); + vlog(logger.trace, "OIDC groups: {}", res.assume_value().groups); return request_auth_result{ security::credential_user{principal}, security::credential_password{auth_hdr}, security::oidc::sasl_authenticator::name, - request_auth_result::superuser{superuser}}; + request_auth_result::superuser{superuser}, + std::move(res.assume_value()).groups}; } else if (!auth_hdr.empty()) { throw ss::httpd::bad_request_exception( "Unsupported Authorization method"); @@ -231,6 +234,7 @@ request_auth_result::request_auth_result(request_auth_result&& other) noexcept : _username{std::move(other._username)} , _password{std::move(other._password)} , _sasl_mechanism{std::move(other._sasl_mechanism)} + , _groups{std::move(other._groups)} , _authenticated{other._authenticated} , _superuser{other._superuser} , _auth_required{other._auth_required} diff --git a/src/v/security/request_auth.h b/src/v/security/request_auth.h index 09e714bb7c367..59961be7f7f54 100644 --- a/src/v/security/request_auth.h +++ b/src/v/security/request_auth.h @@ -13,6 +13,7 @@ #include "cluster/fwd.h" #include "config/property.h" +#include "security/acl.h" #include "security/fwd.h" #include "security/types.h" @@ -59,10 +60,12 @@ class [[nodiscard]] request_auth_result { security::credential_user username, security::credential_password password, ss::sstring sasl_mechanism, - superuser is_superuser) + superuser is_superuser, + chunked_vector groups) : _username(std::move(username)) , _password(std::move(password)) , _sasl_mechanism(std::move(sasl_mechanism)) + , _groups(std::move(groups)) , _authenticated(true) , _superuser(is_superuser) , _auth_required(true) {}; @@ -83,7 +86,16 @@ class [[nodiscard]] request_auth_result { request_auth_result operator=(request_auth_result&&) = delete; request_auth_result operator=(const request_auth_result&) = delete; - request_auth_result(const request_auth_result&) = default; + request_auth_result(const request_auth_result& rhs) + : _username(rhs._username) + , _password(rhs._password) + , _sasl_mechanism(rhs._sasl_mechanism) + , _groups(rhs._groups.copy()) + , _authenticated(rhs._authenticated) + , _superuser(rhs._superuser) + , _auth_required(rhs._auth_required) + , _checked(rhs._checked) {} + request_auth_result(request_auth_result&&) noexcept; ~request_auth_result() noexcept(false); @@ -105,6 +117,9 @@ class [[nodiscard]] request_auth_result { const ss::sstring& get_username() const { return _username; } const ss::sstring& get_password() const { return _password; } const ss::sstring& get_sasl_mechanism() const { return _sasl_mechanism; } + const chunked_vector& get_groups() const { + return _groups; + } bool is_authenticated() const { return _authenticated; }; bool is_superuser() const { return _superuser; } @@ -114,6 +129,7 @@ class [[nodiscard]] request_auth_result { security::credential_user _username; security::credential_password _password; ss::sstring _sasl_mechanism; + chunked_vector _groups; bool _authenticated{false}; bool _superuser{false}; bool _auth_required{false}; diff --git a/src/v/security/sasl_authentication.h b/src/v/security/sasl_authentication.h index e8ad0cb2a7eaa..936c4d0f44092 100644 --- a/src/v/security/sasl_authentication.h +++ b/src/v/security/sasl_authentication.h @@ -39,6 +39,10 @@ class sasl_mechanism { } virtual const audit::user& audit_user() const = 0; virtual const char* mechanism_name() const = 0; + virtual const chunked_vector& groups() const { + static const chunked_vector empty; + return empty; + } }; /* diff --git a/src/v/security/tests/authorizer_test.cc b/src/v/security/tests/authorizer_test.cc index 5feab1183273a..2e03262d40e5b 100644 --- a/src/v/security/tests/authorizer_test.cc +++ b/src/v/security/tests/authorizer_test.cc @@ -126,7 +126,8 @@ TEST(AUTHORIZER_TEST, authz_empty_resource_name) { acl_operation::read, user, host, - security::superuser_required::no)); + security::superuser_required::no, + {})); acl_entry acl( acl_wildcard_user, @@ -145,7 +146,8 @@ TEST(AUTHORIZER_TEST, authz_empty_resource_name) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, acl); ASSERT_EQ(result.resource_pattern, resource); @@ -185,7 +187,8 @@ TEST(AUTHORIZER_TEST, authz_deny_applies_first) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_EQ(result.acl, deny); @@ -221,7 +224,8 @@ TEST(AUTHORIZER_TEST, authz_allow_all) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, acl); ASSERT_EQ(result.resource_pattern, resource); @@ -260,7 +264,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user1, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_EQ(result.acl, acl); @@ -277,7 +282,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user2, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_EQ(result.acl, acl); @@ -297,7 +303,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user1, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -314,7 +321,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user2, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -334,7 +342,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user1, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_EQ(result.acl, acl); @@ -351,7 +360,8 @@ TEST(AUTHORIZER_TEST, authz_super_user_allow) { acl_operation::read, user2, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -375,7 +385,8 @@ TEST(AUTHORIZER_TEST, authz_wildcards) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -404,7 +415,8 @@ TEST(AUTHORIZER_TEST, authz_wildcards) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, read_acl); @@ -437,7 +449,8 @@ TEST(AUTHORIZER_TEST, authz_wildcards) { acl_operation::write, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_EQ(result.acl, deny_write_acl); ASSERT_EQ(result.resource_pattern, wildcard_resource); @@ -460,7 +473,8 @@ TEST(AUTHORIZER_TEST, authz_no_acls_deny) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -484,7 +498,8 @@ TEST(AUTHORIZER_TEST, authz_no_acls_allow) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_FALSE(result.acl.has_value()); @@ -499,13 +514,15 @@ TEST(AUTHORIZER_TEST, authz_no_acls_allow) { static void do_implied_acls( const acl_principal& bind_principal, - std::optional roles = std::nullopt) { - auto test_allow = [&bind_principal, &roles]( + std::optional roles = std::nullopt, + chunked_vector groups = {}) { + auto test_allow = [&bind_principal, &roles, &groups]( acl_operation op, std::set allowed) { acl_principal user(principal_type::user, "alice"); ASSERT_TRUE( user == bind_principal - || bind_principal.type() == principal_type::role); + || bind_principal.type() == principal_type::role + || bind_principal.type() == principal_type::group); acl_host host("192.168.3.1"); @@ -540,7 +557,8 @@ static void do_implied_acls( test_op, user, host, - security::superuser_required::no); + security::superuser_required::no, + groups); if (allowed.contains(test_op) || test_op == op) { ASSERT_TRUE(ok.authorized); ASSERT_EQ(ok.acl, acl); @@ -561,12 +579,13 @@ static void do_implied_acls( } }; - auto test_deny = [&bind_principal, &roles]( + auto test_deny = [&bind_principal, &roles, &groups]( acl_operation op, std::set denied) { acl_principal user(principal_type::user, "alice"); ASSERT_TRUE( user == bind_principal - || bind_principal.type() == principal_type::role); + || bind_principal.type() == principal_type::role + || bind_principal.type() == principal_type::group); acl_host host("192.168.3.1"); @@ -608,7 +627,8 @@ static void do_implied_acls( test_op, user, host, - security::superuser_required::no); + security::superuser_required::no, + groups); if (denied.contains(test_op) || test_op == op) { ASSERT_FALSE(ok.authorized); ASSERT_EQ(ok.acl, deny); @@ -694,7 +714,8 @@ TEST(AUTHORIZER_TEST, authz_allow_for_all_wildcard_resource) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, acl); @@ -767,7 +788,8 @@ TEST(AUTHORIZER_TEST, authz_allow_for_all_prefixed_resource) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, acl); @@ -943,7 +965,8 @@ TEST(AUTHORIZER_TEST, authz_auth_prefix_resource) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.authorized); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -963,7 +986,8 @@ TEST(AUTHORIZER_TEST, authz_auth_prefix_resource) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.authorized); ASSERT_EQ(result.acl, allow_read_acl); @@ -992,7 +1016,8 @@ TEST(AUTHORIZER_TEST, authz_single_char) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, allow_read_acl); ASSERT_EQ(result.resource_pattern, resource); @@ -1002,7 +1027,8 @@ TEST(AUTHORIZER_TEST, authz_single_char) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -1018,7 +1044,8 @@ TEST(AUTHORIZER_TEST, authz_single_char) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, allow_read_acl); ASSERT_EQ(result.resource_pattern, resource1); @@ -1028,7 +1055,8 @@ TEST(AUTHORIZER_TEST, authz_single_char) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, allow_read_acl); ASSERT_EQ(result.resource_pattern, resource1); @@ -1038,7 +1066,8 @@ TEST(AUTHORIZER_TEST, authz_single_char) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -1203,7 +1232,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::read, user1, host2, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl2); ASSERT_EQ(result.resource_pattern, resource); @@ -1213,7 +1243,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_EQ(result.acl, acl3); ASSERT_EQ(result.resource_pattern, resource); @@ -1223,7 +1254,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::write, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl4); ASSERT_EQ(result.resource_pattern, resource); @@ -1233,7 +1265,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::write, user1, host2, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -1243,7 +1276,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::describe, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl5); ASSERT_EQ(result.resource_pattern, resource); @@ -1253,7 +1287,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::describe, user1, host2, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl5); ASSERT_EQ(result.resource_pattern, resource); @@ -1263,7 +1298,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::alter, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -1273,7 +1309,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::alter, user1, host2, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result); ASSERT_FALSE(result.acl.has_value()); ASSERT_FALSE(result.resource_pattern.has_value()); @@ -1283,7 +1320,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::describe, user2, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl6); ASSERT_EQ(result.resource_pattern, resource); @@ -1293,7 +1331,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::describe, user3, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl7); ASSERT_EQ(result.resource_pattern, resource); @@ -1303,7 +1342,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::read, user2, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl6); ASSERT_EQ(result.resource_pattern, resource); @@ -1313,7 +1353,8 @@ TEST(AUTHORIZER_TEST, authz_topic_acl) { acl_operation::write, user3, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(bool(result)); ASSERT_EQ(result.acl, acl7); ASSERT_EQ(result.resource_pattern, resource); @@ -1340,13 +1381,15 @@ TEST(AUTHORIZER_TEST, authz_topic_group_same_name) { acl_operation::read, user, host, - security::superuser_required::no)); + security::superuser_required::no, + {})); ASSERT_FALSE(auth.authorized( kafka::group_id("topic-foo-xxx"), acl_operation::read, user, host, - security::superuser_required::no)); + security::superuser_required::no, + {})); } TEST(AUTHORIZER_TEST, role_authz_principal_view) { @@ -1423,7 +1466,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(bool(result)); EXPECT_FALSE(result.acl.has_value()); EXPECT_FALSE(result.resource_pattern.has_value()); @@ -1439,7 +1483,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::read, user, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_TRUE(bool(result)); EXPECT_EQ(result.acl, acl1); EXPECT_EQ(result.resource_pattern, resource); @@ -1452,7 +1497,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::read, user3, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(bool(result)); EXPECT_FALSE(result.acl.has_value()); EXPECT_FALSE(result.resource_pattern.has_value()); @@ -1464,7 +1510,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::write, user4, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_TRUE(bool(result)); EXPECT_EQ(result.acl, acl3); EXPECT_EQ(result.resource_pattern, resource); @@ -1477,7 +1524,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::write, user, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(bool(result)); EXPECT_FALSE(result.acl.has_value()); EXPECT_FALSE(result.resource_pattern.has_value()); @@ -1500,7 +1548,8 @@ TEST(AUTHORIZER_TEST, role_authz_simple_allow) { acl_operation::read, user3, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_TRUE(bool(result)); EXPECT_EQ(result.acl, acl1); EXPECT_EQ(result.resource_pattern, resource); @@ -1541,7 +1590,12 @@ TEST(AUTHORIZER_TEST, role_authz_user_deny_applies_first) { for (auto op : {acl_operation::read, acl_operation::write, acl_operation::describe}) { auto result = auth.authorized( - default_topic, op, user1, host1, security::superuser_required::no); + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {}); EXPECT_FALSE(bool(result)); if (op == acl_operation::write) { EXPECT_EQ(result.acl, deny_user); @@ -1565,7 +1619,12 @@ TEST(AUTHORIZER_TEST, role_authz_user_deny_applies_first) { for (auto op : {acl_operation::read, acl_operation::write, acl_operation::describe}) { auto result = auth.authorized( - default_topic, op, user1, host1, security::superuser_required::no); + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {}); if (op == acl_operation::write) { EXPECT_FALSE(bool(result)); EXPECT_EQ(result.acl, deny_user); @@ -1614,7 +1673,12 @@ TEST(AUTHORIZER_TEST, role_authz_role_deny_applies_first) { for (auto op : {acl_operation::read, acl_operation::write, acl_operation::describe}) { auto result = auth.authorized( - default_topic, op, user1, host1, security::superuser_required::no); + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {}); EXPECT_TRUE(bool(result)); EXPECT_EQ(result.acl, allow_user); EXPECT_EQ(result.resource_pattern, resource); @@ -1634,7 +1698,12 @@ TEST(AUTHORIZER_TEST, role_authz_role_deny_applies_first) { for (auto op : {acl_operation::read, acl_operation::write, acl_operation::describe}) { auto result = auth.authorized( - default_topic, op, user1, host1, security::superuser_required::no); + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {}); if (op == acl_operation::write) { EXPECT_FALSE(bool(result)); EXPECT_EQ(result.acl, deny_role); @@ -1721,7 +1790,8 @@ TEST(AUTHORIZER_TEST, role_authz_wildcard_no_auth) { acl_operation::write, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(result.authorized); } @@ -1766,7 +1836,8 @@ TEST(AUTHORIZER_TEST, role_authz_user_same_name) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_TRUE(result.authorized); EXPECT_EQ(result.acl, allow_user); EXPECT_EQ(result.principal, user1); @@ -1776,7 +1847,8 @@ TEST(AUTHORIZER_TEST, role_authz_user_same_name) { acl_operation::read, user2, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(result.authorized); EXPECT_FALSE(result.acl.has_value()); EXPECT_EQ(result.principal, user2); @@ -1791,7 +1863,8 @@ TEST(AUTHORIZER_TEST, role_authz_user_same_name) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_TRUE(result.authorized); EXPECT_EQ(result.acl, allow_user); EXPECT_EQ(result.principal, user1); @@ -1801,7 +1874,8 @@ TEST(AUTHORIZER_TEST, role_authz_user_same_name) { acl_operation::read, user2, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); EXPECT_FALSE(result.authorized); EXPECT_EQ(result.acl, deny_role); EXPECT_EQ(result.principal, user2); @@ -1883,7 +1957,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.is_authorized()); auto kafka_acls = get_acls(auth, acl_binding_filter::any()); @@ -1901,7 +1976,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.is_authorized()); // Check read implies describe @@ -1910,7 +1986,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::describe, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.is_authorized()); // Check read does not imply write @@ -1919,7 +1996,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::write, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.is_authorized()); // Check global resource @@ -1928,7 +2006,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::describe, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_TRUE(result.is_authorized()); // Check that describe does not imply read @@ -1937,7 +2016,8 @@ TEST(AUTHORIZER_TEST, authz_filter_out_non_kafka_resources) { acl_operation::read, user, host, - security::superuser_required::no); + security::superuser_required::no, + {}); ASSERT_FALSE(result.is_authorized()); } @@ -1968,7 +2048,8 @@ TEST(AUTHORIZER_TEST, authz_superuser_required) { acl_operation::read, normaluser, host, - security::superuser_required::no); + security::superuser_required::no, + {}); // Verify that the normal user is authorized still EXPECT_TRUE(result.is_authorized()); @@ -1979,7 +2060,8 @@ TEST(AUTHORIZER_TEST, authz_superuser_required) { acl_operation::read, normaluser, host, - security::superuser_required::yes); + security::superuser_required::yes, + {}); EXPECT_FALSE(result.is_authorized()); EXPECT_TRUE(result.required_superuser); @@ -1988,9 +2070,694 @@ TEST(AUTHORIZER_TEST, authz_superuser_required) { acl_operation::read, superuser, host, - security::superuser_required::yes); + security::superuser_required::yes, + {}); EXPECT_TRUE(result.is_authorized()); EXPECT_TRUE(result.is_superuser); } +TEST(AUTHORIZER_TEST, group_authz_principal_view) { + ss::sstring s1{"foor"}; + ss::sstring s2{"bar"}; + + ASSERT_NE(s1, s2); + acl_principal p1{principal_type::user, s1}; + acl_principal p2{principal_type::user, s2}; + acl_principal_view pv1{p1}; + + EXPECT_NE(p1, p2); + EXPECT_EQ(p1, pv1); + EXPECT_NE(p2, pv1); + + acl_principal p3{principal_type::group, s1}; + acl_principal p4{principal_type::group, s2}; + acl_principal_view pv3{p3}; + + EXPECT_NE(p3, p4); + EXPECT_EQ(p3, pv3); + EXPECT_NE(p4, pv3); + + // respects principal type + EXPECT_NE(p1, p3); + EXPECT_NE(p2, p4); + EXPECT_NE(p1, pv3); + EXPECT_NE(pv1, pv3); +} + +TEST(AUTHORIZER_TEST, group_authz_simple_allow) { + acl_principal user1(principal_type::user, "phyllis"); + acl_principal group1(principal_type::group, "group-lola"); + + acl_host host1("192.168.1.2"); + auto host_any = acl_host::wildcard_host(); + const model::topic topic1("topic1"); + + acl_entry acl1( + group1, host_any, acl_operation::read, acl_permission::allow); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, topic1(), pattern_type::literal); + + bindings.emplace_back(resource, acl1); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + auto result = auth.authorized( + topic1, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(bool(result)); + EXPECT_EQ(result.acl, acl1); + EXPECT_EQ(result.group, group1); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); +} + +TEST(AUTHORIZER_TEST, group_authz_user_deny_applies_first) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + + acl_host host1("192.168.1.2"); + acl_entry deny_user( + user1, + acl_host::wildcard_host(), + acl_operation::write, + acl_permission::deny); + + acl_entry allow_group( + group1, + acl_host::wildcard_host(), + acl_operation::all, + acl_permission::allow); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, deny_user); + bindings.emplace_back(resource, allow_group); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // user1 should be denied write access to the topic + + for (auto op : + {acl_operation::read, acl_operation::write, acl_operation::describe}) { + auto result = auth.authorized( + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {group1}); + if (op == acl_operation::write) { + EXPECT_FALSE(bool(result)); + EXPECT_EQ(result.acl, deny_user); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_FALSE(result.group.has_value()); + } else { + EXPECT_TRUE(bool(result)); + EXPECT_EQ(result.acl, allow_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group1); + } + } +} + +TEST(AUTHORIZER_TEST, group_authz_group_deny_applies_first) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + + acl_host host1("192.168.1.2"); + + acl_entry allow_user( + user1, + acl_host::wildcard_host(), + acl_operation::all, + acl_permission::allow); + + acl_entry deny_group( + group1, + acl_host::wildcard_host(), + acl_operation::write, + acl_permission::deny); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_user); + bindings.emplace_back(resource, deny_group); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // user1 should still have read and describe access, but the deny acl from + // the group should take precedence + for (auto op : + {acl_operation::read, acl_operation::write, acl_operation::describe}) { + auto result = auth.authorized( + default_topic, + op, + user1, + host1, + security::superuser_required::no, + {group1}); + if (op == acl_operation::write) { + EXPECT_FALSE(bool(result)); + EXPECT_EQ(result.acl, deny_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group1); + } else { + EXPECT_TRUE(bool(result)); + EXPECT_EQ(result.acl, allow_user); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_FALSE(result.group.has_value()); + } + } +} + +TEST(AUTHORIZER_TEST, group_authz_multiple_groups_deny_precedence) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group_allow(principal_type::group, "group_allow"); + acl_principal group_deny(principal_type::group, "group_deny"); + + acl_host host1("192.168.1.2"); + + acl_entry allow_all_group( + group_allow, + acl_host::wildcard_host(), + acl_operation::all, + acl_permission::allow); + + acl_entry deny_write_group( + group_deny, + acl_host::wildcard_host(), + acl_operation::write, + acl_permission::deny); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_all_group); + bindings.emplace_back(resource, deny_write_group); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // Test with both groups - deny should take precedence for write operations + chunked_vector both_groups{group_allow, group_deny}; + + for (auto op : + {acl_operation::read, acl_operation::write, acl_operation::describe}) { + auto result = auth.authorized( + default_topic, + op, + user1, + host1, + security::superuser_required::no, + both_groups); + + if (op == acl_operation::write) { + // Write should be denied due to group_deny ACL taking precedence + EXPECT_FALSE(bool(result)); + EXPECT_EQ(result.acl, deny_write_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group_deny); + } else { + // Read and describe should be allowed via group_allow ACL + EXPECT_TRUE(bool(result)); + EXPECT_EQ(result.acl, allow_all_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group_allow); + } + } + + // Test with only the allow group - all operations should succeed + chunked_vector allow_group_only{group_allow}; + + for (auto op : + {acl_operation::read, acl_operation::write, acl_operation::describe}) { + auto result = auth.authorized( + default_topic, + op, + user1, + host1, + security::superuser_required::no, + allow_group_only); + + EXPECT_TRUE(bool(result)); + EXPECT_EQ(result.acl, allow_all_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group_allow); + } + + // Test with only the deny group - only write should be denied + chunked_vector deny_group_only{group_deny}; + + for (auto op : + {acl_operation::read, acl_operation::write, acl_operation::describe}) { + auto result = auth.authorized( + default_topic, + op, + user1, + host1, + security::superuser_required::no, + deny_group_only); + + if (op == acl_operation::write) { + EXPECT_FALSE(bool(result)); + EXPECT_EQ(result.acl, deny_write_group); + EXPECT_EQ(result.resource_pattern, resource); + EXPECT_FALSE(result.role.has_value()); + EXPECT_EQ(result.group, group_deny); + } else { + // No ACL allows read/describe for group_deny, so should fail + EXPECT_FALSE(bool(result)); + EXPECT_FALSE(result.acl.has_value()); + EXPECT_FALSE(result.resource_pattern.has_value()); + EXPECT_FALSE(result.role.has_value()); + EXPECT_FALSE(result.group.has_value()); + EXPECT_TRUE(result.empty_matches); + } + } +} + +TEST(AUTHORIZER_TEST, group_authz_implied_acls) { + acl_principal group1(principal_type::group, "group-admins"); + + do_implied_acls(group1, std::nullopt, {group1}); +} + +TEST(AUTHORIZER_TEST, group_authz_empty_groups_no_auth) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + acl_host host1("192.168.1.2"); + + acl_entry allow_group( + group1, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::allow); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_group); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // User with empty groups should not be authorized + auto result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {}); + + EXPECT_FALSE(result.authorized); + EXPECT_FALSE(result.acl.has_value()); + EXPECT_FALSE(result.group.has_value()); + EXPECT_TRUE(result.empty_matches); +} + +TEST(AUTHORIZER_TEST, group_authz_host_specific) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + acl_host host1("192.168.1.2"); + acl_host host2("192.168.1.3"); + + acl_entry allow_group_host1( + group1, host1, acl_operation::read, acl_permission::allow); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_group_host1); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // Should be authorized from correct host + auto result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.acl, allow_group_host1); + EXPECT_EQ(result.group, group1); + + // Should NOT be authorized from different host + result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host2, + security::superuser_required::no, + {group1}); + + EXPECT_FALSE(result.authorized); + EXPECT_FALSE(result.acl.has_value()); + EXPECT_FALSE(result.group.has_value()); +} + +TEST(AUTHORIZER_TEST, group_authz_roles_and_groups_priority) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + role_name role_name1("role1"); + acl_principal role1 = role::to_principal(role_name1()); + acl_host host1("192.168.1.2"); + + // Role allows read + acl_entry allow_role( + role1, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::allow); + + // Group denies read + acl_entry deny_group( + group1, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::deny); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_role); + bindings.emplace_back(resource, deny_group); + + role_store roles; + roles.put(role_name1, role{{role_member::from_principal(user1)}}); + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // Group deny should take precedence over role allow (deny always wins) + auto result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_FALSE(result.authorized); + EXPECT_EQ(result.acl, deny_group); + EXPECT_EQ(result.group, group1); + EXPECT_FALSE(result.role.has_value()); +} + +TEST(AUTHORIZER_TEST, group_authz_different_resource_types) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + acl_host host1("192.168.1.2"); + + // Group has read access to topics + acl_entry allow_topic_read( + group1, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::allow); + + // Group has write access to consumer groups + acl_entry allow_group_write( + group1, + acl_host::wildcard_host(), + acl_operation::write, + acl_permission::allow); + + std::vector bindings; + resource_pattern topic_resource( + resource_type::topic, default_topic(), pattern_type::literal); + resource_pattern group_resource( + resource_type::group, "consumer-group", pattern_type::literal); + + bindings.emplace_back(topic_resource, allow_topic_read); + bindings.emplace_back(group_resource, allow_group_write); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // Should have read access to topic + auto result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.acl, allow_topic_read); + EXPECT_EQ(result.group, group1); + + // Should have write access to consumer group + result = auth.authorized( + kafka::group_id("consumer-group"), + acl_operation::write, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.acl, allow_group_write); + EXPECT_EQ(result.group, group1); + + // Should NOT have write access to topic + result = auth.authorized( + default_topic, + acl_operation::write, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_FALSE(result.authorized); + EXPECT_FALSE(result.group.has_value()); +} + +TEST(AUTHORIZER_TEST, group_authz_prefixed_and_wildcard_resources) { + acl_principal user1(principal_type::user, "user1"); + acl_principal group1(principal_type::group, "group1"); + acl_host host1("192.168.1.2"); + + acl_entry allow_prefixed( + group1, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::allow); + + acl_entry allow_wildcard( + group1, + acl_host::wildcard_host(), + acl_operation::write, + acl_permission::allow); + + std::vector bindings; + resource_pattern prefixed_resource( + resource_type::topic, "test-", pattern_type::prefixed); + resource_pattern wildcard_resource( + resource_type::topic, resource_pattern::wildcard, pattern_type::literal); + + bindings.emplace_back(prefixed_resource, allow_prefixed); + bindings.emplace_back(wildcard_resource, allow_wildcard); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + auth.add_bindings(bindings); + + // Should match prefixed resource + auto result = auth.authorized( + model::topic("test-topic"), + acl_operation::read, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.acl, allow_prefixed); + EXPECT_EQ(result.resource_pattern, prefixed_resource); + EXPECT_EQ(result.group, group1); + + // Should match wildcard resource + result = auth.authorized( + model::topic("any-topic"), + acl_operation::write, + user1, + host1, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.acl, allow_wildcard); + EXPECT_EQ(result.resource_pattern, wildcard_resource); + EXPECT_EQ(result.group, group1); +} + +TEST(AUTHORIZER_TEST, group_authz_get_acls_by_group_principal) { + acl_principal group1(principal_type::group, "group1"); + acl_principal group2(principal_type::group, "group2"); + + auto auth = make_test_instance(); + + acl_entry group1_acl( + group1, acl_wildcard_host, acl_operation::read, acl_permission::allow); + acl_entry group2_acl( + group2, acl_wildcard_host, acl_operation::write, acl_permission::allow); + + std::vector bindings; + bindings.emplace_back(default_resource, group1_acl); + bindings.emplace_back(default_resource, group2_acl); + auth.add_bindings(bindings); + + // Should find ACLs for specific group + auto group1_acls = get_acls(auth, group1); + ASSERT_EQ(group1_acls.size(), 1); + ASSERT_TRUE(group1_acls.contains(group1_acl)); + + auto group2_acls = get_acls(auth, group2); + ASSERT_EQ(group2_acls.size(), 1); + ASSERT_TRUE(group2_acls.contains(group2_acl)); + + // Should not cross-contaminate + ASSERT_FALSE(group1_acls.contains(group2_acl)); + ASSERT_FALSE(group2_acls.contains(group1_acl)); +} + +TEST(AUTHORIZER_TEST, group_authz_superuser_overrides_group_deny) { + acl_principal superuser(principal_type::user, "superuser1"); + acl_principal group1(principal_type::group, "group1"); + acl_host host("192.0.4.4"); + + config::mock_property> superuser_config_prop( + std::vector{"superuser1"}); + role_store roles; + authorizer auth(superuser_config_prop.bind(), &roles); + + acl_entry deny_group( + group1, acl_wildcard_host, acl_operation::all, acl_permission::deny); + + std::vector bindings; + resource_pattern resource( + resource_type::topic, resource_pattern::wildcard, pattern_type::literal); + bindings.emplace_back(resource, deny_group); + auth.add_bindings(bindings); + + // Superuser should be authorized despite group deny ACL + auto result = auth.authorized( + default_topic, + acl_operation::read, + superuser, + host, + security::superuser_required::no, + {group1}); + + EXPECT_TRUE(result.authorized); + EXPECT_TRUE(result.is_superuser); + EXPECT_FALSE(result.acl.has_value()); + EXPECT_FALSE(result.group.has_value()); +} + +TEST(AUTHORIZER_TEST, group_authz_remove_bindings_with_groups) { + acl_principal group1(principal_type::group, "group1"); + acl_principal group2(principal_type::group, "group2"); + + auto auth = make_test_instance(); + + acl_entry group1_read( + group1, acl_wildcard_host, acl_operation::read, acl_permission::allow); + acl_entry group1_write( + group1, acl_wildcard_host, acl_operation::write, acl_permission::allow); + acl_entry group2_read( + group2, acl_wildcard_host, acl_operation::read, acl_permission::allow); + + std::vector bindings; + bindings.emplace_back(default_resource, group1_read); + bindings.emplace_back(default_resource, group1_write); + bindings.emplace_back(default_resource, group2_read); + auth.add_bindings(bindings); + + // Remove only group1's read permission + { + std::vector filters; + filters.emplace_back(default_resource, group1_read); + auth.remove_bindings(filters); + } + + auto remaining_acls = get_acls(auth, default_resource); + absl::flat_hash_set expected{group1_write, group2_read}; + ASSERT_EQ(remaining_acls, expected); +} + +TEST(AUTHORIZER_TEST, group_authz_large_number_of_groups) { + acl_principal user1(principal_type::user, "user1"); + acl_host host1("192.168.1.2"); + + role_store roles; + auto auth = make_test_instance(authorizer::allow_empty_matches::no, &roles); + + // Create many groups (simulate realistic scenario) + chunked_vector many_groups; + std::vector bindings; + + for (int i = 0; i < 50; ++i) { + acl_principal group(principal_type::group, fmt::format("group{}", i)); + many_groups.push_back(group); + + // Only group42 has permissions + if (i == 42) { + acl_entry allow_read( + group, + acl_host::wildcard_host(), + acl_operation::read, + acl_permission::allow); + + resource_pattern resource( + resource_type::topic, default_topic(), pattern_type::literal); + bindings.emplace_back(resource, allow_read); + } + } + + auth.add_bindings(bindings); + + // Should find the one group with permissions + auto result = auth.authorized( + default_topic, + acl_operation::read, + user1, + host1, + security::superuser_required::no, + many_groups); + + EXPECT_TRUE(result.authorized); + EXPECT_EQ(result.group.value().name(), "group42"); +} } // namespace security diff --git a/src/v/security/tests/role_store_bench.cc b/src/v/security/tests/role_store_bench.cc index 33fbc6e127200..5908af4d3631e 100644 --- a/src/v/security/tests/role_store_bench.cc +++ b/src/v/security/tests/role_store_bench.cc @@ -240,7 +240,12 @@ void run_authz( perf_tests::start_measuring_time(); auto result = auth.authorized( - topic1, acl_operation::read, p, host1, security::superuser_required::no); + topic1, + acl_operation::read, + p, + host1, + security::superuser_required::no, + {}); perf_tests::do_not_optimize(result); perf_tests::stop_measuring_time(); } @@ -319,7 +324,8 @@ PERF_TEST(role_store_bench, role_authz_empty_store) { acl_operation::read, user1, host1, - security::superuser_required::no); + security::superuser_required::no, + {}); perf_tests::do_not_optimize(result); perf_tests::stop_measuring_time(); } 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..1c0edd4ff0112 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,7 +11,7 @@ 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') +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"l\n\x1bResolveOidcIdentityResponse\x12\x11\n\tprincipal\x18\x01 \x01(\t\x12*\n\x06expire\x18\x02 \x01(\x0b2\x1a.google.protobuf.Timestamp\x12\x0e\n\x06groups\x18\x03 \x03(\t"\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') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'proto.redpanda.core.admin.v2.security_pb2', _globals) @@ -91,18 +91,18 @@ _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['_RESOLVEOIDCIDENTITYRESPONSE']._serialized_end = 1600 + _globals['_REFRESHOIDCKEYSREQUEST']._serialized_start = 1602 + _globals['_REFRESHOIDCKEYSREQUEST']._serialized_end = 1626 + _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_start = 1628 + _globals['_REFRESHOIDCKEYSRESPONSE']._serialized_end = 1653 + _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_start = 1655 + _globals['_REVOKEOIDCSESSIONSREQUEST']._serialized_end = 1682 + _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_start = 1684 + _globals['_REVOKEOIDCSESSIONSRESPONSE']._serialized_end = 1712 + _globals['_ROLEUSER']._serialized_start = 1714 + _globals['_ROLEUSER']._serialized_end = 1738 + _globals['_ROLEMEMBER']._serialized_start = 1740 + _globals['_ROLEMEMBER']._serialized_end = 1812 + _globals['_SECURITYSERVICE']._serialized_start = 1815 + _globals['_SECURITYSERVICE']._serialized_end = 3043 \ 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..0154317704e0d 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 @@ -312,6 +312,7 @@ class ResolveOidcIdentityResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor PRINCIPAL_FIELD_NUMBER: builtins.int EXPIRE_FIELD_NUMBER: builtins.int + GROUPS_FIELD_NUMBER: builtins.int principal: builtins.str 'The principal resolved from the OIDC token.' @@ -319,13 +320,17 @@ class ResolveOidcIdentityResponse(google.protobuf.message.Message): def expire(self) -> google.protobuf.timestamp_pb2.Timestamp: """The timestamp of the token's expiry.""" - def __init__(self, *, principal: builtins.str=..., expire: google.protobuf.timestamp_pb2.Timestamp | None=...) -> None: + @property + def groups(self) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """The groups resolved from the OIDC token.""" + + def __init__(self, *, principal: builtins.str=..., expire: google.protobuf.timestamp_pb2.Timestamp | None=..., groups: collections.abc.Iterable[builtins.str] | None=...) -> None: ... def HasField(self, field_name: typing.Literal['expire', b'expire']) -> builtins.bool: ... - def ClearField(self, field_name: typing.Literal['expire', b'expire', 'principal', b'principal']) -> None: + def ClearField(self, field_name: typing.Literal['expire', b'expire', 'groups', b'groups', 'principal', b'principal']) -> None: ... Global___ResolveOidcIdentityResponse: typing_extensions.TypeAlias = ResolveOidcIdentityResponse diff --git a/tests/rptest/services/keycloak.py b/tests/rptest/services/keycloak.py index ef13dbfb9a3cf..5540cc00fa877 100644 --- a/tests/rptest/services/keycloak.py +++ b/tests/rptest/services/keycloak.py @@ -6,6 +6,7 @@ from ducktape.services.service import Service from ducktape.utils.util import wait_until from keycloak import KeycloakAdmin +from keycloak.exceptions import KeycloakGetError KC_INSTALL_DIR = os.path.join("/", "opt", "keycloak") KC_DATA_DIR = os.path.join(KC_INSTALL_DIR, "data") @@ -96,6 +97,8 @@ def __init__( class KeycloakAdminClient: + GROUP_MAPPER_NAME = "groups-mapper" + def __init__( self, logger, @@ -142,6 +145,90 @@ def create_client(self, client_id, **kwargs): self.logger.debug(f"client_id: {id}") return id + def create_group_mapper(self, client_id: str, use_full_path: bool = True): + id = self.kc_admin.get_client_id(client_id) + assert id is not None, f"Client {client_id} not found" + self.logger.debug(f"Creating group mapper for client {client_id} (id: {id})") + + try: + mappers = self.kc_admin.get_mappers_from_client(id) + mapper = next( + (m for m in mappers if m["name"] == self.GROUP_MAPPER_NAME), None + ) + except KeycloakGetError: + mapper = None + + mapper_representation = { + "name": self.GROUP_MAPPER_NAME, + "protocol": "openid-connect", + "protocolMapper": "oidc-group-membership-mapper", + "config": { + "access.token.claim": "true", + "id.token.claim": "true", + "userinfo.token.claim": "true", + "full.path": f"{use_full_path}".lower(), + "claim.name": "groups", + "jsonType.label": "String", + }, + } + + if mapper is None: + self.logger.debug("Creating group mapper") + self.kc_admin.add_mapper_to_client(id, mapper_representation) + + def create_group( + self, + group_name: str, + parent: str | None = None, + skip_exists: bool = True, + **kwargs, + ) -> str: + self.logger.debug(f"Creating group named {group_name}") + payload = {"name": group_name} + payload.update(kwargs) + + group_id = self.kc_admin.create_group( + payload=payload, parent=parent, skip_exists=skip_exists + ) + + if group_id is None: + if not skip_exists: + raise RuntimeError(f"Failed to create group named {group_name}") + group_id = self.kc_admin.get_groups({"name": group_name})[0]["id"] + + return group_id + + def _get_group_id(self, group_name: str) -> str | None: + groups = self.kc_admin.get_groups() + group = next(g for g in groups if g["name"] == group_name) + if group is None: + return None + return group["id"] + + def _add_user_to_group(self, user_id: str, group_id: str): + self.kc_admin.group_user_add(user_id=user_id, group_id=group_id) + + def _remove_user_from_group(self, user_id: str, group_id: str): + self.kc_admin.group_user_remove(user_id=user_id, group_id=group_id) + + def add_service_user_to_group(self, client_id: str, group_name: str): + id = self.kc_admin.get_client_id(client_id) + assert id is not None, f"Client {client_id} not found" + service_account_user = self.kc_admin.get_client_service_account_user(id) + account_id = service_account_user["id"] + group_id = self._get_group_id(group_name=group_name) + assert group_id is not None, f"Group {group_name} not found" + self._add_user_to_group(user_id=account_id, group_id=group_id) + + def remove_service_user_from_group(self, client_id: str, group_name: str): + id = self.kc_admin.get_client_id(client_id) + assert id is not None, f"Client {client_id} not found" + service_account_user = self.kc_admin.get_client_service_account_user(id) + account_id = service_account_user["id"] + group_id = self._get_group_id(group_name=group_name) + assert group_id is not None, f"Group {group_name} not found" + self._remove_user_from_group(user_id=account_id, group_id=group_id) + def generate_client_secret(self, client_id): id = self.kc_admin.get_client_id(client_id) assert id is not None, f"Client {client_id} not found" @@ -221,7 +308,7 @@ def __init__( self.https_port = https_port @property - def admin(self): + def admin(self) -> KeycloakAdminClient: assert self._admin is not None return self._admin diff --git a/tests/rptest/tests/redpanda_oauth_test.py b/tests/rptest/tests/redpanda_oauth_test.py index 9e9206d09ec05..f2af064e741bf 100644 --- a/tests/rptest/tests/redpanda_oauth_test.py +++ b/tests/rptest/tests/redpanda_oauth_test.py @@ -8,6 +8,7 @@ # by the Apache License, Version 2.0 import datetime +from enum import Enum import json import socket import threading @@ -17,7 +18,7 @@ import requests from connectrpc.errors import ConnectError, ConnectErrorCode from ducktape.cluster.cluster import ClusterNode -from ducktape.mark import parametrize +from ducktape.mark import matrix, parametrize from ducktape.tests.test import Test from ducktape.utils.util import wait_until from keycloak import KeycloakOpenID @@ -65,10 +66,33 @@ "kafka/client": "trace", "kafka": "debug", "http": "trace", + "request_auth": "trace", }, ) +class NestedGroupType(str, Enum): + """ + How nested identity provider group names are mapped to Redpanda principals. + The enum values are used directly as string configuration values. + NONE + No special handling for nested groups. The full group name as provided + by the identity provider (including any nesting or path components) is + used when deriving principals. + SUFFIX + Use only the leaf (suffix) component of a nested group name when + deriving principals. For example, a group like ``/team/platform/admins`` + is treated as ``admins``. + """ + + NONE = "none" + SUFFIX = "suffix" + + +def get_nested_group_types() -> list[NestedGroupType]: + return [NestedGroupType.NONE, NestedGroupType.SUFFIX] + + class RedpandaOIDCTestBase(Test): """ Base class for tests that use the Redpanda service with OIDC @@ -179,7 +203,7 @@ def create_service_user(self, client_id=CLIENT_ID): self.keycloak.admin.update_user(service_user, email="myapp@customer.com") return self.keycloak.admin_ll.get_user_id(service_user) - def get_client_credentials_token(self, cfg): + def get_client_credentials_token(self, cfg) -> dict: token_endpoint_url = urlparse(cfg.token_endpoint) openid = KeycloakOpenID( server_url=f"{token_endpoint_url.scheme}://{token_endpoint_url.netloc}", @@ -799,6 +823,269 @@ def has_group(): consumer.close() + @cluster(num_nodes=4) + @matrix( + full_group=[True, False], + nested_group_mode=get_nested_group_types(), + ) + def test_group_claim(self, full_group: bool, nested_group_mode: NestedGroupType): + """ + Test that group claim mapping works as expected for topic authorization. + + This test verifies that OIDC group claims from the identity provider (Keycloak) + are correctly mapped to Redpanda ACL principals, allowing group-based authorization. + + Parameters: + full_group: When True, Keycloak includes the full group path (e.g., "/test-group") + in the token. When False, only the group name is included. + nested_group_mode: Controls how Redpanda handles nested group paths: + - NONE: Group paths are used as-is + - SUFFIX: Only the leaf group name is used + + Test flow: + 1. Configure Redpanda's nested_group_behavior setting + 2. Create a service user in Keycloak with a group mapper + 3. Create a group and add the service user to it + 4. Create a topic and grant access via a Group:* ACL principal + 5. Authenticate using OIDC and verify the user can access the topic + 6. Verify the resolved OIDC identity includes the expected group + """ + kc_node = self.keycloak.nodes[0] + + # Determine the qualified group name based on test parameters. + # When full_group=True and nested_group_mode=NONE, Keycloak returns "/test-group" + # Otherwise, just "test-group" is used. + group_name = "test-group" + qualified_group_name = f"{'/' if full_group and nested_group_mode == NestedGroupType.NONE else ''}{group_name}" + group_acl = f"Group:{qualified_group_name}" + + self.logger.debug(f'Qualified group name: "{qualified_group_name}"') + self.logger.debug(f'Group ACL: "{group_acl}"') + + # Configure how Redpanda handles nested group paths from OIDC tokens + self.redpanda.set_cluster_config( + {"nested_group_behavior": nested_group_mode.value} + ) + + # Set up the OIDC client and service user in Keycloak + client_id = CLIENT_ID + self.create_service_user() + + # Create a group mapper that includes group membership in the access token. + # use_full_path determines whether the full path ("/group") or just name ("group") is included. + self.keycloak.admin.create_group_mapper(client_id, full_group) + + # Create the group and add the service account to it + self.keycloak.admin.create_group(group_name) + self.keycloak.admin.add_service_user_to_group(client_id, group_name) + + # Create a topic and grant access to it via the group ACL principal. + # This allows any user with the matching group claim to access the topic. + self.rpk.create_topic(EXAMPLE_TOPIC) + self.rpk.sasl_allow_principal( + group_acl, + ["all"], + "topic", + EXAMPLE_TOPIC, + self.su_username, + self.su_password, + self.su_algorithm, + ) + + # Create a Kafka client that authenticates using OIDC + cfg = self.keycloak.generate_oauth_config(kc_node, client_id) + token = self.get_client_credentials_token(cfg) + assert cfg.client_secret is not None + assert cfg.token_endpoint is not None + k_client = PythonLibrdkafka( + self.redpanda, + algorithm="OAUTHBEARER", + oauth_config=cfg, + tls_cert=self.client_cert, + ) + producer = k_client.get_producer() + + # Explicit poll triggers OIDC token flow. Required for librdkafka + # metadata requests to behave nicely. + producer.poll(0.0) + + # Verify the user can see the topic (proving group-based authorization works) + expected_topics = set([EXAMPLE_TOPIC]) + self.logger.debug(f"expected_topics: {expected_topics}") + + wait_until( + lambda: set(producer.list_topics(timeout=5).topics.keys()) + == expected_topics, + timeout_sec=5, + err_msg="Failed to list topics using group claim for authorization", + ) + + # Verify the resolved OIDC identity includes the expected group claim + def resolve_oidc_identity( + token: dict, + ) -> security_pb2.ResolveOidcIdentityResponse: + admin_v2 = AdminV2(self.redpanda) + req = security_pb2.ResolveOidcIdentityRequest() + self.logger.debug(f'Using access token "{token["access_token"]}"') + return admin_v2.security().resolve_oidc_identity( + req, + extra_headers={"Authorization": f"Bearer {token['access_token']}"}, + ) + + resp = resolve_oidc_identity(token=token) + assert resp.groups == [qualified_group_name], ( + f"Unexpected groups: {resp.groups}, did not match {[qualified_group_name]}" + ) + + @cluster(num_nodes=4) + def test_group_membership_change(self): + """ + Test that changing group membership dynamically changes topic access permissions. + + This test: + 1. Creates two topics (topic1 and topic2) and three groups (group1, group2, group3) + 2. Sets up ACLs so group1 can access topic1 and group2 can access topic2 (group3 has no permissions) + 3. Adds service user to group1, verifies it can see topic1 but not topic2 + 4. Removes service user from group1, adds to group2 + 5. Verifies service user can now see topic2 but not topic1 + 6. Removes service user from group2, adds to group3 (no permissions) + 7. Verifies service user cannot see any topics + 8. Adds service user to all groups, verifies it can see both topics + """ + kc_node = self.keycloak.nodes[0] + + topic1 = "topic1" + topic2 = "topic2" + group1 = "group1" + group2 = "group2" + group3 = "group3" + + client_id = CLIENT_ID + self.create_service_user() + + # Create group mapper (use full path = False for simpler group names) + self.keycloak.admin.create_group_mapper(client_id, use_full_path=False) + + # Create all groups in Keycloak + self.keycloak.admin.create_group(group1) + self.keycloak.admin.create_group(group2) + self.keycloak.admin.create_group(group3) + + # Create both topics + self.rpk.create_topic(topic1) + self.rpk.create_topic(topic2) + + # Set up ACLs: group1 can describe topic1, group2 can describe topic2 + # group3 has no permissions + self.rpk.sasl_allow_principal( + f"Group:{group1}", + ["describe"], + "topic", + topic1, + self.su_username, + self.su_password, + self.su_algorithm, + ) + self.rpk.sasl_allow_principal( + f"Group:{group2}", + ["describe"], + "topic", + topic2, + self.su_username, + self.su_password, + self.su_algorithm, + ) + + cfg = self.keycloak.generate_oauth_config(kc_node, client_id) + assert cfg.client_secret is not None + assert cfg.token_endpoint is not None + + def get_visible_topics() -> set[str]: + """Get a fresh token and list visible topics.""" + k_client = PythonLibrdkafka( + self.redpanda, + algorithm="OAUTHBEARER", + oauth_config=cfg, + tls_cert=self.client_cert, + ) + producer = k_client.get_producer() + producer.poll(0.0) + return set(producer.list_topics(timeout=5).topics.keys()) + + # Phase 1: Add service user to group1 + self.logger.info("Phase 1: Adding service user to group1") + self.keycloak.admin.add_service_user_to_group(client_id, group1) + + # Verify service user can see topic1 but not topic2 + wait_until( + lambda: get_visible_topics() == {topic1}, + timeout_sec=10, + backoff_sec=1, + err_msg=f"Expected to see only {topic1} when in group1, got: {get_visible_topics()}", + ) + self.logger.info("Verified: service user in group1 can see topic1 only") + + # Phase 2: Remove from group1, add to group2 + self.logger.info("Phase 2: Removing service user from group1, adding to group2") + self.keycloak.admin.remove_service_user_from_group(client_id, group1) + self.keycloak.admin.add_service_user_to_group(client_id, group2) + + # Verify service user can now see topic2 but not topic1 + wait_until( + lambda: get_visible_topics() == {topic2}, + timeout_sec=10, + backoff_sec=1, + err_msg=f"Expected to see only {topic2} when in group2, got: {get_visible_topics()}", + ) + self.logger.info("Verified: service user in group2 can see topic2 only") + + # Phase 3: Remove from group2, add to group3 (no permissions) + self.logger.info( + "Phase 3: Removing service user from group2, adding to group3 (no permissions)" + ) + self.keycloak.admin.remove_service_user_from_group(client_id, group2) + self.keycloak.admin.add_service_user_to_group(client_id, group3) + + # Verify service user cannot see any topics + wait_until( + lambda: get_visible_topics() == set(), + timeout_sec=10, + backoff_sec=1, + err_msg=f"Expected to see no topics when in group3, got: {get_visible_topics()}", + ) + self.logger.info( + "Verified: service user in group3 (no permissions) cannot see any topics" + ) + + # Phase 4: Add service user to all groups + self.logger.info("Phase 4: Adding service user to all groups") + self.keycloak.admin.add_service_user_to_group(client_id, group1) + self.keycloak.admin.add_service_user_to_group(client_id, group2) + + # Verify service user can see both topics + wait_until( + lambda: get_visible_topics() == {topic1, topic2}, + timeout_sec=10, + backoff_sec=1, + err_msg=f"Expected to see both topics when in all groups, got: {get_visible_topics()}", + ) + self.logger.info("Verified: service user in all groups can see both topics") + + # Verify the resolved OIDC identity includes all three groups + token = self.get_client_credentials_token(cfg) + admin_v2 = AdminV2(self.redpanda) + req = security_pb2.ResolveOidcIdentityRequest() + resp = admin_v2.security().resolve_oidc_identity( + req, + extra_headers={"Authorization": f"Bearer {token['access_token']}"}, + ) + assert set(resp.groups) == {group1, group2, group3}, ( + f"Expected groups {[group1, group2, group3]}, got {resp.groups}" + ) + self.logger.info( + f"Verified: resolved OIDC identity includes all three groups: {resp.groups}" + ) + class RedpandaOIDCTest(RedpandaOIDCTestMethods): def __init__(self, test_context, **kwargs):