Skip to content

Commit

Permalink
Merge pull request #5486 from graphcareful/license-bugfix
Browse files Browse the repository at this point in the history
Upgrade testing for license checks
  • Loading branch information
Rob Blafford committed Jul 21, 2022
2 parents 374a4d2 + bd87380 commit fdc9cab
Show file tree
Hide file tree
Showing 6 changed files with 194 additions and 41 deletions.
57 changes: 42 additions & 15 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -163,22 +163,49 @@ ss::future<> feature_manager::stop() {
}

ss::future<> feature_manager::maybe_log_license_check_info() {
static constexpr std::chrono::seconds license_check_retry = 5min;
const auto& cfg = config::shard_local_cfg();
std::stringstream warn_ss;
if (cfg.cloud_storage_enabled) {
fmt::print(warn_ss, "{}", "Tired Storage(cloud_storage)");
}
const auto& warn_log = warn_ss.str();
if (!warn_log.empty()) {
const auto& license = _feature_table.local().get_license();
if (!license || license->is_expired()) {
auto license_check_retry = std::chrono::seconds(60 * 5);
auto interval_override = std::getenv(
"__REDPANDA_LICENSE_CHECK_INTERVAL_SEC");
if (interval_override != nullptr) {
try {
license_check_retry = std::min(
std::chrono::seconds{license_check_retry},
std::chrono::seconds{std::stoi(interval_override)});
vlog(
clusterlog.info,
"Overriding default license log annoy interval to: {}s",
license_check_retry.count());
} catch (...) {
vlog(
clusterlog.warn,
"Enterprise feature(s) {} detected as enabled without a valid "
"license, please contact support and/or upload a valid redpanda "
"license",
warn_log);
clusterlog.error,
"Invalid license check interval override '{}'",
interval_override);
}
}
if (_feature_table.local().is_active(feature::license)) {
const auto& cfg = config::shard_local_cfg();
std::stringstream warn_ss;
if (cfg.cloud_storage_enabled) {
fmt::print(warn_ss, "{}", "Tiered Storage(cloud_storage)");
}
if (
cfg.partition_autobalancing_mode
== model::partition_autobalancing_mode::continuous) {
fmt::print(warn_ss, "{} & ", "Continuous partition autobalancing");
}
const auto& warn_log = warn_ss.str();
if (!warn_log.empty()) {
const auto& license = _feature_table.local().get_license();
if (!license || license->is_expired()) {
vlog(
clusterlog.warn,
"Enterprise feature(s) ({}) detected as enabled without a "
"valid "
"license, please contact support and/or upload a valid "
"redpanda "
"license",
warn_log);
}
}
}
try {
Expand Down
18 changes: 18 additions & 0 deletions tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -1772,3 +1772,21 @@ def save_executable(self):
else:
self._saved_executable = True
self._context.log_collect['executable', self] = True

def search_log(self, pattern):
"""
Test helper for grepping the redpanda log
:return: true if any instances of `pattern` found
"""
for node in self.nodes:
for line in node.account.ssh_capture(
f"grep \"{pattern}\" {RedpandaService.STDOUT_STDERR_CAPTURE} || true"
):
# We got a match
self.logger.debug(
f"Found {pattern} on node {node.name}: {line}")
return True

# Fall through, no matches
return False
27 changes: 4 additions & 23 deletions tests/rptest/tests/cluster_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,25 +33,6 @@
SECRET_CONFIG_NAMES = frozenset(["cloud_storage_secret_key"])


def search_log(redpanda, pattern):
"""
Test helper for grepping the redpanda log
:return: true if any instances of `pattern` found
"""
for node in redpanda.nodes:
for line in node.account.ssh_capture(
f"grep \"{pattern}\" {redpanda.STDOUT_STDERR_CAPTURE} || true"
):
# We got a match
redpanda.logger.debug(
f"Found {pattern} on node {node.name}: {line}")
return True

# Fall through, no matches
return False


class ClusterConfigUpgradeTest(RedpandaTest):
def __init__(self, *args, **kwargs):
super().__init__(*args, extra_rp_conf={}, **kwargs)
Expand Down Expand Up @@ -84,8 +65,8 @@ def test_upgrade_redpanda_yaml(self):
self.redpanda.restart_nodes(
[node], override_cfg_params={'delete_retention_ms': '1234'})
assert admin.get_cluster_config()['delete_retention_ms'] == 9876
assert search_log(self.redpanda,
"Ignoring value for 'delete_retention_ms'")
assert self.redpanda.search_log(
"Ignoring value for 'delete_retention_ms'")


class ClusterConfigTest(RedpandaTest):
Expand Down Expand Up @@ -948,11 +929,11 @@ def set_and_search(key, value, expect_log):
self._wait_for_version_sync(patch_result['config_version'])

# Check value was/was not printed to log while applying
assert search_log(self.redpanda, value) is expect_log
assert self.redpanda.search_log(value) is expect_log

# Check we do/don't print on next startup
self.redpanda.restart_nodes(self.redpanda.nodes)
assert search_log(self.redpanda, value) is expect_log
assert self.redpanda.search_log(value) is expect_log

# Default valued secrets are still shown.
self._check_value_everywhere("cloud_storage_secret_key", None)
Expand Down
5 changes: 2 additions & 3 deletions tests/rptest/tests/cluster_features_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
import time
import datetime

from rptest.utils.rpenv import sample_license
from rptest.services.admin import Admin
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
from rptest.tests.redpanda_test import RedpandaTest
Expand Down Expand Up @@ -152,10 +153,8 @@ def test_license_upload_and_query(self):
"""
Test uploading and retrieval of license
"""
license = os.environ.get("REDPANDA_SAMPLE_LICENSE", None)
license = sample_license()
if license is None:
is_ci = os.environ.get("CI", "false")
assert is_ci == "false"
self.logger.info(
"Skipping test, REDPANDA_SAMPLE_LICENSE env var not found")
return
Expand Down
103 changes: 103 additions & 0 deletions tests/rptest/tests/license_upgrade_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
# Copyright 2022 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 os
import re
import time

from ducktape.utils.util import wait_until
from rptest.utils.rpenv import sample_license
from rptest.services.admin import Admin
from ducktape.utils.util import wait_until
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda import SISettings
from rptest.services.cluster import cluster
from requests.exceptions import HTTPError
from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST
from rptest.services.redpanda_installer import RedpandaInstaller, wait_for_num_versions


class UpgradeToLicenseChecks(RedpandaTest):
"""
Test that ensures the licensing work does not incorrectly print license
enforcement errors during upgrade when a guarded feature is already
enabled. Also tests that the license can only be uploaded once the cluster
has completed upgrade to the latest version.
"""
LICENSE_CHECK_INTERVAL_SEC = 1

def __init__(self, test_context):
# Setting 'si_settings' enables a licensed feature, however at v22.1.4 there
# are no license checks present. This test verifies behavior between versions
# of redpanda that do and do not have the licensing feature built-in.
super(UpgradeToLicenseChecks, self).__init__(test_context=test_context,
num_brokers=3,
si_settings=SISettings())
self.installer = self.redpanda._installer
self.admin = Admin(self.redpanda)

def setUp(self):
self.installer.install(self.redpanda.nodes, (22, 1, 4))
super(UpgradeToLicenseChecks, self).setUp()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_basic_upgrade(self):
# Modified environment variables apply to processes restarted from this point onwards
self.redpanda.set_environment({
'__REDPANDA_LICENSE_CHECK_INTERVAL_SEC':
f'{UpgradeToLicenseChecks.LICENSE_CHECK_INTERVAL_SEC}'
})

license = sample_license()
if license is None:
self.logger.info(
"Skipping test, REDPANDA_SAMPLE_LICENSE env var not found")
return

unique_versions = wait_for_num_versions(self.redpanda, 1)
assert 'v22.1.4' in unique_versions, unique_versions

# These logs can't exist in v22.1.4 but double check anyway...
assert self.redpanda.search_log("Enterprise feature(s).*") is False

# Update one node to newest version
self.installer.install([self.redpanda.nodes[0]],
RedpandaInstaller.HEAD)
self.redpanda.restart_nodes([self.redpanda.nodes[0]])
unique_versions = wait_for_num_versions(self.redpanda, 2)

try:
# Ensure a valid license cannot be uploaded in this cluster state
self.admin.put_license(license)
assert False
except HTTPError as e:
assert e.response.status_code == 400

# Ensure the log is not written, if the fiber was enabled a log should
# appear within one interval of the license check fiber
time.sleep(UpgradeToLicenseChecks.LICENSE_CHECK_INTERVAL_SEC * 2)
assert self.redpanda.search_log("Enterprise feature(s).*") is False

# Install new version on all nodes
self.installer.install(self.redpanda.nodes, RedpandaInstaller.HEAD)

# Restart nodes 2 and 3
self.redpanda.restart_nodes(
[self.redpanda.nodes[1], self.redpanda.nodes[2]])
_ = wait_for_num_versions(self.redpanda, 1)

# Assert that the log was found
wait_until(
lambda: self.redpanda.search_log("Enterprise feature(s).*"),
timeout_sec=UpgradeToLicenseChecks.LICENSE_CHECK_INTERVAL_SEC * 4,
backoff_sec=1,
err_msg="Timeout waiting for enterprise nag log")

# Install license
assert self.admin.put_license(license).status_code == 200
25 changes: 25 additions & 0 deletions tests/rptest/utils/rpenv.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
# Copyright 2022 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

# Utilities for checking the environment of a test

import os


def sample_license():
"""
Returns the sample license from the env if it exists, asserts if its
missing and the environment is CI
"""
license = os.environ.get("REDPANDA_SAMPLE_LICENSE", None)
if license is None:
is_ci = os.environ.get("CI", "false")
assert is_ci == "false"
return None
return license

0 comments on commit fdc9cab

Please sign in to comment.