Skip to content

Commit

Permalink
Merge pull request #23367 from pgellert/elf/expiry-metric
Browse files Browse the repository at this point in the history
CORE-7338 license: add `enterprise_license_expiry_sec` metric
  • Loading branch information
pgellert authored Sep 24, 2024
2 parents 3cef408 + baece2c commit 3ff5d86
Show file tree
Hide file tree
Showing 13 changed files with 216 additions and 6 deletions.
3 changes: 3 additions & 0 deletions src/v/features/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@ redpanda_cc_library(
"fwd.h",
"logger.h",
],
implementation_deps = [
"//src/v/metrics",
],
include_prefix = "features",
visibility = ["//visibility:public"],
deps = [
Expand Down
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 @@ -15,12 +15,18 @@
#include "cluster/version.h"
#include "config/configuration.h"
#include "features/logger.h"
#include "metrics/metrics.h"
#include "metrics/prometheus_sanitize.h"
#include "version/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 @@ -200,6 +206,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 @@ -232,9 +292,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 @@ -697,6 +763,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 @@ -17,6 +17,7 @@
#include "utils/waiter_queue.h"

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

Expand Down Expand Up @@ -497,6 +498,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 @@ -599,7 +605,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 @@ -655,6 +669,7 @@ class feature_table {

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

} // namespace features
Expand Down
17 changes: 17 additions & 0 deletions src/v/features/tests/BUILD
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
load("//bazel:test.bzl", "redpanda_cc_btest")

redpanda_cc_btest(
name = "feature_table_test",
timeout = "short",
srcs = [
"feature_table_test.cc",
],
deps = [
"//src/v/cluster:features",
"//src/v/features",
"//src/v/security:license",
"//src/v/test_utils:seastar_boost",
"@seastar",
"@seastar//:testing",
],
)
24 changes: 23 additions & 1 deletion src/v/features/tests/feature_table_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -9,10 +9,10 @@
* by the Apache License, Version 2.0
*/

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

#include <seastar/core/sleep.hh>
Expand Down Expand Up @@ -335,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);
}
13 changes: 10 additions & 3 deletions src/v/security/license.cc
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,11 @@
#include "json/validator.h"
#include "utils/base64.h"

#include <algorithm>
#include <chrono>

using namespace std::chrono_literals;

namespace security {

namespace crypto {
Expand Down Expand Up @@ -178,9 +183,11 @@ license make_license(const ss::sstring& raw_license) {
}

bool license::is_expired() const noexcept {
const auto now = std::chrono::duration_cast<std::chrono::seconds>(
std::chrono::system_clock::now().time_since_epoch());
return now > expiry;
return clock::now() > expiration();
}

license::clock::time_point license::expiration() const noexcept {
return clock::time_point{expiry};
}

} // namespace security
Expand Down
6 changes: 6 additions & 0 deletions src/v/security/license.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

#include <fmt/core.h>

#include <chrono>
#include <exception>
#include <fstream>
#include <vector>
Expand Down Expand Up @@ -67,6 +68,8 @@ inline std::ostream& operator<<(std::ostream& os, license_type lt) {

struct license
: serde::envelope<license, serde::version<1>, serde::compat_version<0>> {
using clock = std::chrono::system_clock;

/// Expected encoded contents
uint8_t format_version;
license_type type;
Expand All @@ -84,6 +87,9 @@ struct license
/// Seconds since epoch until license expiration
std::chrono::seconds expires() const noexcept;

/// Expiration timepoint
clock::time_point expiration() const noexcept;

auto operator<=>(const license&) const = delete;

private:
Expand Down
8 changes: 8 additions & 0 deletions src/v/security/tests/license_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,11 @@
#include <boost/filesystem/operations.hpp>
#include <boost/filesystem/path.hpp>
#include <boost/test/unit_test.hpp>

#include <chrono>

using namespace std::chrono_literals;

namespace security {

BOOST_AUTO_TEST_CASE(test_license_invalid_signature) {
Expand Down Expand Up @@ -76,7 +81,10 @@ BOOST_AUTO_TEST_CASE(test_license_valid_content) {
BOOST_CHECK_EQUAL(license.format_version, 0);
BOOST_CHECK_EQUAL(license.type, license_type::enterprise);
BOOST_CHECK_EQUAL(license.organization, "redpanda-testing");
BOOST_CHECK(!license.is_expired());
BOOST_CHECK_EQUAL(license.expiry.count(), 4813252273);
BOOST_CHECK(
license.expiration() == license::clock::time_point{4813252273s});
BOOST_CHECK_EQUAL(
license.checksum,
"2730125070a934ca1067ed073d7159acc9975dc61015892308aae186f7455daf");
Expand Down
1 change: 1 addition & 0 deletions src/v/storage/tests/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,7 @@ redpanda_cc_gtest(
deps = [
":disk_log_builder",
":disk_log_builder_fixture",
"//src/v/config",
"//src/v/model",
"//src/v/model/tests:random",
"//src/v/random:generators",
Expand Down
12 changes: 12 additions & 0 deletions src/v/storage/tests/segment_deduplication_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0

#include "config/configuration.h"
#include "gmock/gmock.h"
#include "model/record_batch_types.h"
#include "model/tests/random_batch.h"
Expand Down Expand Up @@ -220,6 +221,17 @@ TEST(FindSlidingRangeTest, TestEmptySegmentNoCompactibleRecords) {
}

TEST(BuildOffsetMap, TestBuildSimpleMap) {
ss::smp::invoke_on_all([] {
config::shard_local_cfg().disable_metrics.set_value(true);
config::shard_local_cfg().disable_public_metrics.set_value(true);
}).get();
auto defer_config_reset = ss::defer([] {
ss::smp::invoke_on_all([] {
config::shard_local_cfg().disable_metrics.reset();
config::shard_local_cfg().disable_public_metrics.reset();
}).get();
});

storage::disk_log_builder b;
build_segments(b, 3);
auto cleanup = ss::defer([&] { b.stop().get(); });
Expand Down
8 changes: 7 additions & 1 deletion src/v/storage/tests/storage_test_fixture.h
Original file line number Diff line number Diff line change
Expand Up @@ -209,9 +209,14 @@ class storage_test_fixture {
resources,
feature_table) {
configure_unit_test_logging();
// avoid double metric registrations
// avoid double metric registrations - disk_log_builder and other
// helpers also start a feature_table and other structs that register
// metrics
ss::smp::invoke_on_all([] {
config::shard_local_cfg().get("disable_metrics").set_value(true);
config::shard_local_cfg()
.get("disable_public_metrics")
.set_value(true);
config::shard_local_cfg()
.get("log_segment_size_min")
.set_value(std::optional<uint64_t>{});
Expand All @@ -230,6 +235,7 @@ class storage_test_fixture {
feature_table.stop().get();
ss::smp::invoke_on_all([] {
config::shard_local_cfg().get("disable_metrics").reset();
config::shard_local_cfg().get("disable_public_metrics").reset();
config::shard_local_cfg().get("log_segment_size_min").reset();
}).get();
}
Expand Down
Loading

0 comments on commit 3ff5d86

Please sign in to comment.