Skip to content

Commit

Permalink
Merge pull request redpanda-data#13477 from oleiman/tls_cert_expiry
Browse files Browse the repository at this point in the history
TLS Certificate Metrics
  • Loading branch information
oleiman authored Nov 8, 2023
2 parents ee55229 + e360ed3 commit 7d9722c
Show file tree
Hide file tree
Showing 15 changed files with 1,078 additions and 80 deletions.
5 changes: 3 additions & 2 deletions src/v/cloud_roles/aws_refresh_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -202,15 +202,16 @@ ss::future<api_response> aws_refresh_impl::fetch_instance_metadata_token() {
token_request.target("/latest/api/token");

co_return co_await make_request(
co_await make_api_client(), std::move(token_request));
co_await make_api_client("aws"), std::move(token_request));
}

ss::future<api_response> aws_refresh_impl::make_request_with_token(
http::client::request_header req, std::optional<std::string_view> token) {
if (token.has_value()) {
add_metadata_token_to_request(req, token.value());
}
co_return co_await make_request(co_await make_api_client(), std::move(req));
co_return co_await make_request(
co_await make_api_client("aws"), std::move(req));
}

std::ostream& aws_refresh_impl::print(std::ostream& os) const {
Expand Down
2 changes: 1 addition & 1 deletion src/v/cloud_roles/aws_sts_refresh_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ ss::future<api_response> aws_sts_refresh_impl::fetch_credentials() {
}

co_return co_await request_with_payload(
co_await make_api_client(tls_enabled),
co_await make_api_client("aws_sts", tls_enabled),
std::move(assume_req),
std::move(body));
}
Expand Down
2 changes: 1 addition & 1 deletion src/v/cloud_roles/gcp_refresh_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ ss::future<api_response> gcp_refresh_impl::fetch_credentials() {
metadata_flavor::header_name.data(), metadata_flavor::value.data());

co_return co_await make_request(
co_await make_api_client(), std::move(oauth_req));
co_await make_api_client("gcp"), std::move(oauth_req));
}

api_response_parse_result gcp_refresh_impl::parse_response(iobuf response) {
Expand Down
13 changes: 8 additions & 5 deletions src/v/cloud_roles/refresh_credentials.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
#include "config/configuration.h"
#include "model/metadata.h"
#include "net/tls.h"
#include "net/tls_certificate_probe.h"
#include "vlog.h"

#include <seastar/core/abort_source.hh>
Expand Down Expand Up @@ -316,11 +317,11 @@ ss::future<> refresh_credentials::impl::sleep_until_expiry() const {
}
}

