Skip to content

Commit

Permalink
Merge pull request redpanda-data#14378 from BenPope/oidc_authenticator
Browse files Browse the repository at this point in the history
security/oidc: Basic support for OIDC with Kafka and HTTP
  • Loading branch information
BenPope authored Oct 31, 2023
2 parents 07668a7 + 16063c9 commit 0dfb673
Show file tree
Hide file tree
Showing 41 changed files with 1,344 additions and 77 deletions.
7 changes: 6 additions & 1 deletion cmake/dependencies.cmake
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,10 @@ fetch_dep(GTest
REPO https://github.com/google/googletest
TAG v1.14.0)

fetch_dep(ada
REPO https://github.com/ada-url/ada
TAG v2.7.2)

if(${CMAKE_SYSTEM_PROCESSOR} MATCHES "x86_64")
set(TINYGO_TARBALL "tinygo-linux-amd64.tar.gz")
set(TINYGO_MD5 "5b4ba9d8dd8ce10f7939de441ad46a80")
Expand Down Expand Up @@ -115,7 +119,8 @@ FetchContent_MakeAvailable(
avro
tinygo
wasmtime
hdrhistogram)
hdrhistogram
ada)

add_library(Crc32c::crc32c ALIAS crc32c)
add_library(aklomp::base64 ALIAS base64)
Expand Down
23 changes: 23 additions & 0 deletions src/v/cluster/controller.cc
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
#include "security/authorizer.h"
#include "security/credential_store.h"
#include "security/ephemeral_credential_store.h"
#include "security/oidc_service.h"
#include "ssx/future-util.h"

