Skip to content

Commit

Permalink
redpanda: verify enterprise license on upgrade
Browse files Browse the repository at this point in the history
Introduces stict checking of the enterprise license on upgrade when
enterprise features are used in the cluster.
  • Loading branch information
pgellert committed Sep 17, 2024
1 parent dbbc199 commit 878a553
Show file tree
Hide file tree
Showing 4 changed files with 202 additions and 1 deletion.
59 changes: 59 additions & 0 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,66 @@ ss::future<> feature_manager::maybe_log_license_check_info() {
}
}

bool feature_manager::need_to_verify_enterprise_license() {
auto trying_to_upgrade
= _feature_table.local().get_active_version()
< _feature_table.local().get_latest_logical_version();
return trying_to_upgrade;
}

void feature_manager::verify_enterprise_license() {
vlog(clusterlog.info, "Verifying enterprise license...");

if (!need_to_verify_enterprise_license()) {
vlog(clusterlog.info, "Enterprise license verification skipped...");
_verified_enterprise_license = true;
return;
}

const auto& license = _feature_table.local().get_license();
auto license_missing_or_expired = !license || license->is_expired();

vlog(
clusterlog.info,
"Verifying enterprise license: active_version={}, latest_version={}, "
"enterprise_features_used={}, license_missing_or_expired={}",
_feature_table.local().get_active_version(),
_feature_table.local().get_latest_logical_version(),
license_required_feature_enabled(),
license_missing_or_expired);

if (license_required_feature_enabled() && license_missing_or_expired) {
vassert(
false,
"Looks like you’ve enabled a Redpanda Enterprise feature(s) "
"without a valid license. Please enter an active Redpanda "
"license key (e.g. rpk cluster license set <key>). If you "
"don’t have one, please request a new/trial license at "
"https://redpanda.com/license-request");
}

_verified_enterprise_license = true;
}

