Skip to content

Commit

Permalink
admin: add GET /v1/cluster/partitions handler
Browse files Browse the repository at this point in the history
Viewing cluster-wide info for all partitions + disabled filter.
  • Loading branch information
ztlpn committed Oct 26, 2023
1 parent abb63b8 commit 825b518
Show file tree
Hide file tree
Showing 3 changed files with 97 additions and 0 deletions.
31 changes: 31 additions & 0 deletions src/v/redpanda/admin/api-doc/cluster.json
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,37 @@
}
]
},
{
"path": "/v1/cluster/partitions",
"operations": [
{
"method": "GET",
"summary": "Get cluster-level metadata for all partitions",
"nickname": "get_cluster_partitions",
"type": "array",
"items": {
"type": "cluster_partition"
},
"produces": [
"application/json"
],
"parameters": [
{
"name": "disabled",
"in": "query",
"required": false,
"type": "boolean"
},
{
"name": "with_internal",
"in": "query",
"required": false,
"type": "boolean"
}
]
}
]
},
{
"path": "/v1/cluster/partitions/{namespace}/{topic}",
"operations": [
Expand Down
64 changes: 64 additions & 0 deletions src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -5106,6 +5106,65 @@ fragmented_vector<cluster_partition_info> topic2cluster_partitions(

} // namespace

ss::future<ss::json::json_return_type>
admin_server::get_cluster_partitions_handler(
std::unique_ptr<ss::http::request> req) {
std::optional<bool> disabled_filter;
if (req->query_parameters.contains("disabled")) {
disabled_filter = get_boolean_query_param(*req, "disabled");
}

bool with_internal = get_boolean_query_param(*req, "with_internal");

const auto& topics_state = _controller->get_topics_state().local();

fragmented_vector<model::topic_namespace> topics;
auto fill_topics = [&](const auto& map) {
for (const auto& [ns_tp, _] : map) {
if (!with_internal && !model::is_user_topic(ns_tp)) {
continue;
}
topics.push_back(ns_tp);
}
};

if (disabled_filter && *disabled_filter) {
// optimization: if disabled filter is on, iterate only over disabled
// topics;
fill_topics(topics_state.get_disabled_partitions());
} else {
fill_topics(topics_state.topics_map());
}

std::sort(topics.begin(), topics.end());

ss::chunked_fifo<cluster_partition_info> partitions;
for (const auto& ns_tp : topics) {
auto topic_it = topics_state.topics_map().find(ns_tp);
if (topic_it == topics_state.topics_map().end()) {
// probably got deleted while we were iterating.
continue;
}

auto topic_partitions = topic2cluster_partitions(
ns_tp,
topic_it->second.get_assignments(),
topics_state.get_disabled_partitions(),
disabled_filter);

std::move(
topic_partitions.begin(),
topic_partitions.end(),
std::back_inserter(partitions));

co_await ss::coroutine::maybe_yield();
}

co_return ss::json::json_return_type(ss::json::stream_range_as_array(
lw_shared_container{std::move(partitions)},
[](const auto& p) { return p.to_json(); }));
}

ss::future<ss::json::json_return_type>
admin_server::get_cluster_partitions_topic_handler(
std::unique_ptr<ss::http::request> req) {
Expand Down Expand Up @@ -5261,6 +5320,11 @@ void admin_server::register_cluster_partitions_routes() {
std::move(req));
});

register_route<user>(
ss::httpd::cluster_json::get_cluster_partitions,
[this](std::unique_ptr<ss::http::request> req) {
return get_cluster_partitions_handler(std::move(req));
});
register_route<user>(
ss::httpd::cluster_json::get_cluster_partitions_topic,
[this](std::unique_ptr<ss::http::request> req) {
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 @@ -489,6 +489,8 @@ class admin_server {
ss::future<ss::json::json_return_type>
post_cluster_partitions_topic_partition_handler(
std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
get_cluster_partitions_handler(std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
get_cluster_partitions_topic_handler(std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
Expand Down

0 comments on commit 825b518

Please sign in to comment.