Skip to content

Commit

Permalink
admin: add a scrubbing metadata reset endpoint
Browse files Browse the repository at this point in the history
This commit introduces a new admin api endpoint which can be used to
reset the scrubbing metadata of one partition. It does this by
replicating the `reset_scrubbing_metadata` which was introduced in the
previous patch.
  • Loading branch information
Vlad Lazar committed Oct 23, 2023
1 parent 0dea418 commit f6bc4ed
Show file tree
Hide file tree
Showing 4 changed files with 91 additions and 0 deletions.
31 changes: 31 additions & 0 deletions src/v/redpanda/admin/api-doc/shadow_indexing.json
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,37 @@
]
}
]
},
{
"path": "/v1/cloud_storage/reset_scrubbing_metadata/{namespace}/{topic}/{partition}",
"operations": [
{
"method": "POST",
"summary": "Reset scrubbing related metadata and anomalies for given partition",
"operationId": "reset_scrubbing_metadata",
"nickname": "reset_scrubbing_metadata",
"parameters": [
{
"name": "namespace",
"in": "path",
"required": true,
"type": "string"
},
{
"name": "topic",
"in": "path",
"required": true,
"type": "string"
},
{
"name": "partition",
"in": "path",
"required": true,
"type": "integer"
}
]
}
]
}
],
"models": {
Expand Down
48 changes: 48 additions & 0 deletions src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -5595,6 +5595,50 @@ admin_server::get_cloud_storage_anomalies(
co_return map_anomalies_to_json(ntp, *initial_rev, *status);
}

ss::future<ss::json::json_return_type>
admin_server::reset_scrubbing_metadata(std::unique_ptr<ss::http::request> req) {
const model::ntp ntp = parse_ntp_from_request(
req->param, model::kafka_namespace);

if (need_redirect_to_leader(ntp, _metadata_cache)) {
throw co_await redirect_to_leader(*req, ntp);
}

const auto shard = _shard_table.local().shard_for(ntp);
if (!shard) {
throw ss::httpd::not_found_exception(fmt::format(
"{} could not be found on the node. Perhaps it has been moved "
"during the redirect.",
ntp));
}

auto status = co_await _partition_manager.invoke_on(
*shard, [&ntp, shard](const auto& pm) {
const auto& partitions = pm.partitions();
auto partition_iter = partitions.find(ntp);

if (partition_iter == partitions.end()) {
throw ss::httpd::not_found_exception(
fmt::format("{} could not be found on shard {}.", ntp, *shard));
}

auto archiver = partition_iter->second->archiver();
if (!archiver) {
throw ss::httpd::not_found_exception(
fmt::format("{} has no archiver on shard {}.", ntp, *shard));
}

return archiver.value().get().reset_scrubbing_metadata();
});

if (status != cluster::errc::success) {
throw ss::httpd::server_error_exception{
"Failed to replicate or apply scrubber metadata reset command"};
}

co_return ss::json::json_return_type(ss::json::json_void());
}

void admin_server::register_shadow_indexing_routes() {
register_route<superuser>(
ss::httpd::shadow_indexing_json::sync_local_state,
Expand Down Expand Up @@ -5648,6 +5692,10 @@ void admin_server::register_shadow_indexing_routes() {
register_route<user>(
ss::httpd::shadow_indexing_json::get_cloud_storage_anomalies,
[this](auto req) { return get_cloud_storage_anomalies(std::move(req)); });

register_route<user>(
ss::httpd::shadow_indexing_json::reset_scrubbing_metadata,
[this](auto req) { return reset_scrubbing_metadata(std::move(req)); });
}

constexpr std::string_view to_string_view(service_kind kind) {
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 @@ -459,6 +459,8 @@ class admin_server {
std::unique_ptr<ss::http::reply> rep);
ss::future<ss::json::json_return_type>
get_cloud_storage_anomalies(std::unique_ptr<ss::http::request>);
ss::future<ss::json::json_return_type>
reset_scrubbing_metadata(std::unique_ptr<ss::http::request>);

/// Self test routes
ss::future<ss::json::json_return_type>
Expand Down
10 changes: 10 additions & 0 deletions tests/rptest/services/admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -1112,3 +1112,13 @@ def get_cloud_storage_anomalies(self, namespace: str, topic: str,
return self._request(
"GET",
f"cloud_storage/anomalies/{namespace}/{topic}/{partition}").json()

def reset_scrubbing_metadata(self,
namespace: str,
topic: str,
partition: int,
node: Optional[ClusterNode] = None):
return self._request(
"POST",
f"cloud_storage/reset_scrubbing_metadata/{namespace}/{topic}/{partition}",
node=node)

0 comments on commit f6bc4ed

Please sign in to comment.