ss::future<http::client>
refresh_credentials::impl::make_api_client(client_tls_enabled enable_tls) {
ss::future<http::client> refresh_credentials::impl::make_api_client(
ss::sstring name, client_tls_enabled enable_tls) {
if (enable_tls == client_tls_enabled::yes) {
if (_tls_certs == nullptr) {
co_await init_tls_certs();
co_await init_tls_certs(std::move(name));
}

co_return http::client{
Expand All @@ -342,7 +343,7 @@ refresh_credentials::impl::make_api_client(client_tls_enabled enable_tls) {
_as};
}

ss::future<> refresh_credentials::impl::init_tls_certs() {
ss::future<> refresh_credentials::impl::init_tls_certs(ss::sstring name) {
ss::tls::credentials_builder b;
b.set_client_auth(ss::tls::client_auth::NONE);

Expand All @@ -365,7 +366,9 @@ ss::future<> refresh_credentials::impl::init_tls_certs() {
co_await b.set_system_trust();
}

_tls_certs = co_await b.build_reloadable_certificate_credentials();
_tls_certs = co_await net::build_reloadable_credentials_with_probe<
ss::tls::certificate_credentials>(
std::move(b), "cloud_provider_client", std::move(name));
}

refresh_credentials make_refresh_credentials(
Expand Down
7 changes: 4 additions & 3 deletions src/v/cloud_roles/refresh_credentials.h
Original file line number Diff line number Diff line change
Expand Up @@ -71,8 +71,9 @@ class refresh_credentials {

protected:
/// Returns an http client with the API host and port applied
ss::future<http::client>
make_api_client(client_tls_enabled enable_tls = client_tls_enabled::no);
ss::future<http::client> make_api_client(
ss::sstring name = "",
client_tls_enabled enable_tls = client_tls_enabled::no);

/// Helper to parse the iobuf returned from API into a credentials
/// object, customized to API response structure
Expand Down Expand Up @@ -104,7 +105,7 @@ class refresh_credentials {
private:
/// Initializes certificate_credentials on first client creation.
/// Subsequent clients which are created will reuse the certs.
ss::future<> init_tls_certs();
ss::future<> init_tls_certs(ss::sstring name);

/// The address to query for credentials. Can be overridden using env
/// variable `RP_SI_CREDS_API_ADDRESS`
Expand Down
12 changes: 9 additions & 3 deletions src/v/cloud_storage_clients/configuration.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,15 @@
#include "cloud_storage_clients/logger.h"
#include "config/configuration.h"
#include "net/tls.h"
#include "net/tls_certificate_probe.h"

#include <seastar/net/tls.hh>

namespace {

ss::future<ss::shared_ptr<ss::tls::certificate_credentials>>
build_tls_credentials(
ss::sstring name,
std::optional<cloud_storage_clients::ca_trust_file> trust_file,
ss::logger& log) {
ss::tls::credentials_builder cred_builder;
Expand Down Expand Up @@ -49,7 +53,9 @@ build_tls_credentials(
co_await cred_builder.set_system_trust();
}
}
co_return co_await cred_builder.build_reloadable_certificate_credentials();
co_return co_await net::build_reloadable_credentials_with_probe<
ss::tls::certificate_credentials>(
std::move(cred_builder), "cloud_storage_client", std::move(name));
};

} // namespace
Expand Down Expand Up @@ -87,7 +93,7 @@ ss::future<s3_configuration> s3_configuration::make_configuration(
client_cfg.uri = access_point_uri(endpoint_uri);
if (overrides.disable_tls == false) {
client_cfg.credentials = co_await build_tls_credentials(
overrides.trust_file, s3_log);
"s3", overrides.trust_file, s3_log);
}

client_cfg.server_addr = net::unresolved_address(
Expand Down Expand Up @@ -147,7 +153,7 @@ ss::future<abs_configuration> abs_configuration::make_configuration(
client_cfg.uri = access_point_uri{endpoint_uri};
if (overrides.disable_tls == false) {
client_cfg.credentials = co_await build_tls_credentials(
overrides.trust_file, abs_log);
"abs", overrides.trust_file, abs_log);
}

client_cfg.server_addr = net::unresolved_address(
Expand Down
6 changes: 5 additions & 1 deletion src/v/cluster/metrics_reporter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
#include "model/record_batch_types.h"
#include "model/timeout_clock.h"
#include "net/tls.h"
#include "net/tls_certificate_probe.h"
#include "net/unresolved_address.h"
#include "reflection/adl.h"
#include "rpc/types.h"
Expand All @@ -39,6 +40,7 @@
#include <seastar/core/coroutine.hh>
#include <seastar/core/lowres_clock.hh>
#include <seastar/core/shared_ptr.hh>
#include <seastar/net/tls.hh>

#include <absl/algorithm/container.h>
#include <absl/container/node_hash_map.h>
Expand Down Expand Up @@ -400,7 +402,9 @@ ss::future<http::client> metrics_reporter::make_http_client() {
}

client_configuration.credentials
= co_await builder.build_reloadable_certificate_credentials();
= co_await net::build_reloadable_credentials_with_probe<
ss::tls::certificate_credentials>(
std::move(builder), "metrics_reporter", "httpclient");
client_configuration.tls_sni_hostname = _address.host;
}
co_return http::client(client_configuration, _as.local());
Expand Down
203 changes: 203 additions & 0 deletions src/v/net/probes.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,13 +11,30 @@
#include "metrics/metrics.h"
#include "net/client_probe.h"
#include "net/server_probe.h"
#include "net/tls_certificate_probe.h"
#include "net/types.h"
#include "prometheus/prometheus_sanitize.h"
#include "ssx/sformat.h"

#include <seastar/core/lowres_clock.hh>
#include <seastar/core/metrics.hh>
#include <seastar/core/smp.hh>
#include <seastar/net/inet_address.hh>
#include <seastar/net/tls.hh>
#include <seastar/util/defer.hh>

#include <boost/lexical_cast.hpp>
#include <fmt/chrono.h>
#include <fmt/ranges.h>
#include <gnutls/gnutls.h>
#include <gnutls/gnutlsxx.h>
#include <gnutls/x509-ext.h>
#include <gnutls/x509.h>

#include <chrono>
#include <ostream>
#include <span>
#include <string>

namespace net {
void server_probe::setup_metrics(
Expand Down Expand Up @@ -282,4 +299,190 @@ std::ostream& operator<<(std::ostream& o, const client_probe& p) {
<< ", requests_blocked_memory: " << p._requests_blocked_memory << " }";
return o;
}

ss::future<ss::shared_ptr<ss::tls::server_credentials>>
build_reloadable_server_credentials_with_probe(
config::tls_config config,
ss::sstring service,
ss::sstring listener_name,
ss::tls::reload_callback cb) {
auto builder = co_await config.get_credentials_builder();
if (!builder) {
co_return nullptr;
}
co_return co_await build_reloadable_credentials_with_probe<
ss::tls::server_credentials>(
std::move(*builder),
std::move(service),
std::move(listener_name),
std::move(cb));
}

template<TLSCreds T>
ss::future<ss::shared_ptr<T>> build_reloadable_credentials_with_probe(
ss::tls::credentials_builder builder,
ss::sstring area,
ss::sstring detail,
ss::tls::reload_callback cb) {
auto probe = ss::make_lw_shared<net::tls_certificate_probe>();
auto wrap_cb = [probe, cb = std::move(cb)](
const std::unordered_set<ss::sstring>& updated,
const ss::tls::certificate_credentials& creds,
const std::exception_ptr& eptr) {
if (cb) {
cb(updated, eptr);
}
probe->loaded(creds, eptr);
};

ss::shared_ptr<T> cred;
if constexpr (std::is_same<T, ss::tls::server_credentials>::value) {
cred = co_await builder.build_reloadable_server_credentials(wrap_cb);
} else {
cred = co_await builder.build_reloadable_certificate_credentials(
wrap_cb);
}

probe->setup_metrics(std::move(area), std::move(detail));
probe->loaded(*cred, nullptr);
co_return cred;
}

template ss::future<ss::shared_ptr<ss::tls::server_credentials>>
build_reloadable_credentials_with_probe(
ss::tls::credentials_builder builder,
ss::sstring area,
ss::sstring detail,
ss::tls::reload_callback cb);

template ss::future<ss::shared_ptr<ss::tls::certificate_credentials>>
build_reloadable_credentials_with_probe(
ss::tls::credentials_builder builder,
ss::sstring area,
ss::sstring detail,
ss::tls::reload_callback cb);

void tls_certificate_probe::loaded(
const ss::tls::certificate_credentials& creds, std::exception_ptr ex) {
_load_time = clock_type::now();

if (ex) {
reset();
return;
}

auto to_tls_serial = [](bytes_view b) {
using T = tls_serial_number::type;
T result = 0;
const auto end = std::min(b.size(), sizeof(T));
std::memcpy(&result, b.data(), end);
return tls_serial_number{result};
};

_cert_loaded = true;

auto certs_info = creds.get_cert_info();
auto ts_info = creds.get_trust_list_info();

if (!certs_info.has_value() || !ts_info.has_value()) {
reset();
return;
}

for (auto& info : certs_info.value()) {
auto exp = clock_type::from_time_t(info.expiry);
auto srl = to_tls_serial(info.serial);
if (exp < _cert_expiry_time) {
_cert_expiry_time = exp;
_cert_serial = srl;
}
}

for (auto& info : ts_info.value()) {
auto exp = clock_type::from_time_t(info.expiry);
auto srl = to_tls_serial(info.serial);
if (exp < _ca_expiry_time) {
_ca_expiry_time = exp;
_ca_serial = srl;
}
}
}

void tls_certificate_probe::setup_metrics(
std::string_view area, std::string_view detail) {
if (ss::this_shard_id() != 0) {
return;
}

namespace sm = ss::metrics;
const auto area_label = sm::label("area");
const auto detail_label = sm::label("detail");

const std::vector<sm::label_instance> labels = {
area_label(area), detail_label(detail)};

auto setup = [this,
&labels](const std::vector<sm::label>& aggregate_labels) {
using namespace std::literals::chrono_literals;
std::vector<sm::metric_definition> defs;
defs.emplace_back(
sm::make_gauge(
"truststore_expires_at_timestamp_seconds",
[this] { return _ca_expiry_time.time_since_epoch() / 1s; },
sm::description(
"Expiry time of the shortest-lived CA in the truststore"
"(seconds since epoch)"),
labels)
.aggregate(aggregate_labels));
defs.emplace_back(
sm::make_gauge(
"certificate_expires_at_timestamp_seconds",
[this] { return _cert_expiry_time.time_since_epoch() / 1s; },
sm::description(
"Expiry time of the server certificate (seconds since epoch)"),
labels)
.aggregate(aggregate_labels));
defs.emplace_back(
sm::make_gauge(
"certificate_serial",
[this] { return _cert_serial; },
sm::description("Least significant four bytes of the server "
"certificate serial number"),
labels)
.aggregate(aggregate_labels));
defs.emplace_back(
sm::make_gauge(
"loaded_at_timestamp_seconds",
[this] { return _load_time.time_since_epoch() / 1s; },
sm::description(
"Load time of the server certificate (seconds since epoch)."),
labels)
.aggregate(aggregate_labels));
defs.emplace_back(
sm::make_gauge(
"certificate_valid",
[this] { return cert_valid() ? 1 : 0; },
sm::description("The value is one if the certificate is valid with "
"the given truststore, otherwise zero."),
labels)
.aggregate(aggregate_labels));
return defs;
};

if (!config::shard_local_cfg().disable_metrics()) {
const auto aggregate_labels
= config::shard_local_cfg().aggregate_metrics()
? std::vector<sm::label>{sm::shard_label}
: std::vector<sm::label>{};
_metrics.add_group(
prometheus_sanitize::metrics_name("tls"), setup(aggregate_labels));
}

if (!config::shard_local_cfg().disable_public_metrics()) {
_public_metrics.add_group(
prometheus_sanitize::metrics_name("tls"),
setup(std::vector<sm::label>{sm::shard_label}));
}
}

} // namespace net
Loading

0 comments on commit 7d9722c

Please sign in to comment.