Skip to content

Commit

Permalink
cst/cache: allow admin api trim only if cache is initialized
Browse files Browse the repository at this point in the history
Fixes UBSAN assertion
  • Loading branch information
nvartolomei committed Dec 20, 2024
1 parent 1c93893 commit 6787f49
Show file tree
Hide file tree
Showing 2 changed files with 35 additions and 0 deletions.
9 changes: 9 additions & 0 deletions src/v/redpanda/admin/server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@
#include <seastar/core/map_reduce.hh>
#include <seastar/core/prometheus.hh>
#include <seastar/core/reactor.hh>
#include <seastar/core/shard_id.hh>
#include <seastar/core/sharded.hh>
#include <seastar/core/shared_ptr.hh>
#include <seastar/core/smp.hh>
Expand Down Expand Up @@ -4347,6 +4348,14 @@ admin_server::delete_cloud_storage_lifecycle(
ss::future<ss::json::json_return_type>
admin_server::post_cloud_storage_cache_trim(
std::unique_ptr<ss::http::request> req) {
co_await ss::smp::submit_to(ss::shard_id{0}, [this] {
if (!_cloud_storage_cache.local_is_initialized()) {
throw ss::httpd::bad_request_exception(
"Cloud Storage Cache is not available. Is cloud storage "
"enabled?");
}
});

auto max_objects = get_integer_query_param(*req, "objects");
auto max_bytes = static_cast<std::optional<size_t>>(
get_integer_query_param(*req, "bytes"));
Expand Down
26 changes: 26 additions & 0 deletions tests/rptest/tests/cloud_storage_cache_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
import pytest
from requests import HTTPError

from rptest.services.admin import Admin
from rptest.services.cluster import cluster
from rptest.tests.redpanda_test import RedpandaTest


class CloudStorageCacheAdminApisNoCacheTest(RedpandaTest):
"""
Test the Cloud Storage Cache Admin APIs when tiered storage is not configured.
"""
def __init__(self, test_context):
super().__init__(test_context, num_brokers=1)
self.admin = Admin(self.redpanda)

@cluster(num_nodes=1)
def test_admin_apis(self):
for node in self.redpanda.nodes:
with pytest.raises(HTTPError) as excinfo:
self.admin.cloud_storage_trim(byte_limit=None,
object_limit=None,
node=node)

assert "Cloud Storage Cache is not available. Is cloud storage enabled?" == excinfo.value.response.json(
)['message']

0 comments on commit 6787f49

Please sign in to comment.