diff --git a/CMakeLists.txt b/CMakeLists.txt index c223513227..7f7238e69e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -661,6 +661,7 @@ set(SOURCES cache/compressed_secondary_cache.cc cache/lru_cache.cc cache/secondary_cache.cc + cache/secondary_cache_adapter.cc cache/sharded_cache.cc db/arena_wrapped_db_iter.cc db/blob/blob_contents.cc diff --git a/HISTORY.md b/HISTORY.md index a7afd99db4..589085a376 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -14,6 +14,10 @@ * Add statistics rocksdb.secondary.cache.filter.hits, rocksdb.secondary.cache.index.hits, and rocksdb.secondary.cache.filter.hits * Added a new PerfContext counter `internal_merge_point_lookup_count` which tracks the number of Merge operands applied while serving point lookup queries. * Add new statistics rocksdb.table.open.prefetch.tail.read.bytes, rocksdb.table.open.prefetch.tail.{miss|hit} +* Add support for SecondaryCache with HyperClockCache (`HyperClockCacheOptions` inherits `secondary_cache` option from `ShardedCacheOptions`) + +### Public API Changes +* Changed various functions and features in `Cache` that are mostly relevant to custom implementations or wrappers. Especially, asychronous lookup functionality is moved from `Lookup()` to a new `StartAsyncLookup()` function. ## 8.0.0 (02/19/2023) ### Behavior changes diff --git a/TARGETS b/TARGETS index 9fe58577ae..b6b4eee786 100644 --- a/TARGETS +++ b/TARGETS @@ -19,6 +19,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "cache/compressed_secondary_cache.cc", "cache/lru_cache.cc", "cache/secondary_cache.cc", + "cache/secondary_cache_adapter.cc", "cache/sharded_cache.cc", "db/arena_wrapped_db_iter.cc", "db/blob/blob_contents.cc", @@ -363,6 +364,7 @@ cpp_library_wrapper(name="rocksdb_whole_archive_lib", srcs=[ "cache/compressed_secondary_cache.cc", "cache/lru_cache.cc", "cache/secondary_cache.cc", + "cache/secondary_cache_adapter.cc", "cache/sharded_cache.cc", "db/arena_wrapped_db_iter.cc", "db/blob/blob_contents.cc", diff --git a/cache/cache.cc b/cache/cache.cc index eb3002cc35..a65f5ec4f8 100644 --- a/cache/cache.cc +++ b/cache/cache.cc @@ -114,4 +114,45 @@ Status Cache::CreateFromString(const ConfigOptions& config_options, } return status; } + +bool Cache::AsyncLookupHandle::IsReady() { + return pending_handle == nullptr || pending_handle->IsReady(); +} + +bool Cache::AsyncLookupHandle::IsPending() { return pending_handle != nullptr; } + +Cache::Handle* Cache::AsyncLookupHandle::Result() { + assert(!IsPending()); + return result_handle; +} + +void Cache::StartAsyncLookup(AsyncLookupHandle& async_handle) { + async_handle.found_dummy_entry = false; // in case re-used + assert(!async_handle.IsPending()); + async_handle.result_handle = + Lookup(async_handle.key, async_handle.helper, async_handle.create_context, + async_handle.priority, async_handle.stats); +} + +Cache::Handle* Cache::Wait(AsyncLookupHandle& async_handle) { + WaitAll(&async_handle, 1); + return async_handle.Result(); +} + +void Cache::WaitAll(AsyncLookupHandle* async_handles, size_t count) { + for (size_t i = 0; i < count; ++i) { + if (async_handles[i].IsPending()) { + // If a pending handle gets here, it should be marked at "to be handled + // by a caller" by that caller erasing the pending_cache on it. + assert(async_handles[i].pending_cache == nullptr); + } + } +} + +void Cache::SetEvictionCallback(EvictionCallback&& fn) { + // Overwriting non-empty with non-empty could indicate a bug + assert(!eviction_callback_ || !fn); + eviction_callback_ = std::move(fn); +} + } // namespace ROCKSDB_NAMESPACE diff --git a/cache/cache_bench_tool.cc b/cache/cache_bench_tool.cc index a1f698f21f..1d93c1d960 100644 --- a/cache/cache_bench_tool.cc +++ b/cache/cache_bench_tool.cc @@ -547,7 +547,7 @@ class CacheBench { } // do lookup handle = cache_->Lookup(key, &helper2, /*context*/ nullptr, - Cache::Priority::LOW, true); + Cache::Priority::LOW); if (handle) { if (!FLAGS_lean) { // do something with the data @@ -576,7 +576,7 @@ class CacheBench { } // do lookup handle = cache_->Lookup(key, &helper2, /*context*/ nullptr, - Cache::Priority::LOW, true); + Cache::Priority::LOW); if (handle) { if (!FLAGS_lean) { // do something with the data diff --git a/cache/cache_test.cc b/cache/cache_test.cc index 9c4144b68f..febed5b427 100644 --- a/cache/cache_test.cc +++ b/cache/cache_test.cc @@ -18,6 +18,7 @@ #include "cache/lru_cache.h" #include "cache/typed_cache.h" #include "port/stack_trace.h" +#include "test_util/secondary_cache_test_util.h" #include "test_util/testharness.h" #include "util/coding.h" #include "util/string_util.h" @@ -81,13 +82,10 @@ const Cache::CacheItemHelper kEraseOnDeleteHelper2{ Cache* cache = static_cast(value); cache->Erase(EncodeKey16Bytes(1234)); }}; - -const std::string kLRU = "lru"; -const std::string kHyperClock = "hyper_clock"; - } // anonymous namespace -class CacheTest : public testing::TestWithParam { +class CacheTest : public testing::Test, + public secondary_cache_test_util::WithCacheTypeParam { public: static CacheTest* current_; static std::string type_; @@ -107,8 +105,6 @@ class CacheTest : public testing::TestWithParam { std::shared_ptr cache_; std::shared_ptr cache2_; - size_t estimated_value_size_ = 1; - CacheTest() : cache_(NewCache(kCacheSize, kNumShardBits, false)), cache2_(NewCache(kCacheSize2, kNumShardBits2, false)) { @@ -118,41 +114,6 @@ class CacheTest : public testing::TestWithParam { ~CacheTest() override {} - std::shared_ptr NewCache(size_t capacity) { - auto type = GetParam(); - if (type == kLRU) { - return NewLRUCache(capacity); - } - if (type == kHyperClock) { - return HyperClockCacheOptions( - capacity, estimated_value_size_ /*estimated_value_size*/) - .MakeSharedCache(); - } - return nullptr; - } - - std::shared_ptr NewCache( - size_t capacity, int num_shard_bits, bool strict_capacity_limit, - CacheMetadataChargePolicy charge_policy = kDontChargeCacheMetadata) { - auto type = GetParam(); - if (type == kLRU) { - LRUCacheOptions co; - co.capacity = capacity; - co.num_shard_bits = num_shard_bits; - co.strict_capacity_limit = strict_capacity_limit; - co.high_pri_pool_ratio = 0; - co.metadata_charge_policy = charge_policy; - return NewLRUCache(co); - } - if (type == kHyperClock) { - return HyperClockCacheOptions(capacity, 1 /*estimated_value_size*/, - num_shard_bits, strict_capacity_limit, - nullptr /*allocator*/, charge_policy) - .MakeSharedCache(); - } - return nullptr; - } - // These functions encode/decode keys in tests cases that use // int keys. // Currently, HyperClockCache requires keys to be 16B long, whereas @@ -186,8 +147,8 @@ class CacheTest : public testing::TestWithParam { void Insert(std::shared_ptr cache, int key, int value, int charge = 1) { - EXPECT_OK( - cache->Insert(EncodeKey(key), EncodeValue(value), &kHelper, charge)); + EXPECT_OK(cache->Insert(EncodeKey(key), EncodeValue(value), &kHelper, + charge, /*handle*/ nullptr, Cache::Priority::HIGH)); } void Erase(std::shared_ptr cache, int key) { @@ -995,8 +956,9 @@ TEST_P(CacheTest, GetChargeAndDeleter) { } INSTANTIATE_TEST_CASE_P(CacheTestInstance, CacheTest, - testing::Values(kLRU, kHyperClock)); -INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest, testing::Values(kLRU)); + secondary_cache_test_util::GetTestingCacheTypes()); +INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest, + testing::Values(secondary_cache_test_util::kLRU)); } // namespace ROCKSDB_NAMESPACE diff --git a/cache/charged_cache.cc b/cache/charged_cache.cc index 3aec09ce86..e44288ecd6 100644 --- a/cache/charged_cache.cc +++ b/cache/charged_cache.cc @@ -35,10 +35,8 @@ Status ChargedCache::Insert(const Slice& key, ObjectPtr obj, Cache::Handle* ChargedCache::Lookup(const Slice& key, const CacheItemHelper* helper, CreateContext* create_context, - Priority priority, bool wait, - Statistics* stats) { - auto handle = - target_->Lookup(key, helper, create_context, priority, wait, stats); + Priority priority, Statistics* stats) { + auto handle = target_->Lookup(key, helper, create_context, priority, stats); // Lookup may promote the KV pair from the secondary cache to the primary // cache. So we directly call the reservation manager to update the total // memory used in the cache. @@ -50,6 +48,16 @@ Cache::Handle* ChargedCache::Lookup(const Slice& key, return handle; } +void ChargedCache::WaitAll(AsyncLookupHandle* async_handles, size_t count) { + target_->WaitAll(async_handles, count); + // In case of any promotions. Although some could finish by return of + // StartAsyncLookup, Wait/WaitAll will generally be used, so simpler to + // update here. + assert(cache_res_mgr_); + cache_res_mgr_->UpdateCacheReservation(target_->GetUsage()) + .PermitUncheckedError(); +} + bool ChargedCache::Release(Cache::Handle* handle, bool useful, bool erase_if_last_ref) { size_t memory_used_delta = target_->GetUsage(handle); diff --git a/cache/charged_cache.h b/cache/charged_cache.h index f39e2cb868..f2eacb9edd 100644 --- a/cache/charged_cache.h +++ b/cache/charged_cache.h @@ -28,9 +28,11 @@ class ChargedCache : public CacheWrapper { Cache::Handle* Lookup(const Slice& key, const CacheItemHelper* helper, CreateContext* create_context, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override; + void WaitAll(AsyncLookupHandle* async_handles, size_t count) override; + bool Release(Cache::Handle* handle, bool useful, bool erase_if_last_ref = false) override; bool Release(Cache::Handle* handle, bool erase_if_last_ref = false) override; diff --git a/cache/clock_cache.cc b/cache/clock_cache.cc index 2a26cf07f2..12be0babef 100644 --- a/cache/clock_cache.cc +++ b/cache/clock_cache.cc @@ -13,6 +13,7 @@ #include #include "cache/cache_key.h" +#include "cache/secondary_cache_adapter.h" #include "logging/logging.h" #include "monitoring/perf_context_imp.h" #include "monitoring/statistics.h" @@ -50,13 +51,7 @@ inline uint64_t GetInitialCountdown(Cache::Priority priority) { } } -inline void FreeDataMarkEmpty(ClockHandle& h, MemoryAllocator* allocator) { - // NOTE: in theory there's more room for parallelism if we copy the handle - // data and delay actions like this until after marking the entry as empty, - // but performance tests only show a regression by copying the few words - // of data. - h.FreeData(allocator); - +inline void MarkEmpty(ClockHandle& h) { #ifndef NDEBUG // Mark slot as empty, with assertion uint64_t meta = h.meta.exchange(0, std::memory_order_release); @@ -67,6 +62,16 @@ inline void FreeDataMarkEmpty(ClockHandle& h, MemoryAllocator* allocator) { #endif } +inline void FreeDataMarkEmpty(ClockHandle& h, MemoryAllocator* allocator) { + // NOTE: in theory there's more room for parallelism if we copy the handle + // data and delay actions like this until after marking the entry as empty, + // but performance tests only show a regression by copying the few words + // of data. + h.FreeData(allocator); + + MarkEmpty(h); +} + inline bool ClockUpdate(ClockHandle& h) { uint64_t meta = h.meta.load(std::memory_order_relaxed); @@ -124,14 +129,16 @@ void ClockHandleBasicData::FreeData(MemoryAllocator* allocator) const { HyperClockTable::HyperClockTable( size_t capacity, bool /*strict_capacity_limit*/, CacheMetadataChargePolicy metadata_charge_policy, - MemoryAllocator* allocator, const Opts& opts) + MemoryAllocator* allocator, + const Cache::EvictionCallback* eviction_callback, const Opts& opts) : length_bits_(CalcHashBits(capacity, opts.estimated_value_size, metadata_charge_policy)), length_bits_mask_((size_t{1} << length_bits_) - 1), occupancy_limit_(static_cast((uint64_t{1} << length_bits_) * kStrictLoadFactor)), array_(new HandleImpl[size_t{1} << length_bits_]), - allocator_(allocator) { + allocator_(allocator), + eviction_callback_(*eviction_callback) { if (metadata_charge_policy == CacheMetadataChargePolicy::kFullChargeCacheMetadata) { usage_ += size_t{GetTableSize()} * sizeof(HandleImpl); @@ -574,6 +581,34 @@ Status HyperClockTable::Insert(const ClockHandleBasicData& proto, return Status::OkOverwritten(); } +HyperClockTable::HandleImpl* HyperClockTable::CreateStandalone( + ClockHandleBasicData& proto, size_t capacity, bool strict_capacity_limit, + bool allow_uncharged) { + const size_t total_charge = proto.GetTotalCharge(); + if (strict_capacity_limit) { + Status s = ChargeUsageMaybeEvictStrict(total_charge, capacity, + /*need_evict_for_occupancy=*/false); + if (!s.ok()) { + if (allow_uncharged) { + proto.total_charge = 0; + } else { + return nullptr; + } + } + } else { + // Case strict_capacity_limit == false + bool success = + ChargeUsageMaybeEvictNonStrict(total_charge, capacity, + /*need_evict_for_occupancy=*/false); + if (!success) { + // Force the issue + usage_.fetch_add(total_charge, std::memory_order_relaxed); + } + } + + return StandaloneInsert(proto); +} + HyperClockTable::HandleImpl* HyperClockTable::Lookup( const UniqueId64x2& hashed_key) { size_t probe = 0; @@ -960,6 +995,9 @@ inline void HyperClockTable::Evict(size_t requested_charge, uint64_t max_clock_pointer = old_clock_pointer + (ClockHandle::kMaxCountdown << length_bits_); + // For key reconstructed from hash + UniqueId64x2 unhashed; + for (;;) { for (size_t i = 0; i < step_size; i++) { HandleImpl& h = array_[ModTableSize(Lower32of64(old_clock_pointer + i))]; @@ -968,7 +1006,17 @@ inline void HyperClockTable::Evict(size_t requested_charge, Rollback(h.hashed_key, &h); *freed_charge += h.GetTotalCharge(); *freed_count += 1; - FreeDataMarkEmpty(h, allocator_); + bool took_ownership = false; + if (eviction_callback_) { + took_ownership = + eviction_callback_(ClockCacheShard::ReverseHash( + h.GetHash(), &unhashed), + reinterpret_cast(&h)); + } + if (!took_ownership) { + h.FreeData(allocator_); + } + MarkEmpty(h); } } @@ -990,10 +1038,12 @@ template ClockCacheShard::ClockCacheShard( size_t capacity, bool strict_capacity_limit, CacheMetadataChargePolicy metadata_charge_policy, - MemoryAllocator* allocator, const typename Table::Opts& opts) + MemoryAllocator* allocator, + const Cache::EvictionCallback* eviction_callback, + const typename Table::Opts& opts) : CacheShardBase(metadata_charge_policy), table_(capacity, strict_capacity_limit, metadata_charge_policy, allocator, - opts), + eviction_callback, opts), capacity_(capacity), strict_capacity_limit_(strict_capacity_limit) { // Initial charge metadata should not exceed capacity @@ -1093,10 +1143,29 @@ Status ClockCacheShard
::Insert(const Slice& key, proto.value = value; proto.helper = helper; proto.total_charge = charge; - Status s = table_.Insert( - proto, handle, priority, capacity_.load(std::memory_order_relaxed), - strict_capacity_limit_.load(std::memory_order_relaxed)); - return s; + return table_.Insert(proto, handle, priority, + capacity_.load(std::memory_order_relaxed), + strict_capacity_limit_.load(std::memory_order_relaxed)); +} + +template +typename ClockCacheShard
::HandleImpl* +ClockCacheShard
::CreateStandalone(const Slice& key, + const UniqueId64x2& hashed_key, + Cache::ObjectPtr obj, + const Cache::CacheItemHelper* helper, + size_t charge, bool allow_uncharged) { + if (UNLIKELY(key.size() != kCacheKeySize)) { + return nullptr; + } + ClockHandleBasicData proto; + proto.hashed_key = hashed_key; + proto.value = obj; + proto.helper = helper; + proto.total_charge = charge; + return table_.CreateStandalone( + proto, capacity_.load(std::memory_order_relaxed), + strict_capacity_limit_.load(std::memory_order_relaxed), allow_uncharged); } template @@ -1226,11 +1295,12 @@ HyperClockCache::HyperClockCache( // get to table entries size_t per_shard = GetPerShardCapacity(); MemoryAllocator* alloc = this->memory_allocator(); + const Cache::EvictionCallback* eviction_callback = &eviction_callback_; InitShards([=](Shard* cs) { HyperClockTable::Opts opts; opts.estimated_value_size = estimated_value_size; new (cs) Shard(per_shard, strict_capacity_limit, metadata_charge_policy, - alloc, opts); + alloc, eviction_callback, opts); }); } @@ -1400,9 +1470,13 @@ std::shared_ptr HyperClockCacheOptions::MakeSharedCache() const { constexpr size_t min_shard_size = 32U * 1024U * 1024U; my_num_shard_bits = GetDefaultCacheShardBits(capacity, min_shard_size); } - return std::make_shared( + std::shared_ptr cache = std::make_shared( capacity, estimated_entry_charge, my_num_shard_bits, strict_capacity_limit, metadata_charge_policy, memory_allocator); + if (secondary_cache) { + cache = std::make_shared(cache, secondary_cache); + } + return cache; } } // namespace ROCKSDB_NAMESPACE diff --git a/cache/clock_cache.h b/cache/clock_cache.h index 454ee17047..fc5aef6cb4 100644 --- a/cache/clock_cache.h +++ b/cache/clock_cache.h @@ -405,13 +405,19 @@ class HyperClockTable { HyperClockTable(size_t capacity, bool strict_capacity_limit, CacheMetadataChargePolicy metadata_charge_policy, - MemoryAllocator* allocator, const Opts& opts); + MemoryAllocator* allocator, + const Cache::EvictionCallback* eviction_callback, + const Opts& opts); ~HyperClockTable(); Status Insert(const ClockHandleBasicData& proto, HandleImpl** handle, Cache::Priority priority, size_t capacity, bool strict_capacity_limit); + HandleImpl* CreateStandalone(ClockHandleBasicData& proto, size_t capacity, + bool strict_capacity_limit, + bool allow_uncharged); + HandleImpl* Lookup(const UniqueId64x2& hashed_key); bool Release(HandleImpl* handle, bool useful, bool erase_if_last_ref); @@ -537,6 +543,9 @@ class HyperClockTable { // From Cache, for deleter MemoryAllocator* const allocator_; + // A reference to Cache::eviction_callback_ + const Cache::EvictionCallback& eviction_callback_; + // We partition the following members into different cache lines // to avoid false sharing among Lookup, Release, Erase and Insert // operations in ClockCacheShard. @@ -562,7 +571,9 @@ class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase { public: ClockCacheShard(size_t capacity, bool strict_capacity_limit, CacheMetadataChargePolicy metadata_charge_policy, - MemoryAllocator* allocator, const typename Table::Opts& opts); + MemoryAllocator* allocator, + const Cache::EvictionCallback* eviction_callback, + const typename Table::Opts& opts); // For CacheShard concept using HandleImpl = typename Table::HandleImpl; @@ -603,6 +614,11 @@ class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase { Cache::ObjectPtr value, const Cache::CacheItemHelper* helper, size_t charge, HandleImpl** handle, Cache::Priority priority); + HandleImpl* CreateStandalone(const Slice& key, const UniqueId64x2& hashed_key, + Cache::ObjectPtr obj, + const Cache::CacheItemHelper* helper, + size_t charge, bool allow_uncharged); + HandleImpl* Lookup(const Slice& key, const UniqueId64x2& hashed_key); bool Release(HandleImpl* handle, bool useful, bool erase_if_last_ref); @@ -640,15 +656,10 @@ class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase { HandleImpl* Lookup(const Slice& key, const UniqueId64x2& hashed_key, const Cache::CacheItemHelper* /*helper*/, Cache::CreateContext* /*create_context*/, - Cache::Priority /*priority*/, bool /*wait*/, - Statistics* /*stats*/) { + Cache::Priority /*priority*/, Statistics* /*stats*/) { return Lookup(key, hashed_key); } - bool IsReady(HandleImpl* /*handle*/) { return true; } - - void Wait(HandleImpl* /*handle*/) {} - // Acquire/release N references void TEST_RefN(HandleImpl* handle, size_t n); void TEST_ReleaseN(HandleImpl* handle, size_t n); diff --git a/cache/compressed_secondary_cache.h b/cache/compressed_secondary_cache.h index 0e90945cb2..7b45ca8bd9 100644 --- a/cache/compressed_secondary_cache.h +++ b/cache/compressed_secondary_cache.h @@ -106,7 +106,7 @@ class CompressedSecondaryCache : public SecondaryCache { std::string GetPrintableOptions() const override; private: - friend class CompressedSecondaryCacheTest; + friend class CompressedSecondaryCacheTestBase; static constexpr std::array malloc_bin_sizes_{ 128, 256, 512, 1024, 2048, 4096, 8192, 16384}; diff --git a/cache/compressed_secondary_cache_test.cc b/cache/compressed_secondary_cache_test.cc index 399c9efded..1e41fc142b 100644 --- a/cache/compressed_secondary_cache_test.cc +++ b/cache/compressed_secondary_cache_test.cc @@ -18,16 +18,20 @@ namespace ROCKSDB_NAMESPACE { -using secondary_cache_test_util::GetHelper; -using secondary_cache_test_util::GetHelperFail; -using secondary_cache_test_util::TestCreateContext; -using secondary_cache_test_util::TestItem; +using secondary_cache_test_util::GetTestingCacheTypes; +using secondary_cache_test_util::WithCacheType; -class CompressedSecondaryCacheTest : public testing::Test, - public TestCreateContext { +// 16 bytes for HCC compatibility +const std::string key0 = "____ ____key0"; +const std::string key1 = "____ ____key1"; +const std::string key2 = "____ ____key2"; +const std::string key3 = "____ ____key3"; + +class CompressedSecondaryCacheTestBase : public testing::Test, + public WithCacheType { public: - CompressedSecondaryCacheTest() {} - ~CompressedSecondaryCacheTest() override = default; + CompressedSecondaryCacheTestBase() {} + ~CompressedSecondaryCacheTestBase() override = default; protected: void BasicTestHelper(std::shared_ptr sec_cache, @@ -36,7 +40,7 @@ class CompressedSecondaryCacheTest : public testing::Test, bool kept_in_sec_cache{true}; // Lookup an non-existent key. std::unique_ptr handle0 = - sec_cache->Lookup("k0", GetHelper(), this, true, /*advise_erase=*/true, + sec_cache->Lookup(key0, GetHelper(), this, true, /*advise_erase=*/true, kept_in_sec_cache); ASSERT_EQ(handle0, nullptr); @@ -45,22 +49,22 @@ class CompressedSecondaryCacheTest : public testing::Test, std::string str1(rnd.RandomString(1000)); TestItem item1(str1.data(), str1.length()); // A dummy handle is inserted if the item is inserted for the first time. - ASSERT_OK(sec_cache->Insert("k1", &item1, GetHelper())); + ASSERT_OK(sec_cache->Insert(key1, &item1, GetHelper())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 1); ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, 0); ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes, 0); std::unique_ptr handle1_1 = - sec_cache->Lookup("k1", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key1, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_EQ(handle1_1, nullptr); // Insert and Lookup the item k1 for the second time and advise erasing it. - ASSERT_OK(sec_cache->Insert("k1", &item1, GetHelper())); + ASSERT_OK(sec_cache->Insert(key1, &item1, GetHelper())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 1); std::unique_ptr handle1_2 = - sec_cache->Lookup("k1", GetHelper(), this, true, /*advise_erase=*/true, + sec_cache->Lookup(key1, GetHelper(), this, true, /*advise_erase=*/true, kept_in_sec_cache); ASSERT_NE(handle1_2, nullptr); ASSERT_FALSE(kept_in_sec_cache); @@ -81,21 +85,21 @@ class CompressedSecondaryCacheTest : public testing::Test, // Lookup the item k1 again. std::unique_ptr handle1_3 = - sec_cache->Lookup("k1", GetHelper(), this, true, /*advise_erase=*/true, + sec_cache->Lookup(key1, GetHelper(), this, true, /*advise_erase=*/true, kept_in_sec_cache); ASSERT_EQ(handle1_3, nullptr); // Insert and Lookup the item k2. std::string str2(rnd.RandomString(1000)); TestItem item2(str2.data(), str2.length()); - ASSERT_OK(sec_cache->Insert("k2", &item2, GetHelper())); + ASSERT_OK(sec_cache->Insert(key2, &item2, GetHelper())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 2); std::unique_ptr handle2_1 = - sec_cache->Lookup("k2", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key2, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_EQ(handle2_1, nullptr); - ASSERT_OK(sec_cache->Insert("k2", &item2, GetHelper())); + ASSERT_OK(sec_cache->Insert(key2, &item2, GetHelper())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 2); if (sec_cache_is_compressed) { ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, @@ -107,7 +111,7 @@ class CompressedSecondaryCacheTest : public testing::Test, ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes, 0); } std::unique_ptr handle2_2 = - sec_cache->Lookup("k2", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key2, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_NE(handle2_2, nullptr); std::unique_ptr val2 = @@ -177,25 +181,25 @@ class CompressedSecondaryCacheTest : public testing::Test, std::string str1(rnd.RandomString(1000)); TestItem item1(str1.data(), str1.length()); // Insert a dummy handle. - ASSERT_OK(sec_cache->Insert("k1", &item1, GetHelper())); + ASSERT_OK(sec_cache->Insert(key1, &item1, GetHelper())); // Insert k1. - ASSERT_OK(sec_cache->Insert("k1", &item1, GetHelper())); + ASSERT_OK(sec_cache->Insert(key1, &item1, GetHelper())); // Insert and Lookup the second item. std::string str2(rnd.RandomString(200)); TestItem item2(str2.data(), str2.length()); // Insert a dummy handle, k1 is not evicted. - ASSERT_OK(sec_cache->Insert("k2", &item2, GetHelper())); + ASSERT_OK(sec_cache->Insert(key2, &item2, GetHelper())); bool kept_in_sec_cache{false}; std::unique_ptr handle1 = - sec_cache->Lookup("k1", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key1, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_EQ(handle1, nullptr); // Insert k2 and k1 is evicted. - ASSERT_OK(sec_cache->Insert("k2", &item2, GetHelper())); + ASSERT_OK(sec_cache->Insert(key2, &item2, GetHelper())); std::unique_ptr handle2 = - sec_cache->Lookup("k2", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key2, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_NE(handle2, nullptr); std::unique_ptr val2 = @@ -204,17 +208,17 @@ class CompressedSecondaryCacheTest : public testing::Test, ASSERT_EQ(memcmp(val2->Buf(), item2.Buf(), item2.Size()), 0); // Insert k1 again and a dummy handle is inserted. - ASSERT_OK(sec_cache->Insert("k1", &item1, GetHelper())); + ASSERT_OK(sec_cache->Insert(key1, &item1, GetHelper())); std::unique_ptr handle1_1 = - sec_cache->Lookup("k1", GetHelper(), this, true, /*advise_erase=*/false, + sec_cache->Lookup(key1, GetHelper(), this, true, /*advise_erase=*/false, kept_in_sec_cache); ASSERT_EQ(handle1_1, nullptr); // Create Fails. SetFailCreate(true); std::unique_ptr handle2_1 = - sec_cache->Lookup("k2", GetHelper(), this, true, /*advise_erase=*/true, + sec_cache->Lookup(key2, GetHelper(), this, true, /*advise_erase=*/true, kept_in_sec_cache); ASSERT_EQ(handle2_1, nullptr); @@ -222,8 +226,8 @@ class CompressedSecondaryCacheTest : public testing::Test, std::string str3 = rnd.RandomString(10); TestItem item3(str3.data(), str3.length()); // The Status is OK because a dummy handle is inserted. - ASSERT_OK(sec_cache->Insert("k3", &item3, GetHelperFail())); - ASSERT_NOK(sec_cache->Insert("k3", &item3, GetHelperFail())); + ASSERT_OK(sec_cache->Insert(key3, &item3, GetHelperFail())); + ASSERT_NOK(sec_cache->Insert(key3, &item3, GetHelperFail())); sec_cache.reset(); } @@ -247,26 +251,22 @@ class CompressedSecondaryCacheTest : public testing::Test, secondary_cache_opts.enable_custom_split_merge = enable_custom_split_merge; std::shared_ptr secondary_cache = NewCompressedSecondaryCache(secondary_cache_opts); - LRUCacheOptions lru_cache_opts( + std::shared_ptr cache = NewCache( /*_capacity =*/1300, /*_num_shard_bits =*/0, - /*_strict_capacity_limit =*/false, /*_high_pri_pool_ratio =*/0.5, - /*_memory_allocator =*/nullptr, kDefaultToAdaptiveMutex, - kDefaultCacheMetadataChargePolicy, /*_low_pri_pool_ratio =*/0.0); - lru_cache_opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(lru_cache_opts); + /*_strict_capacity_limit =*/true, secondary_cache); std::shared_ptr stats = CreateDBStatistics(); get_perf_context()->Reset(); Random rnd(301); std::string str1 = rnd.RandomString(1001); auto item1_1 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1_1, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1_1, GetHelper(), str1.length())); std::string str2 = rnd.RandomString(1012); auto item2_1 = new TestItem(str2.data(), str2.length()); // After this Insert, primary cache contains k2 and secondary cache contains // k1's dummy item. - ASSERT_OK(cache->Insert("k2", item2_1, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2_1, GetHelper(), str2.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 1); ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, 0); ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes, 0); @@ -275,19 +275,19 @@ class CompressedSecondaryCacheTest : public testing::Test, auto item3_1 = new TestItem(str3.data(), str3.length()); // After this Insert, primary cache contains k3 and secondary cache contains // k1's dummy item and k2's dummy item. - ASSERT_OK(cache->Insert("k3", item3_1, GetHelper(), str3.length())); + ASSERT_OK(cache->Insert(key3, item3_1, GetHelper(), str3.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 2); // After this Insert, primary cache contains k1 and secondary cache contains // k1's dummy item, k2's dummy item, and k3's dummy item. auto item1_2 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1_2, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1_2, GetHelper(), str1.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 3); // After this Insert, primary cache contains k2 and secondary cache contains // k1's item, k2's dummy item, and k3's dummy item. auto item2_2 = new TestItem(str2.data(), str2.length()); - ASSERT_OK(cache->Insert("k2", item2_2, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2_2, GetHelper(), str2.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 1); if (sec_cache_is_compressed) { ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, @@ -302,7 +302,7 @@ class CompressedSecondaryCacheTest : public testing::Test, // After this Insert, primary cache contains k3 and secondary cache contains // k1's item and k2's item. auto item3_2 = new TestItem(str3.data(), str3.length()); - ASSERT_OK(cache->Insert("k3", item3_2, GetHelper(), str3.length())); + ASSERT_OK(cache->Insert(key3, item3_2, GetHelper(), str3.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 2); if (sec_cache_is_compressed) { ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, @@ -315,7 +315,7 @@ class CompressedSecondaryCacheTest : public testing::Test, } Cache::Handle* handle; - handle = cache->Lookup("k3", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key3, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); auto val3 = static_cast(cache->Value(handle)); @@ -324,13 +324,13 @@ class CompressedSecondaryCacheTest : public testing::Test, cache->Release(handle); // Lookup an non-existent key. - handle = cache->Lookup("k0", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key0, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_EQ(handle, nullptr); // This Lookup should just insert a dummy handle in the primary cache // and the k1 is still in the secondary cache. - handle = cache->Lookup("k1", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key1, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 1); @@ -342,7 +342,7 @@ class CompressedSecondaryCacheTest : public testing::Test, // This Lookup should erase k1 from the secondary cache and insert // it into primary cache; then k3 is demoted. // k2 and k3 are in secondary cache. - handle = cache->Lookup("k1", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key1, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 1); @@ -350,7 +350,7 @@ class CompressedSecondaryCacheTest : public testing::Test, cache->Release(handle); // k2 is still in secondary cache. - handle = cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 2); @@ -358,7 +358,7 @@ class CompressedSecondaryCacheTest : public testing::Test, // Testing SetCapacity(). ASSERT_OK(secondary_cache->SetCapacity(0)); - handle = cache->Lookup("k3", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key3, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_EQ(handle, nullptr); @@ -368,30 +368,30 @@ class CompressedSecondaryCacheTest : public testing::Test, ASSERT_EQ(capacity, 7000); auto item1_3 = new TestItem(str1.data(), str1.length()); // After this Insert, primary cache contains k1. - ASSERT_OK(cache->Insert("k1", item1_3, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key1, item1_3, GetHelper(), str2.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 3); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 4); auto item2_3 = new TestItem(str2.data(), str2.length()); // After this Insert, primary cache contains k2 and secondary cache contains // k1's dummy item. - ASSERT_OK(cache->Insert("k2", item2_3, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key2, item2_3, GetHelper(), str1.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 4); auto item1_4 = new TestItem(str1.data(), str1.length()); // After this Insert, primary cache contains k1 and secondary cache contains // k1's dummy item and k2's dummy item. - ASSERT_OK(cache->Insert("k1", item1_4, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key1, item1_4, GetHelper(), str2.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 5); auto item2_4 = new TestItem(str2.data(), str2.length()); // After this Insert, primary cache contains k2 and secondary cache contains // k1's real item and k2's dummy item. - ASSERT_OK(cache->Insert("k2", item2_4, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2_4, GetHelper(), str2.length())); ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 5); // This Lookup should just insert a dummy handle in the primary cache // and the k1 is still in the secondary cache. - handle = cache->Lookup("k1", GetHelper(), this, Cache::Priority::LOW, true, + handle = cache->Lookup(key1, GetHelper(), this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); @@ -419,27 +419,31 @@ class CompressedSecondaryCacheTest : public testing::Test, std::shared_ptr secondary_cache = NewCompressedSecondaryCache(secondary_cache_opts); - LRUCacheOptions opts( + std::shared_ptr cache = NewCache( /*_capacity=*/1300, /*_num_shard_bits=*/0, - /*_strict_capacity_limit=*/false, /*_high_pri_pool_ratio=*/0.5, - /*_memory_allocator=*/nullptr, kDefaultToAdaptiveMutex, - kDefaultCacheMetadataChargePolicy, /*_low_pri_pool_ratio=*/0.0); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + /*_strict_capacity_limit=*/false, secondary_cache); Random rnd(301); std::string str1 = rnd.RandomString(1001); auto item1 = std::make_unique(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1.get(), GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1.get(), GetHelper(), str1.length())); item1.release(); // Appease clang-analyze "potential memory leak" Cache::Handle* handle; - handle = cache->Lookup("k2", nullptr, this, Cache::Priority::LOW, true); + handle = cache->Lookup(key2, nullptr, this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); - handle = - cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, false); + handle = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); + Cache::AsyncLookupHandle ah; + ah.key = key2; + ah.helper = GetHelper(); + ah.create_context = this; + ah.priority = Cache::Priority::LOW; + cache->StartAsyncLookup(ah); + cache->Wait(ah); + ASSERT_EQ(ah.Result(), nullptr); + cache.reset(); secondary_cache.reset(); } @@ -462,36 +466,29 @@ class CompressedSecondaryCacheTest : public testing::Test, std::shared_ptr secondary_cache = NewCompressedSecondaryCache(secondary_cache_opts); - LRUCacheOptions opts( + std::shared_ptr cache = NewCache( /*_capacity=*/1300, /*_num_shard_bits=*/0, - /*_strict_capacity_limit=*/false, /*_high_pri_pool_ratio=*/0.5, - /*_memory_allocator=*/nullptr, kDefaultToAdaptiveMutex, - kDefaultCacheMetadataChargePolicy, /*_low_pri_pool_ratio=*/0.0); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + /*_strict_capacity_limit=*/true, secondary_cache); Random rnd(301); std::string str1 = rnd.RandomString(1001); auto item1 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1, GetHelperFail(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1, GetHelperFail(), str1.length())); std::string str2 = rnd.RandomString(1002); auto item2 = new TestItem(str2.data(), str2.length()); // k1 should be demoted to the secondary cache. - ASSERT_OK(cache->Insert("k2", item2, GetHelperFail(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2, GetHelperFail(), str2.length())); Cache::Handle* handle; - handle = - cache->Lookup("k2", GetHelperFail(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key2, GetHelperFail(), this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); // This lookup should fail, since k1 demotion would have failed. - handle = - cache->Lookup("k1", GetHelperFail(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key1, GetHelperFail(), this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); // Since k1 was not promoted, k2 should still be in cache. - handle = - cache->Lookup("k2", GetHelperFail(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key2, GetHelperFail(), this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); @@ -517,34 +514,30 @@ class CompressedSecondaryCacheTest : public testing::Test, std::shared_ptr secondary_cache = NewCompressedSecondaryCache(secondary_cache_opts); - LRUCacheOptions opts( + std::shared_ptr cache = NewCache( /*_capacity=*/1300, /*_num_shard_bits=*/0, - /*_strict_capacity_limit=*/false, /*_high_pri_pool_ratio=*/0.5, - /*_memory_allocator=*/nullptr, kDefaultToAdaptiveMutex, - kDefaultCacheMetadataChargePolicy, /*_low_pri_pool_ratio=*/0.0); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + /*_strict_capacity_limit=*/true, secondary_cache); Random rnd(301); std::string str1 = rnd.RandomString(1001); auto item1 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1, GetHelper(), str1.length())); std::string str2 = rnd.RandomString(1002); auto item2 = new TestItem(str2.data(), str2.length()); // k1 should be demoted to the secondary cache. - ASSERT_OK(cache->Insert("k2", item2, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2, GetHelper(), str2.length())); Cache::Handle* handle; SetFailCreate(true); - handle = cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); // This lookup should fail, since k1 creation would have failed - handle = cache->Lookup("k1", GetHelper(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key1, GetHelper(), this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); // Since k1 didn't get promoted, k2 should still be in cache - handle = cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, true); + handle = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); @@ -570,39 +563,34 @@ class CompressedSecondaryCacheTest : public testing::Test, std::shared_ptr secondary_cache = NewCompressedSecondaryCache(secondary_cache_opts); - LRUCacheOptions opts( + std::shared_ptr cache = NewCache( /*_capacity=*/1300, /*_num_shard_bits=*/0, - /*_strict_capacity_limit=*/false, /*_high_pri_pool_ratio=*/0.5, - /*_memory_allocator=*/nullptr, kDefaultToAdaptiveMutex, - kDefaultCacheMetadataChargePolicy, /*_low_pri_pool_ratio=*/0.0); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + /*_strict_capacity_limit=*/false, secondary_cache); Random rnd(301); std::string str1 = rnd.RandomString(1001); auto item1_1 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1_1, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1_1, GetHelper(), str1.length())); std::string str2 = rnd.RandomString(1002); std::string str2_clone{str2}; auto item2 = new TestItem(str2.data(), str2.length()); // After this Insert, primary cache contains k2 and secondary cache contains // k1's dummy item. - ASSERT_OK(cache->Insert("k2", item2, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2, GetHelper(), str2.length())); // After this Insert, primary cache contains k1 and secondary cache contains // k1's dummy item and k2's dummy item. auto item1_2 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert("k1", item1_2, GetHelper(), str1.length())); + ASSERT_OK(cache->Insert(key1, item1_2, GetHelper(), str1.length())); auto item2_2 = new TestItem(str2.data(), str2.length()); // After this Insert, primary cache contains k2 and secondary cache contains // k1's item and k2's dummy item. - ASSERT_OK(cache->Insert("k2", item2_2, GetHelper(), str2.length())); + ASSERT_OK(cache->Insert(key2, item2_2, GetHelper(), str2.length())); Cache::Handle* handle2; - handle2 = - cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, true); + handle2 = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW); ASSERT_NE(handle2, nullptr); cache->Release(handle2); @@ -610,14 +598,12 @@ class CompressedSecondaryCacheTest : public testing::Test, // strict_capacity_limit is true, but the lookup should still succeed. // A k1's dummy item is inserted into primary cache. Cache::Handle* handle1; - handle1 = - cache->Lookup("k1", GetHelper(), this, Cache::Priority::LOW, true); + handle1 = cache->Lookup(key1, GetHelper(), this, Cache::Priority::LOW); ASSERT_NE(handle1, nullptr); cache->Release(handle1); // Since k1 didn't get inserted, k2 should still be in cache - handle2 = - cache->Lookup("k2", GetHelper(), this, Cache::Priority::LOW, true); + handle2 = cache->Lookup(key2, GetHelper(), this, Cache::Priority::LOW); ASSERT_NE(handle2, nullptr); cache->Release(handle2); @@ -743,14 +729,25 @@ class CompressedSecondaryCacheTest : public testing::Test, } }; +class CompressedSecondaryCacheTest + : public CompressedSecondaryCacheTestBase, + public testing::WithParamInterface { + const std::string& Type() override { return GetParam(); } +}; + +INSTANTIATE_TEST_CASE_P(CompressedSecondaryCacheTest, + CompressedSecondaryCacheTest, GetTestingCacheTypes()); + class CompressedSecCacheTestWithCompressAndAllocatorParam - : public CompressedSecondaryCacheTest, - public ::testing::WithParamInterface> { + : public CompressedSecondaryCacheTestBase, + public ::testing::WithParamInterface< + std::tuple> { public: CompressedSecCacheTestWithCompressAndAllocatorParam() { sec_cache_is_compressed_ = std::get<0>(GetParam()); use_jemalloc_ = std::get<1>(GetParam()); } + const std::string& Type() override { return std::get<2>(GetParam()); } bool sec_cache_is_compressed_; bool use_jemalloc_; }; @@ -761,19 +758,20 @@ TEST_P(CompressedSecCacheTestWithCompressAndAllocatorParam, BasicTes) { INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests, CompressedSecCacheTestWithCompressAndAllocatorParam, - ::testing::Combine(testing::Bool(), testing::Bool())); + ::testing::Combine(testing::Bool(), testing::Bool(), + GetTestingCacheTypes())); class CompressedSecondaryCacheTestWithCompressionParam - : public CompressedSecondaryCacheTest, - public ::testing::WithParamInterface { + : public CompressedSecondaryCacheTestBase, + public ::testing::WithParamInterface> { public: CompressedSecondaryCacheTestWithCompressionParam() { - sec_cache_is_compressed_ = GetParam(); + sec_cache_is_compressed_ = std::get<0>(GetParam()); } + const std::string& Type() override { return std::get<1>(GetParam()); } bool sec_cache_is_compressed_; }; - TEST_P(CompressedSecondaryCacheTestWithCompressionParam, BasicTestFromString) { std::shared_ptr sec_cache{nullptr}; std::string sec_cache_uri; @@ -898,7 +896,7 @@ TEST_P(CompressedSecondaryCacheTestWithCompressionParam, EntryRoles) { // Uniquify `junk` junk[0] = static_cast(i); TestItem item{junk.data(), junk.length()}; - Slice ith_key = Slice(junk.data(), 5); + Slice ith_key = Slice(junk.data(), 16); get_perf_context()->Reset(); ASSERT_OK(sec_cache->Insert(ith_key, &item, GetHelper(role))); @@ -935,16 +933,19 @@ TEST_P(CompressedSecondaryCacheTestWithCompressionParam, EntryRoles) { INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests, CompressedSecondaryCacheTestWithCompressionParam, - testing::Bool()); + testing::Combine(testing::Bool(), + GetTestingCacheTypes())); class CompressedSecCacheTestWithCompressAndSplitParam - : public CompressedSecondaryCacheTest, - public ::testing::WithParamInterface> { + : public CompressedSecondaryCacheTestBase, + public ::testing::WithParamInterface< + std::tuple> { public: CompressedSecCacheTestWithCompressAndSplitParam() { sec_cache_is_compressed_ = std::get<0>(GetParam()); enable_custom_split_merge_ = std::get<1>(GetParam()); } + const std::string& Type() override { return std::get<2>(GetParam()); } bool sec_cache_is_compressed_; bool enable_custom_split_merge_; }; @@ -955,17 +956,18 @@ TEST_P(CompressedSecCacheTestWithCompressAndSplitParam, BasicIntegrationTest) { INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests, CompressedSecCacheTestWithCompressAndSplitParam, - ::testing::Combine(testing::Bool(), testing::Bool())); + ::testing::Combine(testing::Bool(), testing::Bool(), + GetTestingCacheTypes())); -TEST_F(CompressedSecondaryCacheTest, SplitValueIntoChunksTest) { +TEST_P(CompressedSecondaryCacheTest, SplitValueIntoChunksTest) { SplitValueIntoChunksTest(); } -TEST_F(CompressedSecondaryCacheTest, MergeChunksIntoValueTest) { +TEST_P(CompressedSecondaryCacheTest, MergeChunksIntoValueTest) { MergeChunksIntoValueTest(); } -TEST_F(CompressedSecondaryCacheTest, SplictValueAndMergeChunksTest) { +TEST_P(CompressedSecondaryCacheTest, SplictValueAndMergeChunksTest) { SplictValueAndMergeChunksTest(); } diff --git a/cache/lru_cache.cc b/cache/lru_cache.cc index ab130b4610..3b4e80ef87 100644 --- a/cache/lru_cache.cc +++ b/cache/lru_cache.cc @@ -14,6 +14,7 @@ #include #include +#include "cache/secondary_cache_adapter.h" #include "monitoring/perf_context_imp.h" #include "monitoring/statistics.h" #include "port/lang.h" @@ -22,14 +23,6 @@ namespace ROCKSDB_NAMESPACE { namespace lru_cache { -namespace { -// A distinct pointer value for marking "dummy" cache entries -struct DummyValue { - char val[12] = "kDummyValue"; -}; -DummyValue kDummyValue{}; -} // namespace - LRUHandleTable::LRUHandleTable(int max_upper_hash_bits, MemoryAllocator* allocator) : length_bits_(/* historical starting size*/ 4), @@ -127,7 +120,7 @@ LRUCacheShard::LRUCacheShard(size_t capacity, bool strict_capacity_limit, CacheMetadataChargePolicy metadata_charge_policy, int max_upper_hash_bits, MemoryAllocator* allocator, - SecondaryCache* secondary_cache) + const Cache::EvictionCallback* eviction_callback) : CacheShardBase(metadata_charge_policy), capacity_(0), high_pri_pool_usage_(0), @@ -141,7 +134,7 @@ LRUCacheShard::LRUCacheShard(size_t capacity, bool strict_capacity_limit, usage_(0), lru_usage_(0), mutex_(use_adaptive_mutex), - secondary_cache_(secondary_cache) { + eviction_callback_(*eviction_callback) { // Make empty circular linked list. lru_.next = &lru_; lru_.prev = &lru_; @@ -341,15 +334,19 @@ void LRUCacheShard::EvictFromLRU(size_t charge, } } -void LRUCacheShard::TryInsertIntoSecondaryCache( - autovector evicted_handles) { - for (auto entry : evicted_handles) { - if (secondary_cache_ && entry->IsSecondaryCacheCompatible()) { - secondary_cache_->Insert(entry->key(), entry->value, entry->helper) - .PermitUncheckedError(); +void LRUCacheShard::NotifyEvicted( + const autovector& evicted_handles) { + MemoryAllocator* alloc = table_.GetAllocator(); + for (LRUHandle* entry : evicted_handles) { + if (eviction_callback_ && + eviction_callback_(entry->key(), + reinterpret_cast(entry))) { + // Callback took ownership of obj; just free handle + free(entry); + } else { + // Free the entries here outside of mutex for performance reasons. + entry->Free(alloc); } - // Free the entries here outside of mutex for performance reasons. - entry->Free(table_.GetAllocator()); } } @@ -363,7 +360,7 @@ void LRUCacheShard::SetCapacity(size_t capacity) { EvictFromLRU(0, &last_reference_list); } - TryInsertIntoSecondaryCache(last_reference_list); + NotifyEvicted(last_reference_list); } void LRUCacheShard::SetStrictCapacityLimit(bool strict_capacity_limit) { @@ -371,8 +368,7 @@ void LRUCacheShard::SetStrictCapacityLimit(bool strict_capacity_limit) { strict_capacity_limit_ = strict_capacity_limit; } -Status LRUCacheShard::InsertItem(LRUHandle* e, LRUHandle** handle, - bool free_handle_on_fail) { +Status LRUCacheShard::InsertItem(LRUHandle* e, LRUHandle** handle) { Status s = Status::OK(); autovector last_reference_list; @@ -391,10 +387,9 @@ Status LRUCacheShard::InsertItem(LRUHandle* e, LRUHandle** handle, // into cache and get evicted immediately. last_reference_list.push_back(e); } else { - if (free_handle_on_fail) { - free(e); - *handle = nullptr; - } + free(e); + e = nullptr; + *handle = nullptr; s = Status::MemoryLimit("Insert failed due to LRU cache being full."); } } else { @@ -426,202 +421,27 @@ Status LRUCacheShard::InsertItem(LRUHandle* e, LRUHandle** handle, } } - TryInsertIntoSecondaryCache(last_reference_list); + NotifyEvicted(last_reference_list); return s; } -void LRUCacheShard::Promote(LRUHandle* e) { - SecondaryCacheResultHandle* secondary_handle = e->sec_handle; - - assert(secondary_handle->IsReady()); - // e is not thread-shared here; OK to modify "immutable" fields as well as - // "mutable" (normally requiring mutex) - e->SetIsPending(false); - e->value = secondary_handle->Value(); - assert(e->total_charge == 0); - size_t value_size = secondary_handle->Size(); - delete secondary_handle; - - if (e->value) { - e->CalcTotalCharge(value_size, metadata_charge_policy_); - Status s; - if (e->IsStandalone()) { - assert(secondary_cache_ && secondary_cache_->SupportForceErase()); - - // Insert a dummy handle and return a standalone handle to caller. - // Charge the standalone handle. - autovector last_reference_list; - bool free_standalone_handle{false}; - { - DMutexLock l(mutex_); - - // Free the space following strict LRU policy until enough space - // is freed or the lru list is empty. - EvictFromLRU(e->total_charge, &last_reference_list); - - if ((usage_ + e->total_charge) > capacity_ && strict_capacity_limit_) { - free_standalone_handle = true; - } else { - usage_ += e->total_charge; - } - } - - TryInsertIntoSecondaryCache(last_reference_list); - if (free_standalone_handle) { - e->Unref(); - e->Free(table_.GetAllocator()); - e = nullptr; - } else { - PERF_COUNTER_ADD(block_cache_standalone_handle_count, 1); - } - - // Insert a dummy handle into the primary cache. This dummy handle is - // not IsSecondaryCacheCompatible(). - // FIXME? This should not overwrite an existing non-dummy entry in the - // rare case that one exists - Cache::Priority priority = - e->IsHighPri() ? Cache::Priority::HIGH : Cache::Priority::LOW; - s = Insert(e->key(), e->hash, &kDummyValue, &kNoopCacheItemHelper, - /*charge=*/0, - /*handle=*/nullptr, priority); - } else { - e->SetInCache(true); - LRUHandle* handle = e; - // This InsertItem() could fail if the cache is over capacity and - // strict_capacity_limit_ is true. In such a case, we don't want - // InsertItem() to free the handle, since the item is already in memory - // and the caller will most likely just read it from disk if we erase it - // here. - s = InsertItem(e, &handle, /*free_handle_on_fail=*/false); - if (s.ok()) { - PERF_COUNTER_ADD(block_cache_real_handle_count, 1); - } - } - - if (!s.ok()) { - // Item is in memory, but not accounted against the cache capacity. - // When the handle is released, the item should get deleted. - assert(!e->InCache()); - } - } else { - // Secondary cache lookup failed. The caller will take care of detecting - // this and eventually releasing e. - assert(!e->value); - assert(!e->InCache()); - } -} - LRUHandle* LRUCacheShard::Lookup(const Slice& key, uint32_t hash, - const Cache::CacheItemHelper* helper, - Cache::CreateContext* create_context, - Cache::Priority priority, bool wait, - Statistics* stats) { - LRUHandle* e = nullptr; - bool found_dummy_entry{false}; - { - DMutexLock l(mutex_); - e = table_.Lookup(key, hash); - if (e != nullptr) { - assert(e->InCache()); - if (e->value == &kDummyValue) { - // For a dummy handle, if it was retrieved from secondary cache, - // it may still exist in secondary cache. - // If the handle exists in secondary cache, the value should be - // erased from sec cache and be inserted into primary cache. - found_dummy_entry = true; - // Let the dummy entry be overwritten - e = nullptr; - } else { - if (!e->HasRefs()) { - // The entry is in LRU since it's in hash and has no external - // references. - LRU_Remove(e); - } - e->Ref(); - e->SetHit(); - } - } - } - - // If handle table lookup failed or the handle is a dummy one, allocate - // a handle outside the mutex if we re going to lookup in the secondary cache. - // - // When a block is firstly Lookup from CompressedSecondaryCache, we just - // insert a dummy block into the primary cache (charging the actual size of - // the block) and don't erase the block from CompressedSecondaryCache. A - // standalone handle is returned to the caller. Only if the block is hit - // again, we erase it from CompressedSecondaryCache and add it into the - // primary cache. - if (!e && secondary_cache_ && helper && helper->create_cb) { - bool kept_in_sec_cache{false}; - std::unique_ptr secondary_handle = - secondary_cache_->Lookup(key, helper, create_context, wait, - found_dummy_entry, kept_in_sec_cache); - if (secondary_handle != nullptr) { - e = static_cast(malloc(sizeof(LRUHandle) - 1 + key.size())); - - // For entries already in secondary cache, prevent re-insertion by - // using a helper that is not secondary cache compatible - if (kept_in_sec_cache) { - helper = helper->without_secondary_compat; - } - - e->m_flags = 0; - e->im_flags = 0; - e->helper = helper; - e->key_length = key.size(); - e->hash = hash; - e->refs = 0; - e->next = e->prev = nullptr; - e->SetPriority(priority); - memcpy(e->key_data, key.data(), key.size()); - e->value = nullptr; - e->sec_handle = secondary_handle.release(); - e->total_charge = 0; - e->Ref(); - e->SetIsStandalone(secondary_cache_->SupportForceErase() && - !found_dummy_entry); - - if (wait) { - Promote(e); - if (e) { - if (!e->value) { - // The secondary cache returned a handle, but the lookup failed. - e->Unref(); - e->Free(table_.GetAllocator()); - e = nullptr; - } - } - } else { - // If wait is false, we always return a handle and let the caller - // release the handle after checking for success or failure. - e->SetIsPending(true); - } - if (e) { - // This may be slightly inaccurate, if the lookup eventually fails. - // But the probability is very low. - switch (helper->role) { - case CacheEntryRole::kFilterBlock: - RecordTick(stats, SECONDARY_CACHE_FILTER_HITS); - break; - case CacheEntryRole::kIndexBlock: - RecordTick(stats, SECONDARY_CACHE_INDEX_HITS); - break; - case CacheEntryRole::kDataBlock: - RecordTick(stats, SECONDARY_CACHE_DATA_HITS); - break; - default: - break; - } - PERF_COUNTER_ADD(secondary_cache_hit_count, 1); - RecordTick(stats, SECONDARY_CACHE_HITS); - } - } else { - // Caller will most likely overwrite the dummy entry with an Insert - // after this Lookup fails - assert(e == nullptr); + const Cache::CacheItemHelper* /*helper*/, + Cache::CreateContext* /*create_context*/, + Cache::Priority /*priority*/, + Statistics* /*stats*/) { + DMutexLock l(mutex_); + LRUHandle* e = table_.Lookup(key, hash); + if (e != nullptr) { + assert(e->InCache()); + if (!e->HasRefs()) { + // The entry is in LRU since it's in hash and has no external + // references. + LRU_Remove(e); } + e->Ref(); + e->SetHit(); } return e; } @@ -630,8 +450,6 @@ bool LRUCacheShard::Ref(LRUHandle* e) { DMutexLock l(mutex_); // To create another reference - entry must be already externally referenced. assert(e->HasRefs()); - // Pending handles are not for sharing - assert(!e->IsPending()); e->Ref(); return true; } @@ -655,14 +473,13 @@ bool LRUCacheShard::Release(LRUHandle* e, bool /*useful*/, if (e == nullptr) { return false; } - bool last_reference = false; - // Must Wait or WaitAll first on pending handles. Otherwise, would leak - // a secondary cache handle. - assert(!e->IsPending()); + bool must_free; + bool was_in_cache; { DMutexLock l(mutex_); - last_reference = e->Unref(); - if (last_reference && e->InCache()) { + must_free = e->Unref(); + was_in_cache = e->InCache(); + if (must_free && was_in_cache) { // The item is still in cache, and nobody else holds a reference to it. if (usage_ > capacity_ || erase_if_last_ref) { // The LRU list must be empty since the cache is full. @@ -673,29 +490,38 @@ bool LRUCacheShard::Release(LRUHandle* e, bool /*useful*/, } else { // Put the item back on the LRU list, and don't free it. LRU_Insert(e); - last_reference = false; + must_free = false; } } - // If it was the last reference, then decrement the cache usage. - if (last_reference) { + // If about to be freed, then decrement the cache usage. + if (must_free) { assert(usage_ >= e->total_charge); usage_ -= e->total_charge; } } // Free the entry here outside of mutex for performance reasons. - if (last_reference) { - e->Free(table_.GetAllocator()); + if (must_free) { + // Only call eviction callback if we're sure no one requested erasure + // FIXME: disabled because of test churn + if (false && was_in_cache && !erase_if_last_ref && eviction_callback_ && + eviction_callback_(e->key(), reinterpret_cast(e))) { + // Callback took ownership of obj; just free handle + free(e); + } else { + e->Free(table_.GetAllocator()); + } } - return last_reference; + return must_free; } -Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, - Cache::ObjectPtr value, - const Cache::CacheItemHelper* helper, - size_t charge, LRUHandle** handle, - Cache::Priority priority) { +LRUHandle* LRUCacheShard::CreateHandle(const Slice& key, uint32_t hash, + Cache::ObjectPtr value, + const Cache::CacheItemHelper* helper, + size_t charge) { assert(helper); + // value == nullptr is reserved for indicating failure in SecondaryCache + assert(!(helper->IsSecondaryCacheCompatible() && value == nullptr)); // Allocate the memory here outside of the mutex. // If the cache is full, we'll have to release it. @@ -711,16 +537,53 @@ Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, e->hash = hash; e->refs = 0; e->next = e->prev = nullptr; - e->SetInCache(true); - e->SetPriority(priority); memcpy(e->key_data, key.data(), key.size()); e->CalcTotalCharge(charge, metadata_charge_policy_); - // value == nullptr is reserved for indicating failure for when secondary - // cache compatible - assert(!(e->IsSecondaryCacheCompatible() && value == nullptr)); + return e; +} - return InsertItem(e, handle, /* free_handle_on_fail */ true); +Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, + Cache::ObjectPtr value, + const Cache::CacheItemHelper* helper, + size_t charge, LRUHandle** handle, + Cache::Priority priority) { + LRUHandle* e = CreateHandle(key, hash, value, helper, charge); + e->SetPriority(priority); + e->SetInCache(true); + return InsertItem(e, handle); +} + +LRUHandle* LRUCacheShard::CreateStandalone(const Slice& key, uint32_t hash, + Cache::ObjectPtr value, + const Cache::CacheItemHelper* helper, + size_t charge, + bool allow_uncharged) { + LRUHandle* e = CreateHandle(key, hash, value, helper, charge); + e->SetIsStandalone(true); + e->Ref(); + + autovector last_reference_list; + + { + DMutexLock l(mutex_); + + EvictFromLRU(e->total_charge, &last_reference_list); + + if (strict_capacity_limit_ && (usage_ + e->total_charge) > capacity_) { + if (allow_uncharged) { + e->total_charge = 0; + } else { + free(e); + e = nullptr; + } + } else { + usage_ += e->total_charge; + } + } + + NotifyEvicted(last_reference_list); + return e; } void LRUCacheShard::Erase(const Slice& key, uint32_t hash) { @@ -749,16 +612,6 @@ void LRUCacheShard::Erase(const Slice& key, uint32_t hash) { } } -bool LRUCacheShard::IsReady(LRUHandle* e) { - bool ready = true; - if (e->IsPending()) { - assert(secondary_cache_); - assert(e->sec_handle); - ready = e->sec_handle->IsReady(); - } - return ready; -} - size_t LRUCacheShard::GetUsage() const { DMutexLock l(mutex_); return usage_; @@ -798,26 +651,23 @@ LRUCache::LRUCache(size_t capacity, int num_shard_bits, double low_pri_pool_ratio, std::shared_ptr allocator, bool use_adaptive_mutex, - CacheMetadataChargePolicy metadata_charge_policy, - std::shared_ptr _secondary_cache) + CacheMetadataChargePolicy metadata_charge_policy) : ShardedCache(capacity, num_shard_bits, strict_capacity_limit, - std::move(allocator)), - secondary_cache_(std::move(_secondary_cache)) { + std::move(allocator)) { size_t per_shard = GetPerShardCapacity(); - SecondaryCache* secondary_cache = secondary_cache_.get(); MemoryAllocator* alloc = memory_allocator(); + const EvictionCallback* eviction_callback = &eviction_callback_; InitShards([=](LRUCacheShard* cs) { - new (cs) LRUCacheShard( - per_shard, strict_capacity_limit, high_pri_pool_ratio, - low_pri_pool_ratio, use_adaptive_mutex, metadata_charge_policy, - /* max_upper_hash_bits */ 32 - num_shard_bits, alloc, secondary_cache); + new (cs) LRUCacheShard(per_shard, strict_capacity_limit, + high_pri_pool_ratio, low_pri_pool_ratio, + use_adaptive_mutex, metadata_charge_policy, + /* max_upper_hash_bits */ 32 - num_shard_bits, alloc, + eviction_callback); }); } Cache::ObjectPtr LRUCache::Value(Handle* handle) { auto h = reinterpret_cast(handle); - assert(!h->IsPending() || h->value == nullptr); - assert(h->value != &kDummyValue); return h->value; } @@ -840,42 +690,6 @@ double LRUCache::GetHighPriPoolRatio() { return GetShard(0).GetHighPriPoolRatio(); } -void LRUCache::WaitAll(std::vector& handles) { - if (secondary_cache_) { - std::vector sec_handles; - sec_handles.reserve(handles.size()); - for (Handle* handle : handles) { - if (!handle) { - continue; - } - LRUHandle* lru_handle = reinterpret_cast(handle); - if (!lru_handle->IsPending()) { - continue; - } - sec_handles.emplace_back(lru_handle->sec_handle); - } - secondary_cache_->WaitAll(sec_handles); - for (Handle* handle : handles) { - if (!handle) { - continue; - } - LRUHandle* lru_handle = reinterpret_cast(handle); - if (!lru_handle->IsPending()) { - continue; - } - GetShard(lru_handle->hash).Promote(lru_handle); - } - } -} - -void LRUCache::AppendPrintableOptions(std::string& str) const { - ShardedCache::AppendPrintableOptions(str); // options from shard - if (secondary_cache_) { - str.append(" secondary_cache:\n"); - str.append(secondary_cache_->GetPrintableOptions()); - } -} - } // namespace lru_cache std::shared_ptr NewLRUCache( @@ -903,10 +717,14 @@ std::shared_ptr NewLRUCache( if (num_shard_bits < 0) { num_shard_bits = GetDefaultCacheShardBits(capacity); } - return std::make_shared( + std::shared_ptr cache = std::make_shared( capacity, num_shard_bits, strict_capacity_limit, high_pri_pool_ratio, low_pri_pool_ratio, std::move(memory_allocator), use_adaptive_mutex, - metadata_charge_policy, secondary_cache); + metadata_charge_policy); + if (secondary_cache) { + cache = std::make_shared(cache, secondary_cache); + } + return cache; } std::shared_ptr NewLRUCache(const LRUCacheOptions& cache_opts) { diff --git a/cache/lru_cache.h b/cache/lru_cache.h index 4c0caccb93..554907b3be 100644 --- a/cache/lru_cache.h +++ b/cache/lru_cache.h @@ -16,7 +16,6 @@ #include "port/likely.h" #include "port/malloc.h" #include "port/port.h" -#include "rocksdb/secondary_cache.h" #include "util/autovector.h" #include "util/distributed_mutex.h" @@ -51,12 +50,7 @@ namespace lru_cache { struct LRUHandle { Cache::ObjectPtr value; const Cache::CacheItemHelper* helper; - // An entry is not added to the LRUHandleTable until the secondary cache - // lookup is complete, so its safe to have this union. - union { - LRUHandle* next_hash; - SecondaryCacheResultHandle* sec_handle; - }; + LRUHandle* next_hash; LRUHandle* next; LRUHandle* prev; size_t total_charge; // TODO(opt): Only allow uint32_t? @@ -89,10 +83,8 @@ struct LRUHandle { IM_IS_HIGH_PRI = (1 << 0), // Whether this entry is low priority entry. IM_IS_LOW_PRI = (1 << 1), - // Is the handle still being read from a lower tier. - IM_IS_PENDING = (1 << 2), // Marks result handles that should not be inserted into cache - IM_IS_STANDALONE = (1 << 3), + IM_IS_STANDALONE = (1 << 2), }; // Beginning of the key (MUST BE THE LAST FIELD IN THIS STRUCT!) @@ -122,8 +114,6 @@ struct LRUHandle { bool IsLowPri() const { return im_flags & IM_IS_LOW_PRI; } bool InLowPriPool() const { return m_flags & M_IN_LOW_PRI_POOL; } bool HasHit() const { return m_flags & M_HAS_HIT; } - bool IsSecondaryCacheCompatible() const { return helper->size_cb != nullptr; } - bool IsPending() const { return im_flags & IM_IS_PENDING; } bool IsStandalone() const { return im_flags & IM_IS_STANDALONE; } void SetInCache(bool in_cache) { @@ -165,14 +155,6 @@ struct LRUHandle { void SetHit() { m_flags |= M_HAS_HIT; } - void SetIsPending(bool pending) { - if (pending) { - im_flags |= IM_IS_PENDING; - } else { - im_flags &= ~IM_IS_PENDING; - } - } - void SetIsStandalone(bool is_standalone) { if (is_standalone) { im_flags |= IM_IS_STANDALONE; @@ -183,14 +165,6 @@ struct LRUHandle { void Free(MemoryAllocator* allocator) { assert(refs == 0); - - if (UNLIKELY(IsPending())) { - assert(sec_handle != nullptr); - SecondaryCacheResultHandle* tmp_sec_handle = sec_handle; - tmp_sec_handle->Wait(); - value = tmp_sec_handle->Value(); - delete tmp_sec_handle; - } assert(helper); if (helper->del_cb) { helper->del_cb(value, allocator); @@ -290,12 +264,14 @@ class LRUHandleTable { // A single shard of sharded cache. class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase { public: + // NOTE: the eviction_callback ptr is saved, as is it assumed to be kept + // alive in Cache. LRUCacheShard(size_t capacity, bool strict_capacity_limit, double high_pri_pool_ratio, double low_pri_pool_ratio, bool use_adaptive_mutex, CacheMetadataChargePolicy metadata_charge_policy, int max_upper_hash_bits, MemoryAllocator* allocator, - SecondaryCache* secondary_cache); + const Cache::EvictionCallback* eviction_callback); public: // Type definitions expected as parameter to ShardedCache using HandleImpl = LRUHandle; @@ -326,14 +302,17 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase { const Cache::CacheItemHelper* helper, size_t charge, LRUHandle** handle, Cache::Priority priority); + LRUHandle* CreateStandalone(const Slice& key, uint32_t hash, + Cache::ObjectPtr obj, + const Cache::CacheItemHelper* helper, + size_t charge, bool allow_uncharged); + LRUHandle* Lookup(const Slice& key, uint32_t hash, const Cache::CacheItemHelper* helper, Cache::CreateContext* create_context, - Cache::Priority priority, bool wait, Statistics* stats); + Cache::Priority priority, Statistics* stats); bool Release(LRUHandle* handle, bool useful, bool erase_if_last_ref); - bool IsReady(LRUHandle* /*handle*/); - void Wait(LRUHandle* /*handle*/) {} bool Ref(LRUHandle* handle); void Erase(const Slice& key, uint32_t hash); @@ -373,20 +352,10 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase { private: friend class LRUCache; // Insert an item into the hash table and, if handle is null, insert into - // the LRU list. Older items are evicted as necessary. If the cache is full - // and free_handle_on_fail is true, the item is deleted and handle is set to - // nullptr. - Status InsertItem(LRUHandle* item, LRUHandle** handle, - bool free_handle_on_fail); - // Promote an item looked up from the secondary cache to the LRU cache. - // The item may be still in the secondary cache. - // It is only inserted into the hash table and not the LRU list, and only - // if the cache is not at full capacity, as is the case during Insert. The - // caller should hold a reference on the LRUHandle. When the caller releases - // the last reference, the item is added to the LRU list. - // The item is promoted to the high pri or low pri pool as specified by the - // caller in Lookup. - void Promote(LRUHandle* e); + // the LRU list. Older items are evicted as necessary. Frees `item` on + // non-OK status. + Status InsertItem(LRUHandle* item, LRUHandle** handle); + void LRU_Remove(LRUHandle* e); void LRU_Insert(LRUHandle* e); @@ -400,8 +369,11 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase { // holding the mutex_. void EvictFromLRU(size_t charge, autovector* deleted); - // Try to insert the evicted handles into the secondary cache. - void TryInsertIntoSecondaryCache(autovector evicted_handles); + void NotifyEvicted(const autovector& evicted_handles); + + LRUHandle* CreateHandle(const Slice& key, uint32_t hash, + Cache::ObjectPtr value, + const Cache::CacheItemHelper* helper, size_t charge); // Initialized before use. size_t capacity_; @@ -464,8 +436,8 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase { // don't mind mutex_ invoking the non-const actions. mutable DMutex mutex_; - // Owned by LRUCache - SecondaryCache* secondary_cache_; + // A reference to Cache::eviction_callback_ + const Cache::EvictionCallback& eviction_callback_; }; class LRUCache @@ -479,23 +451,16 @@ class LRUCache std::shared_ptr memory_allocator = nullptr, bool use_adaptive_mutex = kDefaultToAdaptiveMutex, CacheMetadataChargePolicy metadata_charge_policy = - kDontChargeCacheMetadata, - std::shared_ptr secondary_cache = nullptr); + kDontChargeCacheMetadata); const char* Name() const override { return "LRUCache"; } ObjectPtr Value(Handle* handle) override; size_t GetCharge(Handle* handle) const override; const CacheItemHelper* GetCacheItemHelper(Handle* handle) const override; - void WaitAll(std::vector& handles) override; // Retrieves number of elements in LRU, for unit test purpose only. size_t TEST_GetLRUSize(); // Retrieves high pri pool ratio. double GetHighPriPoolRatio(); - - void AppendPrintableOptions(std::string& str) const override; - - private: - std::shared_ptr secondary_cache_; }; } // namespace lru_cache diff --git a/cache/lru_cache_test.cc b/cache/lru_cache_test.cc index 29d5c5f87d..c4f3929765 100644 --- a/cache/lru_cache_test.cc +++ b/cache/lru_cache_test.cc @@ -5,6 +5,7 @@ #include "cache/lru_cache.h" +#include #include #include @@ -52,8 +53,7 @@ class LRUCacheTest : public testing::Test { high_pri_pool_ratio, low_pri_pool_ratio, use_adaptive_mutex, kDontChargeCacheMetadata, /*max_upper_hash_bits=*/24, - /*allocator*/ nullptr, - /*secondary_cache=*/nullptr); + /*allocator*/ nullptr, &eviction_callback_); } void Insert(const std::string& key, @@ -69,7 +69,7 @@ class LRUCacheTest : public testing::Test { bool Lookup(const std::string& key) { auto handle = cache_->Lookup(key, 0 /*hash*/, nullptr, nullptr, - Cache::Priority::LOW, true, nullptr); + Cache::Priority::LOW, nullptr); if (handle) { cache_->Release(handle, true /*useful*/, false /*erase*/); return true; @@ -146,6 +146,7 @@ class LRUCacheTest : public testing::Test { private: LRUCacheShard* cache_ = nullptr; + Cache::EvictionCallback eviction_callback_; }; TEST_F(LRUCacheTest, BasicLRU) { @@ -394,8 +395,9 @@ class ClockCacheTest : public testing::Test { Table::Opts opts; opts.estimated_value_size = 1; - new (shard_) Shard(capacity, strict_capacity_limit, - kDontChargeCacheMetadata, /*allocator*/ nullptr, opts); + new (shard_) + Shard(capacity, strict_capacity_limit, kDontChargeCacheMetadata, + /*allocator*/ nullptr, &eviction_callback_, opts); } Status Insert(const UniqueId64x2& hashed_key, @@ -450,6 +452,9 @@ class ClockCacheTest : public testing::Test { } Shard* shard_ = nullptr; + + private: + Cache::EvictionCallback eviction_callback_; }; TEST_F(ClockCacheTest, Misc) { @@ -1053,12 +1058,16 @@ class TestSecondaryCache : public SecondaryCache { ResultMap result_map_; }; -using secondary_cache_test_util::GetHelper; -using secondary_cache_test_util::GetHelperFail; -using secondary_cache_test_util::TestCreateContext; -using secondary_cache_test_util::TestItem; +using secondary_cache_test_util::GetTestingCacheTypes; +using secondary_cache_test_util::WithCacheTypeParam; -class DBSecondaryCacheTest : public DBTestBase { +class BasicSecondaryCacheTest : public testing::Test, + public WithCacheTypeParam {}; + +INSTANTIATE_TEST_CASE_P(BasicSecondaryCacheTest, BasicSecondaryCacheTest, + GetTestingCacheTypes()); + +class DBSecondaryCacheTest : public DBTestBase, public WithCacheTypeParam { public: DBSecondaryCacheTest() : DBTestBase("db_secondary_cache_test", /*env_do_fsync=*/true) { @@ -1070,19 +1079,15 @@ class DBSecondaryCacheTest : public DBTestBase { std::unique_ptr fault_env_; }; -class LRUCacheSecondaryCacheTest : public LRUCacheTest, - public TestCreateContext {}; +INSTANTIATE_TEST_CASE_P(DBSecondaryCacheTest, DBSecondaryCacheTest, + GetTestingCacheTypes()); -TEST_F(LRUCacheSecondaryCacheTest, BasicTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, BasicTest) { std::shared_ptr secondary_cache = std::make_shared(4096); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); std::shared_ptr stats = CreateDBStatistics(); CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); @@ -1093,7 +1098,7 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicTest) { std::string str3 = rnd.RandomString(1021); ASSERT_OK(secondary_cache->InsertSaved(k3.AsSlice(), str3)); - std::string str1 = rnd.RandomString(1020); + std::string str1 = rnd.RandomString(1021); TestItem* item1 = new TestItem(str1.data(), str1.length()); ASSERT_OK(cache->Insert(k1.AsSlice(), item1, GetHelper(), str1.length())); std::string str2 = rnd.RandomString(1021); @@ -1103,25 +1108,22 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicTest) { get_perf_context()->Reset(); Cache::Handle* handle; - handle = - cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k2.AsSlice(), GetHelper(), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str2.size()); cache->Release(handle); // This lookup should promote k1 and demote k2 - handle = - cache->Lookup(k1.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k1.AsSlice(), GetHelper(), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str1.size()); cache->Release(handle); // This lookup should promote k3 and demote k1 - handle = - cache->Lookup(k3.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k3.AsSlice(), GetHelper(), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str3.size()); cache->Release(handle); @@ -1137,16 +1139,12 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicTest) { secondary_cache.reset(); } -TEST_F(LRUCacheSecondaryCacheTest, StatsTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, StatsTest) { std::shared_ptr secondary_cache = std::make_shared(4096); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); std::shared_ptr stats = CreateDBStatistics(); CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); @@ -1163,23 +1161,20 @@ TEST_F(LRUCacheSecondaryCacheTest, StatsTest) { get_perf_context()->Reset(); Cache::Handle* handle; - handle = - cache->Lookup(k1.AsSlice(), GetHelper(CacheEntryRole::kFilterBlock), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k1.AsSlice(), GetHelper(CacheEntryRole::kFilterBlock), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str1.size()); cache->Release(handle); - handle = - cache->Lookup(k2.AsSlice(), GetHelper(CacheEntryRole::kIndexBlock), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k2.AsSlice(), GetHelper(CacheEntryRole::kIndexBlock), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str2.size()); cache->Release(handle); - handle = - cache->Lookup(k3.AsSlice(), GetHelper(CacheEntryRole::kDataBlock), - /*context*/ this, Cache::Priority::LOW, true, stats.get()); + handle = cache->Lookup(k3.AsSlice(), GetHelper(CacheEntryRole::kDataBlock), + /*context*/ this, Cache::Priority::LOW, stats.get()); ASSERT_NE(handle, nullptr); ASSERT_EQ(static_cast(cache->Value(handle))->Size(), str3.size()); cache->Release(handle); @@ -1198,16 +1193,12 @@ TEST_F(LRUCacheSecondaryCacheTest, StatsTest) { secondary_cache.reset(); } -TEST_F(LRUCacheSecondaryCacheTest, BasicFailTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, BasicFailTest) { std::shared_ptr secondary_cache = std::make_shared(2048); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); @@ -1223,26 +1214,33 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicFailTest) { Cache::Handle* handle; handle = cache->Lookup(k2.AsSlice(), nullptr, /*context*/ this, - Cache::Priority::LOW, true); + Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); + handle = cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, false); + /*context*/ this, Cache::Priority::LOW); + ASSERT_EQ(handle, nullptr); + + Cache::AsyncLookupHandle async_handle; + async_handle.key = k2.AsSlice(); + async_handle.helper = GetHelper(); + async_handle.create_context = this; + async_handle.priority = Cache::Priority::LOW; + cache->StartAsyncLookup(async_handle); + cache->Wait(async_handle); + handle = async_handle.Result(); ASSERT_EQ(handle, nullptr); cache.reset(); secondary_cache.reset(); } -TEST_F(LRUCacheSecondaryCacheTest, SaveFailTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, SaveFailTest) { std::shared_ptr secondary_cache = std::make_shared(2048); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); @@ -1259,16 +1257,16 @@ TEST_F(LRUCacheSecondaryCacheTest, SaveFailTest) { Cache::Handle* handle; handle = cache->Lookup(k2.AsSlice(), GetHelperFail(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); // This lookup should fail, since k1 demotion would have failed handle = cache->Lookup(k1.AsSlice(), GetHelperFail(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); // Since k1 didn't get promoted, k2 should still be in cache handle = cache->Lookup(k2.AsSlice(), GetHelperFail(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); ASSERT_EQ(secondary_cache->num_inserts(), 1u); @@ -1278,16 +1276,12 @@ TEST_F(LRUCacheSecondaryCacheTest, SaveFailTest) { secondary_cache.reset(); } -TEST_F(LRUCacheSecondaryCacheTest, CreateFailTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, CreateFailTest) { std::shared_ptr secondary_cache = std::make_shared(2048); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); @@ -1303,16 +1297,16 @@ TEST_F(LRUCacheSecondaryCacheTest, CreateFailTest) { Cache::Handle* handle; SetFailCreate(true); handle = cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); // This lookup should fail, since k1 creation would have failed handle = cache->Lookup(k1.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_EQ(handle, nullptr); // Since k1 didn't get promoted, k2 should still be in cache handle = cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); + /*context*/ this, Cache::Priority::LOW); ASSERT_NE(handle, nullptr); cache->Release(handle); ASSERT_EQ(secondary_cache->num_inserts(), 1u); @@ -1322,50 +1316,70 @@ TEST_F(LRUCacheSecondaryCacheTest, CreateFailTest) { secondary_cache.reset(); } -TEST_F(LRUCacheSecondaryCacheTest, FullCapacityTest) { - LRUCacheOptions opts(1024 /* capacity */, 0 /* num_shard_bits */, - true /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); - std::shared_ptr secondary_cache = - std::make_shared(2048); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); - CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); - CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); +TEST_P(BasicSecondaryCacheTest, FullCapacityTest) { + for (bool strict_capacity_limit : {false, true}) { + std::shared_ptr secondary_cache = + std::make_shared(2048); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 0 /* num_shard_bits */, + strict_capacity_limit, secondary_cache); + CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); + CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get()); - Random rnd(301); - std::string str1 = rnd.RandomString(1020); - TestItem* item1 = new TestItem(str1.data(), str1.length()); - ASSERT_OK(cache->Insert(k1.AsSlice(), item1, GetHelper(), str1.length())); - std::string str2 = rnd.RandomString(1020); - TestItem* item2 = new TestItem(str2.data(), str2.length()); - // k1 should be demoted to NVM - ASSERT_OK(cache->Insert(k2.AsSlice(), item2, GetHelper(), str2.length())); + Random rnd(301); + std::string str1 = rnd.RandomString(1020); + TestItem* item1 = new TestItem(str1.data(), str1.length()); + ASSERT_OK(cache->Insert(k1.AsSlice(), item1, GetHelper(), str1.length())); + std::string str2 = rnd.RandomString(1020); + TestItem* item2 = new TestItem(str2.data(), str2.length()); + // k1 should be demoted to NVM + ASSERT_OK(cache->Insert(k2.AsSlice(), item2, GetHelper(), str2.length())); - Cache::Handle* handle; - handle = cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); - ASSERT_NE(handle, nullptr); - // k1 promotion should fail due to the block cache being at capacity, - // but the lookup should still succeed - Cache::Handle* handle2; - handle2 = cache->Lookup(k1.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); - ASSERT_NE(handle2, nullptr); - // Since k1 didn't get inserted, k2 should still be in cache - cache->Release(handle); - cache->Release(handle2); - handle = cache->Lookup(k2.AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, true); - ASSERT_NE(handle, nullptr); - cache->Release(handle); - ASSERT_EQ(secondary_cache->num_inserts(), 1u); - ASSERT_EQ(secondary_cache->num_lookups(), 1u); + Cache::Handle* handle2; + handle2 = cache->Lookup(k2.AsSlice(), GetHelper(), + /*context*/ this, Cache::Priority::LOW); + ASSERT_NE(handle2, nullptr); + // k1 lookup fails without secondary cache support + Cache::Handle* handle1; + handle1 = cache->Lookup( + k1.AsSlice(), + GetHelper(CacheEntryRole::kDataBlock, /*secondary_compatible=*/false), + /*context*/ this, Cache::Priority::LOW); + ASSERT_EQ(handle1, nullptr); - cache.reset(); - secondary_cache.reset(); + // k1 promotion can fail with strict_capacit_limit=true, but Lookup still + // succeeds using a standalone handle + handle1 = cache->Lookup(k1.AsSlice(), GetHelper(), + /*context*/ this, Cache::Priority::LOW); + ASSERT_NE(handle1, nullptr); + + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 1u); + + // Releasing k2's handle first, k2 is evicted from primary iff k1 promotion + // was charged to the cache (except HCC doesn't erase in Release() over + // capacity) + // FIXME: Insert to secondary from Release disabled + cache->Release(handle2); + cache->Release(handle1); + handle2 = cache->Lookup( + k2.AsSlice(), + GetHelper(CacheEntryRole::kDataBlock, /*secondary_compatible=*/false), + /*context*/ this, Cache::Priority::LOW); + if (strict_capacity_limit || GetParam() == kHyperClock) { + ASSERT_NE(handle2, nullptr); + cache->Release(handle2); + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + } else { + ASSERT_EQ(handle2, nullptr); + // FIXME: Insert to secondary from Release disabled + // ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + } + + cache.reset(); + secondary_cache.reset(); + } } // In this test, the block cache size is set to 4096, after insert 6 KV-pairs @@ -1374,16 +1388,24 @@ TEST_F(LRUCacheSecondaryCacheTest, FullCapacityTest) { // of the meta blocks are about 900 to 1000. Therefore, in any situation, // if we try to insert block_1 to the block cache, it will always fails. Only // block_2 will be successfully inserted into the block cache. -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { - LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +// CORRECTION: this is not quite right. block_1 can be inserted into the block +// cache because strict_capacity_limit=false, but it is removed from the cache +// in Release() because of being over-capacity, without demoting to secondary +// cache. HyperClockCache doesn't check capacity on release (for efficiency) +// so can demote the over-capacity item to secondary cache. Also, we intend to +// add support for demotion in Release, but that currently causes too much +// unit test churn. +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { + if (GetParam() == kHyperClock) { + // See CORRECTION above + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1471,16 +1493,16 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { // of the meta blocks are about 900 to 1000. Therefore, we can successfully // insert and cache block_1 in the block cache (this is the different place // from TestSecondaryCacheCorrectness1) -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { - LRUCacheOptions opts(6100 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { + if (GetParam() == kHyperClock) { + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(6100 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1564,16 +1586,12 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { // of the meta blocks are about 900 to 1000. Therefore, we can successfully // cache all the blocks in the block cache and there is not secondary cache // insertion. 2 lookup is needed for the blocks. -TEST_F(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { - LRUCacheOptions opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1618,16 +1636,12 @@ TEST_F(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { Destroy(options); } -TEST_F(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { - LRUCacheOptions opts(8 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(8 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1667,16 +1681,16 @@ TEST_F(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { // of the meta blocks are about 900 to 1000. Therefore, in any situation, // if we try to insert block_1 to the block cache, it will always fails. Only // block_2 will be successfully inserted into the block cache. -TEST_F(DBSecondaryCacheTest, SecondaryCacheFailureTest) { - LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, SecondaryCacheFailureTest) { + if (GetParam() == kHyperClock) { + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1759,16 +1773,12 @@ TEST_F(DBSecondaryCacheTest, SecondaryCacheFailureTest) { Destroy(options); } -TEST_F(LRUCacheSecondaryCacheTest, BasicWaitAllTest) { - LRUCacheOptions opts(1024 /* capacity */, 2 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(BasicSecondaryCacheTest, BasicWaitAllTest) { std::shared_ptr secondary_cache = std::make_shared(32 * 1024); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1024 /* capacity */, 2 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); const int num_keys = 32; OffsetableCacheKey ock{"foo", "bar", 1}; @@ -1782,7 +1792,15 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicWaitAllTest) { str.length())); } // Force all entries to be evicted to the secondary cache - cache->SetCapacity(0); + if (GetParam() == kHyperClock) { + // HCC doesn't respond immediately to SetCapacity + for (int i = 9000; i < 9030; ++i) { + ASSERT_OK(cache->Insert(ock.WithOffset(i).AsSlice(), nullptr, + &kNoopCacheItemHelper, 256)); + } + } else { + cache->SetCapacity(0); + } ASSERT_EQ(secondary_cache->num_inserts(), 32u); cache->SetCapacity(32 * 1024); @@ -1793,24 +1811,31 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicWaitAllTest) { TestSecondaryCache::ResultType::DEFER_AND_FAIL}, {ock.WithOffset(5).AsSlice().ToString(), TestSecondaryCache::ResultType::FAIL}}); - std::vector results; - for (int i = 0; i < 6; ++i) { - results.emplace_back(cache->Lookup(ock.WithOffset(i).AsSlice(), GetHelper(), - /*context*/ this, Cache::Priority::LOW, - false)); + + std::array async_handles; + std::array cache_keys; + for (size_t i = 0; i < async_handles.size(); ++i) { + auto& ah = async_handles[i]; + cache_keys[i] = ock.WithOffset(i); + ah.key = cache_keys[i].AsSlice(); + ah.helper = GetHelper(); + ah.create_context = this; + ah.priority = Cache::Priority::LOW; + cache->StartAsyncLookup(ah); } - cache->WaitAll(results); - for (int i = 0; i < 6; ++i) { - if (i == 4) { - ASSERT_EQ(cache->Value(results[i]), nullptr); - } else if (i == 5) { - ASSERT_EQ(results[i], nullptr); + cache->WaitAll(&async_handles[0], async_handles.size()); + for (size_t i = 0; i < async_handles.size(); ++i) { + SCOPED_TRACE("i = " + std::to_string(i)); + Cache::Handle* result = async_handles[i].Result(); + if (i == 4 || i == 5) { + ASSERT_EQ(result, nullptr); continue; } else { - TestItem* item = static_cast(cache->Value(results[i])); + ASSERT_NE(result, nullptr); + TestItem* item = static_cast(cache->Value(result)); ASSERT_EQ(item->ToString(), values[i]); } - cache->Release(results[i]); + cache->Release(result); } cache.reset(); @@ -1821,16 +1846,16 @@ TEST_F(LRUCacheSecondaryCacheTest, BasicWaitAllTest) { // the cache key associated with each data block (and thus each KV) by using // a sync point callback in TestSecondaryCache::Lookup. We then control the // lookup result by setting the ResultMap. -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheMultiGet) { - LRUCacheOptions opts(1 << 20 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheMultiGet) { + if (GetParam() == kHyperClock) { + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(1 << 20 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -1919,10 +1944,10 @@ class CacheWithStats : public CacheWrapper { return target_->Insert(key, value, helper, charge, handle, priority); } Handle* Lookup(const Slice& key, const CacheItemHelper* helper, - CreateContext* create_context, Priority priority, bool wait, + CreateContext* create_context, Priority priority, Statistics* stats = nullptr) override { lookup_count_++; - return target_->Lookup(key, helper, create_context, priority, wait, stats); + return target_->Lookup(key, helper, create_context, priority, stats); } uint32_t GetInsertCount() { return insert_count_; } @@ -1937,14 +1962,12 @@ class CacheWithStats : public CacheWrapper { uint32_t lookup_count_ = 0; }; -TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { - LRUCacheOptions cache_opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { + std::shared_ptr base_cache = + NewCache(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */); std::shared_ptr cache = - std::make_shared(NewLRUCache(cache_opts)); + std::make_shared(base_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -2010,8 +2033,10 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { // dumpload std::shared_ptr secondary_cache = std::make_shared(2048 * 1024); - cache_opts.secondary_cache = secondary_cache; - cache = std::make_shared(NewLRUCache(cache_opts)); + // This time with secondary cache + base_cache = NewCache(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); + cache = std::make_shared(base_cache); table_options.block_cache = cache; table_options.block_size = 4 * 1024; options.create_if_missing = true; @@ -2066,14 +2091,12 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadBasic) { Destroy(options); } -TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { - LRUCacheOptions cache_opts(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, - kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { + std::shared_ptr base_cache = + NewCache(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */); std::shared_ptr cache = - std::make_shared(NewLRUCache(cache_opts)); + std::make_shared(base_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -2166,8 +2189,10 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { // dumpload std::shared_ptr secondary_cache = std::make_shared(2048 * 1024); - cache_opts.secondary_cache = secondary_cache; - cache = std::make_shared(NewLRUCache(cache_opts)); + // This time with secondary_cache + base_cache = NewCache(1024 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); + cache = std::make_shared(base_cache); table_options.block_cache = cache; table_options.block_size = 4 * 1024; options.create_if_missing = true; @@ -2231,16 +2256,12 @@ TEST_F(DBSecondaryCacheTest, LRUCacheDumpLoadWithFilter) { } // Test the option not to use the secondary cache in a certain DB. -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionBasic) { - LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheOptionBasic) { std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -2326,16 +2347,16 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionBasic) { // We disable the secondary cache in DBOptions at first. Close and reopen the DB // with new options, which set the lowest_used_cache_tier to // kNonVolatileBlockTier. So secondary cache will be used. -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionChange) { - LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheOptionChange) { + if (GetParam() == kHyperClock) { + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -2421,16 +2442,16 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionChange) { // Two DB test. We create 2 DBs sharing the same block cache and secondary // cache. We diable the secondary cache option for DB2. -TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionTwoDB) { - LRUCacheOptions opts(4 * 1024 /* capacity */, 0 /* num_shard_bits */, - false /* strict_capacity_limit */, - 0.5 /* high_pri_pool_ratio */, - nullptr /* memory_allocator */, kDefaultToAdaptiveMutex, - kDontChargeCacheMetadata); +TEST_P(DBSecondaryCacheTest, TestSecondaryCacheOptionTwoDB) { + if (GetParam() == kHyperClock) { + ROCKSDB_GTEST_BYPASS("Test depends on LRUCache-specific behaviors"); + return; + } std::shared_ptr secondary_cache( new TestSecondaryCache(2048 * 1024)); - opts.secondary_cache = secondary_cache; - std::shared_ptr cache = NewLRUCache(opts); + std::shared_ptr cache = + NewCache(4 * 1024 /* capacity */, 0 /* num_shard_bits */, + false /* strict_capacity_limit */, secondary_cache); BlockBasedTableOptions table_options; table_options.block_cache = cache; table_options.block_size = 4 * 1024; @@ -2528,7 +2549,6 @@ TEST_F(DBSecondaryCacheTest, TestSecondaryCacheOptionTwoDB) { ASSERT_OK(DestroyDB(dbname2, options)); } - } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/cache/secondary_cache_adapter.cc b/cache/secondary_cache_adapter.cc new file mode 100644 index 0000000000..772b7a69a2 --- /dev/null +++ b/cache/secondary_cache_adapter.cc @@ -0,0 +1,296 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "cache/secondary_cache_adapter.h" + +#include "monitoring/perf_context_imp.h" + +namespace ROCKSDB_NAMESPACE { + +namespace { +// A distinct pointer value for marking "dummy" cache entries +struct Dummy { + char val[7] = "kDummy"; +}; +const Dummy kDummy{}; +Cache::ObjectPtr const kDummyObj = const_cast(&kDummy); +} // namespace + +CacheWithSecondaryAdapter::CacheWithSecondaryAdapter( + std::shared_ptr target, + std::shared_ptr secondary_cache) + : CacheWrapper(std::move(target)), + secondary_cache_(std::move(secondary_cache)) { + target_->SetEvictionCallback([this](const Slice& key, Handle* handle) { + return EvictionHandler(key, handle); + }); +} + +CacheWithSecondaryAdapter::~CacheWithSecondaryAdapter() { + // `*this` will be destroyed before `*target_`, so we have to prevent + // use after free + target_->SetEvictionCallback({}); +} + +bool CacheWithSecondaryAdapter::EvictionHandler(const Slice& key, + Handle* handle) { + auto helper = GetCacheItemHelper(handle); + if (helper->IsSecondaryCacheCompatible()) { + auto obj = target_->Value(handle); + // Ignore dummy entry + if (obj != kDummyObj) { + // Spill into secondary cache. + secondary_cache_->Insert(key, obj, helper).PermitUncheckedError(); + } + } + // Never takes ownership of obj + return false; +} + +bool CacheWithSecondaryAdapter::ProcessDummyResult(Cache::Handle** handle, + bool erase) { + if (*handle && target_->Value(*handle) == kDummyObj) { + target_->Release(*handle, erase); + *handle = nullptr; + return true; + } else { + return false; + } +} + +void CacheWithSecondaryAdapter::CleanupCacheObject( + ObjectPtr obj, const CacheItemHelper* helper) { + if (helper->del_cb) { + helper->del_cb(obj, memory_allocator()); + } +} + +Cache::Handle* CacheWithSecondaryAdapter::Promote( + std::unique_ptr&& secondary_handle, + const Slice& key, const CacheItemHelper* helper, Priority priority, + Statistics* stats, bool found_dummy_entry, bool kept_in_sec_cache) { + assert(secondary_handle->IsReady()); + + ObjectPtr obj = secondary_handle->Value(); + if (!obj) { + // Nothing found. + return nullptr; + } + // Found something. + switch (helper->role) { + case CacheEntryRole::kFilterBlock: + RecordTick(stats, SECONDARY_CACHE_FILTER_HITS); + break; + case CacheEntryRole::kIndexBlock: + RecordTick(stats, SECONDARY_CACHE_INDEX_HITS); + break; + case CacheEntryRole::kDataBlock: + RecordTick(stats, SECONDARY_CACHE_DATA_HITS); + break; + default: + break; + } + PERF_COUNTER_ADD(secondary_cache_hit_count, 1); + RecordTick(stats, SECONDARY_CACHE_HITS); + + // FIXME? using charge=Size() is suspicious but inherited from older + // implementation + size_t charge = secondary_handle->Size(); + Handle* result = nullptr; + // Insert into primary cache, possibly as a standalone+dummy entries. + if (secondary_cache_->SupportForceErase() && !found_dummy_entry) { + // Create standalone and insert dummy + // Allow standalone to be created even if cache is full, to avoid + // reading the entry from storage. + result = + CreateStandalone(key, obj, helper, charge, /*allow_uncharged*/ true); + assert(result); + PERF_COUNTER_ADD(block_cache_standalone_handle_count, 1); + + // Insert dummy to record recent use + // TODO: try to avoid case where inserting this dummy could overwrite a + // regular entry + Status s = Insert(key, kDummyObj, &kNoopCacheItemHelper, /*charge=*/0, + /*handle=*/nullptr, priority); + s.PermitUncheckedError(); + // Nothing to do or clean up on dummy insertion failure + } else { + // Insert regular entry into primary cache. + // Don't allow it to spill into secondary cache again if it was kept there. + Status s = Insert( + key, obj, kept_in_sec_cache ? helper->without_secondary_compat : helper, + charge, &result, priority); + if (s.ok()) { + assert(result); + PERF_COUNTER_ADD(block_cache_real_handle_count, 1); + } else { + // Create standalone result instead, even if cache is full, to avoid + // reading the entry from storage. + result = + CreateStandalone(key, obj, helper, charge, /*allow_uncharged*/ true); + assert(result); + PERF_COUNTER_ADD(block_cache_standalone_handle_count, 1); + } + } + return result; +} + +Cache::Handle* CacheWithSecondaryAdapter::Lookup(const Slice& key, + const CacheItemHelper* helper, + CreateContext* create_context, + Priority priority, + Statistics* stats) { + // NOTE: we could just StartAsyncLookup() and Wait(), but this should be a bit + // more efficient + Handle* result = + target_->Lookup(key, helper, create_context, priority, stats); + bool secondary_compatible = helper && helper->IsSecondaryCacheCompatible(); + bool found_dummy_entry = + ProcessDummyResult(&result, /*erase=*/secondary_compatible); + if (!result && secondary_compatible) { + // Try our secondary cache + bool kept_in_sec_cache = false; + std::unique_ptr secondary_handle = + secondary_cache_->Lookup(key, helper, create_context, /*wait*/ true, + found_dummy_entry, /*out*/ kept_in_sec_cache); + if (secondary_handle) { + result = Promote(std::move(secondary_handle), key, helper, priority, + stats, found_dummy_entry, kept_in_sec_cache); + } + } + return result; +} + +Cache::ObjectPtr CacheWithSecondaryAdapter::Value(Handle* handle) { + ObjectPtr v = target_->Value(handle); + // TODO with stacked secondaries: might fail in EvictionHandler + assert(v != kDummyObj); + return v; +} + +void CacheWithSecondaryAdapter::StartAsyncLookupOnMySecondary( + AsyncLookupHandle& async_handle) { + assert(!async_handle.IsPending()); + assert(async_handle.result_handle == nullptr); + + std::unique_ptr secondary_handle = + secondary_cache_->Lookup(async_handle.key, async_handle.helper, + async_handle.create_context, /*wait*/ false, + async_handle.found_dummy_entry, + /*out*/ async_handle.kept_in_sec_cache); + if (secondary_handle) { + // TODO with stacked secondaries: Check & process if already ready? + async_handle.pending_handle = secondary_handle.release(); + async_handle.pending_cache = secondary_cache_.get(); + } +} + +void CacheWithSecondaryAdapter::StartAsyncLookup( + AsyncLookupHandle& async_handle) { + target_->StartAsyncLookup(async_handle); + if (!async_handle.IsPending()) { + bool secondary_compatible = + async_handle.helper && + async_handle.helper->IsSecondaryCacheCompatible(); + async_handle.found_dummy_entry |= ProcessDummyResult( + &async_handle.result_handle, /*erase=*/secondary_compatible); + + if (async_handle.Result() == nullptr && secondary_compatible) { + // Not found and not pending on another secondary cache + StartAsyncLookupOnMySecondary(async_handle); + } + } +} + +void CacheWithSecondaryAdapter::WaitAll(AsyncLookupHandle* async_handles, + size_t count) { + if (count == 0) { + // Nothing to do + return; + } + // Requests that are pending on *my* secondary cache, at the start of this + // function + std::vector my_pending; + // Requests that are pending on an "inner" secondary cache (managed somewhere + // under target_), as of the start of this function + std::vector inner_pending; + + // Initial accounting of pending handles, excluding those already handled + // by "outer" secondary caches. (See cur->pending_cache = nullptr.) + for (size_t i = 0; i < count; ++i) { + AsyncLookupHandle* cur = async_handles + i; + if (cur->pending_cache) { + assert(cur->IsPending()); + assert(cur->helper); + assert(cur->helper->IsSecondaryCacheCompatible()); + if (cur->pending_cache == secondary_cache_.get()) { + my_pending.push_back(cur); + // Mark as "to be handled by this caller" + cur->pending_cache = nullptr; + } else { + // Remember as potentially needing a lookup in my secondary + inner_pending.push_back(cur); + } + } + } + + // Wait on inner-most cache lookups first + // TODO with stacked secondaries: because we are not using proper + // async/await constructs here yet, there is a false synchronization point + // here where all the results at one level are needed before initiating + // any lookups at the next level. Probably not a big deal, but worth noting. + if (!inner_pending.empty()) { + target_->WaitAll(async_handles, count); + } + + // For those that failed to find something, convert to lookup in my + // secondary cache. + for (AsyncLookupHandle* cur : inner_pending) { + if (cur->Result() == nullptr) { + // Not found, try my secondary + StartAsyncLookupOnMySecondary(*cur); + if (cur->IsPending()) { + assert(cur->pending_cache == secondary_cache_.get()); + my_pending.push_back(cur); + // Mark as "to be handled by this caller" + cur->pending_cache = nullptr; + } + } + } + + // Wait on all lookups on my secondary cache + { + std::vector my_secondary_handles; + for (AsyncLookupHandle* cur : my_pending) { + my_secondary_handles.push_back(cur->pending_handle); + } + secondary_cache_->WaitAll(my_secondary_handles); + } + + // Process results + for (AsyncLookupHandle* cur : my_pending) { + std::unique_ptr secondary_handle( + cur->pending_handle); + cur->pending_handle = nullptr; + cur->result_handle = Promote( + std::move(secondary_handle), cur->key, cur->helper, cur->priority, + cur->stats, cur->found_dummy_entry, cur->kept_in_sec_cache); + assert(cur->pending_cache == nullptr); + } +} + +std::string CacheWithSecondaryAdapter::GetPrintableOptions() const { + std::string str = target_->GetPrintableOptions(); + str.append(" secondary_cache:\n"); + str.append(secondary_cache_->GetPrintableOptions()); + return str; +} + +const char* CacheWithSecondaryAdapter::Name() const { + // To the user, at least for now, configure the underlying cache with + // a secondary cache. So we pretend to be that cache + return target_->Name(); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/cache/secondary_cache_adapter.h b/cache/secondary_cache_adapter.h new file mode 100644 index 0000000000..4264a8d041 --- /dev/null +++ b/cache/secondary_cache_adapter.h @@ -0,0 +1,52 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include "rocksdb/secondary_cache.h" + +namespace ROCKSDB_NAMESPACE { + +class CacheWithSecondaryAdapter : public CacheWrapper { + public: + explicit CacheWithSecondaryAdapter( + std::shared_ptr target, + std::shared_ptr secondary_cache); + + ~CacheWithSecondaryAdapter() override; + + Handle* Lookup(const Slice& key, const CacheItemHelper* helper, + CreateContext* create_context, + Priority priority = Priority::LOW, + Statistics* stats = nullptr) override; + + ObjectPtr Value(Handle* handle) override; + + void StartAsyncLookup(AsyncLookupHandle& async_handle) override; + + void WaitAll(AsyncLookupHandle* async_handles, size_t count) override; + + std::string GetPrintableOptions() const override; + + const char* Name() const override; + + private: + bool EvictionHandler(const Slice& key, Handle* handle); + + void StartAsyncLookupOnMySecondary(AsyncLookupHandle& async_handle); + + Handle* Promote( + std::unique_ptr&& secondary_handle, + const Slice& key, const CacheItemHelper* helper, Priority priority, + Statistics* stats, bool found_dummy_entry, bool kept_in_sec_cache); + + bool ProcessDummyResult(Cache::Handle** handle, bool erase); + + void CleanupCacheObject(ObjectPtr obj, const CacheItemHelper* helper); + + std::shared_ptr secondary_cache_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/cache/sharded_cache.h b/cache/sharded_cache.h index 36b9aef77e..c8eb58aad5 100644 --- a/cache/sharded_cache.h +++ b/cache/sharded_cache.h @@ -51,15 +51,17 @@ class CacheShardBase { }; Status Insert(const Slice& key, HashCref hash, Cache::ObjectPtr value, const Cache::CacheItemHelper* helper, size_t charge, - HandleImpl** handle, Cache::Priority priority) = 0; + HandleImpl** handle, Cache::Priority priority, + bool standalone) = 0; + Handle* CreateStandalone(const Slice& key, HashCref hash, ObjectPtr obj, + const CacheItemHelper* helper, + size_t charge, bool allow_uncharged) = 0; HandleImpl* Lookup(const Slice& key, HashCref hash, const Cache::CacheItemHelper* helper, Cache::CreateContext* create_context, - Cache::Priority priority, bool wait, + Cache::Priority priority, Statistics* stats) = 0; bool Release(HandleImpl* handle, bool useful, bool erase_if_last_ref) = 0; - bool IsReady(HandleImpl* handle) = 0; - void Wait(HandleImpl* handle) = 0; bool Ref(HandleImpl* handle) = 0; void Erase(const Slice& key, HashCref hash) = 0; void SetCapacity(size_t capacity) = 0; @@ -169,24 +171,33 @@ class ShardedCache : public ShardedCacheBase { [s_c_l](CacheShard* cs) { cs->SetStrictCapacityLimit(s_c_l); }); } - Status Insert(const Slice& key, ObjectPtr value, - const CacheItemHelper* helper, size_t charge, - Handle** handle = nullptr, + Status Insert(const Slice& key, ObjectPtr obj, const CacheItemHelper* helper, + size_t charge, Handle** handle = nullptr, Priority priority = Priority::LOW) override { assert(helper); HashVal hash = CacheShard::ComputeHash(key); auto h_out = reinterpret_cast(handle); - return GetShard(hash).Insert(key, hash, value, helper, charge, h_out, + return GetShard(hash).Insert(key, hash, obj, helper, charge, h_out, priority); } + Handle* CreateStandalone(const Slice& key, ObjectPtr obj, + const CacheItemHelper* helper, size_t charge, + bool allow_uncharged) override { + assert(helper); + HashVal hash = CacheShard::ComputeHash(key); + HandleImpl* result = GetShard(hash).CreateStandalone( + key, hash, obj, helper, charge, allow_uncharged); + return reinterpret_cast(result); + } + Handle* Lookup(const Slice& key, const CacheItemHelper* helper = nullptr, CreateContext* create_context = nullptr, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override { HashVal hash = CacheShard::ComputeHash(key); - HandleImpl* result = GetShard(hash).Lookup( - key, hash, helper, create_context, priority, wait, stats); + HandleImpl* result = GetShard(hash).Lookup(key, hash, helper, + create_context, priority, stats); return reinterpret_cast(result); } @@ -200,14 +211,6 @@ class ShardedCache : public ShardedCacheBase { auto h = reinterpret_cast(handle); return GetShard(h->GetHash()).Release(h, useful, erase_if_last_ref); } - bool IsReady(Handle* handle) override { - auto h = reinterpret_cast(handle); - return GetShard(h->GetHash()).IsReady(h); - } - void Wait(Handle* handle) override { - auto h = reinterpret_cast(handle); - GetShard(h->GetHash()).Wait(h); - } bool Ref(Handle* handle) override { auto h = reinterpret_cast(handle); return GetShard(h->GetHash()).Ref(h); diff --git a/cache/typed_cache.h b/cache/typed_cache.h index 56e14bbfde..e42aa4c260 100644 --- a/cache/typed_cache.h +++ b/cache/typed_cache.h @@ -153,6 +153,11 @@ class BasicTypedCacheInterface : public BaseCacheInterface, using BasicTypedCacheHelper::GetBasicHelper; // ctor using BaseCacheInterface::BaseCacheInterface; + struct TypedAsyncLookupHandle : public Cache::AsyncLookupHandle { + TypedHandle* Result() { + return reinterpret_cast(Cache::AsyncLookupHandle::Result()); + } + }; inline Status Insert(const Slice& key, TValuePtr value, size_t charge, TypedHandle** handle = nullptr, @@ -168,6 +173,11 @@ class BasicTypedCacheInterface : public BaseCacheInterface, this->cache_->BasicLookup(key, stats)); } + inline void StartAsyncLookup(TypedAsyncLookupHandle& async_handle) { + assert(async_handle.helper == nullptr); + this->cache_->StartAsyncLookup(async_handle); + } + inline CacheHandleGuard Guard(TypedHandle* handle) { if (handle) { return CacheHandleGuard(&*this->cache_, handle); @@ -275,6 +285,8 @@ class FullTypedCacheInterface public: CACHE_TYPE_DEFS(); using typename BasicTypedCacheInterface::TypedHandle; + using typename BasicTypedCacheInterface::TypedAsyncLookupHandle; using typename BasicTypedCacheHelperFns::TValuePtr; using BasicTypedCacheHelper::GetBasicHelper; using FullTypedCacheHelper::GetFullHelper; @@ -326,17 +338,28 @@ class FullTypedCacheInterface // (Basic Lookup() also inherited.) inline TypedHandle* LookupFull( const Slice& key, TCreateContext* create_context = nullptr, - Priority priority = Priority::LOW, bool wait = true, - Statistics* stats = nullptr, + Priority priority = Priority::LOW, Statistics* stats = nullptr, CacheTier lowest_used_cache_tier = CacheTier::kNonVolatileBlockTier) { if (lowest_used_cache_tier == CacheTier::kNonVolatileBlockTier) { return reinterpret_cast(this->cache_->Lookup( - key, GetFullHelper(), create_context, priority, wait, stats)); + key, GetFullHelper(), create_context, priority, stats)); } else { return BasicTypedCacheInterface::Lookup(key, stats); } } + + inline void StartAsyncLookupFull( + TypedAsyncLookupHandle& async_handle, + CacheTier lowest_used_cache_tier = CacheTier::kNonVolatileBlockTier) { + if (lowest_used_cache_tier == CacheTier::kNonVolatileBlockTier) { + async_handle.helper = GetFullHelper(); + this->cache_->StartAsyncLookup(async_handle); + } else { + BasicTypedCacheInterface::StartAsyncLookup( + async_handle); + } + } }; // FullTypedSharedCacheInterface - Like FullTypedCacheInterface but with a diff --git a/db/blob/blob_source.cc b/db/blob/blob_source.cc index 9634435007..1e866c7dd4 100644 --- a/db/blob/blob_source.cc +++ b/db/blob/blob_source.cc @@ -104,9 +104,9 @@ Status BlobSource::PutBlobIntoCache( } BlobSource::TypedHandle* BlobSource::GetEntryFromCache(const Slice& key) const { - return blob_cache_.LookupFull( - key, nullptr /* context */, Cache::Priority::BOTTOM, - true /* wait_for_cache */, statistics_, lowest_used_cache_tier_); + return blob_cache_.LookupFull(key, nullptr /* context */, + Cache::Priority::BOTTOM, statistics_, + lowest_used_cache_tier_); } void BlobSource::PinCachedBlob(CacheHandleGuard* cached_blob, diff --git a/db/db_basic_test.cc b/db/db_basic_test.cc index ad73d292ed..0178fe4801 100644 --- a/db/db_basic_test.cc +++ b/db/db_basic_test.cc @@ -3629,11 +3629,11 @@ class DBBasicTestMultiGet : public DBTestBase { Handle* Lookup(const Slice& key, const CacheItemHelper* helper, CreateContext* create_context, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override { num_lookups_++; Handle* handle = - target_->Lookup(key, helper, create_context, priority, wait, stats); + target_->Lookup(key, helper, create_context, priority, stats); if (handle != nullptr) { num_found_++; } diff --git a/db/db_block_cache_test.cc b/db/db_block_cache_test.cc index 1e7f1215a3..1a13663533 100644 --- a/db/db_block_cache_test.cc +++ b/db/db_block_cache_test.cc @@ -717,7 +717,7 @@ class LookupLiarCache : public CacheWrapper { Handle* Lookup(const Slice& key, const CacheItemHelper* helper = nullptr, CreateContext* create_context = nullptr, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override { if (nth_lookup_not_found_ == 1) { nth_lookup_not_found_ = 0; @@ -726,8 +726,7 @@ class LookupLiarCache : public CacheWrapper { if (nth_lookup_not_found_ > 1) { --nth_lookup_not_found_; } - return CacheWrapper::Lookup(key, helper, create_context, priority, wait, - stats); + return CacheWrapper::Lookup(key, helper, create_context, priority, stats); } // 1 == next lookup, 2 == after next, etc. diff --git a/db/error_handler_fs_test.cc b/db/error_handler_fs_test.cc index 03411aa170..82008705d6 100644 --- a/db/error_handler_fs_test.cc +++ b/db/error_handler_fs_test.cc @@ -2860,4 +2860,3 @@ int main(int argc, char** argv) { ::testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); } - diff --git a/db_stress_tool/db_stress_test_base.cc b/db_stress_tool/db_stress_test_base.cc index 215367760e..610826f4b3 100644 --- a/db_stress_tool/db_stress_test_base.cc +++ b/db_stress_tool/db_stress_test_base.cc @@ -109,36 +109,37 @@ std::shared_ptr StressTest::NewCache(size_t capacity, return nullptr; } + std::shared_ptr secondary_cache; + if (!FLAGS_secondary_cache_uri.empty()) { + Status s = SecondaryCache::CreateFromString( + config_options, FLAGS_secondary_cache_uri, &secondary_cache); + if (secondary_cache == nullptr) { + fprintf(stderr, + "No secondary cache registered matching string: %s status=%s\n", + FLAGS_secondary_cache_uri.c_str(), s.ToString().c_str()); + exit(1); + } + if (FLAGS_secondary_cache_fault_one_in > 0) { + secondary_cache = std::make_shared( + secondary_cache, static_cast(FLAGS_seed), + FLAGS_secondary_cache_fault_one_in); + } + } + if (FLAGS_cache_type == "clock_cache") { fprintf(stderr, "Old clock cache implementation has been removed.\n"); exit(1); } else if (FLAGS_cache_type == "hyper_clock_cache") { - return HyperClockCacheOptions(static_cast(capacity), - FLAGS_block_size /*estimated_entry_charge*/, - num_shard_bits) - .MakeSharedCache(); + HyperClockCacheOptions opts(static_cast(capacity), + FLAGS_block_size /*estimated_entry_charge*/, + num_shard_bits); + opts.secondary_cache = std::move(secondary_cache); + return opts.MakeSharedCache(); } else if (FLAGS_cache_type == "lru_cache") { LRUCacheOptions opts; opts.capacity = capacity; opts.num_shard_bits = num_shard_bits; - std::shared_ptr secondary_cache; - if (!FLAGS_secondary_cache_uri.empty()) { - Status s = SecondaryCache::CreateFromString( - config_options, FLAGS_secondary_cache_uri, &secondary_cache); - if (secondary_cache == nullptr) { - fprintf(stderr, - "No secondary cache registered matching string: %s status=%s\n", - FLAGS_secondary_cache_uri.c_str(), s.ToString().c_str()); - exit(1); - } - if (FLAGS_secondary_cache_fault_one_in > 0) { - secondary_cache = std::make_shared( - secondary_cache, static_cast(FLAGS_seed), - FLAGS_secondary_cache_fault_one_in); - } - opts.secondary_cache = secondary_cache; - } - + opts.secondary_cache = std::move(secondary_cache); return NewLRUCache(opts); } else { fprintf(stderr, "Cache type not supported."); diff --git a/include/rocksdb/advanced_cache.h b/include/rocksdb/advanced_cache.h index 3440fcea56..bd7d5b09c8 100644 --- a/include/rocksdb/advanced_cache.h +++ b/include/rocksdb/advanced_cache.h @@ -20,6 +20,7 @@ namespace ROCKSDB_NAMESPACE { class Logger; +class SecondaryCacheResultHandle; class Statistics; // A Cache maps keys to objects resident in memory, tracks reference counts @@ -248,6 +249,19 @@ class Cache { Handle** handle = nullptr, Priority priority = Priority::LOW) = 0; + // Similar to Insert, but used for creating cache entries that cannot + // be found with Lookup, such as for memory charging purposes. The + // key is needed for cache sharding purposes. + // * If allow_uncharged==true or strict_capacity_limit=false, the operation + // always succeeds and returns a valid Handle. + // * If strict_capacity_limit=true and the requested charge cannot be freed + // up in the cache, then + // * If allow_uncharged==true, it's created anyway (GetCharge() == 0). + // * If allow_uncharged==false, returns nullptr to indicate failure. + virtual Handle* CreateStandalone(const Slice& key, ObjectPtr obj, + const CacheItemHelper* helper, size_t charge, + bool allow_uncharged) = 0; + // Lookup the key, returning nullptr if not found. If found, returns // a handle to the mapping that must eventually be passed to Release(). // @@ -258,41 +272,15 @@ class Cache { // used to promote the entry to an object in the primary cache. // In that case, the helper may be saved and used later when the object // is evicted, so as usual, the pointed-to helper must outlive the cache. - // - // ======================== Async Lookup (wait=false) ====================== - // When wait=false, the handle returned might be in any of three states: - // * Present - If Value() != nullptr, then the result is present and - // the handle can be used just as if wait=true. - // * Pending, not ready (IsReady() == false) - secondary cache is still - // working to retrieve the value. Might become ready any time. - // * Pending, ready (IsReady() == true) - secondary cache has the value - // but it has not been loaded as an object into primary cache. Call to - // Wait()/WaitAll() will not block. - // - // IMPORTANT: Pending handles are not thread-safe, and only these functions - // are allowed on them: Value(), IsReady(), Wait(), WaitAll(). Even Release() - // can only come after Wait() or WaitAll() even though a reference is held. - // - // Only Wait()/WaitAll() gets a Handle out of a Pending state. (Waiting is - // safe and has no effect on other handle states.) After waiting on a Handle, - // it is in one of two states: - // * Present - if Value() != nullptr - // * Failed - if Value() == nullptr, such as if the secondary cache - // initially thought it had the value but actually did not. - // - // Note that given an arbitrary Handle, the only way to distinguish the - // Pending+ready state from the Failed state is to Wait() on it. A cache - // entry not compatible with secondary cache can also have Value()==nullptr - // like the Failed state, but this is not generally a concern. virtual Handle* Lookup(const Slice& key, const CacheItemHelper* helper = nullptr, CreateContext* create_context = nullptr, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) = 0; // Convenience wrapper when secondary cache not supported inline Handle* BasicLookup(const Slice& key, Statistics* stats) { - return Lookup(key, nullptr, nullptr, Priority::LOW, true, stats); + return Lookup(key, nullptr, nullptr, Priority::LOW, stats); } // Increments the reference count for the handle if it refers to an entry in @@ -429,28 +417,109 @@ class Cache { return Release(handle, erase_if_last_ref); } - // Determines if the handle returned by Lookup() can give a value without - // blocking, though Wait()/WaitAll() might be required to publish it to - // Value(). See secondary cache compatible Lookup() above for details. - // This call is not thread safe on "pending" handles. - virtual bool IsReady(Handle* /*handle*/) { return true; } + // A temporary handle structure for managing async lookups, which callers + // of AsyncLookup() can allocate on the call stack for efficiency. + // An AsyncLookupHandle should not be used concurrently across threads. + struct AsyncLookupHandle { + // Inputs, populated by caller: + // NOTE: at least in case of stacked secondary caches, the underlying + // key buffer must last until handle is completely waited on. + Slice key; + const CacheItemHelper* helper = nullptr; + CreateContext* create_context = nullptr; + Priority priority = Priority::LOW; + Statistics* stats = nullptr; - // Convert a "pending" handle into a full thread-shareable handle by - // * If necessary, wait until secondary cache finishes loading the value. - // * Construct the object for primary cache and set it in the handle. - // Even after Wait() on a pending handle, the caller must check for - // Value() == nullptr in case of failure. This call is not thread-safe - // on pending handles. This call has no effect on non-pending handles. - // See secondary cache compatible Lookup() above for details. - virtual void Wait(Handle* /*handle*/) {} + AsyncLookupHandle() {} + AsyncLookupHandle(const Slice& _key, const CacheItemHelper* _helper, + CreateContext* _create_context, + Priority _priority = Priority::LOW, + Statistics* _stats = nullptr) + : key(_key), + helper(_helper), + create_context(_create_context), + priority(_priority), + stats(_stats) {} - // Wait for a vector of handles to become ready. As with Wait(), the user - // should check the Value() of each handle for nullptr. This call is not - // thread-safe on pending handles. - virtual void WaitAll(std::vector& /*handles*/) {} + // AsyncLookupHandle should only be destroyed when no longer pending + ~AsyncLookupHandle() { assert(!IsPending()); } - private: + // No copies or moves (StartAsyncLookup may save a pointer to this) + AsyncLookupHandle(const AsyncLookupHandle&) = delete; + AsyncLookupHandle operator=(const AsyncLookupHandle&) = delete; + AsyncLookupHandle(AsyncLookupHandle&&) = delete; + AsyncLookupHandle operator=(AsyncLookupHandle&&) = delete; + + // Determines if the handle returned by Lookup() can give a value without + // blocking, though Wait()/WaitAll() might be required to publish it to + // Value(). See secondary cache compatible Lookup() above for details. + // This call is not thread safe on "pending" handles. + // WART/TODO with stacked secondaries: might indicate ready when one + // result is ready (a miss) but the next lookup will block. + bool IsReady(); + + // Returns true if Wait/WaitAll is required before calling Result(). + bool IsPending(); + + // Returns a Lookup()-like result if this AsyncHandle is not pending. + // (Undefined behavior on a pending AsyncHandle.) Like Lookup(), the + // caller is responsible for eventually Release()ing a non-nullptr + // Handle* result. + Handle* Result(); + + // Implementation details, for RocksDB internal use only + Handle* result_handle = nullptr; + SecondaryCacheResultHandle* pending_handle = nullptr; + SecondaryCache* pending_cache = nullptr; + bool found_dummy_entry = false; + bool kept_in_sec_cache = false; + }; + + // Starts a potentially asynchronous Lookup(), based on the populated + // "input" fields of the async_handle. The caller is responsible for + // keeping the AsyncLookupHandle and the key it references alive through + // WaitAll(), and the AsyncLookupHandle alive through + // AsyncLookupHandle::Result(). WaitAll() can only be skipped if + // AsyncLookupHandle::IsPending() is already false after StartAsyncLookup. + // Calling AsyncLookupHandle::Result() is essentially required so that + // Release() can be called on non-nullptr Handle result. Wait() is a + // concise version of WaitAll()+Result() on a single handle. After an + // AsyncLookupHandle has completed this cycle, its input fields can be + // updated and re-used for another StartAsyncLookup. + // + // Handle is thread-safe while AsyncLookupHandle is not thread-safe. + // + // Default implementation is appropriate for Caches without + // true asynchronous support: defers to synchronous Lookup(). + // (AsyncLookupHandles will only get into the "pending" state with + // SecondaryCache configured.) + virtual void StartAsyncLookup(AsyncLookupHandle& async_handle); + + // A convenient wrapper around WaitAll() and AsyncLookupHandle::Result() + // for a single async handle. See StartAsyncLookup(). + Handle* Wait(AsyncLookupHandle& async_handle); + + // Wait for an array of async handles to get results, so that none are left + // in the "pending" state. Not thread safe. See StartAsyncLookup(). + // Default implementation is appropriate for Caches without true + // asynchronous support: asserts that all handles are not pending (or not + // expected to be handled by this cache, in case of wrapped/stacked + // WaitAlls()). + virtual void WaitAll(AsyncLookupHandle* /*async_handles*/, size_t /*count*/); + + // For a function called on cache entries about to be evicted. The function + // returns `true` if it has taken ownership of the Value (object), or + // `false` if the cache should destroy it as usual. Regardless, Ref() and + // Release() cannot be called on this Handle that is poised for eviction. + using EvictionCallback = std::function; + // Sets an eviction callback for this Cache. Not thread safe and only + // supports being set once, so should only be used during initialization + // or destruction, guaranteed before or after any thread-shared operations. + void SetEvictionCallback(EvictionCallback&& fn); + + protected: std::shared_ptr memory_allocator_; + EvictionCallback eviction_callback_; }; // A wrapper around Cache that can easily be extended with instrumentation, @@ -470,11 +539,17 @@ class CacheWrapper : public Cache { return target_->Insert(key, value, helper, charge, handle, priority); } + Handle* CreateStandalone(const Slice& key, ObjectPtr obj, + const CacheItemHelper* helper, size_t charge, + bool allow_uncharged) override { + return target_->CreateStandalone(key, obj, helper, charge, allow_uncharged); + } + Handle* Lookup(const Slice& key, const CacheItemHelper* helper, CreateContext* create_context, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override { - return target_->Lookup(key, helper, create_context, priority, wait, stats); + return target_->Lookup(key, helper, create_context, priority, stats); } bool Ref(Handle* handle) override { return target_->Ref(handle); } @@ -526,6 +601,14 @@ class CacheWrapper : public Cache { void EraseUnRefEntries() override { target_->EraseUnRefEntries(); } + void StartAsyncLookup(AsyncLookupHandle& async_handle) override { + target_->StartAsyncLookup(async_handle); + } + + void WaitAll(AsyncLookupHandle* async_handles, size_t count) override { + target_->WaitAll(async_handles, count); + } + protected: std::shared_ptr target_; }; diff --git a/include/rocksdb/cache.h b/include/rocksdb/cache.h index 5dbc301190..387da17539 100644 --- a/include/rocksdb/cache.h +++ b/include/rocksdb/cache.h @@ -135,6 +135,9 @@ struct ShardedCacheOptions { CacheMetadataChargePolicy metadata_charge_policy = kDefaultCacheMetadataChargePolicy; + // A SecondaryCache instance to use the non-volatile tier. + std::shared_ptr secondary_cache; + ShardedCacheOptions() {} ShardedCacheOptions( size_t _capacity, int _num_shard_bits, bool _strict_capacity_limit, @@ -182,9 +185,6 @@ struct LRUCacheOptions : public ShardedCacheOptions { // -DROCKSDB_DEFAULT_TO_ADAPTIVE_MUTEX, false otherwise. bool use_adaptive_mutex = kDefaultToAdaptiveMutex; - // A SecondaryCache instance to use a the non-volatile tier. - std::shared_ptr secondary_cache; - LRUCacheOptions() {} LRUCacheOptions(size_t _capacity, int _num_shard_bits, bool _strict_capacity_limit, double _high_pri_pool_ratio, diff --git a/src.mk b/src.mk index 4425e89c19..df1d39475f 100644 --- a/src.mk +++ b/src.mk @@ -10,6 +10,7 @@ LIB_SOURCES = \ cache/lru_cache.cc \ cache/compressed_secondary_cache.cc \ cache/secondary_cache.cc \ + cache/secondary_cache_adapter.cc \ cache/sharded_cache.cc \ db/arena_wrapped_db_iter.cc \ db/blob/blob_contents.cc \ diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 47bf2bee68..6b0e2f1584 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -94,7 +94,7 @@ CacheAllocationPtr CopyBufferToHeap(MemoryAllocator* allocator, Slice& buf) { const BlockHandle& handle, const UncompressionDict& uncompression_dict, \ CachableEntry* out_parsed_block, GetContext* get_context, \ BlockCacheLookupContext* lookup_context, bool for_compaction, \ - bool use_cache, bool wait_for_cache, bool async_read) const; + bool use_cache, bool async_read) const; INSTANTIATE_RETRIEVE_BLOCK(ParsedFullFilterBlock); INSTANTIATE_RETRIEVE_BLOCK(UncompressionDict); @@ -1254,24 +1254,31 @@ Status BlockBasedTable::ReadMetaIndexBlock( } template -WithBlocklikeCheck BlockBasedTable::GetDataBlockFromCache( - const Slice& cache_key, BlockCacheInterface block_cache, - CachableEntry* out_parsed_block, const bool wait, - GetContext* get_context) const { - assert(out_parsed_block); - assert(out_parsed_block->IsEmpty()); +Cache::Priority BlockBasedTable::GetCachePriority() const { // Here we treat the legacy name "...index_and_filter_blocks..." to mean all // metadata blocks that might go into block cache, EXCEPT only those needed // for the read path (Get, etc.). TableProperties should not be needed on the // read path (prefix extractor setting is an O(1) size special case that we // are working not to require from TableProperties), so it is not given // high-priority treatment if it should go into BlockCache. - const Cache::Priority priority = - rep_->table_options.cache_index_and_filter_blocks_with_high_priority && - TBlocklike::kBlockType != BlockType::kData && - TBlocklike::kBlockType != BlockType::kProperties - ? Cache::Priority::HIGH - : Cache::Priority::LOW; + if constexpr (TBlocklike::kBlockType == BlockType::kData || + TBlocklike::kBlockType == BlockType::kProperties) { + return Cache::Priority::LOW; + } else if (rep_->table_options + .cache_index_and_filter_blocks_with_high_priority) { + return Cache::Priority::HIGH; + } else { + return Cache::Priority::LOW; + } +} + +template +WithBlocklikeCheck BlockBasedTable::GetDataBlockFromCache( + const Slice& cache_key, BlockCacheInterface block_cache, + CachableEntry* out_parsed_block, + GetContext* get_context) const { + assert(out_parsed_block); + assert(out_parsed_block->IsEmpty()); Status s; Statistics* statistics = rep_->ioptions.statistics.get(); @@ -1280,8 +1287,8 @@ WithBlocklikeCheck BlockBasedTable::GetDataBlockFromCache( if (block_cache) { assert(!cache_key.empty()); auto cache_handle = block_cache.LookupFull( - cache_key, &rep_->create_context, priority, wait, statistics, - rep_->ioptions.lowest_used_cache_tier); + cache_key, &rep_->create_context, GetCachePriority(), + statistics, rep_->ioptions.lowest_used_cache_tier); // Avoid updating metrics here if the handle is not complete yet. This // happens with MultiGet and secondary cache. So update the metrics only @@ -1314,11 +1321,6 @@ WithBlocklikeCheck BlockBasedTable::PutDataBlockToCache( MemoryAllocator* memory_allocator, GetContext* get_context) const { const ImmutableOptions& ioptions = rep_->ioptions; const uint32_t format_version = rep_->table_options.format_version; - const Cache::Priority priority = - rep_->table_options.cache_index_and_filter_blocks_with_high_priority && - TBlocklike::kBlockType != BlockType::kData - ? Cache::Priority::HIGH - : Cache::Priority::LOW; assert(out_parsed_block); assert(out_parsed_block->IsEmpty()); @@ -1349,7 +1351,7 @@ WithBlocklikeCheck BlockBasedTable::PutDataBlockToCache( size_t charge = block_holder->ApproximateMemoryUsage(); BlockCacheTypedHandle* cache_handle = nullptr; s = block_cache.InsertFull(cache_key, block_holder.get(), charge, - &cache_handle, priority, + &cache_handle, GetCachePriority(), rep_->ioptions.lowest_used_cache_tier); if (s.ok()) { @@ -1446,10 +1448,9 @@ WithBlocklikeCheck BlockBasedTable::MaybeReadBlockAndLoadToCache( FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, const BlockHandle& handle, const UncompressionDict& uncompression_dict, - const bool wait, const bool for_compaction, - CachableEntry* out_parsed_block, GetContext* get_context, - BlockCacheLookupContext* lookup_context, BlockContents* contents, - bool async_read) const { + bool for_compaction, CachableEntry* out_parsed_block, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents, bool async_read) const { assert(out_parsed_block != nullptr); const bool no_io = (ro.read_tier == kBlockCacheTier); BlockCacheInterface block_cache{ @@ -1468,7 +1469,7 @@ BlockBasedTable::MaybeReadBlockAndLoadToCache( key = key_data.AsSlice(); if (!contents) { - s = GetDataBlockFromCache(key, block_cache, out_parsed_block, wait, + s = GetDataBlockFromCache(key, block_cache, out_parsed_block, get_context); // Value could still be null at this point, so check the cache handle // and update the read pattern for prefetching @@ -1629,15 +1630,15 @@ WithBlocklikeCheck BlockBasedTable::RetrieveBlock( const BlockHandle& handle, const UncompressionDict& uncompression_dict, CachableEntry* out_parsed_block, GetContext* get_context, BlockCacheLookupContext* lookup_context, bool for_compaction, - bool use_cache, bool wait_for_cache, bool async_read) const { + bool use_cache, bool async_read) const { assert(out_parsed_block); assert(out_parsed_block->IsEmpty()); Status s; if (use_cache) { s = MaybeReadBlockAndLoadToCache( - prefetch_buffer, ro, handle, uncompression_dict, wait_for_cache, - for_compaction, out_parsed_block, get_context, lookup_context, + prefetch_buffer, ro, handle, uncompression_dict, for_compaction, + out_parsed_block, get_context, lookup_context, /*contents=*/nullptr, async_read); if (!s.ok()) { diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 9d384edc65..d50ee0a2e5 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -336,10 +336,9 @@ class BlockBasedTable : public TableReader { WithBlocklikeCheck MaybeReadBlockAndLoadToCache( FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, const BlockHandle& handle, const UncompressionDict& uncompression_dict, - const bool wait, const bool for_compaction, - CachableEntry* block_entry, GetContext* get_context, - BlockCacheLookupContext* lookup_context, BlockContents* contents, - bool async_read) const; + bool for_compaction, CachableEntry* block_entry, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents, bool async_read) const; // Similar to the above, with one crucial difference: it will retrieve the // block from the file even if there are no caches configured (assuming the @@ -350,16 +349,14 @@ class BlockBasedTable : public TableReader { const BlockHandle& handle, const UncompressionDict& uncompression_dict, CachableEntry* block_entry, GetContext* get_context, BlockCacheLookupContext* lookup_context, bool for_compaction, - bool use_cache, bool wait_for_cache, bool async_read) const; + bool use_cache, bool async_read) const; DECLARE_SYNC_AND_ASYNC_CONST( void, RetrieveMultipleBlocks, const ReadOptions& options, const MultiGetRange* batch, const autovector* handles, - autovector* statuses, - autovector, MultiGetContext::MAX_BATCH_SIZE>* - results, - char* scratch, const UncompressionDict& uncompression_dict); + Status* statuses, CachableEntry* results, char* scratch, + const UncompressionDict& uncompression_dict); // Get the iterator from the index reader. // @@ -379,6 +376,9 @@ class BlockBasedTable : public TableReader { IndexBlockIter* input_iter, GetContext* get_context, BlockCacheLookupContext* lookup_context) const; + template + Cache::Priority GetCachePriority() const; + // Read block cache from block caches (if set): block_cache. // On success, Status::OK with be returned and @block will be populated with // pointer to the block as well as its block handle. @@ -387,8 +387,7 @@ class BlockBasedTable : public TableReader { template WithBlocklikeCheck GetDataBlockFromCache( const Slice& cache_key, BlockCacheInterface block_cache, - CachableEntry* block, const bool wait, - GetContext* get_context) const; + CachableEntry* block, GetContext* get_context) const; // Put a maybe compressed block to the corresponding block caches. // This method will perform decompression against block_contents if needed diff --git a/table/block_based/block_based_table_reader_impl.h b/table/block_based/block_based_table_reader_impl.h index 105a479f36..50d1477124 100644 --- a/table/block_based/block_based_table_reader_impl.h +++ b/table/block_based/block_based_table_reader_impl.h @@ -77,15 +77,15 @@ TBlockIter* BlockBasedTable::NewDataBlockIterator( const UncompressionDict& dict = uncompression_dict.GetValue() ? *uncompression_dict.GetValue() : UncompressionDict::GetEmptyDict(); - s = RetrieveBlock( - prefetch_buffer, ro, handle, dict, &block.As(), - get_context, lookup_context, for_compaction, - /* use_cache */ true, /* wait_for_cache */ true, async_read); + s = RetrieveBlock(prefetch_buffer, ro, handle, dict, + &block.As(), get_context, lookup_context, + for_compaction, + /* use_cache */ true, async_read); } else { s = RetrieveBlock( prefetch_buffer, ro, handle, UncompressionDict::GetEmptyDict(), &block.As(), get_context, lookup_context, for_compaction, - /* use_cache */ true, /* wait_for_cache */ true, async_read); + /* use_cache */ true, async_read); } if (s.IsTryAgain() && async_read) { diff --git a/table/block_based/block_based_table_reader_sync_and_async.h b/table/block_based/block_based_table_reader_sync_and_async.h index 422631ef10..65d29e8318 100644 --- a/table/block_based/block_based_table_reader_sync_and_async.h +++ b/table/block_based/block_based_table_reader_sync_and_async.h @@ -32,9 +32,8 @@ namespace ROCKSDB_NAMESPACE { DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) (const ReadOptions& options, const MultiGetRange* batch, const autovector* handles, - autovector* statuses, - autovector, MultiGetContext::MAX_BATCH_SIZE>* results, - char* scratch, const UncompressionDict& uncompression_dict) const { + Status* statuses, CachableEntry* results, char* scratch, + const UncompressionDict& uncompression_dict) const { RandomAccessFileReader* file = rep_->file.get(); const Footer& footer = rep_->footer; const ImmutableOptions& ioptions = rep_->ioptions; @@ -52,13 +51,14 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) continue; } - (*statuses)[idx_in_batch] = + statuses[idx_in_batch] = RetrieveBlock(nullptr, options, handle, uncompression_dict, - &(*results)[idx_in_batch].As(), + &results[idx_in_batch].As(), mget_iter->get_context, &lookup_data_block_context, /* for_compaction */ false, /* use_cache */ true, - /* wait_for_cache */ true, /* async_read */ false); + /* async_read */ false); } + assert(idx_in_batch == handles->size()); CO_RETURN; } @@ -261,12 +261,12 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) if (options.fill_cache) { BlockCacheLookupContext lookup_data_block_context( TableReaderCaller::kUserMultiGet); - CachableEntry* block_entry = &(*results)[idx_in_batch]; + CachableEntry* block_entry = &results[idx_in_batch]; // MaybeReadBlockAndLoadToCache will insert into the block caches if // necessary. Since we're passing the serialized block contents, it // will avoid looking up the block cache s = MaybeReadBlockAndLoadToCache( - nullptr, options, handle, uncompression_dict, /*wait=*/true, + nullptr, options, handle, uncompression_dict, /*for_compaction=*/false, &block_entry->As(), mget_iter->get_context, &lookup_data_block_context, &serialized_block, /*async_read=*/false); @@ -301,11 +301,11 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) contents = std::move(serialized_block); } if (s.ok()) { - (*results)[idx_in_batch].SetOwnedValue(std::make_unique( + results[idx_in_batch].SetOwnedValue(std::make_unique( std::move(contents), read_amp_bytes_per_bit, ioptions.stats)); } } - (*statuses)[idx_in_batch] = s; + statuses[idx_in_batch] = s; } } @@ -355,152 +355,147 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) uint64_t prev_offset = std::numeric_limits::max(); autovector block_handles; - autovector, MultiGetContext::MAX_BATCH_SIZE> results; - autovector statuses; + std::array, MultiGetContext::MAX_BATCH_SIZE> results; + std::array statuses; MultiGetContext::Mask reused_mask = 0; char stack_buf[kMultiGetReadStackBufSize]; std::unique_ptr block_buf; { MultiGetRange data_block_range(sst_file_range, sst_file_range.begin(), sst_file_range.end()); - std::vector cache_handles; - bool wait_for_cache_results = false; - CachableEntry uncompression_dict; Status uncompression_dict_status; uncompression_dict_status.PermitUncheckedError(); bool uncompression_dict_inited = false; size_t total_len = 0; - ReadOptions ro = read_options; - ro.read_tier = kBlockCacheTier; - for (auto miter = data_block_range.begin(); - miter != data_block_range.end(); ++miter) { - const Slice& key = miter->ikey; - iiter->Seek(miter->ikey); + // GetContext for any key will do, as the stats will be aggregated + // anyway + GetContext* get_context = sst_file_range.begin()->get_context; - IndexValue v; - if (iiter->Valid()) { - v = iiter->value(); - } - if (!iiter->Valid() || - (!v.first_internal_key.empty() && !skip_filters && - UserComparatorWrapper(rep_->internal_comparator.user_comparator()) - .CompareWithoutTimestamp( - ExtractUserKey(key), - ExtractUserKey(v.first_internal_key)) < 0)) { - // The requested key falls between highest key in previous block and - // lowest key in current block. - if (!iiter->status().IsNotFound()) { - *(miter->s) = iiter->status(); + { + using BCI = BlockCacheInterface; + BCI block_cache{rep_->table_options.block_cache.get()}; + std::array + async_handles; + std::array cache_keys; + size_t cache_lookup_count = 0; + + for (auto miter = data_block_range.begin(); + miter != data_block_range.end(); ++miter) { + const Slice& key = miter->ikey; + iiter->Seek(miter->ikey); + + IndexValue v; + if (iiter->Valid()) { + v = iiter->value(); } - data_block_range.SkipKey(miter); - sst_file_range.SkipKey(miter); - continue; - } - - if (!uncompression_dict_inited && rep_->uncompression_dict_reader) { - uncompression_dict_status = - rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( - nullptr /* prefetch_buffer */, no_io, - read_options.verify_checksums, - sst_file_range.begin()->get_context, &lookup_context, - &uncompression_dict); - uncompression_dict_inited = true; - } - - if (!uncompression_dict_status.ok()) { - assert(!uncompression_dict_status.IsNotFound()); - *(miter->s) = uncompression_dict_status; - data_block_range.SkipKey(miter); - sst_file_range.SkipKey(miter); - continue; - } - - statuses.emplace_back(); - results.emplace_back(); - if (v.handle.offset() == prev_offset) { - // This key can reuse the previous block (later on). - // Mark previous as "reused" - reused_mask |= MultiGetContext::Mask{1} << (block_handles.size() - 1); - // Use null handle to indicate this one reuses same block as - // previous. - block_handles.emplace_back(BlockHandle::NullBlockHandle()); - continue; - } - // Lookup the cache for the given data block referenced by an index - // iterator value (i.e BlockHandle). If it exists in the cache, - // initialize block to the contents of the data block. - prev_offset = v.handle.offset(); - BlockHandle handle = v.handle; - BlockCacheLookupContext lookup_data_block_context( - TableReaderCaller::kUserMultiGet); - const UncompressionDict& dict = uncompression_dict.GetValue() - ? *uncompression_dict.GetValue() - : UncompressionDict::GetEmptyDict(); - Status s = RetrieveBlock( - nullptr, ro, handle, dict, &(results.back()).As(), - miter->get_context, &lookup_data_block_context, - /* for_compaction */ false, /* use_cache */ true, - /* wait_for_cache */ false, /* async_read */ false); - if (s.IsIncomplete()) { - s = Status::OK(); - } - if (s.ok() && !results.back().IsEmpty()) { - // Since we have a valid handle, check the value. If its nullptr, - // it means the cache is waiting for the final result and we're - // supposed to call WaitAll() to wait for the result. - if (results.back().GetValue() != nullptr) { - // Found it in the cache. Add NULL handle to indicate there is - // nothing to read from disk. - if (results.back().GetCacheHandle()) { - results.back().UpdateCachedValue(); + if (!iiter->Valid() || + (!v.first_internal_key.empty() && !skip_filters && + UserComparatorWrapper( + rep_->internal_comparator.user_comparator()) + .CompareWithoutTimestamp( + ExtractUserKey(key), + ExtractUserKey(v.first_internal_key)) < 0)) { + // The requested key falls between highest key in previous block and + // lowest key in current block. + if (!iiter->status().IsNotFound()) { + *(miter->s) = iiter->status(); } - block_handles.emplace_back(BlockHandle::NullBlockHandle()); - } else { - // We have to wait for the cache lookup to finish in the - // background, and then we may have to read the block from disk - // anyway - assert(results.back().GetCacheHandle()); - wait_for_cache_results = true; - block_handles.emplace_back(handle); - cache_handles.emplace_back(results.back().GetCacheHandle()); + data_block_range.SkipKey(miter); + sst_file_range.SkipKey(miter); + continue; } - } else { - block_handles.emplace_back(handle); - total_len += BlockSizeWithTrailer(handle); - } - } - if (wait_for_cache_results) { - Cache* block_cache = rep_->table_options.block_cache.get(); - block_cache->WaitAll(cache_handles); + if (!uncompression_dict_inited && rep_->uncompression_dict_reader) { + uncompression_dict_status = + rep_->uncompression_dict_reader + ->GetOrReadUncompressionDictionary( + nullptr /* prefetch_buffer */, no_io, + read_options.verify_checksums, get_context, + &lookup_context, &uncompression_dict); + uncompression_dict_inited = true; + } + + if (!uncompression_dict_status.ok()) { + assert(!uncompression_dict_status.IsNotFound()); + *(miter->s) = uncompression_dict_status; + data_block_range.SkipKey(miter); + sst_file_range.SkipKey(miter); + continue; + } + + if (v.handle.offset() == prev_offset) { + // This key can reuse the previous block (later on). + // Mark previous as "reused" + reused_mask |= MultiGetContext::Mask{1} + << (block_handles.size() - 1); + // Use null handle to indicate this one reuses same block as + // previous. + block_handles.emplace_back(BlockHandle::NullBlockHandle()); + continue; + } + prev_offset = v.handle.offset(); + block_handles.emplace_back(v.handle); + + if (block_cache) { + // Lookup the cache for the given data block referenced by an index + // iterator value (i.e BlockHandle). If it exists in the cache, + // initialize block to the contents of the data block. + // TODO? + // BlockCacheLookupContext lookup_data_block_context( + // TableReaderCaller::kUserMultiGet); + + // An async version of MaybeReadBlockAndLoadToCache / + // GetDataBlockFromCache + BCI::TypedAsyncLookupHandle& async_handle = + async_handles[cache_lookup_count]; + cache_keys[cache_lookup_count] = + GetCacheKey(rep_->base_cache_key, v.handle); + async_handle.key = cache_keys[cache_lookup_count].AsSlice(); + // NB: StartAsyncLookupFull populates async_handle.helper + async_handle.create_context = &rep_->create_context; + async_handle.priority = GetCachePriority(); + async_handle.stats = rep_->ioptions.statistics.get(); + + block_cache.StartAsyncLookupFull( + async_handle, rep_->ioptions.lowest_used_cache_tier); + ++cache_lookup_count; + // TODO: stats? + } + } + + if (block_cache) { + block_cache.get()->WaitAll(&async_handles[0], cache_lookup_count); + } + size_t lookup_idx = 0; for (size_t i = 0; i < block_handles.size(); ++i) { // If this block was a success or failure or not needed because // the corresponding key is in the same block as a prior key, skip - if (block_handles[i] == BlockHandle::NullBlockHandle() || - results[i].IsEmpty()) { + if (block_handles[i] == BlockHandle::NullBlockHandle()) { continue; } - results[i].UpdateCachedValue(); - void* val = results[i].GetValue(); - Cache::Handle* handle = results[i].GetCacheHandle(); - // GetContext for any key will do, as the stats will be aggregated - // anyway - GetContext* get_context = sst_file_range.begin()->get_context; - if (!val) { - // The async cache lookup failed - could be due to an error - // or a false positive. We need to read the data block from - // the SST file - results[i].Reset(); + if (!block_cache) { total_len += BlockSizeWithTrailer(block_handles[i]); - UpdateCacheMissMetrics(BlockType::kData, get_context); } else { - block_handles[i] = BlockHandle::NullBlockHandle(); - UpdateCacheHitMetrics(BlockType::kData, get_context, - block_cache->GetUsage(handle)); + BCI::TypedHandle* h = async_handles[lookup_idx].Result(); + if (h) { + // Cache hit + results[i].SetCachedValue(block_cache.Value(h), block_cache.get(), + h); + // Don't need to fetch + block_handles[i] = BlockHandle::NullBlockHandle(); + UpdateCacheHitMetrics(BlockType::kData, get_context, + block_cache.get()->GetUsage(h)); + } else { + // Cache miss + total_len += BlockSizeWithTrailer(block_handles[i]); + UpdateCacheMissMetrics(BlockType::kData, get_context); + } + ++lookup_idx; } } + assert(lookup_idx == cache_lookup_count); } if (total_len) { @@ -530,11 +525,10 @@ DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) } } CO_AWAIT(RetrieveMultipleBlocks) - (read_options, &data_block_range, &block_handles, &statuses, &results, - scratch, dict); - if (sst_file_range.begin()->get_context) { - ++(sst_file_range.begin() - ->get_context->get_context_stats_.num_sst_read); + (read_options, &data_block_range, &block_handles, &statuses[0], + &results[0], scratch, dict); + if (get_context) { + ++(get_context->get_context_stats_.num_sst_read); } } } diff --git a/table/block_based/cachable_entry.h b/table/block_based/cachable_entry.h index 3f7a22ba1e..3cd1bb807a 100644 --- a/table/block_based/cachable_entry.h +++ b/table/block_based/cachable_entry.h @@ -176,22 +176,6 @@ class CachableEntry { assert(!own_value_); } - void UpdateCachedValue() { - assert(cache_ != nullptr); - assert(cache_handle_ != nullptr); - - value_ = static_cast(cache_->Value(cache_handle_)); - } - - bool IsReady() { - if (!own_value_) { - assert(cache_ != nullptr); - assert(cache_handle_ != nullptr); - return cache_->IsReady(cache_handle_); - } - return true; - } - // Since this class is essentially an elaborate pointer, it's sometimes // useful to be able to upcast or downcast the base type of the pointer, // especially when interacting with typed_cache.h. diff --git a/table/block_based/filter_block_reader_common.cc b/table/block_based/filter_block_reader_common.cc index 838fb5296a..12b0eeb464 100644 --- a/table/block_based/filter_block_reader_common.cc +++ b/table/block_based/filter_block_reader_common.cc @@ -33,7 +33,7 @@ Status FilterBlockReaderCommon::ReadFilterBlock( UncompressionDict::GetEmptyDict(), filter_block, get_context, lookup_context, /* for_compaction */ false, use_cache, - /* wait_for_cache */ true, /* async_read */ false); + /* async_read */ false); return s; } diff --git a/table/block_based/index_reader_common.cc b/table/block_based/index_reader_common.cc index 46c276e6be..576d0b1503 100644 --- a/table/block_based/index_reader_common.cc +++ b/table/block_based/index_reader_common.cc @@ -29,7 +29,7 @@ Status BlockBasedTable::IndexReaderCommon::ReadIndexBlock( prefetch_buffer, read_options, rep->footer.index_handle(), UncompressionDict::GetEmptyDict(), &index_block->As(), get_context, lookup_context, /* for_compaction */ false, use_cache, - /* wait_for_cache */ true, /* async_read */ false); + /* async_read */ false); return s; } diff --git a/table/block_based/partitioned_filter_block.cc b/table/block_based/partitioned_filter_block.cc index 092446f022..3429a72567 100644 --- a/table/block_based/partitioned_filter_block.cc +++ b/table/block_based/partitioned_filter_block.cc @@ -318,7 +318,7 @@ Status PartitionedFilterBlockReader::GetFilterPartitionBlock( UncompressionDict::GetEmptyDict(), filter_block, get_context, lookup_context, /* for_compaction */ false, /* use_cache */ true, - /* wait_for_cache */ true, /* async_read */ false); + /* async_read */ false); return s; } @@ -518,9 +518,8 @@ Status PartitionedFilterBlockReader::CacheDependencies(const ReadOptions& ro, // filter blocks s = table()->MaybeReadBlockAndLoadToCache( prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(), - /* wait */ true, /* for_compaction */ false, &block, - nullptr /* get_context */, &lookup_context, nullptr /* contents */, - false); + /* for_compaction */ false, &block, nullptr /* get_context */, + &lookup_context, nullptr /* contents */, false); if (!s.ok()) { return s; } diff --git a/table/block_based/partitioned_index_reader.cc b/table/block_based/partitioned_index_reader.cc index dbe8b2bd48..3fd8a66725 100644 --- a/table/block_based/partitioned_index_reader.cc +++ b/table/block_based/partitioned_index_reader.cc @@ -192,7 +192,7 @@ Status PartitionIndexReader::CacheDependencies(const ReadOptions& ro, // filter blocks Status s = table()->MaybeReadBlockAndLoadToCache( prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(), - /*wait=*/true, /*for_compaction=*/false, &block.As(), + /*for_compaction=*/false, &block.As(), /*get_context=*/nullptr, &lookup_context, /*contents=*/nullptr, /*async_read=*/false); diff --git a/table/block_based/uncompression_dict_reader.cc b/table/block_based/uncompression_dict_reader.cc index 7b0b7c9435..ba1908720d 100644 --- a/table/block_based/uncompression_dict_reader.cc +++ b/table/block_based/uncompression_dict_reader.cc @@ -62,7 +62,7 @@ Status UncompressionDictReader::ReadUncompressionDictionary( prefetch_buffer, read_options, rep->compression_dict_handle, UncompressionDict::GetEmptyDict(), uncompression_dict, get_context, lookup_context, - /* for_compaction */ false, use_cache, /* wait_for_cache */ true, + /* for_compaction */ false, use_cache, /* async_read */ false); if (!s.ok()) { diff --git a/test_util/secondary_cache_test_util.cc b/test_util/secondary_cache_test_util.cc index 48fce1d693..1c62dc4ad7 100644 --- a/test_util/secondary_cache_test_util.cc +++ b/test_util/secondary_cache_test_util.cc @@ -5,14 +5,15 @@ #include "test_util/secondary_cache_test_util.h" -#include - #include namespace ROCKSDB_NAMESPACE { namespace secondary_cache_test_util { +namespace { +using TestItem = WithCacheType::TestItem; + size_t SizeCallback(Cache::ObjectPtr obj) { return static_cast(obj)->Size(); } @@ -50,7 +51,7 @@ Status CreateCallback(const Slice& data, Cache::CreateContext* context, // If helpers without_secondary are provided, returns helpers with secondary // support. If not provided, returns helpers without secondary support. -static auto GenerateHelpersByRole( +auto GenerateHelpersByRole( const std::array* without_secondary, bool fail) { @@ -71,9 +72,10 @@ static auto GenerateHelpersByRole( } return a; } +} // namespace -const Cache::CacheItemHelper* GetHelper(CacheEntryRole r, - bool secondary_compatible, bool fail) { +const Cache::CacheItemHelper* WithCacheType::GetHelper( + CacheEntryRole r, bool secondary_compatible, bool fail) { static const std::array without_secondary = GenerateHelpersByRole(nullptr, false); static const std::array @@ -85,7 +87,7 @@ const Cache::CacheItemHelper* GetHelper(CacheEntryRole r, : without_secondary)[static_cast(r)]; } -const Cache::CacheItemHelper* GetHelperFail(CacheEntryRole r) { +const Cache::CacheItemHelper* WithCacheType::GetHelperFail(CacheEntryRole r) { return GetHelper(r, true, true); } diff --git a/test_util/secondary_cache_test_util.h b/test_util/secondary_cache_test_util.h index 7dfed25ca5..3e6135b64d 100644 --- a/test_util/secondary_cache_test_util.h +++ b/test_util/secondary_cache_test_util.h @@ -5,52 +5,113 @@ #pragma once +#include + +#include + #include "rocksdb/advanced_cache.h" namespace ROCKSDB_NAMESPACE { - namespace secondary_cache_test_util { -class TestItem { - public: - TestItem(const char* buf, size_t size) : buf_(new char[size]), size_(size) { - memcpy(buf_.get(), buf, size); - } - ~TestItem() = default; - - char* Buf() { return buf_.get(); } - [[nodiscard]] size_t Size() const { return size_; } - std::string ToString() { return std::string(Buf(), Size()); } - - private: - std::unique_ptr buf_; - size_t size_; -}; - struct TestCreateContext : public Cache::CreateContext { void SetFailCreate(bool fail) { fail_create_ = fail; } bool fail_create_ = false; }; -size_t SizeCallback(Cache::ObjectPtr obj); -Status SaveToCallback(Cache::ObjectPtr from_obj, size_t from_offset, - size_t length, char* out); -void DeletionCallback(Cache::ObjectPtr obj, MemoryAllocator* alloc); -Status SaveToCallbackFail(Cache::ObjectPtr obj, size_t offset, size_t size, - char* out); +class WithCacheType : public TestCreateContext { + public: + WithCacheType() {} + virtual ~WithCacheType() {} -Status CreateCallback(const Slice& data, Cache::CreateContext* context, - MemoryAllocator* allocator, Cache::ObjectPtr* out_obj, - size_t* out_charge); + class TestItem { + public: + TestItem(const char* buf, size_t size) : buf_(new char[size]), size_(size) { + memcpy(buf_.get(), buf, size); + } + ~TestItem() = default; -const Cache::CacheItemHelper* GetHelper( - CacheEntryRole r = CacheEntryRole::kDataBlock, - bool secondary_compatible = true, bool fail = false); + char* Buf() { return buf_.get(); } + [[nodiscard]] size_t Size() const { return size_; } + std::string ToString() { return std::string(Buf(), Size()); } -const Cache::CacheItemHelper* GetHelperFail( - CacheEntryRole r = CacheEntryRole::kDataBlock); + private: + std::unique_ptr buf_; + size_t size_; + }; + + static constexpr auto kLRU = "lru"; + static constexpr auto kHyperClock = "hyper_clock"; + + // For options other than capacity + size_t estimated_value_size_ = 1; + + virtual const std::string& Type() = 0; + + std::shared_ptr NewCache( + size_t capacity, + std::function modify_opts_fn = {}) { + const auto& type = Type(); + if (type == kLRU) { + LRUCacheOptions lru_opts; + lru_opts.capacity = capacity; + if (modify_opts_fn) { + modify_opts_fn(lru_opts); + } + return NewLRUCache(lru_opts); + } + if (type == kHyperClock) { + HyperClockCacheOptions hc_opts{capacity, estimated_value_size_}; + if (modify_opts_fn) { + modify_opts_fn(hc_opts); + } + return hc_opts.MakeSharedCache(); + } + assert(false); + return nullptr; + } + + std::shared_ptr NewCache( + size_t capacity, int num_shard_bits, bool strict_capacity_limit, + CacheMetadataChargePolicy charge_policy = kDontChargeCacheMetadata) { + return NewCache(capacity, [=](ShardedCacheOptions& opts) { + opts.num_shard_bits = num_shard_bits; + opts.strict_capacity_limit = strict_capacity_limit; + opts.metadata_charge_policy = charge_policy; + }); + } + + std::shared_ptr NewCache( + size_t capacity, int num_shard_bits, bool strict_capacity_limit, + std::shared_ptr secondary_cache) { + return NewCache(capacity, [=](ShardedCacheOptions& opts) { + opts.num_shard_bits = num_shard_bits; + opts.strict_capacity_limit = strict_capacity_limit; + opts.metadata_charge_policy = kDontChargeCacheMetadata; + opts.secondary_cache = secondary_cache; + }); + } + + static const Cache::CacheItemHelper* GetHelper( + CacheEntryRole r = CacheEntryRole::kDataBlock, + bool secondary_compatible = true, bool fail = false); + + static const Cache::CacheItemHelper* GetHelperFail( + CacheEntryRole r = CacheEntryRole::kDataBlock); +}; + +class WithCacheTypeParam : public WithCacheType, + public testing::WithParamInterface { + const std::string& Type() override { return GetParam(); } +}; + +constexpr auto kLRU = WithCacheType::kLRU; +constexpr auto kHyperClock = WithCacheType::kHyperClock; + +inline auto GetTestingCacheTypes() { + return testing::Values(std::string(kLRU), std::string(kHyperClock)); +} } // namespace secondary_cache_test_util - } // namespace ROCKSDB_NAMESPACE diff --git a/tools/db_crashtest.py b/tools/db_crashtest.py index fd1213b544..729f221a2a 100644 --- a/tools/db_crashtest.py +++ b/tools/db_crashtest.py @@ -603,9 +603,6 @@ def finalize_and_sanitize(src_params): dest_params["enable_compaction_filter"] = 0 dest_params["sync"] = 0 dest_params["write_fault_one_in"] = 0 - if dest_params["secondary_cache_uri"] != "": - # Currently the only cache type compatible with a secondary cache is LRUCache - dest_params["cache_type"] = "lru_cache" # Remove the following once write-prepared/write-unprepared with/without # unordered write supports timestamped snapshots if dest_params.get("create_timestamped_snapshot_one_in", 0) > 0: diff --git a/utilities/simulator_cache/sim_cache.cc b/utilities/simulator_cache/sim_cache.cc index 21c4780468..80e535f3b4 100644 --- a/utilities/simulator_cache/sim_cache.cc +++ b/utilities/simulator_cache/sim_cache.cc @@ -194,13 +194,20 @@ class SimCacheImpl : public SimCache { Handle* Lookup(const Slice& key, const CacheItemHelper* helper, CreateContext* create_context, - Priority priority = Priority::LOW, bool wait = true, + Priority priority = Priority::LOW, Statistics* stats = nullptr) override { HandleLookup(key, stats); if (!target_) { return nullptr; } - return target_->Lookup(key, helper, create_context, priority, wait, stats); + return target_->Lookup(key, helper, create_context, priority, stats); + } + + void StartAsyncLookup(AsyncLookupHandle& async_handle) override { + HandleLookup(async_handle.key, async_handle.stats); + if (target_) { + target_->StartAsyncLookup(async_handle); + } } bool Ref(Handle* handle) override { return target_->Ref(handle); }