From 686b54b6d87afad87c3f497b4ad005c7d38ee41e Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Thu, 31 Aug 2023 14:24:18 +0100 Subject: [PATCH] tls cert expiry hackery Signed-off-by: Ben Pope --- src/v/net/probes.cc | 232 +++++++++++++++++++++++++++++++++ src/v/pandaproxy/server.cc | 34 ++++- src/v/redpanda/admin_server.cc | 11 +- src/v/redpanda/application.cc | 43 ++++-- 4 files changed, 304 insertions(+), 16 deletions(-) diff --git a/src/v/net/probes.cc b/src/v/net/probes.cc index 7c4ca5a722562..359fda2ee3759 100644 --- a/src/v/net/probes.cc +++ b/src/v/net/probes.cc @@ -10,14 +10,29 @@ #include "config/configuration.h" #include "net/client_probe.h" #include "net/server_probe.h" +#include "net/tls_credentials_probe.h" +#include "net/types.h" #include "prometheus/prometheus_sanitize.h" #include "ssx/metrics.h" #include "ssx/sformat.h" +#include #include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include + +#include #include +#include namespace net { void server_probe::setup_metrics( @@ -296,4 +311,221 @@ std::ostream& operator<<(std::ostream& o, const client_probe& p) { << ", requests_blocked_memory: " << p._requests_blocked_memory << " }"; return o; } + +void tls_certificate_probe::loaded( + const ss::tls::certificate_credentials& creds, std::exception_ptr ex) { + static ss::logger log("v::net"); + + _load_time = clock_type::now(); + + if (ex) { + _cert_valid = false; + _cert_expiry_time = {}; + _cert_serial = {}; + _ca_expiry_time = {}; + _ca_serial = {}; + return; + } + + auto g_creds = reinterpret_cast( + get_impl(creds)); + + auto certs = [](gnutls_certificate_credentials_t const& creds) { + gnutls_x509_crt_t* crt_list{}; + unsigned int crt_list_size{}; + gnutls_certificate_get_x509_crt(creds, 0, &crt_list, &crt_list_size); + return std::span(crt_list, crt_list_size); + }(g_creds); + + auto cleanup = ss::defer([certs]() { + for (auto cert : certs) { + gnutls_x509_crt_deinit(cert); + } + gnutls_free(&certs.front()); + }); + + auto get_expiry = [](gnutls_x509_crt_t cert) { + return tls_certificate_probe::clock_type::from_time_t( + gnutls_x509_crt_get_expiration_time(cert)); + }; + + auto get_serial = [](gnutls_x509_crt_t cert) { + constexpr size_t serial_max = 128; + size_t serial_size{serial_max}; + bytes serial(bytes::initialized_later{}, serial_size); + gnutls_x509_crt_get_serial(cert, serial.data(), &serial_size); + serial.resize(serial_size); + return serial; + }; + + _cert_valid = true; + + for (auto cert : certs) { + _cert_expiry_time = get_expiry(cert); + _cert_serial = get_serial(cert); + vlog( + log.warn, + "_cert_expiry: {}, _cert_serial: {}", + _cert_expiry_time, + to_hex(_cert_serial)); + } + + { + gnutls_x509_trust_list_t tlist{}; + gnutls_certificate_get_trust_list(g_creds, &tlist); + gnutls_x509_trust_list_iter_t iter{}; + gnutls_x509_crt_t cert{}; + while (GNUTLS_E_REQUESTED_DATA_NOT_AVAILABLE + != gnutls_x509_trust_list_iter_get_ca(tlist, &iter, &cert)) { + _ca_expiry_time = get_expiry(cert); + _ca_serial = get_serial(cert); + vlog( + log.warn, + "_ca_expiry: {}, _ca_serial: {}", + _ca_expiry_time, + to_hex(_ca_serial)); + gnutls_x509_crt_deinit(cert); + } + } + + // auto cert_exp = ss::tls::get_cert_expiry(creds); + // if (!cert_exp.empty()) { + // _cert_expiry_time = cert_exp.front(); + // } + // auto ca_exp = ss::tls::get_private_key_expiry(creds); + // if (!ca_exp.empty()) { + // _cert_expiry_time = ca_exp.front(); + // } +} + +void tls_certificate_probe::setup_metrics( + std::string_view service, std::string_view listener_name) { + if (ss::this_shard_id() != 0 || config::shard_local_cfg().disable_metrics) { + return; + } + + namespace sm = ss::metrics; + const auto service_label = sm::label("service"); + const auto listener_label = sm::label("listener_name"); + const auto serial_label = sm::label("serial"); + const auto loaded = sm::label("loaded"); + + const std::vector labels = { + service_label(service), listener_label(listener_name)}; + + const auto aggregate_labels = config::shard_local_cfg().aggregate_metrics() + ? std::vector{sm::shard_label} + : std::vector{}; + _metrics.add_group( + prometheus_sanitize::metrics_name("tls"), + { + sm::make_gauge( + "truststore_expires_in_seconds", + [this] { + return std::chrono::duration_cast( + _ca_expiry_time - clock_type::now()) + .count(); + }, + sm::description( + "The number of seconds until expiration of the shortest-lived " + "certificate in truststore certificate chain"), + labels) + .aggregate(aggregate_labels), + sm::make_gauge( + "certificate_expires_in_seconds", + [this] { + return std::chrono::duration_cast( + _cert_expiry_time - clock_type::now()) + .count(); + }, + sm::description("The number of seconds until expiration of the " + "server certificate."), + labels) + .aggregate(aggregate_labels), + sm::make_gauge( + "certificate_loaded_seconds", + [this] { + return std::chrono::duration_cast( + clock_type::now() - _load_time) + .count(); + }, + sm::description( + "The number of seconds ago the certifacate was loaded."), + labels) + .aggregate(aggregate_labels), + 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), + }); +} + +void tls_certificate_probe::setup_public_metrics( + std::string_view service, std::string_view listener_name) { + if ( + ss::this_shard_id() != 0 + || config::shard_local_cfg().disable_public_metrics) { + return; + } + + namespace sm = ss::metrics; + const auto service_label = sm::label("service"); + const auto listener_label = sm::label("listener_name"); + + const std::vector labels = { + service_label(service), listener_label(listener_name)}; + + const auto aggregate_labels = config::shard_local_cfg().aggregate_metrics() + ? std::vector{sm::shard_label} + : std::vector{}; + _public_metrics.add_group( + prometheus_sanitize::metrics_name("tls"), + { + sm::make_gauge( + "truststore_expires_in_seconds", + [this] { + return std::chrono::duration_cast( + _ca_expiry_time - clock_type::now()) + .count(); + }, + sm::description( + "The number of seconds until expiration of the shortest-lived " + "certificate in truststore certificate chain"), + labels) + .aggregate(aggregate_labels), + sm::make_gauge( + "certificate_expires_in_seconds", + [this] { + return std::chrono::duration_cast( + _cert_expiry_time - clock_type::now()) + .count(); + }, + sm::description("The number of seconds until expiration of the " + "server certificate."), + labels) + .aggregate(aggregate_labels), + sm::make_gauge( + "certificate_loaded_seconds", + [this] { + return std::chrono::duration_cast( + clock_type::now() - _load_time) + .count(); + }, + sm::description( + "The number of seconds ago the certifacate was loaded."), + labels) + .aggregate(aggregate_labels), + 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), + }); +} + } // namespace net diff --git a/src/v/pandaproxy/server.cc b/src/v/pandaproxy/server.cc index 3c24538fad334..9896ad9d449cb 100644 --- a/src/v/pandaproxy/server.cc +++ b/src/v/pandaproxy/server.cc @@ -11,14 +11,20 @@ #include "cluster/cluster_utils.h" #include "model/metadata.h" +#include "net/tls_credentials_probe.h" #include "pandaproxy/json/types.h" #include "pandaproxy/logger.h" #include "pandaproxy/probe.h" #include "pandaproxy/reply.h" +#include "ssx/metrics.h" #include #include #include +#include + +#include +#include #include #include @@ -205,13 +211,37 @@ ss::future<> server::start( if (it != endpoints_tls.end()) { auto builder = co_await it->config.get_credentials_builder(); if (builder) { + // The callback that captures this must be copyable due to + // std::function + auto probe = std::make_shared(); cred = co_await builder->build_reloadable_server_credentials( - []( + [probe]( const std::unordered_set& updated, - const std::exception_ptr& eptr) { + const ss::tls::certificate_credentials& creds, + const std::exception_ptr& eptr) mutable { cluster::log_certificate_reload_event( plog, "API TLS", updated, eptr); + // if (eptr) { + // vlog( + // plog.warn, + // "Reloading certificate credentials failed: {}", + // eptr); + // } else { + // auto exp = ss::tls::get_cert_expiry(creds); + // auto ser = ss::tls::get_cert_serial(creds); + // vlog( + // plog.warn, + // "Reloading certificate credentials: expiry: {}, " + // "ser: {}", + // exp, + // ser); + // } + probe->loaded(creds, eptr); }); + probe->setup_metrics(_public_metrics_group_name, it->name); + probe->setup_public_metrics( + _public_metrics_group_name, it->name); + probe->loaded(*cred, nullptr); } } co_await _server.listen(addr, cred); diff --git a/src/v/redpanda/admin_server.cc b/src/v/redpanda/admin_server.cc index c0326d7412bd3..653dcea9b6fa2 100644 --- a/src/v/redpanda/admin_server.cc +++ b/src/v/redpanda/admin_server.cc @@ -63,6 +63,7 @@ #include "model/record.h" #include "model/timeout_clock.h" #include "net/dns.h" +#include "net/tls_credentials_probe.h" #include "pandaproxy/rest/api.h" #include "pandaproxy/schema_registry/api.h" #include "pandaproxy/schema_registry/schema_id_validation.h" @@ -114,6 +115,7 @@ #include #include #include +#include #include #include #include @@ -533,13 +535,20 @@ ss::future<> admin_server::configure_listeners() { if (tls_it != _cfg.endpoints_tls.end()) { auto builder = co_await tls_it->config.get_credentials_builder(); if (builder) { + auto probe = ss::make_lw_shared(); cred = co_await builder->build_reloadable_server_credentials( - []( + [probe]( const std::unordered_set& updated, + const ss::tls::certificate_credentials& creds, const std::exception_ptr& eptr) { cluster::log_certificate_reload_event( logger, "API TLS", updated, eptr); + probe->loaded(creds, eptr); }); + std::string_view service = "admin"; + probe->setup_metrics(service, tls_it->name); + probe->setup_public_metrics(service, tls_it->name); + probe->loaded(*cred, nullptr); } if (!localhost && !tls_it->config.get_require_client_auth()) { diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 02d296eb2e790..4c4fc32fb745a 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -75,6 +75,7 @@ #include "model/fundamental.h" #include "model/metadata.h" #include "net/server.h" +#include "net/tls_credentials_probe.h" #include "pandaproxy/rest/api.h" #include "pandaproxy/rest/configuration.h" #include "pandaproxy/schema_registry/api.h" @@ -1601,19 +1602,28 @@ void application::wire_up_redpanda_services( .get(); auto kafka_builder = it->config.get_credentials_builder().get0(); - credentails - = kafka_builder - ? kafka_builder + if (kafka_builder) { + auto probe + = std::make_shared(); + credentails + = kafka_builder ->build_reloadable_server_credentials( - [this, name = it->name]( + [this, probe, name = it->name]( const std::unordered_set& updated, + const ss::tls::certificate_credentials& + creds, const std::exception_ptr& eptr) { cluster::log_certificate_reload_event( _log, "Kafka RPC TLS", updated, eptr); + probe->loaded(creds, eptr); }) - .get0() - : nullptr; + .get0(); + std::string_view service = "kafka"; + probe->setup_metrics(service, it->name); + probe->setup_public_metrics(service, it->name); + probe->loaded(*credentails, nullptr); + } } c.addrs.emplace_back( @@ -1806,19 +1816,26 @@ void application::wire_up_bootstrap_services() { .rpc_server_tls() .get_credentials_builder() .get0(); - auto credentials - = rpc_builder - ? rpc_builder + if (rpc_builder) { + auto probe = std::make_shared(); + auto credentials + = rpc_builder ->build_reloadable_server_credentials( - [this]( + [this, probe]( const std::unordered_set& updated, + const ss::tls::certificate_credentials& creds, const std::exception_ptr& eptr) { cluster::log_certificate_reload_event( _log, "Internal RPC TLS", updated, eptr); + probe->loaded(creds, eptr); }) - .get0() - : nullptr; - c.addrs.emplace_back(rpc_server_addr, credentials); + .get0(); + std::string_view service = "rpc"; + probe->setup_metrics(service, ""); + probe->setup_public_metrics(service, ""); + probe->loaded(*credentials, nullptr); + c.addrs.emplace_back(rpc_server_addr, credentials); + } }); }) .get();