Skip to content

Commit

Permalink
Merge pull request #23761 from oleiman/vbotbuildovich/backport-23314-…
Browse files Browse the repository at this point in the history
…v24.1.x-583

[v24.1.x] [CORE-7152] Admin: Introduce  endpoint
  • Loading branch information
oleiman authored Oct 17, 2024
2 parents 488947b + ef43c87 commit 169ac87
Show file tree
Hide file tree
Showing 11 changed files with 499 additions and 28 deletions.
1 change: 1 addition & 0 deletions src/v/cluster/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,7 @@ v_cc_library(
v::cloud_storage
v::features
v::version
v::enterprise_features
)
add_subdirectory(tests)
add_subdirectory(cloud_metadata/tests)
Expand Down
71 changes: 45 additions & 26 deletions src/v/cluster/feature_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,50 @@ ss::future<> feature_manager::stop() {
co_await _gate.close();
}

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();
auto has_gssapi = [&cfg]() {
return absl::c_any_of(
cfg.sasl_mechanisms(), [](const auto& m) { return m == "GSSAPI"; });
};
auto has_oidc = []() {
return config::oidc_is_enabled_kafka()
|| config::oidc_is_enabled_http();
};

auto has_schema_id_validation = [&cfg]() {
return cfg.enable_schema_id_validation()
!= pandaproxy::schema_registry::schema_id_validation_mode::none;
};
auto n_roles = _role_store.local().size();
auto has_non_default_roles
= n_roles >= 2
|| (n_roles == 1 && !_role_store.local().contains(security::default_role));

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::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);
return report;
}

ss::future<> feature_manager::maybe_log_license_check_info() {
auto license_check_retry = std::chrono::seconds(60 * 5);
auto interval_override = std::getenv(
Expand All @@ -213,32 +257,7 @@ ss::future<> feature_manager::maybe_log_license_check_info() {
}
}
if (_feature_table.local().is_active(features::feature::license)) {
const auto& cfg = config::shard_local_cfg();
auto has_gssapi = [&cfg]() {
return absl::c_any_of(cfg.sasl_mechanisms(), [](const auto& m) {
return m == "GSSAPI";
});
};
auto has_oidc = []() {
return config::oidc_is_enabled_kafka()
|| config::oidc_is_enabled_http();
};
auto has_schma_id_validation = [&cfg]() {
return cfg.enable_schema_id_validation()
!= pandaproxy::schema_registry::schema_id_validation_mode::
none;
};
auto n_roles = _role_store.local().size();
auto has_non_default_roles
= n_roles >= 2
|| (n_roles == 1 && !_role_store.local().contains(security::default_role));

if (
cfg.audit_enabled || cfg.cloud_storage_enabled
|| cfg.partition_autobalancing_mode
== model::partition_autobalancing_mode::continuous
|| has_gssapi() || has_oidc() || has_schma_id_validation()
|| has_non_default_roles) {
if (license_required_feature_enabled()) {
const auto& license = _feature_table.local().get_license();
if (!license || license->is_expired()) {
vlog(
Expand Down
23 changes: 23 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 Expand Up @@ -141,6 +144,26 @@ class feature_manager {
std::vector<std::reference_wrapper<const features::feature_spec>>
auto_activate_features(cluster_version, cluster_version);

// This method returns true if there are any feature(s) enabled that require
// the enterprise license. Currently the following features require a
// license:
// +-------------+---------------------------------+---------------+
// | Config Type | Config Name | Value(s) |
// +-------------+---------------------------------+---------------+
// | Cluster | `audit_enabled` | `true` |
// | Cluster | `cloud_storage_enabled` | `true` |
// | Cluster | `partition_auto_balancing_mode` | `continuous` |
// | Cluster | `core_balancing_continous` | `true` |
// | Cluster | `sasl_mechanisms` | `GSSAPI` |
// | Cluster | `sasl_mechanisms` | `OAUTHBEARER` |
// | Cluster | `http_authentication` | `OIDC` |
// | Cluster | `enable_schema_id_validation` | `redpanda` |
// | Cluster | `enable_schema_id_validation` | `compat` |
// +-------------+---------------------------------+---------------+
//
// Also if there are any non default roles in the role store.
bool license_required_feature_enabled() const;

ss::sharded<controller_stm>& _stm;
ss::sharded<ss::abort_source>& _as;
ss::gate _gate;
Expand Down
8 changes: 8 additions & 0 deletions src/v/features/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -16,4 +16,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)
57 changes: 57 additions & 0 deletions src/v/features/enterprise_features.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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::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";
}
__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
52 changes: 52 additions & 0 deletions src/v/features/enterprise_features.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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,
gssapi,
oidc,
schema_id_validation,
rbac,
};

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)"
}
}
}
}
}
Loading

0 comments on commit 169ac87

Please sign in to comment.