Skip to content

Commit

Permalink
cloud_storage: Match requests using URL
Browse files Browse the repository at this point in the history
In topic recovery tests some assertions simply wait for a count of
requests made to S3 imposter. This may cause tests to fail when requests
not related to the test are observed by the imposter, throwing off the
expected count.

This change adds an optional predicate to the assertion so that the
request count can be matched after filtering for specific requests by
URL, method etc.
  • Loading branch information
abhijat committed Sep 29, 2023
1 parent b3015e0 commit 8b3f32d
Show file tree
Hide file tree
Showing 3 changed files with 42 additions and 11 deletions.
12 changes: 12 additions & 0 deletions src/v/cloud_storage/tests/s3_imposter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,18 @@ s3_imposter_fixture::get_requests() const {
return _requests;
}

std::vector<http_test_utils::request_info> s3_imposter_fixture::get_requests(
s3_imposter_fixture::req_pred_t predicate) const {
std::vector<http_test_utils::request_info> matching_requests;
matching_requests.reserve(_requests.size());
std::copy_if(
_requests.cbegin(),
_requests.cend(),
std::back_inserter(matching_requests),
std::move(predicate));
return matching_requests;
}

const std::multimap<ss::sstring, http_test_utils::request_info>&
s3_imposter_fixture::get_targets() const {
return _targets;
Expand Down
7 changes: 7 additions & 0 deletions src/v/cloud_storage/tests/s3_imposter.h
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,13 @@ class s3_imposter_fixture {
/// Access all http requests ordered by time
const std::vector<http_test_utils::request_info>& get_requests() const;

using req_pred_t
= ss::noncopyable_function<bool(const http_test_utils::request_info&)>;

/// Access http requests matching the given predicate
std::vector<http_test_utils::request_info>
get_requests(req_pred_t predicate) const;

/// Access all http requests ordered by target url
const std::multimap<ss::sstring, http_test_utils::request_info>&
get_targets() const;
Expand Down
34 changes: 23 additions & 11 deletions src/v/cloud_storage/tests/topic_recovery_service_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,11 @@ generate_no_manifests_expectations(
return expectations;
}

bool is_manifest_list_request(const http_test_utils::request_info& req) {
return req.method == "GET" && req.url.starts_with("/?list-type=2&prefix=")
&& req.url.ends_with("0000000/");
}

} // namespace

class fixture
Expand Down Expand Up @@ -151,14 +156,20 @@ class fixture
}

using equals = ss::bool_class<struct equals_tag>;
void wait_for_n_requests(size_t n, equals e = equals::no) {
tests::cooperative_spin_wait_with_timeout(10s, [this, n, e] {
if (e) {
return get_requests().size() == n;
} else {
return get_requests().size() >= n;
}
}).get();
void wait_for_n_requests(
size_t n,
equals e = equals::no,
std::optional<req_pred_t> predicate = std::nullopt) {
tests::cooperative_spin_wait_with_timeout(
10s,
[this, n, e, p = std::move(predicate)]() mutable {
const auto matching_requests_size
= p ? get_requests(std::move(p.value())).size()
: get_requests().size();
return e ? matching_requests_size == n
: matching_requests_size >= n;
})
.get();
}

cloud_storage::init_recovery_result
Expand Down Expand Up @@ -217,7 +228,7 @@ FIXTURE_TEST(recovery_with_no_topics_exits_early, fixture) {
BOOST_REQUIRE_EQUAL(result, expected);

// Wait until one request is received, to list bucket for manifest files
wait_for_n_requests(16, equals::yes);
wait_for_n_requests(16, equals::yes, is_manifest_list_request);

const auto& list_topics_req = get_requests()[0];
BOOST_REQUIRE_EQUAL(list_topics_req.url, "/?list-type=2&prefix=00000000/");
Expand Down Expand Up @@ -291,7 +302,8 @@ FIXTURE_TEST(recovery_with_existing_topic, fixture) {
.message = "recovery started"};

BOOST_REQUIRE_EQUAL(result, expected);
wait_for_n_requests(16, equals::yes);

wait_for_n_requests(16, equals::yes, is_manifest_list_request);

tests::cooperative_spin_wait_with_timeout(10s, [&service] {
return service.local().is_active() == false;
Expand Down Expand Up @@ -388,7 +400,7 @@ FIXTURE_TEST(recovery_with_topic_name_pattern_without_match, fixture) {

start_recovery(R"JSON({"topic_names_pattern": "abc*"})JSON");

wait_for_n_requests(16, equals::yes);
wait_for_n_requests(16, equals::yes, is_manifest_list_request);

auto& service = app.topic_recovery_service;
tests::cooperative_spin_wait_with_timeout(10s, [&service] {
Expand Down

0 comments on commit 8b3f32d

Please sign in to comment.