diff --git a/src/v/cloud_storage/cache_service.cc b/src/v/cloud_storage/cache_service.cc index afc20f8f16b8f..6cb0614200e1e 100644 --- a/src/v/cloud_storage/cache_service.cc +++ b/src/v/cloud_storage/cache_service.cc @@ -356,6 +356,10 @@ ss::future<> cache::trim( target_size *= _cache_size_low_watermark; } + if (!_free_space.has_value()) { + throw std::runtime_error("Free space information is not available."); + } + // In the extreme case where even trimming to the low watermark wouldn't // free enough space to enable writing to the cache, go even further. if (_free_space < config::shard_local_cfg().storage_min_free_bytes()) { @@ -1533,7 +1537,7 @@ bool cache::may_exceed_limits(uint64_t bytes, size_t objects) { auto would_fit_in_cache = _current_cache_size + bytes <= _max_bytes; - return !_block_puts && _free_space > bytes * 10 + return !_block_puts && _free_space.value_or(0) > bytes * 10 && _current_cache_objects + _reserved_cache_objects + objects < _max_objects() && !would_fit_in_cache; @@ -1871,7 +1875,12 @@ ss::future<> cache::do_reserve_space(uint64_t bytes, size_t objects) { // all skipping the cache limit based on the same apparent // free bytes. This counter will get reset to ground // truth the next time we get a disk status notification. - _free_space -= bytes; + if (unlikely(!_free_space.has_value())) { + throw std::runtime_error( + "Free space information must be available by the " + "time we execute this code path"); + } + *_free_space -= bytes; break; } else { // No allowance, and the disk does not have a lot of diff --git a/src/v/cloud_storage/cache_service.h b/src/v/cloud_storage/cache_service.h index 885a464f442f6..386aac358fc1d 100644 --- a/src/v/cloud_storage/cache_service.h +++ b/src/v/cloud_storage/cache_service.h @@ -335,7 +335,7 @@ class cache /// and have to decide whether to block writes, or exceed our configured /// limit. /// (shard 0 only) - uint64_t _free_space{0}; + std::optional _free_space; ssx::semaphore _cleanup_sm{1, "cloud/cache"}; std::set _files_in_progress; diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index bb1b18b5336a0..7c381f2770e7d 100644 --- a/src/v/redpanda/admin/server.cc +++ b/src/v/redpanda/admin/server.cc @@ -103,6 +103,7 @@ #include #include #include +#include #include #include #include @@ -4347,6 +4348,14 @@ admin_server::delete_cloud_storage_lifecycle( ss::future admin_server::post_cloud_storage_cache_trim( std::unique_ptr 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>( get_integer_query_param(*req, "bytes")); diff --git a/tests/rptest/tests/cloud_storage_cache_test.py b/tests/rptest/tests/cloud_storage_cache_test.py new file mode 100644 index 0000000000000..c46524d9f3883 --- /dev/null +++ b/tests/rptest/tests/cloud_storage_cache_test.py @@ -0,0 +1,94 @@ +import pytest +from ducktape.utils.util import wait_until +from requests import HTTPError + +from rptest.services.admin import Admin +from rptest.services.cluster import cluster +from rptest.services.redpanda import SISettings +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'] + + +class CloudStorageCacheAdminApisTest(RedpandaTest): + """ + Test the Cloud Storage Cache Admin APIs when tiered storage is configured. + """ + def __init__(self, test_context): + super().__init__(test_context, + num_brokers=1, + si_settings=SISettings(test_context)) + self.admin = Admin(self.redpanda) + + def setUp(self): + pass + + @cluster(num_nodes=1, + log_allow_list=["Free space information is not available"]) + def test_admin_apis(self): + num_objects = 100 + object_size = 4096 + + for node in self.redpanda.nodes: + node.account.ssh( + f"mkdir -p {self.redpanda.cache_dir} ; " + f"for n in `seq 1 {num_objects}`; do " + f"dd if=/dev/urandom bs={object_size} count=1 of={self.redpanda.cache_dir}/garbage_$n.bin ; done", + ) + + self.redpanda.start(clean_nodes=False) + + # Assert initial conditions. + usage = self.admin.get_local_storage_usage(node) + assert usage['cloud_storage_cache_objects'] == num_objects, usage + + # Trim with default settings. Nothing should be trimmed as we are well + # below reasonable limits. + # Wrapped with wait_until as it will fail until a background fiber + # updates information about free disk space. + wait_until(lambda: self.admin.cloud_storage_trim( + byte_limit=None, object_limit=None, node=node), + timeout_sec=30, + backoff_sec=1, + retry_on_exc=True) + + usage = self.admin.get_local_storage_usage(node) + assert usage['cloud_storage_cache_objects'] == num_objects, usage + + # Trim with byte limit. We should trim half of objects. + self.admin.cloud_storage_trim(byte_limit=object_size * 50, + object_limit=None, + node=node) + usage = self.admin.get_local_storage_usage(node) + + # Although we set the limit to size of 50 objects, the value + # gets multiplied by 0.8 internally so we end up with 40 objects left. + assert usage['cloud_storage_cache_objects'] == 40, usage + + # Trim with object limit. We should trim 20 objects. + self.admin.cloud_storage_trim(byte_limit=None, + object_limit=20, + node=node) + usage = self.admin.get_local_storage_usage(node) + + # Although we set the limit to 20 objects, the value + # gets multiplied by 0.8 internally so we end up with 16 objects left. + assert usage['cloud_storage_cache_objects'] == 16, usage