From b509644972f0897a9f543ecbb2d9b0b406a79785 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 26 Dec 2024 21:47:51 +0200 Subject: [PATCH 1/6] test: lib: eventually: make *EVENTUALLY_EQUAL inline functions rather then macros. This is a first cleanup step before adding a sleep function parameter to support also manual_clock. Also, add a call to BOOST_REQUIRE_EQUAL/BOOST_CHECK_EQUAL, respectively, to make an error more visible in the test log since those entry points print the offending values when not equal. Signed-off-by: Benny Halevy --- test/boost/bloom_filter_test.cc | 14 +++++++------- test/boost/loading_cache_test.cc | 12 ++++++------ test/boost/reader_concurrency_semaphore_test.cc | 14 +++++++------- test/boost/view_build_test.cc | 2 +- test/lib/eventually.hh | 17 +++++++++++++++-- test/raft/replication.hh | 2 +- 6 files changed, 37 insertions(+), 24 deletions(-) diff --git a/test/boost/bloom_filter_test.cc b/test/boost/bloom_filter_test.cc index 83e7e1e8a618..5abb20179856 100644 --- a/test/boost/bloom_filter_test.cc +++ b/test/boost/bloom_filter_test.cc @@ -96,9 +96,9 @@ SEASTAR_TEST_CASE(test_sstable_manager_auto_reclaim_and_reload_of_bloom_filter) // Test auto reload - disposing sst3 should trigger reload of the // smallest filter in the reclaimed list, which is sst1's bloom filter. dispose_and_stop_tracking_bf_memory(std::move(sst3), sst_mgr); - REQUIRE_EVENTUALLY_EQUAL(sst1->filter_memory_size(), sst1_bf_memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst1->filter_memory_size(); }, sst1_bf_memory); // only sst4's bloom filter memory should be reported as reclaimed - REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_total_memory_reclaimed(), sst4_bf_memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst_mgr.get_total_memory_reclaimed(); }, sst4_bf_memory); // sst2 and sst4 remain the same BOOST_REQUIRE_EQUAL(sst2->filter_memory_size(), sst2_bf_memory); BOOST_REQUIRE_EQUAL(sst4->filter_memory_size(), 0); @@ -162,7 +162,7 @@ SEASTAR_TEST_CASE(test_bloom_filter_reclaim_during_reload) { // dispose sst2 to trigger reload of sst1's bloom filter dispose_and_stop_tracking_bf_memory(std::move(sst2), sst_mgr); // _total_reclaimable_memory will be updated when the reload begins; wait for it. - REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_total_reclaimable_memory(), sst1_bf_memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst_mgr.get_total_reclaimable_memory(); }, sst1_bf_memory); // now that the reload is midway and paused, create new sst to verify that its // filter gets evicted immediately as the memory that became available is reserved @@ -175,8 +175,8 @@ SEASTAR_TEST_CASE(test_bloom_filter_reclaim_during_reload) { // resume reloading sst1 filter utils::get_local_injector().receive_message("reload_reclaimed_components/pause"); - REQUIRE_EVENTUALLY_EQUAL(sst1->filter_memory_size(), sst1_bf_memory); - REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_total_memory_reclaimed(), sst3_bf_memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst1->filter_memory_size(); }, sst1_bf_memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst_mgr.get_total_memory_reclaimed(); }, sst3_bf_memory); BOOST_REQUIRE_EQUAL(sst_mgr.get_total_reclaimable_memory(), sst1_bf_memory); utils::get_local_injector().disable("reload_reclaimed_components/pause"); @@ -275,7 +275,7 @@ SEASTAR_TEST_CASE(test_bloom_filter_reload_after_unlink) { utils::get_local_injector().receive_message("test_bloom_filter_reload_after_unlink"); async_sst_holder.get(); - REQUIRE_EVENTUALLY_EQUAL(sst_mgr.get_active_list().size(), 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return sst_mgr.get_active_list().size(); }, 0); }, { // set available memory = 0 to force reclaim the bloom filter .available_memory = 0 @@ -340,7 +340,7 @@ SEASTAR_TEST_CASE(test_bloom_filter_reclaim_after_unlink) { utils::get_local_injector().receive_message("test_bloom_filter_reload_after_unlink"); async_sst_holder.get(); - REQUIRE_EVENTUALLY_EQUAL(active_list.size(), 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return active_list.size(); }, 0); }, { // set available memory = 0 to force reclaim the bloom filter .available_memory = 100 diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index 691ecfc23e94..e60afcaa6e3a 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -269,7 +269,7 @@ SEASTAR_TEST_CASE(test_loading_cache_update_config) { loading_cache.get_ptr(i, loader).discard_result().get(); } - REQUIRE_EVENTUALLY_EQUAL(loading_cache.size(), 2); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 2); }); } @@ -345,14 +345,14 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { // Check unprivileged section eviction BOOST_REQUIRE(loading_cache.size() == 1); sleep(20ms).get(); - REQUIRE_EVENTUALLY_EQUAL(loading_cache.size(), 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); // Check privileged section eviction loading_cache.get_ptr(0, loader).discard_result().get(); BOOST_REQUIRE(loading_cache.find(0) != nullptr); sleep(20ms).get(); - REQUIRE_EVENTUALLY_EQUAL(loading_cache.size(), 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); }); } @@ -385,7 +385,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { } sleep(30ms).get(); - REQUIRE_EVENTUALLY_EQUAL(loading_cache.size(), 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); }); } @@ -505,8 +505,8 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged) { } // Make sure that the value we touched twice is eventually evicted - REQUIRE_EVENTUALLY_EQUAL(loading_cache.find(-1), nullptr); - REQUIRE_EVENTUALLY_EQUAL(loading_cache.size(), 0); + REQUIRE_EVENTUALLY_EQUAL::value_ptr>([&] { return loading_cache.find(-1); }, nullptr); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); }); } diff --git a/test/boost/reader_concurrency_semaphore_test.cc b/test/boost/reader_concurrency_semaphore_test.cc index ac6c4b6ff7aa..19577be8670f 100644 --- a/test/boost/reader_concurrency_semaphore_test.cc +++ b/test/boost/reader_concurrency_semaphore_test.cc @@ -469,7 +469,7 @@ SEASTAR_TEST_CASE(reader_restriction_file_tracking) { } // All units should have been deposited back. - REQUIRE_EVENTUALLY_EQUAL(4 * 1024, semaphore.available_resources().memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return semaphore.available_resources().memory; }, 4 * 1024); }); } @@ -513,7 +513,7 @@ SEASTAR_TEST_CASE(reader_concurrency_semaphore_timeout) { } // All units should have been deposited back. - REQUIRE_EVENTUALLY_EQUAL(replica::new_reader_base_cost, semaphore.available_resources().memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return semaphore.available_resources().memory; }, replica::new_reader_base_cost); }); } @@ -545,7 +545,7 @@ SEASTAR_TEST_CASE(reader_concurrency_semaphore_max_queue_length) { } } - REQUIRE_EVENTUALLY_EQUAL(replica::new_reader_base_cost, semaphore.available_resources().memory); + REQUIRE_EVENTUALLY_EQUAL([&] { return semaphore.available_resources().memory; }, replica::new_reader_base_cost); }); } @@ -1167,7 +1167,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_evict_inactive_reads_ // Marking p2 as awaits should eventually allow p3 to be admitted by evicting p1 rd2.mark_as_awaits(); - REQUIRE_EVENTUALLY_EQUAL(semaphore.get_stats().waiters, 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return semaphore.get_stats().waiters; }, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().need_cpu_permits, 1); BOOST_REQUIRE_EQUAL(semaphore.get_stats().awaits_permits, 1); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); @@ -1212,7 +1212,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_set_resources) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().waiters, 1); semaphore.set_resources({4, 4 * 1024}); - REQUIRE_EVENTUALLY_EQUAL(semaphore.get_stats().waiters, 0); + REQUIRE_EVENTUALLY_EQUAL([&] { return semaphore.get_stats().waiters; }, 0); BOOST_REQUIRE_EQUAL(semaphore.available_resources(), reader_resources(1, 1024)); BOOST_REQUIRE_EQUAL(semaphore.initial_resources(), reader_resources(4, 4 * 1024)); permit3_fut.get(); @@ -2101,7 +2101,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_necessary_evicting) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 1); ncpu_guard.reset(); - REQUIRE_EVENTUALLY_EQUAL(bool(handle), false); + REQUIRE_EVENTUALLY_EQUAL([&] { return bool(handle); }, false); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().permit_based_evictions, ++evicted_reads); @@ -2130,7 +2130,7 @@ SEASTAR_THREAD_TEST_CASE(test_reader_concurrency_semaphore_necessary_evicting) { BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 1); ncpu_guard.reset(); - REQUIRE_EVENTUALLY_EQUAL(bool(handle), false); + REQUIRE_EVENTUALLY_EQUAL([&] { return bool(handle); }, false); BOOST_REQUIRE_EQUAL(semaphore.get_stats().inactive_reads, 0); BOOST_REQUIRE_EQUAL(semaphore.get_stats().permit_based_evictions, ++evicted_reads); diff --git a/test/boost/view_build_test.cc b/test/boost/view_build_test.cc index bf65a25193d7..ab469433cc23 100644 --- a/test/boost/view_build_test.cc +++ b/test/boost/view_build_test.cc @@ -464,7 +464,7 @@ SEASTAR_TEST_CASE(test_view_update_generator) { } const auto qsz = db::view::view_update_generator::registration_queue_size; when_all(register_futures.begin(), register_futures.end()).get(); - REQUIRE_EVENTUALLY_EQUAL(view_update_generator.available_register_units(), qsz); + REQUIRE_EVENTUALLY_EQUAL([&] { return view_update_generator.available_register_units(); }, qsz); }; register_and_check_semaphore(ssts.begin(), ssts.begin() + 10); register_and_check_semaphore(ssts.begin() + 10, ssts.end()); diff --git a/test/lib/eventually.hh b/test/lib/eventually.hh index 1dbc76bc40db..40cee0d73874 100644 --- a/test/lib/eventually.hh +++ b/test/lib/eventually.hh @@ -8,6 +8,8 @@ #pragma once +#include + #include #include @@ -49,5 +51,16 @@ bool eventually_true(noncopyable_function f) { return false; } -#define REQUIRE_EVENTUALLY_EQUAL(a, b) BOOST_REQUIRE(eventually_true([&] { return a == b; })) -#define CHECK_EVENTUALLY_EQUAL(a, b) BOOST_CHECK(eventually_true([&] { return a == b; })) +// Must be called in a seastar thread +template +void REQUIRE_EVENTUALLY_EQUAL(std::function a, T b) { + eventually_true([&] { return a() == b; }); + BOOST_REQUIRE_EQUAL(a(), b); +} + +// Must be called in a seastar thread +template +void CHECK_EVENTUALLY_EQUAL(std::function a, T b) { + eventually_true([&] { return a() == b; }); + BOOST_CHECK_EQUAL(a(), b); +} diff --git a/test/raft/replication.hh b/test/raft/replication.hh index 15f0a09e1220..b2d11de3ef29 100644 --- a/test/raft/replication.hh +++ b/test/raft/replication.hh @@ -1235,7 +1235,7 @@ future<> raft_cluster::check_rpc_config(::check_rpc_config cc) { for (auto& node: cc.nodes) { BOOST_CHECK(node.id < _servers.size()); co_await seastar::async([&] { - CHECK_EVENTUALLY_EQUAL(_servers[node.id].rpc->known_peers(), as); + BOOST_CHECK(eventually_true([&] { return _servers[node.id].rpc->known_peers() == as; })); }); } } From 934a9d3fd68efd76404a57a9851cbc6c433057d0 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 26 Dec 2024 21:50:42 +0200 Subject: [PATCH 2/6] test: lib: eventually: add sleep function param To allow support for manual_clock instead of seastar::sleep. Signed-off-by: Benny Halevy --- configure.py | 8 +++++--- test/lib/CMakeLists.txt | 3 ++- test/lib/eventually.cc | 13 +++++++++++++ test/lib/eventually.hh | 16 ++++++++++------ 4 files changed, 30 insertions(+), 10 deletions(-) create mode 100644 test/lib/eventually.cc diff --git a/configure.py b/configure.py index ad3f3da5a0d3..8e709dbf468b 100755 --- a/configure.py +++ b/configure.py @@ -1338,6 +1338,7 @@ def find_ninja(): 'test/lib/test_utils.cc', 'test/lib/tmpdir.cc', 'test/lib/sstable_run_based_compaction_strategy_for_tests.cc', + 'test/lib/eventually.cc', ] scylla_tests_dependencies = scylla_core + alternator + idls + scylla_tests_generic_dependencies + [ @@ -1379,6 +1380,7 @@ def find_ninja(): 'test/lib/key_utils.cc', 'test/lib/random_schema.cc', 'test/lib/data_model.cc', + 'test/lib/eventually.cc', 'seastar/tests/perf/linux_perf_event.cc'] deps = { @@ -1573,11 +1575,11 @@ def find_ninja(): deps['test/boost/schema_loader_test'] += ['tools/schema_loader.cc', 'tools/read_mutation.cc'] deps['test/boost/rust_test'] += ['rust/inc/src/lib.rs'] -deps['test/raft/replication_test'] = ['test/raft/replication_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies -deps['test/raft/raft_server_test'] = ['test/raft/raft_server_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies +deps['test/raft/replication_test'] = ['test/raft/replication_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc', 'test/lib/eventually.cc'] + scylla_raft_dependencies +deps['test/raft/raft_server_test'] = ['test/raft/raft_server_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc', 'test/lib/eventually.cc'] + scylla_raft_dependencies deps['test/raft/randomized_nemesis_test'] = ['test/raft/randomized_nemesis_test.cc', 'direct_failure_detector/failure_detector.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies deps['test/raft/failure_detector_test'] = ['test/raft/failure_detector_test.cc', 'direct_failure_detector/failure_detector.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies -deps['test/raft/many_test'] = ['test/raft/many_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc'] + scylla_raft_dependencies +deps['test/raft/many_test'] = ['test/raft/many_test.cc', 'test/raft/replication.cc', 'test/raft/helpers.cc', 'test/lib/eventually.cc'] + scylla_raft_dependencies deps['test/raft/fsm_test'] = ['test/raft/fsm_test.cc', 'test/raft/helpers.cc', 'test/lib/log.cc'] + scylla_raft_dependencies deps['test/raft/etcd_test'] = ['test/raft/etcd_test.cc', 'test/raft/helpers.cc', 'test/lib/log.cc'] + scylla_raft_dependencies deps['test/raft/raft_sys_table_storage_test'] = ['test/raft/raft_sys_table_storage_test.cc'] + \ diff --git a/test/lib/CMakeLists.txt b/test/lib/CMakeLists.txt index cf74cbb0615d..c8398a6d59cd 100644 --- a/test/lib/CMakeLists.txt +++ b/test/lib/CMakeLists.txt @@ -17,7 +17,8 @@ target_sources(test-lib result_set_assertions.cc sstable_run_based_compaction_strategy_for_tests.cc sstable_utils.cc - data_model.cc) + data_model.cc + eventually.cc) target_include_directories(test-lib PUBLIC ${CMAKE_SOURCE_DIR}) diff --git a/test/lib/eventually.cc b/test/lib/eventually.cc new file mode 100644 index 000000000000..c90a3d543eca --- /dev/null +++ b/test/lib/eventually.cc @@ -0,0 +1,13 @@ +/* + * Copyright (C) 2019-present ScyllaDB + */ + +/* + * SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0 + */ + +#include "test/lib/eventually.hh" + +sleep_fn seastar_sleep_fn = [] (std::chrono::milliseconds ms) -> future<> { + return seastar::sleep(ms); +}; diff --git a/test/lib/eventually.hh b/test/lib/eventually.hh index 40cee0d73874..7bd085104975 100644 --- a/test/lib/eventually.hh +++ b/test/lib/eventually.hh @@ -15,8 +15,12 @@ #include "seastarx.hh" +using sleep_fn = std::function(std::chrono::milliseconds)>; + +extern sleep_fn seastar_sleep_fn; + inline -void eventually(noncopyable_function f, size_t max_attempts = 17) { +void eventually(noncopyable_function f, size_t max_attempts = 17, sleep_fn sleep = seastar_sleep_fn) { size_t attempts = 0; while (true) { try { @@ -33,7 +37,7 @@ void eventually(noncopyable_function f, size_t max_attempts = 17) { } inline -bool eventually_true(noncopyable_function f) { +bool eventually_true(noncopyable_function f, sleep_fn sleep = seastar_sleep_fn) { const unsigned max_attempts = 15; unsigned attempts = 0; while (true) { @@ -53,14 +57,14 @@ bool eventually_true(noncopyable_function f) { // Must be called in a seastar thread template -void REQUIRE_EVENTUALLY_EQUAL(std::function a, T b) { - eventually_true([&] { return a() == b; }); +void REQUIRE_EVENTUALLY_EQUAL(std::function a, T b, sleep_fn sleep = seastar_sleep_fn) { + eventually_true([&] { return a() == b; }, sleep); BOOST_REQUIRE_EQUAL(a(), b); } // Must be called in a seastar thread template -void CHECK_EVENTUALLY_EQUAL(std::function a, T b) { - eventually_true([&] { return a() == b; }); +void CHECK_EVENTUALLY_EQUAL(std::function a, T b, sleep_fn sleep = seastar_sleep_fn) { + eventually_true([&] { return a() == b; }, sleep); BOOST_CHECK_EQUAL(a(), b); } From d68829243f07c712e86cba0debaf5c55d8ae6713 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Fri, 27 Dec 2024 09:05:12 +0200 Subject: [PATCH 3/6] test: loading_cache_test: simlute loader using sleep This test isn't about reading values from file, but rather it's about the loading_cache. Reading from the file can sometimes take longer than the expected refresh times, causing flakiness (see #20322). Rather than reading a string from a real file, just sleep a random, short time, and co_return the string. Signed-off-by: Benny Halevy --- test/boost/loading_cache_test.cc | 84 +++----------------------------- 1 file changed, 6 insertions(+), 78 deletions(-) diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index e60afcaa6e3a..8e247ce52978 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -9,6 +9,7 @@ #include #include "utils/loading_shared_values.hh" #include "utils/loading_cache.hh" +#include #include #include #include @@ -43,51 +44,16 @@ static int rand_int(int max) { static const sstring test_file_name = "loading_cache_test.txt"; static const sstring test_string = "1"; -static bool file_prepared = false; static constexpr int num_loaders = 1000; static thread_local int load_count; -static const tmpdir& get_tmpdir() { - static thread_local tmpdir tmp; - return tmp; -} - -static future<> prepare() { - if (file_prepared) { - return make_ready_future<>(); - } - - return open_file_dma((get_tmpdir().path() / test_file_name.c_str()).c_str(), open_flags::create | open_flags::wo).then([] (file f) { - return do_with(std::move(f), [] (file& f) { - auto size = test_string.size() + 1; - auto aligned_size = align_up(size, f.disk_write_dma_alignment()); - auto buf = allocate_aligned_buffer(aligned_size, f.disk_write_dma_alignment()); - auto wbuf = buf.get(); - std::copy_n(test_string.c_str(), size, wbuf); - return f.dma_write(0, wbuf, aligned_size).then([aligned_size, buf = std::move(buf)] (size_t s) { - BOOST_REQUIRE_EQUAL(s, aligned_size); - file_prepared = true; - }).finally([&f] () mutable { - return f.close(); - }); - }); - }); -} static future loader(const int& k) { - return open_file_dma((get_tmpdir().path() / test_file_name.c_str()).c_str(), open_flags::ro).then([] (file f) -> future { - return do_with(std::move(f), [] (file& f) -> future { - auto size = align_up(test_string.size() + 1, f.disk_read_dma_alignment()); - return f.dma_read_exactly(0, size).then([] (auto buf) { - sstring str(buf.get()); - BOOST_REQUIRE_EQUAL(str, test_string); - ++load_count; - return make_ready_future(std::move(str)); - }).finally([&f] () mutable { - return f.close(); - }); - }); - }); + testlog.debug("loader: start: load_count={}", load_count); + co_await sleep(tests::random::get_int(1, 5) * 1ms); + ++load_count; + testlog.debug("loader: done: load_count={}", load_count); + co_return test_string; } SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_same_key) { @@ -97,8 +63,6 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_same_key) { utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; - prepare().get(); - std::fill(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { @@ -121,8 +85,6 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_different_keys) { utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; - prepare().get(); - std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { @@ -145,8 +107,6 @@ SEASTAR_TEST_CASE(test_loading_shared_values_rehash) { utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; - prepare().get(); - std::iota(ivec.begin(), ivec.end(), 0); // verify that load factor is always in the (0.25, 0.75) range @@ -177,8 +137,6 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_explicit_eviction) utils::loading_shared_values shared_values; std::vector::entry_ptr> anchors_vec(num_loaders); - prepare().get(); - std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { @@ -202,8 +160,6 @@ SEASTAR_TEST_CASE(test_loading_cache_disable_and_enable) { utils::loading_cache loading_cache({num_loaders, 1h, 50ms}, testlog, loader); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - loading_cache.get(0).discard_result().get(); loading_cache.get(0).discard_result().get(); @@ -233,8 +189,6 @@ SEASTAR_TEST_CASE(test_loading_cache_reset) { utils::loading_cache loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - for (int i = 0; i < num_loaders; ++i) { loading_cache.get_ptr(i, loader).discard_result().get(); } @@ -253,8 +207,6 @@ SEASTAR_TEST_CASE(test_loading_cache_update_config) { utils::loading_cache loading_cache({num_loaders, 1h, 1h}, testlog, loader); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - for (int i = 0; i < num_loaders; ++i) { loading_cache.get_ptr(i, loader).discard_result().get(); } @@ -281,8 +233,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_same_key) { utils::loading_cache loading_cache(num_loaders, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - std::fill(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { @@ -301,8 +251,6 @@ SEASTAR_THREAD_TEST_CASE(test_loading_cache_removing_key) { utils::loading_cache loading_cache(num_loaders, 100s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - loading_cache.get_ptr(0, loader).discard_result().get(); BOOST_REQUIRE_EQUAL(load_count, 1); BOOST_REQUIRE(loading_cache.find(0) != nullptr); @@ -319,8 +267,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_different_keys) { utils::loading_cache loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { @@ -338,8 +284,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { utils::loading_cache loading_cache(num_loaders, 20ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - loading_cache.get_ptr(0, loader).discard_result().get(); // Check unprivileged section eviction @@ -362,8 +306,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { utils::loading_cache loading_cache(num_loaders, 30ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - loading_cache.get_ptr(0, loader).discard_result().get(); auto vp = loading_cache.find(0); auto load_time = steady_clock::now(); @@ -395,8 +337,6 @@ SEASTAR_TEST_CASE(test_loading_cache_move_item_to_mru_list_front_on_sync_op) { utils::loading_cache loading_cache(2, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - for (int i = 0; i < 2; ++i) { loading_cache.get_ptr(i, loader).discard_result().get(); } @@ -421,7 +361,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_privileged_gen) { load_count = 0; utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); // Push the entry into the privileged section. Make sure it's being reloaded. loading_cache.get_ptr(0).discard_result().get(); loading_cache.get_ptr(0).discard_result().get(); @@ -436,7 +375,6 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_unprivileged) { load_count = 0; utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); // Load one entry into the unprivileged section. // Make sure it's reloaded. loading_cache.get_ptr(0).discard_result().get(); @@ -452,8 +390,6 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction) { utils::loading_cache loading_cache(1, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - for (int i = 0; i < num_loaders; ++i) { loading_cache.get_ptr(i % 2, loader).discard_result().get(); } @@ -470,8 +406,6 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction_unprivileged_first) { utils::loading_cache loading_cache(4, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - // Touch the value with the key "-1" twice loading_cache.get_ptr(-1, loader).discard_result().get(); loading_cache.find(-1); @@ -494,8 +428,6 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged) { utils::loading_cache loading_cache(4, 10ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - // Touch the value with the key "-1" twice loading_cache.get_ptr(-1, loader).discard_result().get(); loading_cache.find(-1); @@ -527,8 +459,6 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged_minimum_size) { utils::loading_cache loading_cache(50, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - // Add 49 elements to privileged section for (int i = 0; i < 49; i++) { // Touch the value with the key "i" twice @@ -674,8 +604,6 @@ SEASTAR_TEST_CASE(test_loading_cache_reload_during_eviction) { utils::loading_cache loading_cache({1, 100ms, 10ms}, testlog, loader); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - prepare().get(); - auto curr_time = lowres_clock::now(); int i = 0; From b258f8cc6932c062c5edc08b925cff14320280d6 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 26 Dec 2024 19:29:27 +0200 Subject: [PATCH 4/6] test: loading_cache_test: use function-scope loader Rather than a global function, accessing a thread-local `load_count`. The thread-local load_count cannot be used when multiple test cases run in parallel. Signed-off-by: Benny Halevy --- test/boost/loading_cache_test.cc | 158 ++++++++++++++++++------------- 1 file changed, 93 insertions(+), 65 deletions(-) diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index 8e247ce52978..88a1ed3fe4d0 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -46,33 +46,56 @@ static const sstring test_file_name = "loading_cache_test.txt"; static const sstring test_string = "1"; static constexpr int num_loaders = 1000; -static thread_local int load_count; - -static future loader(const int& k) { - testlog.debug("loader: start: load_count={}", load_count); - co_await sleep(tests::random::get_int(1, 5) * 1ms); - ++load_count; - testlog.debug("loader: done: load_count={}", load_count); - co_return test_string; -} +class loader { + struct impl { + int load_count = 0; + + future load(const int& k) { + testlog.debug("load: start: load_count={}", load_count); + co_await sleep(tests::random::get_int(1, 5) * 1ms); + ++load_count; + testlog.debug("load: done: load_count={}", load_count); + co_return test_string; + } + }; + + lw_shared_ptr _impl; + +public: + loader() : _impl(make_lw_shared()) {} + + int& load_count() { + return _impl->load_count; + } + + const int& load_count() const { + return _impl->load_count; + } + + std::function(const int& k)> get() { + return [impl = _impl] (const int& k) { + return impl->load(k).finally([impl] {}); + }; + } +}; SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_same_key) { return seastar::async([] { std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; std::fill(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { - return shared_values.get_or_load(k, loader).then([&] (auto entry_ptr) { + return shared_values.get_or_load(k, loader.get()).then([&] (auto entry_ptr) { anchors_list.emplace_back(std::move(entry_ptr)); }); }).get(); // "loader" must be called exactly once - BOOST_REQUIRE_EQUAL(load_count, 1); + BOOST_REQUIRE_EQUAL(loader.load_count(), 1); BOOST_REQUIRE_EQUAL(shared_values.size(), 1); anchors_list.clear(); }); @@ -81,20 +104,20 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_same_key) { SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_different_keys) { return seastar::async([] { std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { - return shared_values.get_or_load(k, loader).then([&] (auto entry_ptr) { + return shared_values.get_or_load(k, loader.get()).then([&] (auto entry_ptr) { anchors_list.emplace_back(std::move(entry_ptr)); }); }).get(); // "loader" must be called once for each key - BOOST_REQUIRE_EQUAL(load_count, num_loaders); + BOOST_REQUIRE_EQUAL(loader.load_count(), num_loaders); BOOST_REQUIRE_EQUAL(shared_values.size(), num_loaders); anchors_list.clear(); }); @@ -103,7 +126,7 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_different_keys) { SEASTAR_TEST_CASE(test_loading_shared_values_rehash) { return seastar::async([] { std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_shared_values shared_values; std::list::entry_ptr> anchors_list; @@ -111,7 +134,7 @@ SEASTAR_TEST_CASE(test_loading_shared_values_rehash) { // verify that load factor is always in the (0.25, 0.75) range for (int k = 0; k < num_loaders; ++k) { - shared_values.get_or_load(k, loader).then([&] (auto entry_ptr) { + shared_values.get_or_load(k, loader.get()).then([&] (auto entry_ptr) { anchors_list.emplace_back(std::move(entry_ptr)); }).get(); BOOST_REQUIRE_LE(shared_values.size(), 3 * shared_values.buckets_count() / 4); @@ -133,14 +156,14 @@ SEASTAR_TEST_CASE(test_loading_shared_values_rehash) { SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_explicit_eviction) { return seastar::async([] { std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_shared_values shared_values; std::vector::entry_ptr> anchors_vec(num_loaders); std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { - return shared_values.get_or_load(k, loader).then([&] (auto entry_ptr) { + return shared_values.get_or_load(k, loader.get()).then([&] (auto entry_ptr) { anchors_vec[k] = std::move(entry_ptr); }); }).get(); @@ -156,22 +179,22 @@ SEASTAR_TEST_CASE(test_loading_shared_values_parallel_loading_explicit_eviction) SEASTAR_TEST_CASE(test_loading_cache_disable_and_enable) { return seastar::async([] { using namespace std::chrono; - load_count = 0; - utils::loading_cache loading_cache({num_loaders, 1h, 50ms}, testlog, loader); + loader loader; + utils::loading_cache loading_cache({num_loaders, 1h, 50ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); loading_cache.get(0).discard_result().get(); loading_cache.get(0).discard_result().get(); // Disable - load_count = 0; + loader.load_count() = 0; loading_cache.update_config({num_loaders, 0ms, 50ms}); sleep(150ms).get(); - BOOST_REQUIRE_EQUAL(load_count, 0); + BOOST_REQUIRE_EQUAL(loader.load_count(), 0); // Re-enable - load_count = 0; + loader.load_count() = 0; loading_cache.update_config({num_loaders, 1h, 50ms}); sleep(50ms).get(); @@ -179,18 +202,19 @@ SEASTAR_TEST_CASE(test_loading_cache_disable_and_enable) { loading_cache.get_ptr(0).discard_result().get(); loading_cache.get_ptr(0).discard_result().get(); - BOOST_REQUIRE(eventually_true([&] { return load_count >= 3; })); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; })); }); } SEASTAR_TEST_CASE(test_loading_cache_reset) { return seastar::async([] { using namespace std::chrono; + loader loader; utils::loading_cache loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } BOOST_REQUIRE_EQUAL(loading_cache.size(), num_loaders); @@ -204,11 +228,12 @@ SEASTAR_TEST_CASE(test_loading_cache_reset) { SEASTAR_TEST_CASE(test_loading_cache_update_config) { return seastar::async([] { using namespace std::chrono; - utils::loading_cache loading_cache({num_loaders, 1h, 1h}, testlog, loader); + loader loader; + utils::loading_cache loading_cache({num_loaders, 1h, 1h}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } BOOST_REQUIRE_EQUAL(loading_cache.size(), num_loaders); @@ -218,7 +243,7 @@ SEASTAR_TEST_CASE(test_loading_cache_update_config) { sleep(50ms).get(); for (int i = num_loaders; i < 2 * num_loaders; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 2); @@ -229,30 +254,30 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_same_key) { return seastar::async([] { using namespace std::chrono; std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_cache loading_cache(num_loaders, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); std::fill(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { - return loading_cache.get_ptr(k, loader).discard_result(); + return loading_cache.get_ptr(k, loader.get()).discard_result(); }).get(); // "loader" must be called exactly once - BOOST_REQUIRE_EQUAL(load_count, 1); + BOOST_REQUIRE_EQUAL(loader.load_count(), 1); BOOST_REQUIRE_EQUAL(loading_cache.size(), 1); }); } SEASTAR_THREAD_TEST_CASE(test_loading_cache_removing_key) { using namespace std::chrono; - load_count = 0; + loader loader; utils::loading_cache loading_cache(num_loaders, 100s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - loading_cache.get_ptr(0, loader).discard_result().get(); - BOOST_REQUIRE_EQUAL(load_count, 1); + loading_cache.get_ptr(0, loader.get()).discard_result().get(); + BOOST_REQUIRE_EQUAL(loader.load_count(), 1); BOOST_REQUIRE(loading_cache.find(0) != nullptr); loading_cache.remove(0); @@ -263,17 +288,17 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_different_keys) { return seastar::async([] { using namespace std::chrono; std::vector ivec(num_loaders); - load_count = 0; + loader loader; utils::loading_cache loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); std::iota(ivec.begin(), ivec.end(), 0); parallel_for_each(ivec, [&] (int& k) { - return loading_cache.get_ptr(k, loader).discard_result(); + return loading_cache.get_ptr(k, loader.get()).discard_result(); }).get(); - BOOST_REQUIRE_EQUAL(load_count, num_loaders); + BOOST_REQUIRE_EQUAL(loader.load_count(), num_loaders); BOOST_REQUIRE_EQUAL(loading_cache.size(), num_loaders); }); } @@ -281,10 +306,11 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_different_keys) { SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { return seastar::async([] { using namespace std::chrono; + loader loader; utils::loading_cache loading_cache(num_loaders, 20ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - loading_cache.get_ptr(0, loader).discard_result().get(); + loading_cache.get_ptr(0, loader.get()).discard_result().get(); // Check unprivileged section eviction BOOST_REQUIRE(loading_cache.size() == 1); @@ -292,7 +318,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); // Check privileged section eviction - loading_cache.get_ptr(0, loader).discard_result().get(); + loading_cache.get_ptr(0, loader.get()).discard_result().get(); BOOST_REQUIRE(loading_cache.find(0) != nullptr); sleep(20ms).get(); @@ -303,10 +329,11 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { return seastar::async([] { using namespace std::chrono; + loader loader; utils::loading_cache loading_cache(num_loaders, 30ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); - loading_cache.get_ptr(0, loader).discard_result().get(); + loading_cache.get_ptr(0, loader.get()).discard_result().get(); auto vp = loading_cache.find(0); auto load_time = steady_clock::now(); @@ -318,7 +345,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { } else { // If there was a delay and we weren't able to execute the next loop iteration during 20ms let's repopulate // the cache. - loading_cache.get_ptr(0, loader).discard_result().get(); + loading_cache.get_ptr(0, loader.get()).discard_result().get(); BOOST_TEST_MESSAGE("Test " << i << " was skipped. Repopulating..."); } vp = loading_cache.find(0); @@ -334,17 +361,18 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { SEASTAR_TEST_CASE(test_loading_cache_move_item_to_mru_list_front_on_sync_op) { return seastar::async([] { using namespace std::chrono; + loader loader; utils::loading_cache loading_cache(2, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < 2; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } auto vp0 = loading_cache.find(0); BOOST_REQUIRE(vp0 != nullptr); - loading_cache.get_ptr(2, loader).discard_result().get(); + loading_cache.get_ptr(2, loader.get()).discard_result().get(); // "0" should be at the beginning of the list and "1" right after it before we try to add a new entry to the // cache ("2"). And hence "1" should get evicted. @@ -358,43 +386,43 @@ SEASTAR_TEST_CASE(test_loading_cache_move_item_to_mru_list_front_on_sync_op) { SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_privileged_gen) { return seastar::async([] { using namespace std::chrono; - load_count = 0; - utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader); + loader loader; + utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Push the entry into the privileged section. Make sure it's being reloaded. loading_cache.get_ptr(0).discard_result().get(); loading_cache.get_ptr(0).discard_result().get(); sleep(60ms).get(); - BOOST_REQUIRE(eventually_true([&] { return load_count >= 3; })); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; })); }); } SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_unprivileged) { return seastar::async([] { using namespace std::chrono; - load_count = 0; - utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader); + loader loader; + utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Load one entry into the unprivileged section. // Make sure it's reloaded. loading_cache.get_ptr(0).discard_result().get(); sleep(60ms).get(); - BOOST_REQUIRE(eventually_true([&] { return load_count >= 2; })); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 2; })); }); } SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction) { return seastar::async([] { using namespace std::chrono; - load_count = 0; + loader loader; utils::loading_cache loading_cache(1, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { - loading_cache.get_ptr(i % 2, loader).discard_result().get(); + loading_cache.get_ptr(i % 2, loader.get()).discard_result().get(); } - BOOST_REQUIRE_EQUAL(load_count, num_loaders); + BOOST_REQUIRE_EQUAL(loader.load_count(), num_loaders); BOOST_REQUIRE_EQUAL(loading_cache.size(), 1); }); } @@ -402,19 +430,19 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction) { SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction_unprivileged_first) { return seastar::async([] { using namespace std::chrono; - load_count = 0; + loader loader; utils::loading_cache loading_cache(4, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Touch the value with the key "-1" twice - loading_cache.get_ptr(-1, loader).discard_result().get(); + loading_cache.get_ptr(-1, loader.get()).discard_result().get(); loading_cache.find(-1); for (int i = 0; i < num_loaders; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } - BOOST_REQUIRE_EQUAL(load_count, num_loaders + 1); + BOOST_REQUIRE_EQUAL(loader.load_count(), num_loaders + 1); BOOST_REQUIRE_EQUAL(loading_cache.size(), 4); // Make sure that the value we touched twice is still in the cache BOOST_REQUIRE(loading_cache.find(-1) != nullptr); @@ -424,16 +452,16 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction_unprivileged_first) { SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged) { return seastar::async([] { using namespace std::chrono; - load_count = 0; + loader loader; utils::loading_cache loading_cache(4, 10ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Touch the value with the key "-1" twice - loading_cache.get_ptr(-1, loader).discard_result().get(); + loading_cache.get_ptr(-1, loader.get()).discard_result().get(); loading_cache.find(-1); for (int i = 0; i < num_loaders; ++i) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } // Make sure that the value we touched twice is eventually evicted @@ -456,19 +484,20 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged_minimum_size) { // the same time for the batch to correctly execute. using namespace std::chrono; + loader loader; utils::loading_cache loading_cache(50, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Add 49 elements to privileged section for (int i = 0; i < 49; i++) { // Touch the value with the key "i" twice - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); loading_cache.find(i); } // Add 5 elements to unprivileged section for (int i = 50; i < 55; i++) { - loading_cache.get_ptr(i, loader).discard_result().get(); + loading_cache.get_ptr(i, loader.get()).discard_result().get(); } // Make sure that none of 5 elements were evicted @@ -600,8 +629,8 @@ SEASTAR_TEST_CASE(test_loading_cache_section_size_correctly_calculated) { SEASTAR_TEST_CASE(test_loading_cache_reload_during_eviction) { return seastar::async([] { using namespace std::chrono; - load_count = 0; - utils::loading_cache loading_cache({1, 100ms, 10ms}, testlog, loader); + loader loader; + utils::loading_cache loading_cache({1, 100ms, 10ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); auto curr_time = lowres_clock::now(); @@ -619,7 +648,6 @@ SEASTAR_TEST_CASE(test_loading_cache_reload_during_eviction) { SEASTAR_THREAD_TEST_CASE(test_loading_cache_remove_leaves_no_old_entries_behind) { using namespace std::chrono; - load_count = 0; auto load_v1 = [] (auto key) { return make_ready_future("v1"); }; auto load_v2 = [] (auto key) { return make_ready_future("v2"); }; From 0841483d68522dbcd4ffa333d08011070df4eea7 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 26 Dec 2024 14:57:43 +0200 Subject: [PATCH 5/6] utils: loading_cache: make clock_type a template parameter So the unit test can use manual_clock rather than lowres_clock which can be flaky (in particular in debug mode). Signed-off-by: Benny Halevy --- utils/loading_cache.hh | 55 ++++++++++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 21 deletions(-) diff --git a/utils/loading_cache.hh b/utils/loading_cache.hh index 8f8533679f99..82a3fd3a7b5b 100644 --- a/utils/loading_cache.hh +++ b/utils/loading_cache.hh @@ -33,12 +33,15 @@ namespace utils { enum class loading_cache_reload_enabled { no, yes }; -struct loading_cache_config final { +template +struct loading_cache_config_base final { size_t max_size = 0; - seastar::lowres_clock::duration expiry; - seastar::lowres_clock::duration refresh; + Clock::duration expiry; + Clock::duration refresh; }; +using loading_cache_config = loading_cache_config_base; + template struct simple_entry_size { size_t operator()(const Tp& val) { @@ -111,10 +114,16 @@ template, typename LoadingSharedValuesStats = utils::do_nothing_loading_shared_values_stats, typename LoadingCacheStats = utils::do_nothing_loading_cache_stats, + typename Clock = seastar::lowres_clock, typename Alloc = std::pmr::polymorphic_allocator<>> class loading_cache { +public: + using config = loading_cache_config; - using loading_cache_clock_type = seastar::lowres_clock; +private: + using loading_cache_clock_type = Clock; + using time_point = loading_cache_clock_type::time_point; + using duration = loading_cache_clock_type::duration; using safe_link_list_hook = bi::list_base_hook>; class timestamped_val { @@ -206,7 +215,7 @@ public: class entry_is_too_big : public std::exception {}; private: - loading_cache(loading_cache_config cfg, logging::logger& logger) + loading_cache(config cfg, logging::logger& logger) : _cfg(std::move(cfg)) , _logger(logger) , _timer([this] { on_timer(); }) @@ -214,14 +223,18 @@ private: static_assert(noexcept(LoadingCacheStats::inc_unprivileged_on_cache_size_eviction()), "LoadingCacheStats::inc_unprivileged_on_cache_size_eviction must be non-throwing"); static_assert(noexcept(LoadingCacheStats::inc_privileged_on_cache_size_eviction()), "LoadingCacheStats::inc_privileged_on_cache_size_eviction must be non-throwing"); + _logger.debug("Loading cache; max_size: {}, expiry: {}ms, refresh: {}ms", _cfg.max_size, + std::chrono::duration_cast(_cfg.expiry).count(), + std::chrono::duration_cast(_cfg.refresh).count()); + if (!validate_config(_cfg)) { throw exceptions::configuration_exception("loading_cache: caching is enabled but refresh period and/or max_size are zero"); } } - bool validate_config(const loading_cache_config& cfg) const noexcept { + bool validate_config(const config& cfg) const noexcept { // Sanity check: if expiration period is given then non-zero refresh period and maximal size are required - if (cfg.expiry != loading_cache_clock_type::duration(0) && (cfg.max_size == 0 || cfg.refresh == loading_cache_clock_type::duration(0))) { + if (cfg.expiry != duration(0) && (cfg.max_size == 0 || cfg.refresh == duration(0))) { return false; } @@ -231,7 +244,7 @@ private: public: template requires std::is_invocable_r_v, Func, const key_type&> - loading_cache(loading_cache_config cfg, logging::logger& logger, Func&& load) + loading_cache(config cfg, logging::logger& logger, Func&& load) : loading_cache(std::move(cfg), logger) { static_assert(ReloadEnabled == loading_cache_reload_enabled::yes, "This constructor should only be invoked when ReloadEnabled == loading_cache_reload_enabled::yes"); @@ -247,8 +260,8 @@ public: _timer.arm(_timer_period); } - loading_cache(size_t max_size, lowres_clock::duration expiry, logging::logger& logger) - : loading_cache({max_size, expiry, loading_cache_clock_type::time_point::max().time_since_epoch()}, logger) + loading_cache(size_t max_size, duration expiry, logging::logger& logger) + : loading_cache({max_size, expiry, time_point::max().time_since_epoch()}, logger) { static_assert(ReloadEnabled == loading_cache_reload_enabled::no, "This constructor should only be invoked when ReloadEnabled == loading_cache_reload_enabled::no"); @@ -273,7 +286,7 @@ public: remove_if([](const value_type&){ return true; }); } - bool update_config(utils::loading_cache_config cfg) { + bool update_config(config cfg) { _logger.info("Updating loading cache; max_size: {}, expiry: {}ms, refresh: {}ms", cfg.max_size, std::chrono::duration_cast(cfg.expiry).count(), std::chrono::duration_cast(cfg.refresh).count()); @@ -291,8 +304,8 @@ public: // * If caching is disabled and it's being enabled here on update_config, we also need to arm the timer, so that the changes on config // can take place if (_timer.armed() || - (!caching_enabled() && _updated_cfg->expiry != loading_cache_clock_type::duration(0))) { - _timer.rearm(loading_cache_clock_type::now() + loading_cache_clock_type::duration(std::chrono::milliseconds(1))); + (!caching_enabled() && _updated_cfg->expiry != duration(0))) { + _timer.rearm(loading_cache_clock_type::now() + duration(std::chrono::milliseconds(1))); } return true; @@ -469,7 +482,7 @@ private: } bool caching_enabled() const { - return _cfg.expiry != lowres_clock::duration(0); + return _cfg.expiry != duration(0); } static void destroy_ts_value(ts_value_lru_entry* val) noexcept { @@ -676,7 +689,7 @@ private: // If the config was updated after on_timer and before this continuation finished // it's necessary to run on_timer again to make sure that everything will be reloaded correctly if (_updated_cfg) { - _timer.arm(loading_cache_clock_type::now() + loading_cache_clock_type::duration(std::chrono::milliseconds(1))); + _timer.arm(loading_cache_clock_type::now() + duration(std::chrono::milliseconds(1))); } else { _timer.arm(loading_cache_clock_type::now() + _timer_period); } @@ -690,16 +703,16 @@ private: size_t _privileged_section_size = 0; size_t _unprivileged_section_size = 0; loading_cache_clock_type::duration _timer_period; - loading_cache_config _cfg; - std::optional _updated_cfg; + config _cfg; + std::optional _updated_cfg; logging::logger& _logger; std::function(const Key&)> _load; timer _timer; seastar::gate _timer_reads_gate; }; -template -class loading_cache::timestamped_val::value_ptr { +template +class loading_cache::timestamped_val::value_ptr { private: using loading_values_type = typename timestamped_val::loading_values_type; @@ -728,8 +741,8 @@ public: }; /// \brief This is and LRU list entry which is also an anchor for a loading_cache value. -template -class loading_cache::timestamped_val::lru_entry : public safe_link_list_hook { +template +class loading_cache::timestamped_val::lru_entry : public safe_link_list_hook { private: using loading_values_type = typename timestamped_val::loading_values_type; From 32b7cab917dabca67d8a54d3ba3055261bdf6b40 Mon Sep 17 00:00:00 2001 From: Benny Halevy Date: Thu, 26 Dec 2024 14:57:43 +0200 Subject: [PATCH 6/6] tests: loading_cache_test: use manual_clock Relying on a real-time clock like lowres_clock can be flaky (in particular in debug mode). Use manual_clock instead to harden the test against timing issues. Fixes #20322 Signed-off-by: Benny Halevy --- test/boost/loading_cache_test.cc | 92 ++++++++++++++++++-------------- test/lib/eventually.cc | 8 +++ test/lib/eventually.hh | 7 ++- 3 files changed, 66 insertions(+), 41 deletions(-) diff --git a/test/boost/loading_cache_test.cc b/test/boost/loading_cache_test.cc index 88a1ed3fe4d0..bbf5c5f4ca46 100644 --- a/test/boost/loading_cache_test.cc +++ b/test/boost/loading_cache_test.cc @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -35,6 +36,11 @@ BOOST_AUTO_TEST_SUITE(loading_cache_test) +template > +using loading_cache_for_test = utils::loading_cache, std::equal_to, utils::do_nothing_loading_shared_values_stats, utils::do_nothing_loading_cache_stats, + manual_clock>; + /// Get a random integer in the [0, max) range. /// \param max bound of the random value range /// \return The uniformly distributed random integer from the [0, \ref max) range. @@ -51,10 +57,10 @@ class loader { int load_count = 0; future load(const int& k) { - testlog.debug("load: start: load_count={}", load_count); - co_await sleep(tests::random::get_int(1, 5) * 1ms); + testlog.debug("{}: load: start: load_count={}", now(), load_count); + co_await manual_clock_sleep_fn(tests::random::get_int(1, 5) * 1ms); ++load_count; - testlog.debug("load: done: load_count={}", load_count); + testlog.debug("{}: load: done: load_count={}", now(), load_count); co_return test_string; } }; @@ -64,11 +70,17 @@ class loader { public: loader() : _impl(make_lw_shared()) {} + static long now() { + return std::chrono::duration_cast(manual_clock::now().time_since_epoch()).count(); + } + int& load_count() { + testlog.debug("{}: load_count={}", now(), _impl->load_count); return _impl->load_count; } const int& load_count() const { + testlog.debug("{}: load_count={}", now(), _impl->load_count); return _impl->load_count; } @@ -180,7 +192,7 @@ SEASTAR_TEST_CASE(test_loading_cache_disable_and_enable) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache({num_loaders, 1h, 50ms}, testlog, loader.get()); + loading_cache_for_test loading_cache({num_loaders, 1h, 50ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); loading_cache.get(0).discard_result().get(); @@ -190,19 +202,19 @@ SEASTAR_TEST_CASE(test_loading_cache_disable_and_enable) { loader.load_count() = 0; loading_cache.update_config({num_loaders, 0ms, 50ms}); - sleep(150ms).get(); + manual_clock_sleep_fn(150ms).get(); BOOST_REQUIRE_EQUAL(loader.load_count(), 0); // Re-enable loader.load_count() = 0; loading_cache.update_config({num_loaders, 1h, 50ms}); - sleep(50ms).get(); + manual_clock_sleep_fn(50ms).get(); // Push the entry into the privileged section. Make sure it's being reloaded. loading_cache.get_ptr(0).discard_result().get(); loading_cache.get_ptr(0).discard_result().get(); - BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; })); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; }, manual_clock_sleep_fn)); }); } @@ -210,7 +222,7 @@ SEASTAR_TEST_CASE(test_loading_cache_reset) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(num_loaders, 1h, testlog); + loading_cache_for_test loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { @@ -229,7 +241,7 @@ SEASTAR_TEST_CASE(test_loading_cache_update_config) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache({num_loaders, 1h, 1h}, testlog, loader.get()); + loading_cache_for_test loading_cache({num_loaders, 1h, 1h}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { @@ -240,13 +252,13 @@ SEASTAR_TEST_CASE(test_loading_cache_update_config) { loading_cache.update_config({2, 50ms, 50ms}); - sleep(50ms).get(); + manual_clock_sleep_fn(50ms).get(); for (int i = num_loaders; i < 2 * num_loaders; ++i) { loading_cache.get_ptr(i, loader.get()).discard_result().get(); } - REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 2); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 2, manual_clock_sleep_fn); }); } @@ -255,7 +267,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_same_key) { using namespace std::chrono; std::vector ivec(num_loaders); loader loader; - utils::loading_cache loading_cache(num_loaders, 1s, testlog); + loading_cache_for_test loading_cache(num_loaders, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); std::fill(ivec.begin(), ivec.end(), 0); @@ -273,7 +285,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_same_key) { SEASTAR_THREAD_TEST_CASE(test_loading_cache_removing_key) { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(num_loaders, 100s, testlog); + loading_cache_for_test loading_cache(num_loaders, 100s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); loading_cache.get_ptr(0, loader.get()).discard_result().get(); @@ -289,7 +301,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_different_keys) { using namespace std::chrono; std::vector ivec(num_loaders); loader loader; - utils::loading_cache loading_cache(num_loaders, 1h, testlog); + loading_cache_for_test loading_cache(num_loaders, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); std::iota(ivec.begin(), ivec.end(), 0); @@ -307,22 +319,22 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_eviction) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(num_loaders, 20ms, testlog); + loading_cache_for_test loading_cache(num_loaders, 20ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); loading_cache.get_ptr(0, loader.get()).discard_result().get(); // Check unprivileged section eviction BOOST_REQUIRE(loading_cache.size() == 1); - sleep(20ms).get(); - REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); + manual_clock_sleep_fn(20ms).get(); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0, manual_clock_sleep_fn); // Check privileged section eviction loading_cache.get_ptr(0, loader.get()).discard_result().get(); BOOST_REQUIRE(loading_cache.find(0) != nullptr); - sleep(20ms).get(); - REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); + manual_clock_sleep_fn(20ms).get(); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0, manual_clock_sleep_fn); }); } @@ -330,7 +342,7 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(num_loaders, 30ms, testlog); + loading_cache_for_test loading_cache(num_loaders, 30ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); loading_cache.get_ptr(0, loader.get()).discard_result().get(); @@ -350,11 +362,11 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_expiry_reset_on_sync_op) { } vp = loading_cache.find(0); load_time = steady_clock::now(); - sleep(10ms).get(); + manual_clock_sleep_fn(10ms).get(); } - sleep(30ms).get(); - REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); + manual_clock_sleep_fn(30ms).get(); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0, manual_clock_sleep_fn); }); } @@ -362,7 +374,7 @@ SEASTAR_TEST_CASE(test_loading_cache_move_item_to_mru_list_front_on_sync_op) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(2, 1h, testlog); + loading_cache_for_test loading_cache(2, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < 2; ++i) { @@ -387,13 +399,13 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_privileged_gen) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); + loading_cache_for_test loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Push the entry into the privileged section. Make sure it's being reloaded. loading_cache.get_ptr(0).discard_result().get(); loading_cache.get_ptr(0).discard_result().get(); - sleep(60ms).get(); - BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; })); + manual_clock_sleep_fn(60ms).get(); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 3; }, manual_clock_sleep_fn)); }); } @@ -401,13 +413,13 @@ SEASTAR_TEST_CASE(test_loading_cache_loading_reloading_unprivileged) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); + loading_cache_for_test loading_cache({num_loaders, 100ms, 20ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Load one entry into the unprivileged section. // Make sure it's reloaded. loading_cache.get_ptr(0).discard_result().get(); - sleep(60ms).get(); - BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 2; })); + manual_clock_sleep_fn(60ms).get(); + BOOST_REQUIRE(eventually_true([&] { return loader.load_count() >= 2; }, manual_clock_sleep_fn)); }); } @@ -415,7 +427,7 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(1, 1s, testlog); + loading_cache_for_test loading_cache(1, 1s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); for (int i = 0; i < num_loaders; ++i) { @@ -431,7 +443,7 @@ SEASTAR_TEST_CASE(test_loading_cache_max_size_eviction_unprivileged_first) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(4, 1h, testlog); + loading_cache_for_test loading_cache(4, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Touch the value with the key "-1" twice @@ -453,7 +465,7 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(4, 10ms, testlog); + loading_cache_for_test loading_cache(4, 10ms, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Touch the value with the key "-1" twice @@ -465,8 +477,8 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged) { } // Make sure that the value we touched twice is eventually evicted - REQUIRE_EVENTUALLY_EQUAL::value_ptr>([&] { return loading_cache.find(-1); }, nullptr); - REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0); + REQUIRE_EVENTUALLY_EQUAL::value_ptr>([&] { return loading_cache.find(-1); }, nullptr, manual_clock_sleep_fn); + REQUIRE_EVENTUALLY_EQUAL([&] { return loading_cache.size(); }, 0, manual_clock_sleep_fn); }); } @@ -485,7 +497,7 @@ SEASTAR_TEST_CASE(test_loading_cache_eviction_unprivileged_minimum_size) { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache(50, 1h, testlog); + loading_cache_for_test loading_cache(50, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // Add 49 elements to privileged section @@ -523,7 +535,7 @@ SEASTAR_TEST_CASE(test_loading_cache_section_size_correctly_calculated) { auto load_len95 = [] (const int& key) { return make_ready_future(tests::random::get_sstring(95)); }; using namespace std::chrono; - utils::loading_cache loading_cache(100, 1h, testlog); + loading_cache_for_test loading_cache(100, 1h, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); BOOST_REQUIRE_EQUAL(loading_cache.privileged_section_memory_footprint(), 0); @@ -630,7 +642,7 @@ SEASTAR_TEST_CASE(test_loading_cache_reload_during_eviction) { return seastar::async([] { using namespace std::chrono; loader loader; - utils::loading_cache loading_cache({1, 100ms, 10ms}, testlog, loader.get()); + loading_cache_for_test loading_cache({1, 100ms, 10ms}, testlog, loader.get()); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); auto curr_time = lowres_clock::now(); @@ -654,7 +666,7 @@ SEASTAR_THREAD_TEST_CASE(test_loading_cache_remove_leaves_no_old_entries_behind) auto load_v3 = [] (auto key) { return make_ready_future("v3"); }; { - utils::loading_cache loading_cache(num_loaders, 100s, testlog); + loading_cache_for_test loading_cache(num_loaders, 100s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // @@ -692,7 +704,7 @@ SEASTAR_THREAD_TEST_CASE(test_loading_cache_remove_leaves_no_old_entries_behind) // Test remove_if() { - utils::loading_cache loading_cache(num_loaders, 100s, testlog); + loading_cache_for_test loading_cache(num_loaders, 100s, testlog); auto stop_cache_reload = seastar::defer([&loading_cache] { loading_cache.stop().get(); }); // diff --git a/test/lib/eventually.cc b/test/lib/eventually.cc index c90a3d543eca..c41d76ad2599 100644 --- a/test/lib/eventually.cc +++ b/test/lib/eventually.cc @@ -11,3 +11,11 @@ sleep_fn seastar_sleep_fn = [] (std::chrono::milliseconds ms) -> future<> { return seastar::sleep(ms); }; + +sleep_fn manual_clock_sleep_fn = [] (std::chrono::milliseconds ms) -> future<> { + auto end = manual_clock::now() + ms; + while (manual_clock::now() < end) { + manual_clock::advance(std::chrono::milliseconds(1)); + co_await yield(); + } +}; diff --git a/test/lib/eventually.hh b/test/lib/eventually.hh index 7bd085104975..7ef990171476 100644 --- a/test/lib/eventually.hh +++ b/test/lib/eventually.hh @@ -10,8 +10,11 @@ #include +#include +#include #include #include +#include #include "seastarx.hh" @@ -19,6 +22,8 @@ using sleep_fn = std::function(std::chrono::milliseconds)>; extern sleep_fn seastar_sleep_fn; +extern sleep_fn manual_clock_sleep_fn; + inline void eventually(noncopyable_function f, size_t max_attempts = 17, sleep_fn sleep = seastar_sleep_fn) { size_t attempts = 0; @@ -46,7 +51,7 @@ bool eventually_true(noncopyable_function f, sleep_fn sleep = seastar_s } if (++attempts < max_attempts) { - seastar::sleep(std::chrono::milliseconds(1 << attempts)).get(); + sleep(std::chrono::milliseconds(1 << attempts)).get(); } else { return false; }