Skip to content

Commit 06ae9f3

Browse files
authored
Merge pull request #24631 from nvartolomei/nv/cst-cache-ubsan
cst/cache: ubsan fix, fix highly unlikely full cache trim if free disk info is delayed, more tests
2 parents 3fc4ba8 + 84312e5 commit 06ae9f3

File tree

4 files changed

+115
-3
lines changed

4 files changed

+115
-3
lines changed

src/v/cloud_storage/cache_service.cc

+11-2
Original file line numberDiff line numberDiff line change
@@ -356,6 +356,10 @@ ss::future<> cache::trim(
356356
target_size *= _cache_size_low_watermark;
357357
}
358358

359+
if (!_free_space.has_value()) {
360+
throw std::runtime_error("Free space information is not available.");
361+
}
362+
359363
// In the extreme case where even trimming to the low watermark wouldn't
360364
// free enough space to enable writing to the cache, go even further.
361365
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) {
15331537

15341538
auto would_fit_in_cache = _current_cache_size + bytes <= _max_bytes;
15351539

1536-
return !_block_puts && _free_space > bytes * 10
1540+
return !_block_puts && _free_space.value_or(0) > bytes * 10
15371541
&& _current_cache_objects + _reserved_cache_objects + objects
15381542
< _max_objects()
15391543
&& !would_fit_in_cache;
@@ -1871,7 +1875,12 @@ ss::future<> cache::do_reserve_space(uint64_t bytes, size_t objects) {
18711875
// all skipping the cache limit based on the same apparent
18721876
// free bytes. This counter will get reset to ground
18731877
// truth the next time we get a disk status notification.
1874-
_free_space -= bytes;
1878+
if (unlikely(!_free_space.has_value())) {
1879+
throw std::runtime_error(
1880+
"Free space information must be available by the "
1881+
"time we execute this code path");
1882+
}
1883+
*_free_space -= bytes;
18751884
break;
18761885
} else {
18771886
// No allowance, and the disk does not have a lot of

src/v/cloud_storage/cache_service.h

+1-1
Original file line numberDiff line numberDiff line change
@@ -335,7 +335,7 @@ class cache
335335
/// and have to decide whether to block writes, or exceed our configured
336336
/// limit.
337337
/// (shard 0 only)
338-
uint64_t _free_space{0};
338+
std::optional<uint64_t> _free_space;
339339

340340
ssx::semaphore _cleanup_sm{1, "cloud/cache"};
341341
std::set<std::filesystem::path> _files_in_progress;

src/v/redpanda/admin/server.cc

+9
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,7 @@
103103
#include <seastar/core/map_reduce.hh>
104104
#include <seastar/core/prometheus.hh>
105105
#include <seastar/core/reactor.hh>
106+
#include <seastar/core/shard_id.hh>
106107
#include <seastar/core/sharded.hh>
107108
#include <seastar/core/shared_ptr.hh>
108109
#include <seastar/core/smp.hh>
@@ -4347,6 +4348,14 @@ admin_server::delete_cloud_storage_lifecycle(
43474348
ss::future<ss::json::json_return_type>
43484349
admin_server::post_cloud_storage_cache_trim(
43494350
std::unique_ptr<ss::http::request> req) {
4351+
co_await ss::smp::submit_to(ss::shard_id{0}, [this] {
4352+
if (!_cloud_storage_cache.local_is_initialized()) {
4353+
throw ss::httpd::bad_request_exception(
4354+
"Cloud Storage Cache is not available. Is cloud storage "
4355+
"enabled?");
4356+
}
4357+
});
4358+
43504359
auto max_objects = get_integer_query_param(*req, "objects");
43514360
auto max_bytes = static_cast<std::optional<size_t>>(
43524361
get_integer_query_param(*req, "bytes"));
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,94 @@
1+
import pytest
2+
from ducktape.utils.util import wait_until
3+
from requests import HTTPError
4+
5+
from rptest.services.admin import Admin
6+
from rptest.services.cluster import cluster
7+
from rptest.services.redpanda import SISettings
8+
from rptest.tests.redpanda_test import RedpandaTest
9+
10+
11+
class CloudStorageCacheAdminApisNoCacheTest(RedpandaTest):
12+
"""
13+
Test the Cloud Storage Cache Admin APIs when tiered storage is not configured.
14+
"""
15+
def __init__(self, test_context):
16+
super().__init__(test_context, num_brokers=1)
17+
self.admin = Admin(self.redpanda)
18+
19+
@cluster(num_nodes=1)
20+
def test_admin_apis(self):
21+
for node in self.redpanda.nodes:
22+
with pytest.raises(HTTPError) as excinfo:
23+
self.admin.cloud_storage_trim(byte_limit=None,
24+
object_limit=None,
25+
node=node)
26+
27+
assert "Cloud Storage Cache is not available. Is cloud storage enabled?" == excinfo.value.response.json(
28+
)['message']
29+
30+
31+
class CloudStorageCacheAdminApisTest(RedpandaTest):
32+
"""
33+
Test the Cloud Storage Cache Admin APIs when tiered storage is configured.
34+
"""
35+
def __init__(self, test_context):
36+
super().__init__(test_context,
37+
num_brokers=1,
38+
si_settings=SISettings(test_context))
39+
self.admin = Admin(self.redpanda)
40+
41+
def setUp(self):
42+
pass
43+
44+
@cluster(num_nodes=1,
45+
log_allow_list=["Free space information is not available"])
46+
def test_admin_apis(self):
47+
num_objects = 100
48+
object_size = 4096
49+
50+
for node in self.redpanda.nodes:
51+
node.account.ssh(
52+
f"mkdir -p {self.redpanda.cache_dir} ; "
53+
f"for n in `seq 1 {num_objects}`; do "
54+
f"dd if=/dev/urandom bs={object_size} count=1 of={self.redpanda.cache_dir}/garbage_$n.bin ; done",
55+
)
56+
57+
self.redpanda.start(clean_nodes=False)
58+
59+
# Assert initial conditions.
60+
usage = self.admin.get_local_storage_usage(node)
61+
assert usage['cloud_storage_cache_objects'] == num_objects, usage
62+
63+
# Trim with default settings. Nothing should be trimmed as we are well
64+
# below reasonable limits.
65+
# Wrapped with wait_until as it will fail until a background fiber
66+
# updates information about free disk space.
67+
wait_until(lambda: self.admin.cloud_storage_trim(
68+
byte_limit=None, object_limit=None, node=node),
69+
timeout_sec=30,
70+
backoff_sec=1,
71+
retry_on_exc=True)
72+
73+
usage = self.admin.get_local_storage_usage(node)
74+
assert usage['cloud_storage_cache_objects'] == num_objects, usage
75+
76+
# Trim with byte limit. We should trim half of objects.
77+
self.admin.cloud_storage_trim(byte_limit=object_size * 50,
78+
object_limit=None,
79+
node=node)
80+
usage = self.admin.get_local_storage_usage(node)
81+
82+
# Although we set the limit to size of 50 objects, the value
83+
# gets multiplied by 0.8 internally so we end up with 40 objects left.
84+
assert usage['cloud_storage_cache_objects'] == 40, usage
85+
86+
# Trim with object limit. We should trim 20 objects.
87+
self.admin.cloud_storage_trim(byte_limit=None,
88+
object_limit=20,
89+
node=node)
90+
usage = self.admin.get_local_storage_usage(node)
91+
92+
# Although we set the limit to 20 objects, the value
93+
# gets multiplied by 0.8 internally so we end up with 16 objects left.
94+
assert usage['cloud_storage_cache_objects'] == 16, usage

0 commit comments

Comments
 (0)