Skip to content

Commit

Permalink
tls cert expiry hackery
Browse files Browse the repository at this point in the history
Signed-off-by: Ben Pope <ben@redpanda.com>
  • Loading branch information
BenPope committed Aug 31, 2023
1 parent ee1157b commit 686b54b
Show file tree
Hide file tree
Showing 4 changed files with 304 additions and 16 deletions.
232 changes: 232 additions & 0 deletions src/v/net/probes.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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 <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 <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>

namespace net {
void server_probe::setup_metrics(
Expand Down Expand Up @@ -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<gnutls_certificate_credentials_t>(
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<gnutls_x509_crt_t>(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<sm::label_instance> labels = {
service_label(service), listener_label(listener_name)};

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"),
{
sm::make_gauge(
"truststore_expires_in_seconds",
[this] {
return std::chrono::duration_cast<std::chrono::seconds>(
_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<std::chrono::seconds>(
_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<std::chrono::seconds>(
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<sm::label_instance> labels = {
service_label(service), listener_label(listener_name)};

const auto aggregate_labels = config::shard_local_cfg().aggregate_metrics()
? std::vector<sm::label>{sm::shard_label}
: std::vector<sm::label>{};
_public_metrics.add_group(
prometheus_sanitize::metrics_name("tls"),
{
sm::make_gauge(
"truststore_expires_in_seconds",
[this] {
return std::chrono::duration_cast<std::chrono::seconds>(
_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<std::chrono::seconds>(
_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<std::chrono::seconds>(
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
34 changes: 32 additions & 2 deletions src/v/pandaproxy/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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 <seastar/core/coroutine.hh>
#include <seastar/http/function_handlers.hh>
#include <seastar/http/reply.hh>
#include <seastar/net/tls.hh>

#include <fmt/chrono.h>
#include <fmt/ranges.h>

#include <charconv>
#include <exception>
Expand Down Expand Up @@ -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<net::tls_certificate_probe>();
cred = co_await builder->build_reloadable_server_credentials(
[](
[probe](
const std::unordered_set<ss::sstring>& 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);
Expand Down
11 changes: 10 additions & 1 deletion src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -114,6 +115,7 @@
#include <seastar/http/request.hh>
#include <seastar/http/url.hh>
#include <seastar/json/json_elements.hh>
#include <seastar/net/tls.hh>
#include <seastar/util/later.hh>
#include <seastar/util/log.hh>
#include <seastar/util/short_streams.hh>
Expand Down Expand Up @@ -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<net::tls_certificate_probe>();
cred = co_await builder->build_reloadable_server_credentials(
[](
[probe](
const std::unordered_set<ss::sstring>& 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()) {
Expand Down
Loading

0 comments on commit 686b54b

Please sign in to comment.