#include <seastar/core/future.hh>
Expand Down Expand Up @@ -124,6 +125,24 @@ ss::future<> controller::wire_up() {
return _authorizer.start(
[]() { return config::shard_local_cfg().superusers.bind(); });
})
.then([this] {
return _oidc_service.start(
ss::sharded_parameter(
[] { return config::shard_local_cfg().sasl_mechanisms.bind(); }),
ss::sharded_parameter([] {
return config::shard_local_cfg().http_authentication.bind();
}),
ss::sharded_parameter([] {
return config::shard_local_cfg().oidc_discovery_url.bind();
}),
ss::sharded_parameter([] {
return config::shard_local_cfg().oidc_token_audience.bind();
}),
ss::sharded_parameter([] {
return config::shard_local_cfg()
.oidc_clock_skew_tolerance.bind();
}));
})
.then([this] { return _tp_state.start(); })
.then([this] {
return _partition_balancer_state.start_single(
Expand Down Expand Up @@ -500,6 +519,9 @@ controller::start(cluster_discovery& discovery, ss::abort_source& shard0_as) {
.then([this] {
return _hm_frontend.invoke_on_all(&health_monitor_frontend::start);
})
.then([this] {
return _oidc_service.invoke_on_all(&security::oidc::service::start);
})
.then([this, seed_nodes = std::move(seed_nodes)]() mutable {
return _feature_manager.invoke_on(
feature_manager::backend_shard,
Expand Down Expand Up @@ -636,6 +658,7 @@ ss::future<> controller::stop() {
.then([this] { return _config_frontend.stop(); })
.then([this] { return _feature_backend.stop(); })
.then([this] { return _bootstrap_backend.stop(); })
.then([this] { return _oidc_service.stop(); })
.then([this] { return _authorizer.stop(); })
.then([this] { return _ephemeral_credentials.stop(); })
.then([this] { return _credentials.stop(); })
Expand Down
5 changes: 5 additions & 0 deletions src/v/cluster/controller.h
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,10 @@ class controller {

ss::sharded<security::authorizer>& get_authorizer() { return _authorizer; }

ss::sharded<security::oidc::service>& get_oidc_service() {
return _oidc_service;
}

ss::sharded<controller_api>& get_api() { return _api; }

ss::sharded<members_frontend>& get_members_frontend() {
Expand Down Expand Up @@ -254,6 +258,7 @@ class controller {
ss::sharded<security_frontend> _security_frontend;
ss::sharded<ephemeral_credential_frontend> _ephemeral_credential_frontend;
ss::sharded<security::authorizer> _authorizer;
ss::sharded<security::oidc::service> _oidc_service;
ss::sharded<raft::group_manager>& _raft_manager;
ss::sharded<health_monitor_frontend> _hm_frontend; // instance per core
ss::sharded<health_monitor_backend> _hm_backend; // single instance
Expand Down
42 changes: 40 additions & 2 deletions src/v/config/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
#include "pandaproxy/schema_registry/schema_id_validation.h"
#include "security/gssapi_principal_mapper.h"
#include "security/mtls.h"
#include "security/oidc_url_parser.h"
#include "ssx/sformat.h"
#include "storage/chunk_cache.h"
#include "storage/segment_appender.h"
#include "units.h"
Expand Down Expand Up @@ -1110,7 +1112,8 @@ configuration::configuration()
, sasl_mechanisms(
*this,
"sasl_mechanisms",
"A list of supported SASL mechanisms. `SCRAM` and `GSSAPI` are allowed.",
"A list of supported SASL mechanisms. `SCRAM`, `GSSAPI`, and "
"`OAUTHBEARER` are allowed.",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
{"SCRAM"},
validate_sasl_mechanisms)
Expand Down Expand Up @@ -2696,7 +2699,42 @@ configuration::configuration()
"The sample period for the CPU profiler",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
100ms,
{.min = 1ms}) {}
{.min = 1ms})
, oidc_discovery_url(
*this,
"oidc_discovery_url",
"The URL pointing to the well-known discovery endpoint for the OIDC "
"provider.",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
"https://auth.prd.cloud.redpanda.com/.well-known/openid-configuration",
[](auto const& v) -> std::optional<ss::sstring> {
auto res = security::oidc::parse_url(v);
if (res.has_error()) {
return res.error().message();
}
return std::nullopt;
})
, oidc_token_audience(
*this,
"oidc_token_audience",
"A string representing the intended recipient of the token.",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
"redpanda")
, oidc_clock_skew_tolerance(
*this,
"oidc_clock_skew_tolerance",
"The amount of seconds to allow for when validating the exp, nbf, and "
"iat claims in the token.",
{.needs_restart = needs_restart::no, .visibility = visibility::user},
std::chrono::seconds{} * 30)
, http_authentication(
*this,
"http_authentication",
"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) {}

configuration::error_map_t configuration::load(const YAML::Node& root_node) {
if (!root_node["redpanda"]) {
Expand Down
8 changes: 8 additions & 0 deletions src/v/config/configuration.h
Original file line number Diff line number Diff line change
Expand Up @@ -514,6 +514,14 @@ struct configuration final : public config_store {
property<bool> cpu_profiler_enabled;
bounded_property<std::chrono::milliseconds> cpu_profiler_sample_period_ms;

// oidc authentication
property<ss::sstring> oidc_discovery_url;
property<ss::sstring> oidc_token_audience;
property<std::chrono::seconds> oidc_clock_skew_tolerance;

// HTTP Authentication
property<std::vector<ss::sstring>> http_authentication;

configuration();

error_map_t load(const YAML::Node& root_node);
Expand Down
25 changes: 22 additions & 3 deletions src/v/config/validators.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,12 @@
#include "net/inet_address_wrapper.h"
#include "ssx/sformat.h"

#include <absl/algorithm/container.h>
#include <absl/container/flat_hash_set.h>
#include <absl/container/node_hash_set.h>
#include <fmt/format.h>

#include <array>
#include <optional>
#include <unordered_map>

Expand Down Expand Up @@ -108,18 +110,35 @@ std::optional<ss::sstring> validate_client_groups_byte_rate_quota(

std::optional<ss::sstring>
validate_sasl_mechanisms(const std::vector<ss::sstring>& mechanisms) {
static const absl::flat_hash_set<std::string_view> supported{
"GSSAPI", "SCRAM"};
constexpr auto supported = std::to_array<std::string_view>(
{"GSSAPI", "SCRAM", "OAUTHBEARER"});

// Validate results
for (const auto& m : mechanisms) {
if (!supported.contains(m)) {
if (absl::c_none_of(
supported, [&m](auto const& s) { return s == m; })) {
return ssx::sformat("'{}' is not a supported SASL mechanism", m);
}
}
return std::nullopt;
}

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

// Validate results
for (const auto& m : mechanisms) {
if (absl::c_none_of(
supported, [&m](auto const& s) { return s == m; })) {
return ssx::sformat(
"'{}' is not a supported HTTP authentication mechanism", m);
}
}
return std::nullopt;
}

std::optional<ss::sstring> validate_0_to_1_ratio(const double d) {
if (d < 0 || d > 1) {
return fmt::format("Ratio must be in the [0,1] range, got: {}", d);
Expand Down
3 changes: 3 additions & 0 deletions src/v/config/validators.h
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ std::optional<ss::sstring> validate_client_groups_byte_rate_quota(
std::optional<ss::sstring>
validate_sasl_mechanisms(const std::vector<ss::sstring>& mechanisms);

std::optional<ss::sstring>
validate_http_authn_mechanisms(const std::vector<ss::sstring>& mechanisms);

std::optional<ss::sstring> validate_0_to_1_ratio(const double d);

std::optional<ss::sstring>
Expand Down
21 changes: 20 additions & 1 deletion src/v/kafka/client/sasl_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
#include "kafka/client/sasl_client.h"

#include "kafka/client/logger.h"
#include "security/oidc_authenticator.h"
#include "security/scram_authenticator.h"

namespace kafka::client {
Expand All @@ -33,7 +34,8 @@ do_authenticate(shared_broker_t broker, const configuration& config) {

if (
mechanism != security::scram_sha256_authenticator::name
&& mechanism != security::scram_sha512_authenticator::name) {
&& mechanism != security::scram_sha512_authenticator::name
&& mechanism != security::oidc::sasl_authenticator::name) {
throw broker_error{
broker->id(),
error_code::sasl_authentication_failed,
Expand Down Expand Up @@ -69,6 +71,8 @@ do_authenticate(shared_broker_t broker, const configuration& config) {
} else if (mechanism == security::scram_sha512_authenticator::name) {
co_await do_authenticate_scram512(
broker, std::move(username), std::move(password));
} else if (mechanism == security::oidc::sasl_authenticator::name) {
co_await do_authenticate_oauthbearer(broker, std::move(password));
}
}

Expand Down Expand Up @@ -207,4 +211,19 @@ ss::future<> do_authenticate_scram512(
std::move(broker), std::move(username), std::move(password));
}

ss::future<>
do_authenticate_oauthbearer(shared_broker_t broker, ss::sstring token) {
sasl_authenticate_request req;
req.data.auth_bytes = "n,,\1auth=";
req.data.auth_bytes += bytes{token.cbegin(), token.cend()};
req.data.auth_bytes += "\1\1";
auto res = co_await broker->dispatch(std::move(req));
if (res.data.errored()) {
throw broker_error{
broker->id(),
res.data.error_code,
res.data.error_message.value_or("<no error message>")};
}
}

} // namespace kafka::client
3 changes: 3 additions & 0 deletions src/v/kafka/client/sasl_client.h
Original file line number Diff line number Diff line change
Expand Up @@ -32,4 +32,7 @@ ss::future<> do_authenticate_scram256(
ss::future<> do_authenticate_scram512(
shared_broker_t broker, ss::sstring username, ss::sstring password);

ss::future<>
do_authenticate_oauthbearer(shared_broker_t broker, ss::sstring token);

} // namespace kafka::client
15 changes: 15 additions & 0 deletions src/v/kafka/server/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
#include "security/exceptions.h"
#include "security/gssapi_authenticator.h"
#include "security/mtls.h"
#include "security/oidc_authenticator.h"
#include "security/scram_algorithm.h"
#include "security/scram_authenticator.h"
#include "ssx/future-util.h"
Expand Down Expand Up @@ -97,6 +98,7 @@ server::server(
ss::sharded<security::credential_store>& credentials,
ss::sharded<security::authorizer>& authorizer,
ss::sharded<security::audit::audit_log_manager>& audit_mgr,
ss::sharded<security::oidc::service>& oidc_service,
ss::sharded<cluster::security_frontend>& sec_fe,
ss::sharded<cluster::controller_api>& controller_api,
ss::sharded<cluster::tx_gateway_frontend>& tx_gateway_frontend,
Expand Down Expand Up @@ -128,6 +130,7 @@ server::server(
, _credentials(credentials)
, _authorizer(authorizer)
, _audit_mgr(audit_mgr)
, _oidc_service(oidc_service)
, _security_frontend(sec_fe)
, _controller_api(controller_api)
, _tx_gateway_frontend(tx_gateway_frontend)
Expand Down Expand Up @@ -565,6 +568,18 @@ ss::future<response_ptr> sasl_handshake_handler::handle(
}
}

if (supports(security::oidc::sasl_authenticator::name)) {
supported_sasl_mechanisms.emplace_back(
security::oidc::sasl_authenticator::name);

if (
request.data.mechanism == security::oidc::sasl_authenticator::name) {
ctx.sasl()->set_mechanism(
std::make_unique<security::oidc::sasl_authenticator>(
ctx.connection()->server().oidc_service().local()));
}
}

if (!ctx.sasl()->has_mechanism()) {
error = error_code::unsupported_sasl_mechanism;
}
Expand Down
6 changes: 6 additions & 0 deletions src/v/kafka/server/server.h
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ class server final
ss::sharded<security::credential_store>&,
ss::sharded<security::authorizer>&,
ss::sharded<security::audit::audit_log_manager>&,
ss::sharded<security::oidc::service>&,
ss::sharded<cluster::security_frontend>&,
ss::sharded<cluster::controller_api>&,
ss::sharded<cluster::tx_gateway_frontend>&,
Expand Down Expand Up @@ -135,6 +136,10 @@ class server final
return _audit_mgr.local();
}

ss::sharded<security::oidc::service>& oidc_service() {
return _oidc_service;
}

cluster::security_frontend& security_frontend() {
return _security_frontend.local();
}
Expand Down Expand Up @@ -215,6 +220,7 @@ class server final
ss::sharded<security::credential_store>& _credentials;
ss::sharded<security::authorizer>& _authorizer;
ss::sharded<security::audit::audit_log_manager>& _audit_mgr;
ss::sharded<security::oidc::service>& _oidc_service;
ss::sharded<cluster::security_frontend>& _security_frontend;
ss::sharded<cluster::controller_api>& _controller_api;
ss::sharded<cluster::tx_gateway_frontend>& _tx_gateway_frontend;
Expand Down
2 changes: 1 addition & 1 deletion src/v/pandaproxy/auth_utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
#pragma once
#include "config/rest_authn_endpoint.h"
#include "pandaproxy/types.h"
#include "utils/request_auth.h"
#include "security/request_auth.h"

#include <seastar/http/request.hh>

Expand Down
2 changes: 1 addition & 1 deletion src/v/pandaproxy/rest/proxy.h
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
#include "pandaproxy/server.h"
#include "pandaproxy/util.h"
#include "seastarx.h"
#include "utils/request_auth.h"
#include "security/request_auth.h"

#include <seastar/core/future.hh>
#include <seastar/core/sharded.hh>
Expand Down
2 changes: 1 addition & 1 deletion src/v/pandaproxy/schema_registry/service.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
#include "pandaproxy/server.h"
#include "pandaproxy/util.h"
#include "seastarx.h"
#include "utils/request_auth.h"
#include "security/request_auth.h"

#include <seastar/core/future.hh>
#include <seastar/core/sharded.hh>
Expand Down
2 changes: 1 addition & 1 deletion src/v/pandaproxy/server.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
#include "pandaproxy/kafka_client_cache.h"
#include "pandaproxy/types.h"
#include "seastarx.h"
#include "utils/request_auth.h"
#include "security/request_auth.h"

#include <seastar/core/abort_source.hh>
#include <seastar/core/future.hh>
Expand Down
Loading

0 comments on commit 0dfb673

Please sign in to comment.