ss::future<> feature_manager::maybe_update_feature_table() {
// Before doing any feature enablement or active version update, check that
// the cluster has an enterprise license if they use enterprise features
// It's possible that on ::start() the controller log hasn't synced yet, so
// check this every time we're about to update the active version to ensure
// that we abort before the update
if (need_to_verify_enterprise_license() && !_verified_enterprise_license) {
vlog(
clusterlog.debug,
"Waiting for enterprise license to be verified before checking for "
"active version updates...");
try {
co_await ss::sleep_abortable(status_retry, _as.local());
} catch (const ss::sleep_aborted&) {
// Shutting down - next iteration will drop out
}
co_return;
}

vlog(clusterlog.debug, "Checking for active version update...");
bool failed = false;
try {
Expand Down
6 changes: 6 additions & 0 deletions src/v/cluster/feature_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,9 @@ class feature_manager {

ss::future<std::error_code> update_license(security::license&& license);

bool need_to_verify_enterprise_license();
void verify_enterprise_license();

private:
void update_node_version(model::node_id, cluster_version v);

Expand Down Expand Up @@ -196,6 +199,9 @@ class feature_manager {
// Keep track of whether this node is the controller leader
// via leadership notifications
bool _am_controller_leader{false};

// Blocks cluster upgrades until the enterprise license has been verified
bool _verified_enterprise_license{false};
};

} // namespace cluster
19 changes: 18 additions & 1 deletion src/v/redpanda/application.cc
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
#include "cluster/data_migration_service_handler.h"
#include "cluster/ephemeral_credential_frontend.h"
#include "cluster/ephemeral_credential_service.h"
#include "cluster/feature_manager.h"
#include "cluster/fwd.h"
#include "cluster/id_allocator.h"
#include "cluster/id_allocator_frontend.h"
Expand Down Expand Up @@ -3057,8 +3058,24 @@ void application::start_runtime_services(
.get();
}

_debug_bundle_service.invoke_on_all(&debug_bundle::service::start).get();
// Verify the enterprise license when trying to upgrade Redpanda.
// By this point during startup we have enough information to evaluate the
// state of the license and to evaluate what enterprise features are used.
// If redpanda has been restarted on an existing node, we have already
// loaded the feature table from the local snapshot in
// application::load_feature_table_snapshot. If this is a new node joining
// an existing cluster, by this point we have received a controller snapshot
// from another node in the join response and have waited for the controller
// stm to apply that snapshot above.
controller->get_feature_manager()
.invoke_on(
cluster::feature_manager::backend_shard,
[](cluster::feature_manager& fm) {
return fm.verify_enterprise_license();
})
.get();

_debug_bundle_service.invoke_on_all(&debug_bundle::service::start).get();
if (!config::node().admin().empty()) {
_admin.invoke_on_all(&admin_server::start).get0();
}
Expand Down
119 changes: 119 additions & 0 deletions tests/rptest/tests/license_enforcement_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
# Copyright 2024 Redpanda Data, Inc.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.md
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0

import re

from ducktape.mark import parametrize

from rptest.services.cluster import cluster
from rptest.clients.rpk import RpkTool
from rptest.services.redpanda import LoggingConfig
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda_installer import RedpandaVersionTriple
from rptest.util import get_cluster_license


class LicenseEnforcementTest(RedpandaTest):
# Disable log checks because it is not very useful when we expect to crash nodes on license upgrades
LOG_ALLOW_LIST = [re.compile(".*")]

def __init__(self, *args, **kwargs):
super().__init__(*args,
num_brokers=5,
extra_rp_conf={},
log_config=LoggingConfig('info',
logger_levels={
'cluster': 'debug',
'features': 'debug'
}),
**kwargs)

self.rpk = RpkTool(self.redpanda)

def setUp(self):
# start the nodes manually
pass

@cluster(num_nodes=5, log_allow_list=LOG_ALLOW_LIST)
@parametrize(clean_node=True)
@parametrize(clean_node=False)
def test_license_enforcement(self, clean_node):
installer = self.redpanda._installer

# TODO: use highest_from_prior_feature_version
prev_version = RedpandaVersionTriple((24, 1, 16))
# prev_version = installer.highest_from_prior_feature_version(RedpandaInstaller.HEAD)
latest_version = installer.head_version()
self.logger.info(
f"Testing with versions: {prev_version=} {latest_version=}")

self.logger.info(f"Starting all nodes with version: {prev_version}")
installer.install(self.redpanda.nodes, prev_version)
self.redpanda.start(nodes=self.redpanda.nodes,
omit_seeds_on_idx_one=False)

self.redpanda.wait_until(self.redpanda.healthy, 60, 1)

self.logger.info(f"Enabling an enterprise feature")
self.rpk.create_role("XYZ")

first_upgraded = self.redpanda.nodes[0]
self.logger.info(
f"Upgrading node {first_upgraded} expecting it to crash")
installer.install([first_upgraded], latest_version)
# TODO: consider using maintenance mode to more closely match the recommended upgrade process
self.redpanda.stop_node(first_upgraded)

if clean_node:
self.logger.info(f"Cleaning node {first_upgraded}")
self.redpanda.remove_local_data(first_upgraded)
# We could decomissions the old node here for a cleaner test,
# but having 1 ghost node in a 5(+1) node cluster is still okay

self.redpanda.start_node(first_upgraded,
auto_assign_node_id=True,
omit_seeds_on_idx_one=False,
expect_fail=True)

self.logger.info(
f"Recover the node by downgrading, installing a test license and trying again {first_upgraded}"
)
license = get_cluster_license()
assert license, "Setting the license is required for this test"

output = self.rpk.license_set("", license)
assert "Successfully uploaded license" in output

self.logger.info(
f"Recover the node by downgrading and then upgrading again")
installer.install([first_upgraded], prev_version)
self.redpanda.start_node(first_upgraded,
auto_assign_node_id=True,
omit_seeds_on_idx_one=False)

installer.install([first_upgraded], latest_version)

# TODO: we could clean the data directory here again during restart to verify that
# we can start a fresh node based on the controller snapshot of other nodes

self.redpanda.restart_nodes(first_upgraded,
auto_assign_node_id=True,
omit_seeds_on_idx_one=False)

@cluster(num_nodes=5)
def test_enterprise_cluster_bootstrap(self):
self.logger.info(
"Bootstrapping an enterprise cluster without a license")
# TODO: test with other enterprise features as well
self.redpanda.add_extra_rp_conf(
{"partition_autobalancing_mode": "continuous"})
self.redpanda.start(nodes=self.redpanda.nodes,
omit_seeds_on_idx_one=False)

self.redpanda.wait_until(self.redpanda.healthy, 60, 1)

0 comments on commit 878a553

Please sign in to comment.