Skip to content

Commit

Permalink
Merge pull request #23368 from oleiman/vbotbuildovich/backport-23314-…
Browse files Browse the repository at this point in the history
…v24.2.x-830
  • Loading branch information
oleiman authored Sep 19, 2024
2 parents 5376554 + 430021c commit f65c3be
Show file tree
Hide file tree
Showing 12 changed files with 503 additions and 9 deletions.
1 change: 1 addition & 0 deletions src/v/cluster/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
34 changes: 27 additions & 7 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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]() {
Expand All @@ -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;
};
Expand All @@ -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() {
Expand Down
3 changes: 3 additions & 0 deletions src/v/cluster/feature_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -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 <seastar/core/abort_source.hh>
Expand Down Expand Up @@ -97,6 +98,8 @@ class feature_manager {

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

features::enterprise_feature_report report_enterprise_features() const;

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

Expand Down
17 changes: 17 additions & 0 deletions src/v/features/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)
8 changes: 8 additions & 0 deletions src/v/features/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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)
61 changes: 61 additions & 0 deletions src/v/features/enterprise_features.cc
Original file line number Diff line number Diff line change
@@ -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 <iostream>

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
54 changes: 54 additions & 0 deletions src/v/features/enterprise_features.h
Original file line number Diff line number Diff line change
@@ -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 <absl/container/flat_hash_set.h>
#include <boost/range/iterator_range.hpp>

#include <iosfwd>

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<license_required_feature>;
using range = boost::iterator_range<vtype::const_iterator>;

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
46 changes: 46 additions & 0 deletions src/v/redpanda/admin/api-doc/features.json
Original file line number Diff line number Diff line change
Expand Up @@ -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": {
Expand Down Expand Up @@ -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)"
}
}
}
}
}
60 changes: 60 additions & 0 deletions src/v/redpanda/admin/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -2247,6 +2248,60 @@ admin_server::put_license_handler(std::unique_ptr<ss::http::request> req) {
co_return ss::json::json_void();
}

ss::future<ss::json::json_return_type>
admin_server::get_enterprise_handler(std::unique_ptr<ss::http::request>) {
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<user>(
ss::httpd::features_json::get_features,
Expand Down Expand Up @@ -2341,6 +2396,11 @@ void admin_server::register_features_routes() {
[this](std::unique_ptr<ss::http::request> req) {
return put_license_handler(std::move(req));
});
register_route<user>(
ss::httpd::features_json::get_enterprise,
[this](std::unique_ptr<ss::http::request> req) {
return get_enterprise_handler(std::move(req));
});
}

ss::future<ss::json::json_return_type>
Expand Down
2 changes: 2 additions & 0 deletions src/v/redpanda/admin/server.h
Original file line number Diff line number Diff line change
Expand Up @@ -483,6 +483,8 @@ class admin_server {
put_feature_handler(std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
put_license_handler(std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
get_enterprise_handler(std::unique_ptr<ss::http::request>);

/// Broker routes

Expand Down
Loading

0 comments on commit f65c3be

Please sign in to comment.