Skip to content

Commit

Permalink
cluster: add conditional requests test to cloudcheck self-test
Browse files Browse the repository at this point in the history
  • Loading branch information
WillemKauf committed Nov 8, 2024
1 parent f6ddb89 commit cf26b1c
Show file tree
Hide file tree
Showing 3 changed files with 164 additions and 12 deletions.
157 changes: 152 additions & 5 deletions src/v/cluster/self_test/cloudcheck.cc
Original file line number Diff line number Diff line change
Expand Up @@ -108,9 +108,6 @@ ss::future<std::vector<self_test_result>> cloudcheck::run_benchmarks() {
const auto bucket = cloud_storage_clients::bucket_name{
cloud_storage::configuration::get_bucket_config()().value()};

const auto self_test_prefix = cloud_storage_clients::object_key{
"self-test/"};

const auto uuid = cloud_storage_clients::object_key{
ss::sstring{uuid_t::create()}};
const auto self_test_key = cloud_storage_clients::object_key{
Expand Down Expand Up @@ -209,6 +206,13 @@ ss::future<std::vector<self_test_result>> cloudcheck::run_benchmarks() {
&cloudcheck::verify_deletes, bucket, num_default_objects);
results.push_back(std::move(deletes_test_result.test_result));

// Test Puts and Gets with conditional headers (`If-None-Match`,
// `If-Match`).
auto conditional_puts_and_gets_test_result = co_await do_run_test(
&cloudcheck::verify_conditional_puts_and_gets, bucket);
results.push_back(
std::move(conditional_puts_and_gets_test_result.test_result));

co_return results;
}

Expand Down Expand Up @@ -461,8 +465,10 @@ ss::future<cloudcheck::verify_deletes_result> cloudcheck::verify_deletes(
}

std::vector<cloud_storage_clients::object_key> keys(num_objects);
std::generate(keys.begin(), keys.end(), []() {
return cloud_storage_clients::object_key{ss::sstring{uuid_t::create()}};
std::generate(keys.begin(), keys.end(), [this]() {
const auto uuid = cloud_storage_clients::object_key{
ss::sstring{uuid_t::create()}};
return cloud_storage_clients::object_key{self_test_prefix / uuid};
});

for (const auto& key : keys) {
Expand Down Expand Up @@ -495,4 +501,145 @@ ss::future<cloudcheck::verify_deletes_result> cloudcheck::verify_deletes(
co_return result;
}

ss::future<cloudcheck::verify_upload_result>
cloudcheck::verify_conditional_puts_and_gets(
cloud_storage_clients::bucket_name bucket) {
auto result = self_test_result{
.name = _opts.name,
.info = "Conditional Puts and Gets",
.test_type = "cloud"};

if (_cancelled) {
result.warning = "Run was manually cancelled.";
co_return result;
}

const auto uuid = cloud_storage_clients::object_key{
ss::sstring{uuid_t::create()}};
auto key = cloud_storage_clients::object_key{self_test_prefix / uuid};
auto payload = make_random_payload();

try {
ss::sstring etag = {};
// Attempt the first upload - expect it to succeed.
{
auto rtc = retry_chain_node(_opts.timeout, _opts.backoff, &_rtc);
cloud_storage::upload_request upload_request = make_upload_request(
bucket, key, payload.copy(), rtc);
upload_request.etag = &etag;
const cloud_storage::upload_result upload_result
= co_await _cloud_storage_api.local().upload_object(
std::move(upload_request));
switch (upload_result) {
case cloud_storage::upload_result::success:
break;
case cloud_storage::upload_result::timedout:
[[fallthrough]];
case cloud_storage::upload_result::failed:
[[fallthrough]];
case cloud_storage::upload_result::precondition_failed:
[[fallthrough]];
case cloud_storage::upload_result::cancelled:
result.error = "Failed to upload to cloud storage.";
break;
}
}

// Make a conditional read with the recorded etag - expect it to
// succeed.
{
iobuf download_payload;
auto rtc = retry_chain_node(_opts.timeout, _opts.backoff, &_rtc);
cloud_storage::download_request download_request
= make_download_request(
bucket, key, std::ref(download_payload), rtc);
download_request.set_download_if_matches(etag);
const cloud_storage::download_result download_result
= co_await _cloud_storage_api.local().download_object(
std::move(download_request));

switch (download_result) {
case cloud_storage::download_result::success:
break;
case cloud_storage::download_result::timedout:
[[fallthrough]];
case cloud_storage::download_result::failed:
[[fallthrough]];
case cloud_storage::download_result::precondition_failed:
[[fallthrough]];
case cloud_storage::download_result::notfound:
result.error = "Failed to download from cloud storage using "
"recorded ETag and If-Match header.";
break;
}
}

// Make a conditional read with an invalid etag - expect it to fail.
{
iobuf download_payload;
auto rtc = retry_chain_node(_opts.timeout, _opts.backoff, &_rtc);
cloud_storage::download_request download_request
= make_download_request(
bucket, key, std::ref(download_payload), rtc);
download_request.set_download_if_matches("DEADBEEF");
const cloud_storage::download_result download_result
= co_await _cloud_storage_api.local().download_object(
std::move(download_request));

switch (download_result) {
case cloud_storage::download_result::precondition_failed:
break;
case cloud_storage::download_result::success:
result.error
= "Failed to respect conditional read from cloud storage.";
break;
case cloud_storage::download_result::timedout:
[[fallthrough]];
case cloud_storage::download_result::failed:
[[fallthrough]];
case cloud_storage::download_result::notfound:
result.error = "Failed to download from cloud storage.";
break;
}
}

// Attempt the second upload, with If-None-Match header - expect it to
// fail.
{
auto rtc = retry_chain_node(_opts.timeout, _opts.backoff, &_rtc);
cloud_storage::upload_request upload_request = make_upload_request(
bucket, key, payload.copy(), rtc);
upload_request.set_upload_if_not_exists();
const cloud_storage::upload_result upload_result
= co_await _cloud_storage_api.local().upload_object(
std::move(upload_request));

switch (upload_result) {
case cloud_storage::upload_result::precondition_failed:
break;
case cloud_storage::upload_result::success:
[[fallthrough]];
case cloud_storage::upload_result::timedout:
[[fallthrough]];
case cloud_storage::upload_result::failed:
[[fallthrough]];
case cloud_storage::upload_result::cancelled:
result.error = "Failed to respect If-None-Match header.";
break;
}
}

// Clean-up the uploaded file.
{
auto rtc = retry_chain_node(_opts.timeout, _opts.backoff, &_rtc);
std::ignore = co_await _cloud_storage_api.local().delete_object(
bucket, key, rtc);
}
} catch (const std::exception& e) {
result.error = e.what();
}

co_return result;
}

} // namespace cluster::self_test
6 changes: 6 additions & 0 deletions src/v/cluster/self_test/cloudcheck.h
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,12 @@ class cloudcheck {
cloud_storage_clients::bucket_name bucket,
size_t num_objects = num_default_objects);

// Verify that conditional requests (both Put: write operation, using the
// `If-None-Match` header and Get: read operation, using the `If-Match`
// header) to cloud storage work.
ss::future<verify_upload_result>
verify_conditional_puts_and_gets(cloud_storage_clients::bucket_name bucket);

private:
static constexpr size_t num_default_objects = 5;
bool _cancelled{false};
Expand Down
13 changes: 6 additions & 7 deletions tests/rptest/tests/self_test_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -83,14 +83,13 @@ def test_self_test(self):

cloud_results = [r for r in reports if r['test_type'] == 'cloud']

read_tests = ['List', 'Head', 'Get']
write_tests = ['Put', 'Delete', 'Plural Delete']
cloudcheck_tests = [
'List', 'Head', 'Get', 'Put', 'Delete', 'Plural Delete',
'Conditional Puts and Gets'
]

num_expected_cloud_storage_read_tests = num_nodes * len(read_tests)
num_expected_cloud_storage_write_tests = num_nodes * len(write_tests)
assert len(
cloud_results
) == num_expected_cloud_storage_write_tests + num_expected_cloud_storage_read_tests
num_expected_cloud_storage_tests = num_nodes * len(cloudcheck_tests)
assert len(cloud_results) == num_expected_cloud_storage_tests

# Ensure no other result sets exist
assert len(disk_results) + len(network_results) + len(
Expand Down

0 comments on commit cf26b1c

Please sign in to comment.