diff --git a/src/v/cluster/CMakeLists.txt b/src/v/cluster/CMakeLists.txt index ea2f6ac3dc9d..7590aa74c70e 100644 --- a/src/v/cluster/CMakeLists.txt +++ b/src/v/cluster/CMakeLists.txt @@ -272,6 +272,7 @@ v_cc_library( v::cluster_topic_properties v::features v::version + v::enterprise_features ) add_subdirectory(tests) add_subdirectory(cloud_metadata/tests) diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 5335b105dff3..08e54c61eb05 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -195,6 +195,11 @@ ss::future<> feature_manager::stop() { } bool feature_manager::license_required_feature_enabled() const { + return report_enterprise_features().any(); +} + +features::enterprise_feature_report +feature_manager::report_enterprise_features() const { const auto& cfg = config::shard_local_cfg(); const auto& node_cfg = config::node(); auto has_gssapi = [&cfg]() { @@ -205,7 +210,7 @@ bool feature_manager::license_required_feature_enabled() const { return config::oidc_is_enabled_kafka() || config::oidc_is_enabled_http(); }; - auto has_schma_id_validation = [&cfg]() { + auto has_schema_id_validation = [&cfg]() { return cfg.enable_schema_id_validation() != pandaproxy::schema_registry::schema_id_validation_mode::none; }; @@ -219,12 +224,27 @@ bool feature_manager::license_required_feature_enabled() const { = n_roles >= 2 || (n_roles == 1 && !_role_store.local().contains(security::default_role)); - return cfg.audit_enabled || cfg.cloud_storage_enabled - || cfg.partition_autobalancing_mode - == model::partition_autobalancing_mode::continuous - || cfg.core_balancing_continuous() || has_gssapi() || has_oidc() - || has_schma_id_validation() || has_non_default_roles - || fips_enabled(); + features::enterprise_feature_report report; + report.set( + features::license_required_feature::audit_logging, cfg.audit_enabled()); + report.set( + features::license_required_feature::cloud_storage, + cfg.cloud_storage_enabled()); + report.set( + features::license_required_feature::partition_auto_balancing_continuous, + cfg.partition_autobalancing_mode() + == model::partition_autobalancing_mode::continuous); + report.set( + features::license_required_feature::core_balancing_continuous, + cfg.core_balancing_continuous()); + report.set(features::license_required_feature::gssapi, has_gssapi()); + report.set(features::license_required_feature::oidc, has_oidc()); + report.set( + features::license_required_feature::schema_id_validation, + has_schema_id_validation()); + report.set(features::license_required_feature::rbac, has_non_default_roles); + report.set(features::license_required_feature::fips, fips_enabled()); + return report; } ss::future<> feature_manager::maybe_log_license_check_info() { diff --git a/src/v/cluster/feature_manager.h b/src/v/cluster/feature_manager.h index 145ab52e7194..d8f3162b99c2 100644 --- a/src/v/cluster/feature_manager.h +++ b/src/v/cluster/feature_manager.h @@ -15,6 +15,7 @@ #include "cluster/feature_barrier.h" #include "cluster/fwd.h" #include "cluster/types.h" +#include "features/enterprise_features.h" #include "security/fwd.h" #include @@ -97,6 +98,8 @@ class feature_manager { ss::future update_license(security::license&& license); + features::enterprise_feature_report report_enterprise_features() const; + private: void update_node_version(model::node_id, cluster_version v); diff --git a/src/v/features/BUILD b/src/v/features/BUILD index fb5e353341f4..34c6cdf90471 100644 --- a/src/v/features/BUILD +++ b/src/v/features/BUILD @@ -34,3 +34,20 @@ redpanda_cc_library( "@seastar", ], ) + +redpanda_cc_library( + name = "enterprise_features", + srcs = [ + "enterprise_features.cc", + ], + hdrs = [ + "enterprise_features.h", + ], + include_prefix = "features", + visibility = ["//visibility:public"], + deps = [ + "//src/v/base", + "@abseil-cpp//absl/container:flat_hash_set", + "@boost//:range", + ], +) diff --git a/src/v/features/CMakeLists.txt b/src/v/features/CMakeLists.txt index 449e78492abf..3c0253f0b89d 100644 --- a/src/v/features/CMakeLists.txt +++ b/src/v/features/CMakeLists.txt @@ -14,4 +14,12 @@ v_cc_library( add_dependencies(v_features kafka_codegen_headers) +v_cc_library( + NAME enterprise_features + SRCS + enterprise_features.cc + DEPS + absl::flat_hash_set +) + add_subdirectory(tests) diff --git a/src/v/features/enterprise_features.cc b/src/v/features/enterprise_features.cc new file mode 100644 index 000000000000..7db399f42859 --- /dev/null +++ b/src/v/features/enterprise_features.cc @@ -0,0 +1,61 @@ +/* + * 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 + */ + +#include "enterprise_features.h" + +#include "base/vassert.h" + +#include + +namespace features { + +std::ostream& operator<<(std::ostream& os, license_required_feature f) { + switch (f) { + case license_required_feature::audit_logging: + return os << "audit_logging"; + case license_required_feature::cloud_storage: + return os << "cloud_storage"; + case license_required_feature::partition_auto_balancing_continuous: + return os << "partition_auto_balancing_continuous"; + case license_required_feature::core_balancing_continuous: + return os << "core_balancing_continuous"; + case license_required_feature::gssapi: + return os << "gssapi"; + case license_required_feature::oidc: + return os << "oidc"; + case license_required_feature::schema_id_validation: + return os << "schema_id_validation"; + case license_required_feature::rbac: + return os << "rbac"; + case license_required_feature::fips: + return os << "fips"; + } + __builtin_unreachable(); +} + +void enterprise_feature_report::set( + license_required_feature feat, bool enabled) { + auto insert = [feat](vtype& dest, const vtype& other) { + vassert( + !other.contains(feat), + "feature {{{}}} cannot be both enabled and disabled", + feat); + dest.insert(feat); + }; + + if (enabled) { + insert(_enabled, _disabled); + } else { + insert(_disabled, _enabled); + } +} + +} // namespace features diff --git a/src/v/features/enterprise_features.h b/src/v/features/enterprise_features.h new file mode 100644 index 000000000000..992b38f14a64 --- /dev/null +++ b/src/v/features/enterprise_features.h @@ -0,0 +1,54 @@ +/* + * 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 + */ + +#pragma once + +#include +#include + +#include + +namespace features { + +enum class license_required_feature { + audit_logging, + cloud_storage, + partition_auto_balancing_continuous, + core_balancing_continuous, + gssapi, + oidc, + schema_id_validation, + rbac, + fips, +}; + +std::ostream& operator<<(std::ostream&, license_required_feature); + +/** + * Thin wrapper around two sets to indicate the current state of enterprise + * features in the cluster. + */ +class enterprise_feature_report { + using vtype = absl::flat_hash_set; + using range = boost::iterator_range; + +public: + void set(license_required_feature feat, bool enabled); + range enabled() const { return _enabled; } + range disabled() const { return _disabled; } + bool any() const { return !_enabled.empty(); } + +private: + vtype _enabled; + vtype _disabled; +}; + +} // namespace features diff --git a/src/v/redpanda/admin/api-doc/features.json b/src/v/redpanda/admin/api-doc/features.json index 88c2ea9e5ada..1e7b994ca0db 100644 --- a/src/v/redpanda/admin/api-doc/features.json +++ b/src/v/redpanda/admin/api-doc/features.json @@ -90,6 +90,21 @@ } } ] + }, + { + "path": "/v1/features/enterprise", + "operations": [ + { + "method": "GET", + "summary": "Report license status and enterprise features in use", + "nickname": "get_enterprise", + "type": "enterprise_response", + "produces": [ + "application/json" + ], + "parameters": [] + } + ] } ], "models": { @@ -177,6 +192,37 @@ "description": "Contents of a valid, signed license if loaded" } } + }, + "enterprise_feature": { + "id": "enterprise_feature", + "description": "One enterprise feature by name, and whether it is in use", + "properties": { + "name": { + "type": "string" + }, + "enabled": { + "type": "boolean" + } + } + }, + "enterprise_response": { + "id": "enterprise_response", + "description": "Describe the license status and enterprise features in use.", + "properties": { + "license_status": { + "type": "string", + "enum": ["valid", "expired", "not_present"] + }, + "violation": { + "type": "boolean", + "description": "true if license_status is not 'valid' AND one or more enterprise features are enabled" + }, + "features": { + "type": "array", + "items": { "type": "enterprise_feature" }, + "description": "List of enterprise features (name and whether in use)" + } + } } } } diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index 9282086fba86..0a38c6998f8a 100644 --- a/src/v/redpanda/admin/server.cc +++ b/src/v/redpanda/admin/server.cc @@ -50,6 +50,7 @@ #include "config/endpoint_tls_config.h" #include "container/fragmented_vector.h" #include "container/lw_shared_container.h" +#include "features/enterprise_features.h" #include "features/feature_table.h" #include "finjector/hbadger.h" #include "finjector/stress_fiber.h" @@ -2247,6 +2248,60 @@ admin_server::put_license_handler(std::unique_ptr req) { co_return ss::json::json_void(); } +ss::future +admin_server::get_enterprise_handler(std::unique_ptr) { + if (!_controller->get_feature_table().local().is_active( + features::feature::license)) { + throw ss::httpd::bad_request_exception( + "Feature manager reports the cluster is not fully upgraded to " + "accept get enterprise requests"); + } + using status = ss::httpd::features_json::enterprise_response:: + enterprise_response_license_status; + + const auto& license + = _controller->get_feature_table().local().get_license(); + auto license_status = [&license]() { + auto present = license.has_value(); + auto exp = present && license.value().is_expired(); + if (exp) { + return status::expired; + } + if (present) { + return status::valid; + } + return status::not_present; + }(); + + auto& mgr = _controller->get_feature_manager(); + const auto report = co_await mgr.invoke_on( + cluster::feature_manager::backend_shard, + [](const cluster::feature_manager& fm) { + return fm.report_enterprise_features(); + }); + + ss::httpd::features_json::enterprise_response res; + res.license_status = license_status; + res.violation = license_status != status::valid && report.any(); + auto insert_feature = + [&res](features::license_required_feature feat, bool enabled) { + ss::httpd::features_json::enterprise_feature elt; + elt.name = fmt::format("{}", feat); + elt.enabled = enabled; + res.features.push(elt); + }; + + for (auto feat : report.enabled()) { + insert_feature(feat, true); + } + + for (auto feat : report.disabled()) { + insert_feature(feat, false); + } + + co_return ss::json::json_return_type{res}; +} + void admin_server::register_features_routes() { register_route( ss::httpd::features_json::get_features, @@ -2341,6 +2396,11 @@ void admin_server::register_features_routes() { [this](std::unique_ptr req) { return put_license_handler(std::move(req)); }); + register_route( + ss::httpd::features_json::get_enterprise, + [this](std::unique_ptr req) { + return get_enterprise_handler(std::move(req)); + }); } ss::future diff --git a/src/v/redpanda/admin/server.h b/src/v/redpanda/admin/server.h index 512b2da17f94..1be803acb242 100644 --- a/src/v/redpanda/admin/server.h +++ b/src/v/redpanda/admin/server.h @@ -483,6 +483,8 @@ class admin_server { put_feature_handler(std::unique_ptr); ss::future put_license_handler(std::unique_ptr); + ss::future + get_enterprise_handler(std::unique_ptr); /// Broker routes diff --git a/tests/rptest/services/admin.py b/tests/rptest/services/admin.py index 4b0634d1aaa0..d41fa1f510ef 100644 --- a/tests/rptest/services/admin.py +++ b/tests/rptest/services/admin.py @@ -355,6 +355,12 @@ class MigrationAction(Enum): cancel = "cancel" +class EnterpriseLicenseStatus(Enum): + valid = "valid" + expired = "expired" + not_present = "not_present" + + class Admin: """ Wrapper for Redpanda admin REST API. @@ -818,6 +824,9 @@ def get_license(self, node=None, timeout=None): def put_license(self, license): return self._request("PUT", "features/license", data=license) + def get_enterprise_features(self): + return self._request("GET", "features/enterprise") + def get_loggers(self, node): """ Get the names of all loggers. @@ -1181,7 +1190,8 @@ def delete_role(self, role: str, delete_acls: Optional[bool] = None): def list_roles(self, filter: Optional[str] = None, principal: Optional[str] = None, - principal_type: Optional[str] = None): + principal_type: Optional[str] = None, + node=None): params = {} if filter is not None: params['filter'] = filter @@ -1189,7 +1199,7 @@ def list_roles(self, params['principal'] = principal if principal_type is not None: params['principal_type'] = principal_type - return self._request("get", "security/roles", params=params) + return self._request("get", "security/roles", params=params, node=node) def update_role_members(self, role: str, diff --git a/tests/rptest/tests/enterprise_features_license_test.py b/tests/rptest/tests/enterprise_features_license_test.py new file mode 100644 index 000000000000..63d9d620bd94 --- /dev/null +++ b/tests/rptest/tests/enterprise_features_license_test.py @@ -0,0 +1,212 @@ +import time +import json +from enum import IntEnum + +from rptest.utils.rpenv import sample_license +from rptest.services.admin import Admin, EnterpriseLicenseStatus, RolesList, RoleDescription +from rptest.services.redpanda import RESTART_LOG_ALLOW_LIST, SecurityConfig, SchemaRegistryConfig +from rptest.tests.redpanda_test import RedpandaTest +from rptest.services.cluster import cluster +from rptest.services.redpanda_installer import RedpandaInstaller, wait_for_num_versions +from rptest.util import expect_exception +from rptest.utils.mode_checks import skip_fips_mode + +from ducktape.errors import TimeoutError as DucktapeTimeoutError +from ducktape.utils.util import wait_until +from ducktape.mark import parametrize, matrix +from rptest.util import wait_until_result + + +class EnterpriseFeaturesTestBase(RedpandaTest): + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + self.admin = Admin(self.redpanda) + self.installer = self.redpanda._installer + + def setUp(self): + super().setUp() + + +class Features(IntEnum): + audit_logging = 0 + cloud_storage = 1 + partition_auto_balancing_continuous = 2 + core_balancing_continuous = 3 + gssapi = 4 + oidc = 5 + schema_id_validation = 6 + rbac = 7 + fips = 8 + + +SKIP_FEATURES = [ + Features.audit_logging, # NOTE(oren): omit due to shutdown issues + Features. + cloud_storage, # TODO(oren): initially omitted because it's a bit complicated to initialize infra +] + + +class EnterpriseFeaturesTest(EnterpriseFeaturesTestBase): + def __init__(self, *args, **kwargs): + super().__init__( + *args, + num_brokers=3, + schema_registry_config=SchemaRegistryConfig(), + **kwargs, + ) + + self.su, self.pw, self.mech = self.redpanda.SUPERUSER_CREDENTIALS + + self.security = SecurityConfig() + self.security.enable_sasl = True + self.kafka_enable_authorization = True + self.endpoint_authn_method = 'sasl' + + self.redpanda.set_security_settings(self.security) + + def _put_license(self): + license = sample_license() + if license is None: + return None + assert self.admin.put_license(license).status_code == 200, \ + "License update failed" + + def obtain_license(node): + lic = self.admin.get_license(node=node) + return (lic is not None and lic['loaded'] is True, lic) + + result = None + for n in self.redpanda.nodes: + resp = wait_until_result(lambda: obtain_license(n), + timeout_sec=5, + backoff_sec=1) + assert resp['license'] is not None, "License upload failed!" + result = resp['license'] if result is None else result + + return result + + @skip_fips_mode + @cluster(num_nodes=3) + @matrix(with_license=[ + True, + False, + ]) + def test_get_enterprise(self, with_license): + if with_license: + lic = self._put_license() + if lic is None: + self.logger.info( + "Skipping test, REDPANDA_SAMPLE_LICENSE env var not found") + return + + rsp = self.admin.get_enterprise_features().json() + + expect_status = EnterpriseLicenseStatus.valid if with_license else EnterpriseLicenseStatus.not_present + status = rsp.get('license_status', None) + assert type(status) == str, f"Ill-formed license_status {type(status)}" + try: + assert EnterpriseLicenseStatus(status) == expect_status, \ + f"Unexpected status '{status}'" + except ValueError: + assert False, f"Unexpected status in response: '{status}'" + + violation = rsp.get('violation', None) + assert type(violation) == bool, \ + f"Ill-formed violation flag {type(violation)}" + assert not violation, "Config unexpectedly in violation" + + features_rsp = [f['name'] for f in rsp.get('features', [])] + + assert set([f.name for f in Features]) == set(features_rsp), \ + f"Unexpected feature list: {json.dumps(features_rsp, indent=1)}" + + @skip_fips_mode + @cluster(num_nodes=3) + @matrix(feature=[f for f in Features if f not in SKIP_FEATURES], + with_license=[ + True, + False, + ]) + def test_license_violation(self, feature, with_license): + if with_license: + lic = self._put_license() + if lic is None: + self.logger.info( + "Skipping test, REDPANDA_SAMPLE_LICENSE env var not found") + return + + if feature == Features.audit_logging: + self.redpanda.set_cluster_config( + { + 'audit_enabled': True, + }, + expect_restart=True, + ) + elif feature == Features.cloud_storage: + self.redpanda.set_cluster_config({'cloud_storage_enabled': 'true'}, + expect_restart=True) + elif feature == Features.partition_auto_balancing_continuous: + self.redpanda.set_cluster_config( + {'partition_autobalancing_mode': 'continuous'}) + elif feature == Features.core_balancing_continuous: + self.redpanda.set_cluster_config( + {'core_balancing_continuous': 'true'}) + elif feature == Features.gssapi: + self.redpanda.set_cluster_config( + {'sasl_mechanisms': ['SCRAM', 'GSSAPI']}) + elif feature == Features.oidc: + self.redpanda.set_cluster_config( + {'sasl_mechanisms': ['SCRAM', 'OAUTHBEARER']}) + elif feature == Features.schema_id_validation: + self.redpanda.set_cluster_config( + {'enable_schema_id_validation': 'compat'}) + elif feature == Features.rbac: + + # NOTE(oren): make sure the role has propagated to every node since we don't know + # where the get_enterprise request will go + def has_role(r: str): + return all( + len( + RolesList.from_response( + self.admin.list_roles(filter=r, node=n)).roles) > 0 + for n in self.redpanda.nodes) + + self.admin.create_role('dummy') + wait_until(lambda: has_role('dummy'), + timeout_sec=30, + backoff_sec=1) + elif feature == Features.fips: + self.redpanda.rolling_restart_nodes( + self.redpanda.nodes, + override_cfg_params={ + 'fips_mode': + 'permissive', + "openssl_config_file": + self.redpanda.get_openssl_config_file_path(), + "openssl_module_directory": + self.redpanda.get_openssl_modules_directory() + }) + else: + assert False, f"Unexpected feature={feature}" + + rsp = self.admin.get_enterprise_features().json() + enabled = {f['name']: f['enabled'] for f in rsp['features']} + for f in Features: + if f == feature: + assert enabled[f.name], \ + f"expected {f} enabled, got {json.dumps(enabled, indent=1)}" + else: + assert not enabled[f.name], \ + f"expected {f} not enabled, got {json.dumps(enabled, indent=1)}" + + expect_status = EnterpriseLicenseStatus.valid if with_license else EnterpriseLicenseStatus.not_present + status_str = rsp.get('license_status') + try: + status = EnterpriseLicenseStatus(status_str) + assert status == expect_status, f"Unexpected license status: {status} (expected {expect_status})" + except ValueError: + assert False, f"Unexpected status in response: {status_str}" + + violation = rsp.get('violation') + assert violation == (not with_license), \ + f"Expected{' no' if with_license else ''} enterprise license violation, got violation='{violation}'"