Skip to content

Commit

Permalink
features: new metric enterprise_license_expiry_sec
Browse files Browse the repository at this point in the history
Adds a metric to allow easier monitoring of when the enterprise license
is going to expiry.

The dependency on the `security` module existed even previously in the
bazel build but it was missing from the cmake build.

(cherry picked from commit edb5f48)
  • Loading branch information
pgellert committed Sep 24, 2024
1 parent 678d1bc commit 18da91a
Show file tree
Hide file tree
Showing 4 changed files with 118 additions and 0 deletions.
2 changes: 2 additions & 0 deletions src/v/features/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ v_cc_library(
v::model
v::config
v::version
v::security
v::metrics
)

add_dependencies(v_features kafka_codegen_headers)
Expand Down
78 changes: 78 additions & 0 deletions src/v/features/feature_table.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,18 @@
#include "cluster/types.h"
#include "config/node_config.h"
#include "features/logger.h"
#include "metrics/metrics.h"
#include "prometheus/prometheus_sanitize.h"
#include "version.h"

#include <seastar/core/abort_source.hh>

#include <chrono>
#include <memory>

// The feature table is closely related to cluster and uses many types from it
using namespace cluster;
using namespace std::chrono_literals;

namespace features {

Expand Down Expand Up @@ -189,6 +195,60 @@ static std::array test_extra_schema{
feature_spec::prepare_policy::always},
};

class feature_table::probe {
public:
explicit probe(const feature_table& parent)
: _parent(parent) {}

probe(const probe&) = delete;
probe& operator=(const probe&) = delete;
probe(probe&&) = delete;
probe& operator=(probe&&) = delete;
~probe() noexcept = default;

void setup_metrics() {
if (ss::this_shard_id() != 0) {
return;
}

if (!config::shard_local_cfg().disable_metrics()) {
setup_metrics_for(_metrics);
}

if (!config::shard_local_cfg().disable_public_metrics()) {
setup_metrics_for(_public_metrics);
}
}

void setup_metrics_for(metrics::metric_groups_base& metrics) {
namespace sm = ss::metrics;

static_assert(
!std::is_move_constructible_v<feature_table>
&& !std::is_move_assignable_v<feature_table>
&& !std::is_copy_constructible_v<feature_table>
&& !std::is_copy_assignable_v<feature_table>,
"The probe captures a reference to this");

metrics.add_group(
prometheus_sanitize::metrics_name("cluster:features"),
{
sm::make_gauge(
"enterprise_license_expiry_sec",
[&ft = _parent]() {
return calculate_expiry_metric(ft.get_license());
},
sm::description("Number of seconds remaining until the "
"Enterprise license expires"))
.aggregate({sm::shard_label}),
});
}

const feature_table& _parent;
metrics::internal_metric_groups _metrics;
metrics::public_metric_groups _public_metrics;
};

feature_table::feature_table() {
// Intentionally undocumented environment variable, only for use
// in integration tests.
Expand Down Expand Up @@ -221,9 +281,15 @@ feature_table::feature_table() {
}
}
}

_probe = std::make_unique<probe>(*this);
_probe->setup_metrics();
}

feature_table::~feature_table() noexcept = default;

ss::future<> feature_table::stop() {
_probe.reset();
_as.request_abort();

// Don't trust callers to have fired their abort source in the right
Expand Down Expand Up @@ -686,6 +752,18 @@ void feature_table::assert_compatible_version(bool override) {
}
}

long long feature_table::calculate_expiry_metric(
const std::optional<security::license>& license,
security::license::clock::time_point now) {
if (!license) {
return -1;
}

auto rem = license->expiration() - now;
auto rem_capped = std::max(rem.zero(), rem);
return rem_capped / 1s;
}

} // namespace features

namespace cluster {
Expand Down
15 changes: 15 additions & 0 deletions src/v/features/feature_table.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
#include "utils/waiter_queue.h"

#include <array>
#include <memory>
#include <string_view>
#include <unordered_set>

Expand Down Expand Up @@ -444,6 +445,11 @@ class feature_table {
static cluster::cluster_version get_earliest_logical_version();

feature_table();
feature_table(const feature_table&) = delete;
feature_table& operator=(const feature_table&) = delete;
feature_table(feature_table&&) = delete;
feature_table& operator=(feature_table&&) = delete;
~feature_table() noexcept;

feature_state& get_state(feature f_id);
const feature_state& get_state(feature f_id) const {
Expand Down Expand Up @@ -545,7 +551,15 @@ class feature_table {
// Assert out on startup if we appear to have upgraded too far
void assert_compatible_version(bool);

// Visible for testing
static long long calculate_expiry_metric(
const std::optional<security::license>& license,
security::license::clock::time_point now
= security::license::clock::now());

private:
class probe;

// Only for use by our friends feature backend & manager
void set_active_version(
cluster::cluster_version,
Expand Down Expand Up @@ -601,6 +615,7 @@ class feature_table {

ss::gate _gate;
ss::abort_source _as;
std::unique_ptr<probe> _probe;
};

} // namespace features
Expand Down
23 changes: 23 additions & 0 deletions src/v/features/tests/feature_table_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

#include "features/feature_table.h"
#include "features/feature_table_snapshot.h"
#include "security/license.h"
#include "test_utils/fixture.h"
#include "vlog.h"

Expand Down Expand Up @@ -334,3 +335,25 @@ FIXTURE_TEST(feature_table_old_snapshot, feature_table_fixture) {
ft.get_state(feature::test_alpha).get_state()
== feature_state::state::active);
}

SEASTAR_THREAD_TEST_CASE(feature_table_probe_expiry_metric_test) {
using ft = features::feature_table;
const char* sample_valid_license = std::getenv("REDPANDA_SAMPLE_LICENSE");
if (sample_valid_license == nullptr) {
const char* is_on_ci = std::getenv("CI");
BOOST_TEST_REQUIRE(
!is_on_ci,
"Expecting the REDPANDA_SAMPLE_LICENSE env var in the CI "
"enviornment");
return;
}
const ss::sstring license_str{sample_valid_license};
const auto license = security::make_license(license_str);

auto expiry = security::license::clock::time_point{4813252273s};

BOOST_CHECK_EQUAL(ft::calculate_expiry_metric(license, expiry - 1s), 1);
BOOST_CHECK_EQUAL(ft::calculate_expiry_metric(license, expiry), 0);
BOOST_CHECK_EQUAL(ft::calculate_expiry_metric(license, expiry + 1s), 0);
BOOST_CHECK_EQUAL(ft::calculate_expiry_metric(std::nullopt), -1);
}

0 comments on commit 18da91a

Please sign in to comment.