Skip to content

Commit

Permalink
Merge pull request #23893 from pgellert/trial/poc
Browse files Browse the repository at this point in the history
CORE-7996 Trial: build in an evaluation period license
  • Loading branch information
michael-redpanda authored Oct 30, 2024
2 parents 41882b9 + 7702ed1 commit cc888d0
Show file tree
Hide file tree
Showing 25 changed files with 252 additions and 53 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -205,8 +205,10 @@ TEST_P(ClusterRecoveryBackendLeadershipParamTest, TestRecoveryControllerState) {
});

// Sanity check we have a different cluster.
ASSERT_TRUE(
!app.controller->get_feature_table().local().get_license().has_value());
ASSERT_TRUE(!app.controller->get_feature_table()
.local()
.get_configured_license()
.has_value());
ASSERT_NE(
1, config::shard_local_cfg().log_segment_size_jitter_percent.value());
ASSERT_TRUE(!app.controller->get_credential_store().local().contains(
Expand Down Expand Up @@ -249,7 +251,7 @@ TEST_P(ClusterRecoveryBackendLeadershipParamTest, TestRecoveryControllerState) {
auto validate_post_recovery = [&] {
ASSERT_TRUE(app.controller->get_feature_table()
.local()
.get_license()
.get_configured_license()
.has_value());
ASSERT_EQ(
1, config::shard_local_cfg().log_segment_size_jitter_percent.value());
Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/cluster_recovery_reconciler.cc
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ controller_snapshot_reconciler::get_actions(

controller_actions actions;
const auto& snap_license = snap.features.snap.license;
auto existing_license = _feature_table.get_license();
auto existing_license = _feature_table.get_configured_license();
if (
cur_stage < recovery_stage::recovered_license && !existing_license
&& snap_license.has_value()) {
Expand Down
10 changes: 8 additions & 2 deletions src/v/cluster/controller_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,8 @@
namespace cluster {

ss::future<> controller_stm::on_batch_applied() {
if (!_feature_table.is_active(features::feature::controller_snapshots)) {
if (!_feature_table.local().is_active(
features::feature::controller_snapshots)) {
co_return;
}
if (_gate.is_closed()) {
Expand Down Expand Up @@ -106,7 +107,8 @@ ss::future<std::optional<iobuf>>
controller_stm::maybe_make_snapshot(ssx::semaphore_units apply_mtx_holder) {
auto started_at = ss::steady_clock_type::now();

if (!_feature_table.is_active(features::feature::controller_snapshots)) {
if (!_feature_table.local().is_active(
features::feature::controller_snapshots)) {
vlog(clusterlog.warn, "skipping snapshotting, feature not enabled");
co_return std::nullopt;
}
Expand Down Expand Up @@ -200,6 +202,10 @@ ss::future<> controller_stm::apply_snapshot(
}

_metrics_reporter_cluster_info = snapshot.metrics_reporter.cluster_info;
co_await _feature_table.invoke_on_all([&](features::feature_table& ft) {
ft.set_builtin_trial_license(
_metrics_reporter_cluster_info.creation_timestamp);
});
}

} // namespace cluster
6 changes: 3 additions & 3 deletions src/v/cluster/controller_stm.h
Original file line number Diff line number Diff line change
Expand Up @@ -46,12 +46,12 @@ class controller_stm final
template<typename... Args>
controller_stm(
limiter_configuration limiter_conf,
const ss::sharded<features::feature_table>& feature_table,
ss::sharded<features::feature_table>& feature_table,
config::binding<std::chrono::seconds>&& snapshot_max_age,
Args&&... stm_args)
: mux_state_machine(std::forward<Args>(stm_args)...)
, _limiter(std::move(limiter_conf))
, _feature_table(feature_table.local())
, _feature_table(feature_table)
, _snapshot_max_age(std::move(snapshot_max_age))
, _snapshot_debounce_timer([this] { snapshot_timer_callback(); }) {}

Expand Down Expand Up @@ -91,7 +91,7 @@ class controller_stm final

private:
controller_log_limiter _limiter;
const features::feature_table& _feature_table;
ss::sharded<features::feature_table>& _feature_table;
config::binding<std::chrono::seconds> _snapshot_max_age;

metrics_reporter_cluster_info _metrics_reporter_cluster_info;
Expand Down
6 changes: 2 additions & 4 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -298,8 +298,7 @@ ss::future<> feature_manager::maybe_log_license_check_info() {
if (_feature_table.local().is_active(features::feature::license)) {
auto enterprise_features = report_enterprise_features();
if (enterprise_features.any()) {
const auto& license = _feature_table.local().get_license();
if (!license || license->is_expired()) {
if (_feature_table.local().should_sanction()) {
vlog(
clusterlog.warn,
"A Redpanda Enterprise Edition license is required to use "
Expand Down Expand Up @@ -329,7 +328,6 @@ void feature_manager::verify_enterprise_license() {
return;
}

const auto& license = _feature_table.local().get_license();
std::optional<security::license> fallback_license = std::nullopt;
auto fallback_license_str = std::getenv(
"REDPANDA_FALLBACK_ENTERPRISE_LICENSE");
Expand All @@ -349,7 +347,7 @@ void feature_manager::verify_enterprise_license() {
auto invalid = [](const std::optional<security::license>& license) {
return !license || license->is_expired();
};
auto license_missing_or_expired = invalid(license)
auto license_missing_or_expired = _feature_table.local().should_sanction()
&& invalid(fallback_license);
auto enterprise_features = report_enterprise_features();

Expand Down
7 changes: 6 additions & 1 deletion src/v/cluster/metrics_reporter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
#include "config/configuration.h"
#include "config/validators.h"
#include "features/enterprise_features.h"
#include "features/feature_table.h"
#include "hashing/secure.h"
#include "json/stringbuffer.h"
#include "json/writer.h"
Expand Down Expand Up @@ -312,10 +313,14 @@ ss::future<> metrics_reporter::try_initialize_cluster_info() {

auto& first_cfg = batches.front();

_cluster_info.creation_timestamp = first_cfg.header().first_timestamp;
co_await _feature_table.invoke_on_all([&](features::feature_table& ft) {
ft.set_builtin_trial_license(_cluster_info.creation_timestamp);
});

auto data_bytes = iobuf_to_bytes(first_cfg.data());
hash_sha256 sha256;
sha256.update(data_bytes);
_cluster_info.creation_timestamp = first_cfg.header().first_timestamp;
// use timestamps of first two batches in raft-0 log.
for (int i = 0; i < 2; ++i) {
sha256.update(iobuf_to_bytes(
Expand Down
65 changes: 64 additions & 1 deletion src/v/features/feature_table.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,16 @@
#include "features/logger.h"
#include "metrics/metrics.h"
#include "metrics/prometheus_sanitize.h"
#include "model/timestamp.h"
#include "security/license.h"
#include "version/version.h"

#include <seastar/core/abort_source.hh>
#include <seastar/core/shard_id.hh>

#include <chrono>
#include <memory>
#include <optional>

// The feature table is closely related to cluster and uses many types from it
using namespace cluster;
Expand Down Expand Up @@ -299,6 +303,30 @@ class feature_table::probe {
metrics::public_metric_groups _public_metrics;
};

namespace {

security::license
make_builtin_trial_license(security::license::clock::time_point start_time) {
auto expiry_time = start_time + std::chrono::days{45};
auto expiry = std::chrono::duration_cast<std::chrono::seconds>(
expiry_time.time_since_epoch());

if (std::getenv("__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE") != nullptr) {
// For testing, use an expired trial license
expiry = 0s;
}

return security::license{
.format_version = 0,
.type = security::license_type::free_trial,
.organization = "Redpanda Built-In Evaluation Period",
.expiry = expiry,
.checksum = "",
};
}

} // namespace

feature_table::feature_table() {
// Intentionally undocumented environment variable, only for use
// in integration tests.
Expand Down Expand Up @@ -688,12 +716,47 @@ void feature_table::set_license(security::license license) {
_license = std::move(license);
}

void feature_table::revoke_license() { _license = std::nullopt; }
void feature_table::set_builtin_trial_license(
model::timestamp cluster_creation_timestamp) {
_builtin_trial_license = make_builtin_trial_license(
model::to_time_point(cluster_creation_timestamp));
_builtin_trial_license_initialized = true;

if (ss::this_shard_id() == 0) {
vlog(
featureslog.debug,
"Initialized builtin trial license expirying at "
"{}",
_builtin_trial_license->expiry);
}
}

void feature_table::revoke_license() {
_license = std::nullopt;
_builtin_trial_license = std::nullopt;
}

const std::optional<security::license>& feature_table::get_license() const {
return _license ? _license : _builtin_trial_license;
}

const std::optional<security::license>&
feature_table::get_configured_license() const {
return _license;
}

bool feature_table::should_sanction() const {
if (_license) {
return _license->is_expired();
} else if (_builtin_trial_license) {
return _builtin_trial_license->is_expired();
}

// While we are yet to initialize _builtin_trial_license on cluster
// creation, be permissive
return _builtin_trial_license_initialized;
}

void feature_table::testing_activate_all() {
for (auto& s : _feature_state) {
if (
Expand Down
23 changes: 23 additions & 0 deletions src/v/features/feature_table.h
Original file line number Diff line number Diff line change
Expand Up @@ -592,10 +592,26 @@ class feature_table {

void set_license(security::license license);

/// Sets the builtin trial license based on the cluster creation time
void set_builtin_trial_license(model::timestamp cluster_creation_timestamp);

void revoke_license();

/// Returns the user-configured license or if not set, a built in trial
/// license. The built in trial license is initialized async on cluster
/// bootstrap and during startup on subsequent starts, so consider using
/// `should_sanction` instead for a general, permissive check on whether
/// to act on a missing valid license.
const std::optional<security::license>& get_license() const;

/// Returns the user-configured license without falling back to the
/// evaluation period license
const std::optional<security::license>& get_configured_license() const;

/// Whether to act on an expired license or evaluation period by restricting
/// enterprise feature usage
bool should_sanction() const;

/**
* For use in unit tests: activate all features that would
* be auto-activated after a normal cluster startup of the
Expand Down Expand Up @@ -727,6 +743,13 @@ class feature_table {
// Currently loaded redpanda license details
std::optional<security::license> _license;

// Built in trial license to fall back to if there is no license set
std::optional<security::license> _builtin_trial_license;

// Whether _builtin_trial_license has ever been initialized
// Used for implementing revoking the trial license for testing
bool _builtin_trial_license_initialized{false};

model::offset _applied_offset{};

// feature_manager is a friend so that they can initialize
Expand Down
2 changes: 1 addition & 1 deletion src/v/features/feature_table_snapshot.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ feature_table_snapshot feature_table_snapshot::from(const feature_table& ft) {
fts.states.reserve(feature_schema.size());

fts.version = ft.get_active_version();
fts.license = ft.get_license();
fts.license = ft.get_configured_license();
for (const auto& state : ft._feature_state) {
auto& name = state.spec.name;
fts.states.push_back(feature_state_snapshot{
Expand Down
1 change: 1 addition & 0 deletions src/v/features/tests/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ redpanda_cc_btest(
deps = [
"//src/v/cluster:features",
"//src/v/features",
"//src/v/model",
"//src/v/security:license",
"//src/v/test_utils:seastar_boost",
"@seastar",
Expand Down
40 changes: 40 additions & 0 deletions src/v/features/tests/feature_table_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#include "cluster/feature_update_action.h"
#include "features/feature_table.h"
#include "features/feature_table_snapshot.h"
#include "model/timestamp.h"
#include "security/license.h"
#include "test_utils/fixture.h"

Expand Down Expand Up @@ -336,6 +337,45 @@ FIXTURE_TEST(feature_table_old_snapshot, feature_table_fixture) {
== feature_state::state::active);
}

FIXTURE_TEST(feature_table_trial_license_test, feature_table_fixture) {
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 expired_license = license;
expired_license.expiry = 0s;

BOOST_CHECK_EQUAL(ft.get_license().has_value(), false);
BOOST_CHECK_EQUAL(ft.should_sanction(), false);

ft.set_builtin_trial_license(model::timestamp::now());
BOOST_CHECK_EQUAL(ft.get_license().has_value(), true);
BOOST_CHECK_EQUAL(ft.get_license()->is_expired(), false);
BOOST_CHECK_EQUAL(ft.should_sanction(), false);

ft.set_license(expired_license);
BOOST_CHECK_EQUAL(ft.get_license().has_value(), true);
BOOST_CHECK_EQUAL(ft.get_license()->is_expired(), true);
BOOST_CHECK_EQUAL(ft.should_sanction(), true);

ft.set_license(license);
BOOST_CHECK_EQUAL(ft.get_license().has_value(), true);
BOOST_CHECK_EQUAL(ft.get_license()->is_expired(), false);
BOOST_CHECK_EQUAL(ft.should_sanction(), false);

ft.revoke_license();
BOOST_CHECK_EQUAL(ft.get_license().has_value(), false);
BOOST_CHECK_EQUAL(ft.should_sanction(), true);
}

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");
Expand Down
4 changes: 4 additions & 0 deletions src/v/model/timestamp.h
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,10 @@ inline timestamp_clock::duration duration_since_epoch(timestamp ts) {
std::chrono::milliseconds{ts.value()});
}

inline timestamp_clock::time_point to_time_point(timestamp ts) {
return timestamp_clock::time_point(std::chrono::milliseconds(ts.value()));
}

inline timestamp to_timestamp(timestamp_clock::time_point ts) {
return timestamp(std::chrono::duration_cast<std::chrono::milliseconds>(
ts.time_since_epoch())
Expand Down
19 changes: 19 additions & 0 deletions tests/rptest/services/admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -905,6 +905,25 @@ def get_license(self, node=None, timeout=None):
node=node,
timeout=timeout).json()

@staticmethod
def is_sample_license(resp) -> bool:
"""
Returns true if the given response to a `get_license` request returned the same license as the sample license
configured in `sample_license` ("REDPANDA_SAMPLE_LICENSE" env var). Returns false for the built in evaluation
period license and the second sample license 'REDPANDA_SECOND_SAMPLE_LICENSE'.
"""
# NOTE: the initial implementation of the get license endpoint (before v22.3) didn't return the sha256.
# We could remove those old tests, but it's simpler to use the type and the org to detect the installed
# license instead.

# REDPANDA_SAMPLE_LICENSE: {'loaded': True, 'license': {'format_version': 0, 'org': 'redpanda-testing', 'type': 'enterprise', 'expires': 4813252273, 'sha256': '2730125070a934ca1067ed073d7159acc9975dc61015892308aae186f7455daf'}}
# REDPANDA_SECOND_SAMPLE_LICENSE: {'loaded': True, 'license': {'format_version': 0, 'org': 'redpanda-testing-2', 'type': 'enterprise', 'expires': 4827156118, 'sha256': '54240716865c1196fa6bd0ebb31821ab69160a3ed312b13bc810c17c9ec8852c'}}
# Evaluation Period: {'loaded': True, 'license': {'format_version': 0, 'org': 'Redpanda Built-In Evaluation Period', 'type': 'free_trial', 'expires': 1733992567, 'sha256': ''}}

return resp is not None and resp.get('license', None) is not None \
and resp['license']['type'] == 'enterprise' \
and resp['license']['org'] == 'redpanda-testing'

def put_license(self, license):
return self._request("PUT", "features/license", data=license)

Expand Down
2 changes: 1 addition & 1 deletion tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -5355,7 +5355,7 @@ def install_license(self):
def license_observable():
for node in self.started_nodes():
license = self._admin.get_license(node)
if license is None or license['loaded'] is not True:
if not self._admin.is_sample_license(license):
return False
return True

Expand Down
Loading

0 comments on commit cc888d0

Please sign in to comment.