Skip to content

Commit

Permalink
security/oidc: Support oidc for request_auth
Browse files Browse the repository at this point in the history
Signed-off-by: Ben Pope <ben@redpanda.com>
(cherry picked from commit 16063c9)
  • Loading branch information
BenPope authored and NyaliaLui committed Nov 3, 2023
1 parent 6b77d9b commit b6b6830
Show file tree
Hide file tree
Showing 6 changed files with 94 additions and 7 deletions.
3 changes: 2 additions & 1 deletion src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2748,7 +2748,8 @@ configuration::configuration()
, http_authentication(
*this,
"http_authentication",
"A list of supported HTTP authentication mechanisms. `BASIC` is allowed.",
"A list of supported HTTP authentication mechanisms. `BASIC` and `OIDC` "
"are allowed.",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
{"BASIC"},
validate_http_authn_mechanisms) {}
Expand Down
3 changes: 2 additions & 1 deletion src/v/config/validators.cc
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,8 @@ validate_sasl_mechanisms(const std::vector<ss::sstring>& mechanisms) {

std::optional<ss::sstring>
validate_http_authn_mechanisms(const std::vector<ss::sstring>& mechanisms) {
constexpr auto supported = std::to_array<std::string_view>({"BASIC"});
constexpr auto supported = std::to_array<std::string_view>(
{"BASIC", "OIDC"});

// Validate results
for (const auto& m : mechanisms) {
Expand Down
28 changes: 26 additions & 2 deletions src/v/security/request_auth.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,10 @@
#include "config/configuration.h"
#include "seastar/http/exception.hh"
#include "security/credential_store.h"
#include "security/oidc_authenticator.h"
#include "security/scram_algorithm.h"
#include "security/scram_authenticator.h"
#include "security/types.h"
#include "vlog.h"

#include <seastar/core/sstring.hh>
Expand Down Expand Up @@ -75,8 +77,6 @@ request_auth_result request_authenticator::do_authenticate(
ss::http::request const& req,
security::credential_store const& cred_store,
bool require_auth) {
security::credential_user username;

constexpr auto supports = [](std::string_view m) {
return absl::c_any_of(
config::shard_local_cfg().http_authentication(),
Expand All @@ -85,6 +85,7 @@ request_auth_result request_authenticator::do_authenticate(

auto auth_hdr = req.get_header("authorization");
if (supports("BASIC") && auth_hdr.substr(0, 5) == "Basic") {
security::credential_user username;
// Minimal length: Basic, a space, 1 or more bytes
if (auth_hdr.size() < 7) {
throw ss::httpd::bad_request_exception(
Expand Down Expand Up @@ -160,6 +161,29 @@ request_auth_result request_authenticator::do_authenticate(
request_auth_result::superuser(superuser));
}
}
} else if (supports("OIDC") && auth_hdr.substr(0, 6) == "Bearer") {
// Minimal length: Bearer, a space, 1 or more bytes
if (auth_hdr.size() < 8) {
throw ss::httpd::bad_request_exception(
"Malformed Authorization header");
}
auto auth = security::oidc::authenticator{
_controller->get_oidc_service().local()};
auto res = auth.authenticate(auth_hdr.substr(7));
if (res.has_error()) {
throw ss::httpd::base_exception(
"Unauthorized", ss::http::reply::status_type::unauthorized);
}
auto principal = res.assume_value().name();
const auto& superusers = _superusers();
auto found = std::find(superusers.begin(), superusers.end(), principal);
bool superuser = (found != superusers.end()) || (!require_auth);
vlog(logger.trace, "Authenticated principal {}", principal);
return request_auth_result{
security::credential_user{principal},
security::credential_password{auth_hdr},
security::oidc::sasl_authenticator::name,
request_auth_result::superuser{superuser}};
} else if (!auth_hdr.empty()) {
throw ss::httpd::bad_request_exception(
"Unsupported Authorization method");
Expand Down
8 changes: 8 additions & 0 deletions tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -664,6 +664,7 @@ def __init__(self):
self.enable_sasl = False
self.kafka_enable_authorization: Optional[bool] = None
self.sasl_mechanisms: Optional[list[str]] = None
self.http_authentication: Optional[list[str]] = None
self.endpoint_authn_method: Optional[str] = None
self.tls_provider: Optional[TLSProvider] = None
self.require_client_auth: bool = True
Expand Down Expand Up @@ -3111,6 +3112,13 @@ def write_bootstrap_cluster_config(self):
)
conf.update(dict(sasl_mechanisms=self._security.sasl_mechanisms))

if self._security.http_authentication is not None:
self.logger.debug(
f"Setting http_authentication: {self._security.http_authentication} in cluster configuration"
)
conf.update(
dict(http_authentication=self._security.http_authentication))

conf_yaml = yaml.dump(conf)
for node in self.nodes:
self.logger.info(
Expand Down
2 changes: 1 addition & 1 deletion tests/rptest/tests/cluster_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -607,7 +607,7 @@ def test_valid_settings(self):
# The default value is ['BASIC'], but the array cannot contain
# arbitrary strings because the config system validates them.
# Removing BASIC breaks tests that use the admin API.
valid_value = ['BASIC']
valid_value = ['BASIC', 'OIDC']

if name == 'sasl_kerberos_principal_mapping':
# The default value is ['DEFAULT'], but the array must contain
Expand Down
57 changes: 55 additions & 2 deletions tests/rptest/tests/redpanda_oauth_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,14 @@

from rptest.clients.rpk import RpkTool
from rptest.clients.python_librdkafka import PythonLibrdkafka
from rptest.services.redpanda import LoggingConfig, SecurityConfig, make_redpanda_service
from rptest.services.keycloak import KeycloakService
from rptest.services.redpanda import LoggingConfig, PandaproxyConfig, SchemaRegistryConfig, SecurityConfig, make_redpanda_service
from rptest.services.keycloak import DEFAULT_REALM, KeycloakService
from rptest.services.cluster import cluster

import requests
from keycloak import KeycloakOpenID
from urllib.parse import urlparse

CLIENT_ID = 'myapp'
TOKEN_AUDIENCE = 'account'
EXAMPLE_TOPIC = 'foo'
Expand All @@ -37,6 +41,7 @@ def __init__(self,
test_context,
num_nodes=4,
sasl_mechanisms=['SCRAM', 'OAUTHBEARER'],
http_authentication=["BASIC", "OIDC"],
**kwargs):
super(RedpandaOIDCTestBase, self).__init__(test_context, **kwargs)
self.produce_messages = []
Expand All @@ -54,6 +59,13 @@ def __init__(self,
security = SecurityConfig()
security.enable_sasl = True
security.sasl_mechanisms = sasl_mechanisms
security.http_authentication = http_authentication

pandaproxy_config = PandaproxyConfig()
pandaproxy_config.authn_method = 'http_basic'

schema_reg_config = SchemaRegistryConfig()
schema_reg_config.authn_method = 'http_basic'

self.redpanda = make_redpanda_service(
test_context,
Expand All @@ -63,6 +75,8 @@ def __init__(self,
"oidc_token_audience": TOKEN_AUDIENCE,
},
security=security,
pandaproxy_config=pandaproxy_config,
schema_registry_config=schema_reg_config,
log_config=log_config)

self.su_username, self.su_password, self.su_algorithm = self.redpanda.SUPERUSER_CREDENTIALS
Expand Down Expand Up @@ -123,3 +137,42 @@ def test_init(self):
wait_until(lambda: set(producer.list_topics(timeout=5).topics.keys())
== expected_topics,
timeout_sec=5)

token_endpoint_url = urlparse(cfg.token_endpoint)
openid = KeycloakOpenID(
server_url=
f'{token_endpoint_url.scheme}://{token_endpoint_url.netloc}',
client_id=cfg.client_id,
client_secret_key=cfg.client_secret,
realm_name=DEFAULT_REALM,
verify=True)
token = openid.token(grant_type="client_credentials")

def check_pp_topics():
response = requests.get(
url=
f'http://{self.redpanda.nodes[0].account.hostname}:8082/topics',
headers={
'Accept': 'application/vnd.kafka.v2+json',
'Content-Type': 'application/vnd.kafka.v2+json',
'Authorization': f'Bearer {token["access_token"]}'
},
timeout=5)
return response.status_code == requests.codes.ok and set(
response.json()) == expected_topics

def check_sr_subjects():
response = requests.get(
url=
f'http://{self.redpanda.nodes[0].account.hostname}:8081/subjects',
headers={
'Accept': 'application/vnd.schemaregistry.v1+json',
'Authorization': f'Bearer {token["access_token"]}'
},
timeout=5)
return response.status_code == requests.codes.ok and response.json(
) == []

wait_until(check_pp_topics, timeout_sec=5)

wait_until(check_sr_subjects, timeout_sec=5)

0 comments on commit b6b6830

Please sign in to comment.