Skip to content

Commit

Permalink
Merge pull request #24631 from nvartolomei/nv/cst-cache-ubsan
Browse files Browse the repository at this point in the history
cst/cache: ubsan fix, fix highly unlikely full cache trim if free disk info is delayed, more tests
  • Loading branch information
nvartolomei authored Dec 23, 2024
2 parents 3fc4ba8 + 84312e5 commit 06ae9f3
Show file tree
Hide file tree
Showing 4 changed files with 115 additions and 3 deletions.
13 changes: 11 additions & 2 deletions src/v/cloud_storage/cache_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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()) {
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion src/v/cloud_storage/cache_service.h
Original file line number Diff line number Diff line change
Expand Up @@ -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<uint64_t> _free_space;

ssx::semaphore _cleanup_sm{1, "cloud/cache"};
std::set<std::filesystem::path> _files_in_progress;
Expand Down
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
94 changes: 94 additions & 0 deletions tests/rptest/tests/cloud_storage_cache_test.py
Original file line number Diff line number Diff line change
@@ -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

0 comments on commit 06ae9f3

Please sign in to comment.