summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/cache
diff options
context:
space:
mode:
Diffstat (limited to '')
-rw-r--r--src/rocksdb/cache/cache.cc129
-rw-r--r--src/rocksdb/cache/cache_bench.cc20
-rw-r--r--src/rocksdb/cache/cache_bench_tool.cc973
-rw-r--r--src/rocksdb/cache/cache_entry_roles.cc134
-rw-r--r--src/rocksdb/cache/cache_entry_roles.h103
-rw-r--r--src/rocksdb/cache/cache_entry_stats.h183
-rw-r--r--src/rocksdb/cache/cache_helpers.h145
-rw-r--r--src/rocksdb/cache/cache_key.cc364
-rw-r--r--src/rocksdb/cache/cache_key.h143
-rw-r--r--src/rocksdb/cache/cache_reservation_manager.cc185
-rw-r--r--src/rocksdb/cache/cache_reservation_manager.h316
-rw-r--r--src/rocksdb/cache/cache_reservation_manager_test.cc469
-rw-r--r--src/rocksdb/cache/cache_test.cc1037
-rw-r--r--src/rocksdb/cache/charged_cache.cc117
-rw-r--r--src/rocksdb/cache/charged_cache.h121
-rw-r--r--src/rocksdb/cache/clock_cache.cc1404
-rw-r--r--src/rocksdb/cache/clock_cache.h701
-rw-r--r--src/rocksdb/cache/compressed_secondary_cache.cc325
-rw-r--r--src/rocksdb/cache/compressed_secondary_cache.h139
-rw-r--r--src/rocksdb/cache/compressed_secondary_cache_test.cc1005
-rw-r--r--src/rocksdb/cache/lru_cache.cc921
-rw-r--r--src/rocksdb/cache/lru_cache.h546
-rw-r--r--src/rocksdb/cache/lru_cache_test.cc2624
-rw-r--r--src/rocksdb/cache/secondary_cache.cc32
-rw-r--r--src/rocksdb/cache/sharded_cache.cc100
-rw-r--r--src/rocksdb/cache/sharded_cache.h322
26 files changed, 12558 insertions, 0 deletions
diff --git a/src/rocksdb/cache/cache.cc b/src/rocksdb/cache/cache.cc
new file mode 100644
index 000000000..7d23fb757
--- /dev/null
+++ b/src/rocksdb/cache/cache.cc
@@ -0,0 +1,129 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "rocksdb/cache.h"
+
+#include "cache/lru_cache.h"
+#include "rocksdb/secondary_cache.h"
+#include "rocksdb/utilities/customizable_util.h"
+#include "rocksdb/utilities/options_type.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+#ifndef ROCKSDB_LITE
+static std::unordered_map<std::string, OptionTypeInfo>
+ lru_cache_options_type_info = {
+ {"capacity",
+ {offsetof(struct LRUCacheOptions, capacity), OptionType::kSizeT,
+ OptionVerificationType::kNormal, OptionTypeFlags::kMutable}},
+ {"num_shard_bits",
+ {offsetof(struct LRUCacheOptions, num_shard_bits), OptionType::kInt,
+ OptionVerificationType::kNormal, OptionTypeFlags::kMutable}},
+ {"strict_capacity_limit",
+ {offsetof(struct LRUCacheOptions, strict_capacity_limit),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"high_pri_pool_ratio",
+ {offsetof(struct LRUCacheOptions, high_pri_pool_ratio),
+ OptionType::kDouble, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"low_pri_pool_ratio",
+ {offsetof(struct LRUCacheOptions, low_pri_pool_ratio),
+ OptionType::kDouble, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+};
+
+static std::unordered_map<std::string, OptionTypeInfo>
+ comp_sec_cache_options_type_info = {
+ {"capacity",
+ {offsetof(struct CompressedSecondaryCacheOptions, capacity),
+ OptionType::kSizeT, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"num_shard_bits",
+ {offsetof(struct CompressedSecondaryCacheOptions, num_shard_bits),
+ OptionType::kInt, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"compression_type",
+ {offsetof(struct CompressedSecondaryCacheOptions, compression_type),
+ OptionType::kCompressionType, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"compress_format_version",
+ {offsetof(struct CompressedSecondaryCacheOptions,
+ compress_format_version),
+ OptionType::kUInt32T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"enable_custom_split_merge",
+ {offsetof(struct CompressedSecondaryCacheOptions,
+ enable_custom_split_merge),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+};
+#endif // ROCKSDB_LITE
+
+Status SecondaryCache::CreateFromString(
+ const ConfigOptions& config_options, const std::string& value,
+ std::shared_ptr<SecondaryCache>* result) {
+ if (value.find("compressed_secondary_cache://") == 0) {
+ std::string args = value;
+ args.erase(0, std::strlen("compressed_secondary_cache://"));
+ Status status;
+ std::shared_ptr<SecondaryCache> sec_cache;
+
+#ifndef ROCKSDB_LITE
+ CompressedSecondaryCacheOptions sec_cache_opts;
+ status = OptionTypeInfo::ParseStruct(config_options, "",
+ &comp_sec_cache_options_type_info, "",
+ args, &sec_cache_opts);
+ if (status.ok()) {
+ sec_cache = NewCompressedSecondaryCache(sec_cache_opts);
+ }
+
+#else
+ (void)config_options;
+ status = Status::NotSupported(
+ "Cannot load compressed secondary cache in LITE mode ", args);
+#endif //! ROCKSDB_LITE
+
+ if (status.ok()) {
+ result->swap(sec_cache);
+ }
+ return status;
+ } else {
+ return LoadSharedObject<SecondaryCache>(config_options, value, nullptr,
+ result);
+ }
+}
+
+Status Cache::CreateFromString(const ConfigOptions& config_options,
+ const std::string& value,
+ std::shared_ptr<Cache>* result) {
+ Status status;
+ std::shared_ptr<Cache> cache;
+ if (value.find('=') == std::string::npos) {
+ cache = NewLRUCache(ParseSizeT(value));
+ } else {
+#ifndef ROCKSDB_LITE
+ LRUCacheOptions cache_opts;
+ status = OptionTypeInfo::ParseStruct(config_options, "",
+ &lru_cache_options_type_info, "",
+ value, &cache_opts);
+ if (status.ok()) {
+ cache = NewLRUCache(cache_opts);
+ }
+#else
+ (void)config_options;
+ status = Status::NotSupported("Cannot load cache in LITE mode ", value);
+#endif //! ROCKSDB_LITE
+ }
+ if (status.ok()) {
+ result->swap(cache);
+ }
+ return status;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_bench.cc b/src/rocksdb/cache/cache_bench.cc
new file mode 100644
index 000000000..f836939a3
--- /dev/null
+++ b/src/rocksdb/cache/cache_bench.cc
@@ -0,0 +1,20 @@
+// Copyright (c) 2013-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#ifndef GFLAGS
+#include <cstdio>
+int main() {
+ fprintf(stderr, "Please install gflags to run rocksdb tools\n");
+ return 1;
+}
+#else
+#include "rocksdb/cache_bench_tool.h"
+int main(int argc, char** argv) {
+ return ROCKSDB_NAMESPACE::cache_bench_tool(argc, argv);
+}
+#endif // GFLAGS
diff --git a/src/rocksdb/cache/cache_bench_tool.cc b/src/rocksdb/cache/cache_bench_tool.cc
new file mode 100644
index 000000000..73360f414
--- /dev/null
+++ b/src/rocksdb/cache/cache_bench_tool.cc
@@ -0,0 +1,973 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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_key.h"
+#ifdef GFLAGS
+#include <cinttypes>
+#include <cstddef>
+#include <cstdio>
+#include <limits>
+#include <memory>
+#include <set>
+#include <sstream>
+
+#include "db/db_impl/db_impl.h"
+#include "monitoring/histogram.h"
+#include "port/port.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/db.h"
+#include "rocksdb/env.h"
+#include "rocksdb/secondary_cache.h"
+#include "rocksdb/system_clock.h"
+#include "rocksdb/table_properties.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/cachable_entry.h"
+#include "util/coding.h"
+#include "util/distributed_mutex.h"
+#include "util/gflags_compat.h"
+#include "util/hash.h"
+#include "util/mutexlock.h"
+#include "util/random.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+
+using GFLAGS_NAMESPACE::ParseCommandLineFlags;
+
+static constexpr uint32_t KiB = uint32_t{1} << 10;
+static constexpr uint32_t MiB = KiB << 10;
+static constexpr uint64_t GiB = MiB << 10;
+
+DEFINE_uint32(threads, 16, "Number of concurrent threads to run.");
+DEFINE_uint64(cache_size, 1 * GiB,
+ "Number of bytes to use as a cache of uncompressed data.");
+DEFINE_uint32(num_shard_bits, 6, "shard_bits.");
+
+DEFINE_double(resident_ratio, 0.25,
+ "Ratio of keys fitting in cache to keyspace.");
+DEFINE_uint64(ops_per_thread, 2000000U, "Number of operations per thread.");
+DEFINE_uint32(value_bytes, 8 * KiB, "Size of each value added.");
+
+DEFINE_uint32(skew, 5, "Degree of skew in key selection");
+DEFINE_bool(populate_cache, true, "Populate cache before operations");
+
+DEFINE_uint32(lookup_insert_percent, 87,
+ "Ratio of lookup (+ insert on not found) to total workload "
+ "(expressed as a percentage)");
+DEFINE_uint32(insert_percent, 2,
+ "Ratio of insert to total workload (expressed as a percentage)");
+DEFINE_uint32(lookup_percent, 10,
+ "Ratio of lookup to total workload (expressed as a percentage)");
+DEFINE_uint32(erase_percent, 1,
+ "Ratio of erase to total workload (expressed as a percentage)");
+DEFINE_bool(gather_stats, false,
+ "Whether to periodically simulate gathering block cache stats, "
+ "using one more thread.");
+DEFINE_uint32(
+ gather_stats_sleep_ms, 1000,
+ "How many milliseconds to sleep between each gathering of stats.");
+
+DEFINE_uint32(gather_stats_entries_per_lock, 256,
+ "For Cache::ApplyToAllEntries");
+DEFINE_bool(skewed, false, "If true, skew the key access distribution");
+
+DEFINE_bool(lean, false,
+ "If true, no additional computation is performed besides cache "
+ "operations.");
+
+#ifndef ROCKSDB_LITE
+DEFINE_string(secondary_cache_uri, "",
+ "Full URI for creating a custom secondary cache object");
+static class std::shared_ptr<ROCKSDB_NAMESPACE::SecondaryCache> secondary_cache;
+#endif // ROCKSDB_LITE
+
+DEFINE_string(cache_type, "lru_cache", "Type of block cache.");
+
+// ## BEGIN stress_cache_key sub-tool options ##
+// See class StressCacheKey below.
+DEFINE_bool(stress_cache_key, false,
+ "If true, run cache key stress test instead");
+DEFINE_uint32(
+ sck_files_per_day, 2500000,
+ "(-stress_cache_key) Simulated files generated per simulated day");
+// NOTE: Giving each run a specified lifetime, rather than e.g. "until
+// first collision" ensures equal skew from start-up, when collisions are
+// less likely.
+DEFINE_uint32(sck_days_per_run, 90,
+ "(-stress_cache_key) Number of days to simulate in each run");
+// NOTE: The number of observed collisions directly affects the relative
+// accuracy of the predicted probabilities. 15 observations should be well
+// within factor-of-2 accuracy.
+DEFINE_uint32(
+ sck_min_collision, 15,
+ "(-stress_cache_key) Keep running until this many collisions seen");
+// sck_file_size_mb can be thought of as average file size. The simulation is
+// not precise enough to care about the distribution of file sizes; other
+// simulations (https://github.com/pdillinger/unique_id/tree/main/monte_carlo)
+// indicate the distribution only makes a small difference (e.g. < 2x factor)
+DEFINE_uint32(
+ sck_file_size_mb, 32,
+ "(-stress_cache_key) Simulated file size in MiB, for accounting purposes");
+DEFINE_uint32(sck_reopen_nfiles, 100,
+ "(-stress_cache_key) Simulate DB re-open average every n files");
+DEFINE_uint32(sck_newdb_nreopen, 1000,
+ "(-stress_cache_key) Simulate new DB average every n re-opens");
+DEFINE_uint32(sck_restarts_per_day, 24,
+ "(-stress_cache_key) Average simulated process restarts per day "
+ "(across DBs)");
+DEFINE_uint32(
+ sck_db_count, 100,
+ "(-stress_cache_key) Parallel DBs in simulation sharing a block cache");
+DEFINE_uint32(
+ sck_table_bits, 20,
+ "(-stress_cache_key) Log2 number of tracked (live) files (across DBs)");
+// sck_keep_bits being well below full 128 bits amplifies the collision
+// probability so that the true probability can be estimated through observed
+// collisions. (More explanation below.)
+DEFINE_uint32(
+ sck_keep_bits, 50,
+ "(-stress_cache_key) Number of bits to keep from each cache key (<= 64)");
+// sck_randomize is used to validate whether cache key is performing "better
+// than random." Even with this setting, file offsets are not randomized.
+DEFINE_bool(sck_randomize, false,
+ "(-stress_cache_key) Randomize (hash) cache key");
+// See https://github.com/facebook/rocksdb/pull/9058
+DEFINE_bool(sck_footer_unique_id, false,
+ "(-stress_cache_key) Simulate using proposed footer unique id");
+// ## END stress_cache_key sub-tool options ##
+
+namespace ROCKSDB_NAMESPACE {
+
+class CacheBench;
+namespace {
+// State shared by all concurrent executions of the same benchmark.
+class SharedState {
+ public:
+ explicit SharedState(CacheBench* cache_bench)
+ : cv_(&mu_),
+ num_initialized_(0),
+ start_(false),
+ num_done_(0),
+ cache_bench_(cache_bench) {}
+
+ ~SharedState() {}
+
+ port::Mutex* GetMutex() { return &mu_; }
+
+ port::CondVar* GetCondVar() { return &cv_; }
+
+ CacheBench* GetCacheBench() const { return cache_bench_; }
+
+ void IncInitialized() { num_initialized_++; }
+
+ void IncDone() { num_done_++; }
+
+ bool AllInitialized() const { return num_initialized_ >= FLAGS_threads; }
+
+ bool AllDone() const { return num_done_ >= FLAGS_threads; }
+
+ void SetStart() { start_ = true; }
+
+ bool Started() const { return start_; }
+
+ private:
+ port::Mutex mu_;
+ port::CondVar cv_;
+
+ uint64_t num_initialized_;
+ bool start_;
+ uint64_t num_done_;
+
+ CacheBench* cache_bench_;
+};
+
+// Per-thread state for concurrent executions of the same benchmark.
+struct ThreadState {
+ uint32_t tid;
+ Random64 rnd;
+ SharedState* shared;
+ HistogramImpl latency_ns_hist;
+ uint64_t duration_us = 0;
+
+ ThreadState(uint32_t index, SharedState* _shared)
+ : tid(index), rnd(1000 + index), shared(_shared) {}
+};
+
+struct KeyGen {
+ char key_data[27];
+
+ Slice GetRand(Random64& rnd, uint64_t max_key, int max_log) {
+ uint64_t key = 0;
+ if (!FLAGS_skewed) {
+ uint64_t raw = rnd.Next();
+ // Skew according to setting
+ for (uint32_t i = 0; i < FLAGS_skew; ++i) {
+ raw = std::min(raw, rnd.Next());
+ }
+ key = FastRange64(raw, max_key);
+ } else {
+ key = rnd.Skewed(max_log);
+ if (key > max_key) {
+ key -= max_key;
+ }
+ }
+ // Variable size and alignment
+ size_t off = key % 8;
+ key_data[0] = char{42};
+ EncodeFixed64(key_data + 1, key);
+ key_data[9] = char{11};
+ EncodeFixed64(key_data + 10, key);
+ key_data[18] = char{4};
+ EncodeFixed64(key_data + 19, key);
+ assert(27 >= kCacheKeySize);
+ return Slice(&key_data[off], kCacheKeySize);
+ }
+};
+
+char* createValue(Random64& rnd) {
+ char* rv = new char[FLAGS_value_bytes];
+ // Fill with some filler data, and take some CPU time
+ for (uint32_t i = 0; i < FLAGS_value_bytes; i += 8) {
+ EncodeFixed64(rv + i, rnd.Next());
+ }
+ return rv;
+}
+
+// Callbacks for secondary cache
+size_t SizeFn(void* /*obj*/) { return FLAGS_value_bytes; }
+
+Status SaveToFn(void* obj, size_t /*offset*/, size_t size, void* out) {
+ memcpy(out, obj, size);
+ return Status::OK();
+}
+
+// Different deleters to simulate using deleter to gather
+// stats on the code origin and kind of cache entries.
+void deleter1(const Slice& /*key*/, void* value) {
+ delete[] static_cast<char*>(value);
+}
+void deleter2(const Slice& /*key*/, void* value) {
+ delete[] static_cast<char*>(value);
+}
+void deleter3(const Slice& /*key*/, void* value) {
+ delete[] static_cast<char*>(value);
+}
+
+Cache::CacheItemHelper helper1(SizeFn, SaveToFn, deleter1);
+Cache::CacheItemHelper helper2(SizeFn, SaveToFn, deleter2);
+Cache::CacheItemHelper helper3(SizeFn, SaveToFn, deleter3);
+} // namespace
+
+class CacheBench {
+ static constexpr uint64_t kHundredthUint64 =
+ std::numeric_limits<uint64_t>::max() / 100U;
+
+ public:
+ CacheBench()
+ : max_key_(static_cast<uint64_t>(FLAGS_cache_size / FLAGS_resident_ratio /
+ FLAGS_value_bytes)),
+ lookup_insert_threshold_(kHundredthUint64 *
+ FLAGS_lookup_insert_percent),
+ insert_threshold_(lookup_insert_threshold_ +
+ kHundredthUint64 * FLAGS_insert_percent),
+ lookup_threshold_(insert_threshold_ +
+ kHundredthUint64 * FLAGS_lookup_percent),
+ erase_threshold_(lookup_threshold_ +
+ kHundredthUint64 * FLAGS_erase_percent),
+ skewed_(FLAGS_skewed) {
+ if (erase_threshold_ != 100U * kHundredthUint64) {
+ fprintf(stderr, "Percentages must add to 100.\n");
+ exit(1);
+ }
+
+ max_log_ = 0;
+ if (skewed_) {
+ uint64_t max_key = max_key_;
+ while (max_key >>= 1) max_log_++;
+ if (max_key > (static_cast<uint64_t>(1) << max_log_)) max_log_++;
+ }
+
+ 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") {
+ cache_ = HyperClockCacheOptions(FLAGS_cache_size, FLAGS_value_bytes,
+ FLAGS_num_shard_bits)
+ .MakeSharedCache();
+ } else if (FLAGS_cache_type == "lru_cache") {
+ LRUCacheOptions opts(FLAGS_cache_size, FLAGS_num_shard_bits,
+ false /* strict_capacity_limit */,
+ 0.5 /* high_pri_pool_ratio */);
+#ifndef ROCKSDB_LITE
+ if (!FLAGS_secondary_cache_uri.empty()) {
+ Status s = SecondaryCache::CreateFromString(
+ ConfigOptions(), 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);
+ }
+ opts.secondary_cache = secondary_cache;
+ }
+#endif // ROCKSDB_LITE
+
+ cache_ = NewLRUCache(opts);
+ } else {
+ fprintf(stderr, "Cache type not supported.");
+ exit(1);
+ }
+ }
+
+ ~CacheBench() {}
+
+ void PopulateCache() {
+ Random64 rnd(1);
+ KeyGen keygen;
+ for (uint64_t i = 0; i < 2 * FLAGS_cache_size; i += FLAGS_value_bytes) {
+ Status s = cache_->Insert(keygen.GetRand(rnd, max_key_, max_log_),
+ createValue(rnd), &helper1, FLAGS_value_bytes);
+ assert(s.ok());
+ }
+ }
+
+ bool Run() {
+ const auto clock = SystemClock::Default().get();
+
+ PrintEnv();
+ SharedState shared(this);
+ std::vector<std::unique_ptr<ThreadState> > threads(FLAGS_threads);
+ for (uint32_t i = 0; i < FLAGS_threads; i++) {
+ threads[i].reset(new ThreadState(i, &shared));
+ std::thread(ThreadBody, threads[i].get()).detach();
+ }
+
+ HistogramImpl stats_hist;
+ std::string stats_report;
+ std::thread stats_thread(StatsBody, &shared, &stats_hist, &stats_report);
+
+ uint64_t start_time;
+ {
+ MutexLock l(shared.GetMutex());
+ while (!shared.AllInitialized()) {
+ shared.GetCondVar()->Wait();
+ }
+ // Record start time
+ start_time = clock->NowMicros();
+
+ // Start all threads
+ shared.SetStart();
+ shared.GetCondVar()->SignalAll();
+
+ // Wait threads to complete
+ while (!shared.AllDone()) {
+ shared.GetCondVar()->Wait();
+ }
+ }
+
+ // Stats gathering is considered background work. This time measurement
+ // is for foreground work, and not really ideal for that. See below.
+ uint64_t end_time = clock->NowMicros();
+ stats_thread.join();
+
+ // Wall clock time - includes idle time if threads
+ // finish at different times (not ideal).
+ double elapsed_secs = static_cast<double>(end_time - start_time) * 1e-6;
+ uint32_t ops_per_sec = static_cast<uint32_t>(
+ 1.0 * FLAGS_threads * FLAGS_ops_per_thread / elapsed_secs);
+ printf("Complete in %.3f s; Rough parallel ops/sec = %u\n", elapsed_secs,
+ ops_per_sec);
+
+ // Total time in each thread (more accurate throughput measure)
+ elapsed_secs = 0;
+ for (uint32_t i = 0; i < FLAGS_threads; i++) {
+ elapsed_secs += threads[i]->duration_us * 1e-6;
+ }
+ ops_per_sec = static_cast<uint32_t>(1.0 * FLAGS_threads *
+ FLAGS_ops_per_thread / elapsed_secs);
+ printf("Thread ops/sec = %u\n", ops_per_sec);
+
+ printf("\nOperation latency (ns):\n");
+ HistogramImpl combined;
+ for (uint32_t i = 0; i < FLAGS_threads; i++) {
+ combined.Merge(threads[i]->latency_ns_hist);
+ }
+ printf("%s", combined.ToString().c_str());
+
+ if (FLAGS_gather_stats) {
+ printf("\nGather stats latency (us):\n");
+ printf("%s", stats_hist.ToString().c_str());
+ }
+
+ printf("\n%s", stats_report.c_str());
+
+ return true;
+ }
+
+ private:
+ std::shared_ptr<Cache> cache_;
+ const uint64_t max_key_;
+ // Cumulative thresholds in the space of a random uint64_t
+ const uint64_t lookup_insert_threshold_;
+ const uint64_t insert_threshold_;
+ const uint64_t lookup_threshold_;
+ const uint64_t erase_threshold_;
+ const bool skewed_;
+ int max_log_;
+
+ // A benchmark version of gathering stats on an active block cache by
+ // iterating over it. The primary purpose is to measure the impact of
+ // gathering stats with ApplyToAllEntries on throughput- and
+ // latency-sensitive Cache users. Performance of stats gathering is
+ // also reported. The last set of gathered stats is also reported, for
+ // manual sanity checking for logical errors or other unexpected
+ // behavior of cache_bench or the underlying Cache.
+ static void StatsBody(SharedState* shared, HistogramImpl* stats_hist,
+ std::string* stats_report) {
+ if (!FLAGS_gather_stats) {
+ return;
+ }
+ const auto clock = SystemClock::Default().get();
+ uint64_t total_key_size = 0;
+ uint64_t total_charge = 0;
+ uint64_t total_entry_count = 0;
+ uint64_t table_occupancy = 0;
+ uint64_t table_size = 0;
+ std::set<Cache::DeleterFn> deleters;
+ StopWatchNano timer(clock);
+
+ for (;;) {
+ uint64_t time;
+ time = clock->NowMicros();
+ uint64_t deadline = time + uint64_t{FLAGS_gather_stats_sleep_ms} * 1000;
+
+ {
+ MutexLock l(shared->GetMutex());
+ for (;;) {
+ if (shared->AllDone()) {
+ std::ostringstream ostr;
+ ostr << "Most recent cache entry stats:\n"
+ << "Number of entries: " << total_entry_count << "\n"
+ << "Table occupancy: " << table_occupancy << " / "
+ << table_size << " = "
+ << (100.0 * table_occupancy / table_size) << "%\n"
+ << "Total charge: " << BytesToHumanString(total_charge) << "\n"
+ << "Average key size: "
+ << (1.0 * total_key_size / total_entry_count) << "\n"
+ << "Average charge: "
+ << BytesToHumanString(static_cast<uint64_t>(
+ 1.0 * total_charge / total_entry_count))
+ << "\n"
+ << "Unique deleters: " << deleters.size() << "\n";
+ *stats_report = ostr.str();
+ return;
+ }
+ if (clock->NowMicros() >= deadline) {
+ break;
+ }
+ uint64_t diff = deadline - std::min(clock->NowMicros(), deadline);
+ shared->GetCondVar()->TimedWait(diff + 1);
+ }
+ }
+
+ // Now gather stats, outside of mutex
+ total_key_size = 0;
+ total_charge = 0;
+ total_entry_count = 0;
+ deleters.clear();
+ auto fn = [&](const Slice& key, void* /*value*/, size_t charge,
+ Cache::DeleterFn deleter) {
+ total_key_size += key.size();
+ total_charge += charge;
+ ++total_entry_count;
+ // Something slightly more expensive as in (future) stats by category
+ deleters.insert(deleter);
+ };
+ timer.Start();
+ Cache::ApplyToAllEntriesOptions opts;
+ opts.average_entries_per_lock = FLAGS_gather_stats_entries_per_lock;
+ shared->GetCacheBench()->cache_->ApplyToAllEntries(fn, opts);
+ table_occupancy = shared->GetCacheBench()->cache_->GetOccupancyCount();
+ table_size = shared->GetCacheBench()->cache_->GetTableAddressCount();
+ stats_hist->Add(timer.ElapsedNanos() / 1000);
+ }
+ }
+
+ static void ThreadBody(ThreadState* thread) {
+ SharedState* shared = thread->shared;
+
+ {
+ MutexLock l(shared->GetMutex());
+ shared->IncInitialized();
+ if (shared->AllInitialized()) {
+ shared->GetCondVar()->SignalAll();
+ }
+ while (!shared->Started()) {
+ shared->GetCondVar()->Wait();
+ }
+ }
+ thread->shared->GetCacheBench()->OperateCache(thread);
+
+ {
+ MutexLock l(shared->GetMutex());
+ shared->IncDone();
+ if (shared->AllDone()) {
+ shared->GetCondVar()->SignalAll();
+ }
+ }
+ }
+
+ void OperateCache(ThreadState* thread) {
+ // To use looked-up values
+ uint64_t result = 0;
+ // To hold handles for a non-trivial amount of time
+ Cache::Handle* handle = nullptr;
+ KeyGen gen;
+ const auto clock = SystemClock::Default().get();
+ uint64_t start_time = clock->NowMicros();
+ StopWatchNano timer(clock);
+
+ for (uint64_t i = 0; i < FLAGS_ops_per_thread; i++) {
+ Slice key = gen.GetRand(thread->rnd, max_key_, max_log_);
+ uint64_t random_op = thread->rnd.Next();
+ Cache::CreateCallback create_cb = [](const void* buf, size_t size,
+ void** out_obj,
+ size_t* charge) -> Status {
+ *out_obj = reinterpret_cast<void*>(new char[size]);
+ memcpy(*out_obj, buf, size);
+ *charge = size;
+ return Status::OK();
+ };
+
+ timer.Start();
+
+ if (random_op < lookup_insert_threshold_) {
+ if (handle) {
+ cache_->Release(handle);
+ handle = nullptr;
+ }
+ // do lookup
+ handle = cache_->Lookup(key, &helper2, create_cb, Cache::Priority::LOW,
+ true);
+ if (handle) {
+ if (!FLAGS_lean) {
+ // do something with the data
+ result += NPHash64(static_cast<char*>(cache_->Value(handle)),
+ FLAGS_value_bytes);
+ }
+ } else {
+ // do insert
+ Status s = cache_->Insert(key, createValue(thread->rnd), &helper2,
+ FLAGS_value_bytes, &handle);
+ assert(s.ok());
+ }
+ } else if (random_op < insert_threshold_) {
+ if (handle) {
+ cache_->Release(handle);
+ handle = nullptr;
+ }
+ // do insert
+ Status s = cache_->Insert(key, createValue(thread->rnd), &helper3,
+ FLAGS_value_bytes, &handle);
+ assert(s.ok());
+ } else if (random_op < lookup_threshold_) {
+ if (handle) {
+ cache_->Release(handle);
+ handle = nullptr;
+ }
+ // do lookup
+ handle = cache_->Lookup(key, &helper2, create_cb, Cache::Priority::LOW,
+ true);
+ if (handle) {
+ if (!FLAGS_lean) {
+ // do something with the data
+ result += NPHash64(static_cast<char*>(cache_->Value(handle)),
+ FLAGS_value_bytes);
+ }
+ }
+ } else if (random_op < erase_threshold_) {
+ // do erase
+ cache_->Erase(key);
+ } else {
+ // Should be extremely unlikely (noop)
+ assert(random_op >= kHundredthUint64 * 100U);
+ }
+ thread->latency_ns_hist.Add(timer.ElapsedNanos());
+ }
+ if (handle) {
+ cache_->Release(handle);
+ handle = nullptr;
+ }
+ // Ensure computations on `result` are not optimized away.
+ if (result == 1) {
+ printf("You are extremely unlucky(2). Try again.\n");
+ exit(1);
+ }
+ thread->duration_us = clock->NowMicros() - start_time;
+ }
+
+ void PrintEnv() const {
+#if defined(__GNUC__) && !defined(__OPTIMIZE__)
+ printf(
+ "WARNING: Optimization is disabled: benchmarks unnecessarily slow\n");
+#endif
+#ifndef NDEBUG
+ printf("WARNING: Assertions are enabled; benchmarks unnecessarily slow\n");
+#endif
+ printf("RocksDB version : %d.%d\n", kMajorVersion, kMinorVersion);
+ printf("DMutex impl name : %s\n", DMutex::kName());
+ printf("Number of threads : %u\n", FLAGS_threads);
+ printf("Ops per thread : %" PRIu64 "\n", FLAGS_ops_per_thread);
+ printf("Cache size : %s\n",
+ BytesToHumanString(FLAGS_cache_size).c_str());
+ printf("Num shard bits : %u\n", FLAGS_num_shard_bits);
+ printf("Max key : %" PRIu64 "\n", max_key_);
+ printf("Resident ratio : %g\n", FLAGS_resident_ratio);
+ printf("Skew degree : %u\n", FLAGS_skew);
+ printf("Populate cache : %d\n", int{FLAGS_populate_cache});
+ printf("Lookup+Insert pct : %u%%\n", FLAGS_lookup_insert_percent);
+ printf("Insert percentage : %u%%\n", FLAGS_insert_percent);
+ printf("Lookup percentage : %u%%\n", FLAGS_lookup_percent);
+ printf("Erase percentage : %u%%\n", FLAGS_erase_percent);
+ std::ostringstream stats;
+ if (FLAGS_gather_stats) {
+ stats << "enabled (" << FLAGS_gather_stats_sleep_ms << "ms, "
+ << FLAGS_gather_stats_entries_per_lock << "/lock)";
+ } else {
+ stats << "disabled";
+ }
+ printf("Gather stats : %s\n", stats.str().c_str());
+ printf("----------------------------\n");
+ }
+};
+
+// cache_bench -stress_cache_key is an independent embedded tool for
+// estimating the probability of CacheKey collisions through simulation.
+// At a high level, it simulates generating SST files over many months,
+// keeping them in the DB and/or cache for some lifetime while staying
+// under resource caps, and checking for any cache key collisions that
+// arise among the set of live files. For efficient simulation, we make
+// some simplifying "pessimistic" assumptions (that only increase the
+// chance of the simulation reporting a collision relative to the chance
+// of collision in practice):
+// * Every generated file has a cache entry for every byte offset in the
+// file (contiguous range of cache keys)
+// * All of every file is cached for its entire lifetime. (Here "lifetime"
+// is technically the union of DB and Cache lifetime, though we only
+// model a generous DB lifetime, where space usage is always maximized.
+// In a effective Cache, lifetime in cache can only substantially exceed
+// lifetime in DB if there is little cache activity; cache activity is
+// required to hit cache key collisions.)
+//
+// It would be possible to track an exact set of cache key ranges for the
+// set of live files, but we would have no hope of observing collisions
+// (overlap in live files) in our simulation. We need to employ some way
+// of amplifying collision probability that allows us to predict the real
+// collision probability by extrapolation from observed collisions. Our
+// basic approach is to reduce each cache key range down to some smaller
+// number of bits, and limiting to bits that are shared over the whole
+// range. Now we can observe collisions using a set of smaller stripped-down
+// (reduced) cache keys. Let's do some case analysis to understand why this
+// works:
+// * No collision in reduced key - because the reduction is a pure function
+// this implies no collision in the full keys
+// * Collision detected between two reduced keys - either
+// * The reduction has dropped some structured uniqueness info (from one of
+// session counter or file number; file offsets are never materialized here).
+// This can only artificially inflate the observed and extrapolated collision
+// probabilities. We only have to worry about this in designing the reduction.
+// * The reduction has preserved all the structured uniqueness in the cache
+// key, which means either
+// * REJECTED: We have a uniqueness bug in generating cache keys, where
+// structured uniqueness info should have been different but isn't. In such a
+// case, increasing by 1 the number of bits kept after reduction would not
+// reduce observed probabilities by half. (In our observations, the
+// probabilities are reduced approximately by half.)
+// * ACCEPTED: The lost unstructured uniqueness in the key determines the
+// probability that an observed collision would imply an overlap in ranges.
+// In short, dropping n bits from key would increase collision probability by
+// 2**n, assuming those n bits have full entropy in unstructured uniqueness.
+//
+// But we also have to account for the key ranges based on file size. If file
+// sizes are roughly 2**b offsets, using XOR in 128-bit cache keys for
+// "ranges", we know from other simulations (see
+// https://github.com/pdillinger/unique_id/) that that's roughly equivalent to
+// (less than 2x higher collision probability) using a cache key of size
+// 128 - b bits for the whole file. (This is the only place we make an
+// "optimistic" assumption, which is more than offset by the real
+// implementation stripping off 2 lower bits from block byte offsets for cache
+// keys. The simulation assumes byte offsets, which is net pessimistic.)
+//
+// So to accept the extrapolation as valid, we need to be confident that all
+// "lost" bits, excluding those covered by file offset, are full entropy.
+// Recall that we have assumed (verifiably, safely) that other structured data
+// (file number and session counter) are kept, not lost. Based on the
+// implementation comments for OffsetableCacheKey, the only potential hole here
+// is that we only have ~103 bits of entropy in "all new" session IDs, and in
+// extreme cases, there might be only 1 DB ID. However, because the upper ~39
+// bits of session ID are hashed, the combination of file number and file
+// offset only has to add to 25 bits (or more) to ensure full entropy in
+// unstructured uniqueness lost in the reduction. Typical file size of 32MB
+// suffices (at least for simulation purposes where we assume each file offset
+// occupies a cache key).
+//
+// Example results in comments on OffsetableCacheKey.
+class StressCacheKey {
+ public:
+ void Run() {
+ if (FLAGS_sck_footer_unique_id) {
+ // Proposed footer unique IDs are DB-independent and session-independent
+ // (but process-dependent) which is most easily simulated here by
+ // assuming 1 DB and (later below) no session resets without process
+ // reset.
+ FLAGS_sck_db_count = 1;
+ }
+
+ // Describe the simulated workload
+ uint64_t mb_per_day =
+ uint64_t{FLAGS_sck_files_per_day} * FLAGS_sck_file_size_mb;
+ printf("Total cache or DBs size: %gTiB Writing %g MiB/s or %gTiB/day\n",
+ FLAGS_sck_file_size_mb / 1024.0 / 1024.0 *
+ std::pow(2.0, FLAGS_sck_table_bits),
+ mb_per_day / 86400.0, mb_per_day / 1024.0 / 1024.0);
+ // For extrapolating probability of any collisions from a number of
+ // observed collisions
+ multiplier_ = std::pow(2.0, 128 - FLAGS_sck_keep_bits) /
+ (FLAGS_sck_file_size_mb * 1024.0 * 1024.0);
+ printf(
+ "Multiply by %g to correct for simulation losses (but still assume "
+ "whole file cached)\n",
+ multiplier_);
+ restart_nfiles_ = FLAGS_sck_files_per_day / FLAGS_sck_restarts_per_day;
+ double without_ejection =
+ std::pow(1.414214, FLAGS_sck_keep_bits) / FLAGS_sck_files_per_day;
+ // This should be a lower bound for -sck_randomize, usually a terribly
+ // rough lower bound.
+ // If observation is worse than this, then something has gone wrong.
+ printf(
+ "Without ejection, expect random collision after %g days (%g "
+ "corrected)\n",
+ without_ejection, without_ejection * multiplier_);
+ double with_full_table =
+ std::pow(2.0, FLAGS_sck_keep_bits - FLAGS_sck_table_bits) /
+ FLAGS_sck_files_per_day;
+ // This is an alternate lower bound for -sck_randomize, usually pretty
+ // accurate. Our cache keys should usually perform "better than random"
+ // but always no worse. (If observation is substantially worse than this,
+ // then something has gone wrong.)
+ printf(
+ "With ejection and full table, expect random collision after %g "
+ "days (%g corrected)\n",
+ with_full_table, with_full_table * multiplier_);
+ collisions_ = 0;
+
+ // Run until sufficient number of observed collisions.
+ for (int i = 1; collisions_ < FLAGS_sck_min_collision; i++) {
+ RunOnce();
+ if (collisions_ == 0) {
+ printf(
+ "No collisions after %d x %u days "
+ " \n",
+ i, FLAGS_sck_days_per_run);
+ } else {
+ double est = 1.0 * i * FLAGS_sck_days_per_run / collisions_;
+ printf("%" PRIu64
+ " collisions after %d x %u days, est %g days between (%g "
+ "corrected) \n",
+ collisions_, i, FLAGS_sck_days_per_run, est, est * multiplier_);
+ }
+ }
+ }
+
+ void RunOnce() {
+ // Re-initialized simulated state
+ const size_t db_count = std::max(size_t{FLAGS_sck_db_count}, size_t{1});
+ dbs_.reset(new TableProperties[db_count]{});
+ const size_t table_mask = (size_t{1} << FLAGS_sck_table_bits) - 1;
+ table_.reset(new uint64_t[table_mask + 1]{});
+ if (FLAGS_sck_keep_bits > 64) {
+ FLAGS_sck_keep_bits = 64;
+ }
+
+ // Details of which bits are dropped in reduction
+ uint32_t shift_away = 64 - FLAGS_sck_keep_bits;
+ // Shift away fewer potential file number bits (b) than potential
+ // session counter bits (a).
+ uint32_t shift_away_b = shift_away / 3;
+ uint32_t shift_away_a = shift_away - shift_away_b;
+
+ process_count_ = 0;
+ session_count_ = 0;
+ newdb_count_ = 0;
+ ResetProcess(/*newdbs*/ true);
+
+ Random64 r{std::random_device{}()};
+
+ uint64_t max_file_count =
+ uint64_t{FLAGS_sck_files_per_day} * FLAGS_sck_days_per_run;
+ uint32_t report_count = 0;
+ uint32_t collisions_this_run = 0;
+ size_t db_i = 0;
+
+ for (uint64_t file_count = 1; file_count <= max_file_count;
+ ++file_count, ++db_i) {
+ // Round-robin through DBs (this faster than %)
+ if (db_i >= db_count) {
+ db_i = 0;
+ }
+ // Any other periodic actions before simulating next file
+ if (!FLAGS_sck_footer_unique_id && r.OneIn(FLAGS_sck_reopen_nfiles)) {
+ ResetSession(db_i, /*newdb*/ r.OneIn(FLAGS_sck_newdb_nreopen));
+ } else if (r.OneIn(restart_nfiles_)) {
+ ResetProcess(/*newdbs*/ false);
+ }
+ // Simulate next file
+ OffsetableCacheKey ock;
+ dbs_[db_i].orig_file_number += 1;
+ // skip some file numbers for other file kinds, except in footer unique
+ // ID, orig_file_number here tracks process-wide generated SST file
+ // count.
+ if (!FLAGS_sck_footer_unique_id) {
+ dbs_[db_i].orig_file_number += (r.Next() & 3);
+ }
+ bool is_stable;
+ BlockBasedTable::SetupBaseCacheKey(&dbs_[db_i], /* ignored */ "",
+ /* ignored */ 42, &ock, &is_stable);
+ assert(is_stable);
+ // Get a representative cache key, which later we analytically generalize
+ // to a range.
+ CacheKey ck = ock.WithOffset(0);
+ uint64_t reduced_key;
+ if (FLAGS_sck_randomize) {
+ reduced_key = GetSliceHash64(ck.AsSlice()) >> shift_away;
+ } else if (FLAGS_sck_footer_unique_id) {
+ // Special case: keep only file number, not session counter
+ reduced_key = DecodeFixed64(ck.AsSlice().data()) >> shift_away;
+ } else {
+ // Try to keep file number and session counter (shift away other bits)
+ uint32_t a = DecodeFixed32(ck.AsSlice().data()) << shift_away_a;
+ uint32_t b = DecodeFixed32(ck.AsSlice().data() + 4) >> shift_away_b;
+ reduced_key = (uint64_t{a} << 32) + b;
+ }
+ if (reduced_key == 0) {
+ // Unlikely, but we need to exclude tracking this value because we
+ // use it to mean "empty" in table. This case is OK as long as we
+ // don't hit it often.
+ printf("Hit Zero! \n");
+ file_count--;
+ continue;
+ }
+ uint64_t h =
+ NPHash64(reinterpret_cast<char*>(&reduced_key), sizeof(reduced_key));
+ // Skew expected lifetimes, for high variance (super-Poisson) variance
+ // in actual lifetimes.
+ size_t pos =
+ std::min(Lower32of64(h) & table_mask, Upper32of64(h) & table_mask);
+ if (table_[pos] == reduced_key) {
+ collisions_this_run++;
+ // Our goal is to predict probability of no collisions, not expected
+ // number of collisions. To make the distinction, we have to get rid
+ // of observing correlated collisions, which this takes care of:
+ ResetProcess(/*newdbs*/ false);
+ } else {
+ // Replace (end of lifetime for file that was in this slot)
+ table_[pos] = reduced_key;
+ }
+
+ if (++report_count == FLAGS_sck_files_per_day) {
+ report_count = 0;
+ // Estimate fill %
+ size_t incr = table_mask / 1000;
+ size_t sampled_count = 0;
+ for (size_t i = 0; i <= table_mask; i += incr) {
+ if (table_[i] != 0) {
+ sampled_count++;
+ }
+ }
+ // Report
+ printf(
+ "%" PRIu64 " days, %" PRIu64 " proc, %" PRIu64 " sess, %" PRIu64
+ " newdb, %u coll, occ %g%%, ejected %g%% \r",
+ file_count / FLAGS_sck_files_per_day, process_count_,
+ session_count_, newdb_count_ - FLAGS_sck_db_count,
+ collisions_this_run, 100.0 * sampled_count / 1000.0,
+ 100.0 * (1.0 - sampled_count / 1000.0 * table_mask / file_count));
+ fflush(stdout);
+ }
+ }
+ collisions_ += collisions_this_run;
+ }
+
+ void ResetSession(size_t i, bool newdb) {
+ dbs_[i].db_session_id = DBImpl::GenerateDbSessionId(nullptr);
+ if (newdb) {
+ ++newdb_count_;
+ if (FLAGS_sck_footer_unique_id) {
+ // Simulate how footer id would behave
+ dbs_[i].db_id = "none";
+ } else {
+ // db_id might be ignored, depending on the implementation details
+ dbs_[i].db_id = std::to_string(newdb_count_);
+ dbs_[i].orig_file_number = 0;
+ }
+ }
+ session_count_++;
+ }
+
+ void ResetProcess(bool newdbs) {
+ process_count_++;
+ DBImpl::TEST_ResetDbSessionIdGen();
+ for (size_t i = 0; i < FLAGS_sck_db_count; ++i) {
+ ResetSession(i, newdbs);
+ }
+ if (FLAGS_sck_footer_unique_id) {
+ // For footer unique ID, this tracks process-wide generated SST file
+ // count.
+ dbs_[0].orig_file_number = 0;
+ }
+ }
+
+ private:
+ // Use db_session_id and orig_file_number from TableProperties
+ std::unique_ptr<TableProperties[]> dbs_;
+ std::unique_ptr<uint64_t[]> table_;
+ uint64_t process_count_ = 0;
+ uint64_t session_count_ = 0;
+ uint64_t newdb_count_ = 0;
+ uint64_t collisions_ = 0;
+ uint32_t restart_nfiles_ = 0;
+ double multiplier_ = 0.0;
+};
+
+int cache_bench_tool(int argc, char** argv) {
+ ParseCommandLineFlags(&argc, &argv, true);
+
+ if (FLAGS_stress_cache_key) {
+ // Alternate tool
+ StressCacheKey().Run();
+ return 0;
+ }
+
+ if (FLAGS_threads <= 0) {
+ fprintf(stderr, "threads number <= 0\n");
+ exit(1);
+ }
+
+ ROCKSDB_NAMESPACE::CacheBench bench;
+ if (FLAGS_populate_cache) {
+ bench.PopulateCache();
+ printf("Population complete\n");
+ printf("----------------------------\n");
+ }
+ if (bench.Run()) {
+ return 0;
+ } else {
+ return 1;
+ }
+} // namespace ROCKSDB_NAMESPACE
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // GFLAGS
diff --git a/src/rocksdb/cache/cache_entry_roles.cc b/src/rocksdb/cache/cache_entry_roles.cc
new file mode 100644
index 000000000..b27349554
--- /dev/null
+++ b/src/rocksdb/cache/cache_entry_roles.cc
@@ -0,0 +1,134 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// 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/cache_entry_roles.h"
+
+#include <mutex>
+
+#include "port/lang.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::array<std::string, kNumCacheEntryRoles> kCacheEntryRoleToCamelString{{
+ "DataBlock",
+ "FilterBlock",
+ "FilterMetaBlock",
+ "DeprecatedFilterBlock",
+ "IndexBlock",
+ "OtherBlock",
+ "WriteBuffer",
+ "CompressionDictionaryBuildingBuffer",
+ "FilterConstruction",
+ "BlockBasedTableReader",
+ "FileMetadata",
+ "BlobValue",
+ "BlobCache",
+ "Misc",
+}};
+
+std::array<std::string, kNumCacheEntryRoles> kCacheEntryRoleToHyphenString{{
+ "data-block",
+ "filter-block",
+ "filter-meta-block",
+ "deprecated-filter-block",
+ "index-block",
+ "other-block",
+ "write-buffer",
+ "compression-dictionary-building-buffer",
+ "filter-construction",
+ "block-based-table-reader",
+ "file-metadata",
+ "blob-value",
+ "blob-cache",
+ "misc",
+}};
+
+const std::string& GetCacheEntryRoleName(CacheEntryRole role) {
+ return kCacheEntryRoleToHyphenString[static_cast<size_t>(role)];
+}
+
+const std::string& BlockCacheEntryStatsMapKeys::CacheId() {
+ static const std::string kCacheId = "id";
+ return kCacheId;
+}
+
+const std::string& BlockCacheEntryStatsMapKeys::CacheCapacityBytes() {
+ static const std::string kCacheCapacityBytes = "capacity";
+ return kCacheCapacityBytes;
+}
+
+const std::string&
+BlockCacheEntryStatsMapKeys::LastCollectionDurationSeconds() {
+ static const std::string kLastCollectionDurationSeconds =
+ "secs_for_last_collection";
+ return kLastCollectionDurationSeconds;
+}
+
+const std::string& BlockCacheEntryStatsMapKeys::LastCollectionAgeSeconds() {
+ static const std::string kLastCollectionAgeSeconds =
+ "secs_since_last_collection";
+ return kLastCollectionAgeSeconds;
+}
+
+namespace {
+
+std::string GetPrefixedCacheEntryRoleName(const std::string& prefix,
+ CacheEntryRole role) {
+ const std::string& role_name = GetCacheEntryRoleName(role);
+ std::string prefixed_role_name;
+ prefixed_role_name.reserve(prefix.size() + role_name.size());
+ prefixed_role_name.append(prefix);
+ prefixed_role_name.append(role_name);
+ return prefixed_role_name;
+}
+
+} // namespace
+
+std::string BlockCacheEntryStatsMapKeys::EntryCount(CacheEntryRole role) {
+ const static std::string kPrefix = "count.";
+ return GetPrefixedCacheEntryRoleName(kPrefix, role);
+}
+
+std::string BlockCacheEntryStatsMapKeys::UsedBytes(CacheEntryRole role) {
+ const static std::string kPrefix = "bytes.";
+ return GetPrefixedCacheEntryRoleName(kPrefix, role);
+}
+
+std::string BlockCacheEntryStatsMapKeys::UsedPercent(CacheEntryRole role) {
+ const static std::string kPrefix = "percent.";
+ return GetPrefixedCacheEntryRoleName(kPrefix, role);
+}
+
+namespace {
+
+struct Registry {
+ std::mutex mutex;
+ UnorderedMap<Cache::DeleterFn, CacheEntryRole> role_map;
+ void Register(Cache::DeleterFn fn, CacheEntryRole role) {
+ std::lock_guard<std::mutex> lock(mutex);
+ role_map[fn] = role;
+ }
+ UnorderedMap<Cache::DeleterFn, CacheEntryRole> Copy() {
+ std::lock_guard<std::mutex> lock(mutex);
+ return role_map;
+ }
+};
+
+Registry& GetRegistry() {
+ STATIC_AVOID_DESTRUCTION(Registry, registry);
+ return registry;
+}
+
+} // namespace
+
+void RegisterCacheDeleterRole(Cache::DeleterFn fn, CacheEntryRole role) {
+ GetRegistry().Register(fn, role);
+}
+
+UnorderedMap<Cache::DeleterFn, CacheEntryRole> CopyCacheDeleterRoleMap() {
+ return GetRegistry().Copy();
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_entry_roles.h b/src/rocksdb/cache/cache_entry_roles.h
new file mode 100644
index 000000000..5a49fdfd4
--- /dev/null
+++ b/src/rocksdb/cache/cache_entry_roles.h
@@ -0,0 +1,103 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// 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 <array>
+#include <cstdint>
+#include <memory>
+#include <type_traits>
+
+#include "rocksdb/cache.h"
+#include "util/hash_containers.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+extern std::array<std::string, kNumCacheEntryRoles>
+ kCacheEntryRoleToCamelString;
+extern std::array<std::string, kNumCacheEntryRoles>
+ kCacheEntryRoleToHyphenString;
+
+// To associate cache entries with their role, we use a hack on the
+// existing Cache interface. Because the deleter of an entry can authenticate
+// the code origin of an entry, we can elaborate the choice of deleter to
+// also encode role information, without inferring false role information
+// from entries not choosing to encode a role.
+//
+// The rest of this file is for handling mappings between deleters and
+// roles.
+
+// To infer a role from a deleter, the deleter must be registered. This
+// can be done "manually" with this function. This function is thread-safe,
+// and the registration mappings go into private but static storage. (Note
+// that DeleterFn is a function pointer, not std::function. Registrations
+// should not be too many.)
+void RegisterCacheDeleterRole(Cache::DeleterFn fn, CacheEntryRole role);
+
+// Gets a copy of the registered deleter -> role mappings. This is the only
+// function for reading the mappings made with RegisterCacheDeleterRole.
+// Why only this interface for reading?
+// * This function has to be thread safe, which could incur substantial
+// overhead. We should not pay this overhead for every deleter look-up.
+// * This is suitable for preparing for batch operations, like with
+// CacheEntryStatsCollector.
+// * The number of mappings should be sufficiently small (dozens).
+UnorderedMap<Cache::DeleterFn, CacheEntryRole> CopyCacheDeleterRoleMap();
+
+// ************************************************************** //
+// An automatic registration infrastructure. This enables code
+// to simply ask for a deleter associated with a particular type
+// and role, and registration is automatic. In a sense, this is
+// a small dependency injection infrastructure, because linking
+// in new deleter instantiations is essentially sufficient for
+// making stats collection (using CopyCacheDeleterRoleMap) aware
+// of them.
+
+namespace cache_entry_roles_detail {
+
+template <typename T, CacheEntryRole R>
+struct RegisteredDeleter {
+ RegisteredDeleter() { RegisterCacheDeleterRole(Delete, R); }
+
+ // These have global linkage to help ensure compiler optimizations do not
+ // break uniqueness for each <T,R>
+ static void Delete(const Slice& /* key */, void* value) {
+ // Supports T == Something[], unlike delete operator
+ std::default_delete<T>()(
+ static_cast<typename std::remove_extent<T>::type*>(value));
+ }
+};
+
+template <CacheEntryRole R>
+struct RegisteredNoopDeleter {
+ RegisteredNoopDeleter() { RegisterCacheDeleterRole(Delete, R); }
+
+ static void Delete(const Slice& /* key */, void* /* value */) {
+ // Here was `assert(value == nullptr);` but we can also put pointers
+ // to static data in Cache, for testing at least.
+ }
+};
+
+} // namespace cache_entry_roles_detail
+
+// Get an automatically registered deleter for value type T and role R.
+// Based on C++ semantics, registration is invoked exactly once in a
+// thread-safe way on first call to this function, for each <T, R>.
+template <typename T, CacheEntryRole R>
+Cache::DeleterFn GetCacheEntryDeleterForRole() {
+ static cache_entry_roles_detail::RegisteredDeleter<T, R> reg;
+ return reg.Delete;
+}
+
+// Get an automatically registered no-op deleter (value should be nullptr)
+// and associated with role R. This is used for Cache "reservation" entries
+// such as for WriteBufferManager.
+template <CacheEntryRole R>
+Cache::DeleterFn GetNoopDeleterForRole() {
+ static cache_entry_roles_detail::RegisteredNoopDeleter<R> reg;
+ return reg.Delete;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_entry_stats.h b/src/rocksdb/cache/cache_entry_stats.h
new file mode 100644
index 000000000..63b12735b
--- /dev/null
+++ b/src/rocksdb/cache/cache_entry_stats.h
@@ -0,0 +1,183 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// 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 <array>
+#include <cstdint>
+#include <memory>
+#include <mutex>
+
+#include "cache/cache_helpers.h"
+#include "cache/cache_key.h"
+#include "port/lang.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/status.h"
+#include "rocksdb/system_clock.h"
+#include "test_util/sync_point.h"
+#include "util/coding_lean.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// A generic helper object for gathering stats about cache entries by
+// iterating over them with ApplyToAllEntries. This class essentially
+// solves the problem of slowing down a Cache with too many stats
+// collectors that could be sharing stat results, such as from multiple
+// column families or multiple DBs sharing a Cache. We employ a few
+// mitigations:
+// * Only one collector for a particular kind of Stats is alive
+// for each Cache. This is guaranteed using the Cache itself to hold
+// the collector.
+// * A mutex ensures only one thread is gathering stats for this
+// collector.
+// * The most recent gathered stats are saved and simply copied to
+// satisfy requests within a time window (default: 3 minutes) of
+// completion of the most recent stat gathering.
+//
+// Template parameter Stats must be copyable and trivially constructable,
+// as well as...
+// concept Stats {
+// // Notification before applying callback to all entries
+// void BeginCollection(Cache*, SystemClock*, uint64_t start_time_micros);
+// // Get the callback to apply to all entries. `callback`
+// // type must be compatible with Cache::ApplyToAllEntries
+// callback GetEntryCallback();
+// // Notification after applying callback to all entries
+// void EndCollection(Cache*, SystemClock*, uint64_t end_time_micros);
+// // Notification that a collection was skipped because of
+// // sufficiently recent saved results.
+// void SkippedCollection();
+// }
+template <class Stats>
+class CacheEntryStatsCollector {
+ public:
+ // Gather and save stats if saved stats are too old. (Use GetStats() to
+ // read saved stats.)
+ //
+ // Maximum allowed age for a "hit" on saved results is determined by the
+ // two interval parameters. Both set to 0 forces a re-scan. For example
+ // with min_interval_seconds=300 and min_interval_factor=100, if the last
+ // scan took 10s, we would only rescan ("miss") if the age in seconds of
+ // the saved results is > max(300, 100*10).
+ // Justification: scans can vary wildly in duration, e.g. from 0.02 sec
+ // to as much as 20 seconds, so we want to be able to cap the absolute
+ // and relative frequency of scans.
+ void CollectStats(int min_interval_seconds, int min_interval_factor) {
+ // Waits for any pending reader or writer (collector)
+ std::lock_guard<std::mutex> lock(working_mutex_);
+
+ uint64_t max_age_micros =
+ static_cast<uint64_t>(std::max(min_interval_seconds, 0)) * 1000000U;
+
+ if (last_end_time_micros_ > last_start_time_micros_ &&
+ min_interval_factor > 0) {
+ max_age_micros = std::max(
+ max_age_micros, min_interval_factor * (last_end_time_micros_ -
+ last_start_time_micros_));
+ }
+
+ uint64_t start_time_micros = clock_->NowMicros();
+ if ((start_time_micros - last_end_time_micros_) > max_age_micros) {
+ last_start_time_micros_ = start_time_micros;
+ working_stats_.BeginCollection(cache_, clock_, start_time_micros);
+
+ cache_->ApplyToAllEntries(working_stats_.GetEntryCallback(), {});
+ TEST_SYNC_POINT_CALLBACK(
+ "CacheEntryStatsCollector::GetStats:AfterApplyToAllEntries", nullptr);
+
+ uint64_t end_time_micros = clock_->NowMicros();
+ last_end_time_micros_ = end_time_micros;
+ working_stats_.EndCollection(cache_, clock_, end_time_micros);
+ } else {
+ working_stats_.SkippedCollection();
+ }
+
+ // Save so that we don't need to wait for an outstanding collection in
+ // order to make of copy of the last saved stats
+ std::lock_guard<std::mutex> lock2(saved_mutex_);
+ saved_stats_ = working_stats_;
+ }
+
+ // Gets saved stats, regardless of age
+ void GetStats(Stats *stats) {
+ std::lock_guard<std::mutex> lock(saved_mutex_);
+ *stats = saved_stats_;
+ }
+
+ Cache *GetCache() const { return cache_; }
+
+ // Gets or creates a shared instance of CacheEntryStatsCollector in the
+ // cache itself, and saves into `ptr`. This shared_ptr will hold the
+ // entry in cache until all refs are destroyed.
+ static Status GetShared(Cache *cache, SystemClock *clock,
+ std::shared_ptr<CacheEntryStatsCollector> *ptr) {
+ const Slice &cache_key = GetCacheKey();
+
+ Cache::Handle *h = cache->Lookup(cache_key);
+ if (h == nullptr) {
+ // Not yet in cache, but Cache doesn't provide a built-in way to
+ // avoid racing insert. So we double-check under a shared mutex,
+ // inspired by TableCache.
+ STATIC_AVOID_DESTRUCTION(std::mutex, static_mutex);
+ std::lock_guard<std::mutex> lock(static_mutex);
+
+ h = cache->Lookup(cache_key);
+ if (h == nullptr) {
+ auto new_ptr = new CacheEntryStatsCollector(cache, clock);
+ // TODO: non-zero charge causes some tests that count block cache
+ // usage to go flaky. Fix the problem somehow so we can use an
+ // accurate charge.
+ size_t charge = 0;
+ Status s = cache->Insert(cache_key, new_ptr, charge, Deleter, &h,
+ Cache::Priority::HIGH);
+ if (!s.ok()) {
+ assert(h == nullptr);
+ delete new_ptr;
+ return s;
+ }
+ }
+ }
+ // If we reach here, shared entry is in cache with handle `h`.
+ assert(cache->GetDeleter(h) == Deleter);
+
+ // Build an aliasing shared_ptr that keeps `ptr` in cache while there
+ // are references.
+ *ptr = MakeSharedCacheHandleGuard<CacheEntryStatsCollector>(cache, h);
+ return Status::OK();
+ }
+
+ private:
+ explicit CacheEntryStatsCollector(Cache *cache, SystemClock *clock)
+ : saved_stats_(),
+ working_stats_(),
+ last_start_time_micros_(0),
+ last_end_time_micros_(/*pessimistic*/ 10000000),
+ cache_(cache),
+ clock_(clock) {}
+
+ static void Deleter(const Slice &, void *value) {
+ delete static_cast<CacheEntryStatsCollector *>(value);
+ }
+
+ static const Slice &GetCacheKey() {
+ // For each template instantiation
+ static CacheKey ckey = CacheKey::CreateUniqueForProcessLifetime();
+ static Slice ckey_slice = ckey.AsSlice();
+ return ckey_slice;
+ }
+
+ std::mutex saved_mutex_;
+ Stats saved_stats_;
+
+ std::mutex working_mutex_;
+ Stats working_stats_;
+ uint64_t last_start_time_micros_;
+ uint64_t last_end_time_micros_;
+
+ Cache *const cache_;
+ SystemClock *const clock_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_helpers.h b/src/rocksdb/cache/cache_helpers.h
new file mode 100644
index 000000000..7ea2365b8
--- /dev/null
+++ b/src/rocksdb/cache/cache_helpers.h
@@ -0,0 +1,145 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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 <cassert>
+
+#include "rocksdb/cache.h"
+#include "rocksdb/rocksdb_namespace.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Returns the cached value given a cache handle.
+template <typename T>
+T* GetFromCacheHandle(Cache* cache, Cache::Handle* handle) {
+ assert(cache);
+ assert(handle);
+
+ return static_cast<T*>(cache->Value(handle));
+}
+
+// Simple generic deleter for Cache (to be used with Cache::Insert).
+template <typename T>
+void DeleteCacheEntry(const Slice& /* key */, void* value) {
+ delete static_cast<T*>(value);
+}
+
+// Turns a T* into a Slice so it can be used as a key with Cache.
+template <typename T>
+Slice GetSlice(const T* t) {
+ return Slice(reinterpret_cast<const char*>(t), sizeof(T));
+}
+
+// Generic resource management object for cache handles that releases the handle
+// when destroyed. Has unique ownership of the handle, so copying it is not
+// allowed, while moving it transfers ownership.
+template <typename T>
+class CacheHandleGuard {
+ public:
+ CacheHandleGuard() = default;
+
+ CacheHandleGuard(Cache* cache, Cache::Handle* handle)
+ : cache_(cache),
+ handle_(handle),
+ value_(GetFromCacheHandle<T>(cache, handle)) {
+ assert(cache_ && handle_ && value_);
+ }
+
+ CacheHandleGuard(const CacheHandleGuard&) = delete;
+ CacheHandleGuard& operator=(const CacheHandleGuard&) = delete;
+
+ CacheHandleGuard(CacheHandleGuard&& rhs) noexcept
+ : cache_(rhs.cache_), handle_(rhs.handle_), value_(rhs.value_) {
+ assert((!cache_ && !handle_ && !value_) || (cache_ && handle_ && value_));
+
+ rhs.ResetFields();
+ }
+
+ CacheHandleGuard& operator=(CacheHandleGuard&& rhs) noexcept {
+ if (this == &rhs) {
+ return *this;
+ }
+
+ ReleaseHandle();
+
+ cache_ = rhs.cache_;
+ handle_ = rhs.handle_;
+ value_ = rhs.value_;
+
+ assert((!cache_ && !handle_ && !value_) || (cache_ && handle_ && value_));
+
+ rhs.ResetFields();
+
+ return *this;
+ }
+
+ ~CacheHandleGuard() { ReleaseHandle(); }
+
+ bool IsEmpty() const { return !handle_; }
+
+ Cache* GetCache() const { return cache_; }
+ Cache::Handle* GetCacheHandle() const { return handle_; }
+ T* GetValue() const { return value_; }
+
+ void TransferTo(Cleanable* cleanable) {
+ if (cleanable) {
+ if (handle_ != nullptr) {
+ assert(cache_);
+ cleanable->RegisterCleanup(&ReleaseCacheHandle, cache_, handle_);
+ }
+ }
+ ResetFields();
+ }
+
+ void Reset() {
+ ReleaseHandle();
+ ResetFields();
+ }
+
+ private:
+ void ReleaseHandle() {
+ if (IsEmpty()) {
+ return;
+ }
+
+ assert(cache_);
+ cache_->Release(handle_);
+ }
+
+ void ResetFields() {
+ cache_ = nullptr;
+ handle_ = nullptr;
+ value_ = nullptr;
+ }
+
+ static void ReleaseCacheHandle(void* arg1, void* arg2) {
+ Cache* const cache = static_cast<Cache*>(arg1);
+ assert(cache);
+
+ Cache::Handle* const cache_handle = static_cast<Cache::Handle*>(arg2);
+ assert(cache_handle);
+
+ cache->Release(cache_handle);
+ }
+
+ private:
+ Cache* cache_ = nullptr;
+ Cache::Handle* handle_ = nullptr;
+ T* value_ = nullptr;
+};
+
+// Build an aliasing shared_ptr that keeps `handle` in cache while there
+// are references, but the pointer is to the value for that cache entry,
+// which must be of type T. This is copyable, unlike CacheHandleGuard, but
+// does not provide access to caching details.
+template <typename T>
+std::shared_ptr<T> MakeSharedCacheHandleGuard(Cache* cache,
+ Cache::Handle* handle) {
+ auto wrapper = std::make_shared<CacheHandleGuard<T>>(cache, handle);
+ return std::shared_ptr<T>(wrapper, static_cast<T*>(cache->Value(handle)));
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_key.cc b/src/rocksdb/cache/cache_key.cc
new file mode 100644
index 000000000..a79328972
--- /dev/null
+++ b/src/rocksdb/cache/cache_key.cc
@@ -0,0 +1,364 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// 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/cache_key.h"
+
+#include <algorithm>
+#include <atomic>
+
+#include "rocksdb/cache.h"
+#include "table/unique_id_impl.h"
+#include "util/hash.h"
+#include "util/math.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Value space plan for CacheKey:
+//
+// file_num_etc64_ | offset_etc64_ | Only generated by
+// ---------------+---------------+------------------------------------------
+// 0 | 0 | Reserved for "empty" CacheKey()
+// 0 | > 0, < 1<<63 | CreateUniqueForCacheLifetime
+// 0 | >= 1<<63 | CreateUniqueForProcessLifetime
+// > 0 | any | OffsetableCacheKey.WithOffset
+
+CacheKey CacheKey::CreateUniqueForCacheLifetime(Cache *cache) {
+ // +1 so that we can reserve all zeros for "unset" cache key
+ uint64_t id = cache->NewId() + 1;
+ // Ensure we don't collide with CreateUniqueForProcessLifetime
+ assert((id >> 63) == 0U);
+ return CacheKey(0, id);
+}
+
+CacheKey CacheKey::CreateUniqueForProcessLifetime() {
+ // To avoid colliding with CreateUniqueForCacheLifetime, assuming
+ // Cache::NewId counts up from zero, here we count down from UINT64_MAX.
+ // If this ever becomes a point of contention, we could sub-divide the
+ // space and use CoreLocalArray.
+ static std::atomic<uint64_t> counter{UINT64_MAX};
+ uint64_t id = counter.fetch_sub(1, std::memory_order_relaxed);
+ // Ensure we don't collide with CreateUniqueForCacheLifetime
+ assert((id >> 63) == 1U);
+ return CacheKey(0, id);
+}
+
+// How we generate CacheKeys and base OffsetableCacheKey, assuming that
+// db_session_ids are generated from a base_session_id and
+// session_id_counter (by SemiStructuredUniqueIdGen+EncodeSessionId
+// in DBImpl::GenerateDbSessionId):
+//
+// Conceptual inputs:
+// db_id (unstructured, from GenerateRawUniqueId or equiv)
+// * could be shared between cloned DBs but rare
+// * could be constant, if session id suffices
+// base_session_id (unstructured, from GenerateRawUniqueId)
+// session_id_counter (structured)
+// * usually much smaller than 2**24
+// orig_file_number (structured)
+// * usually smaller than 2**24
+// offset_in_file (structured, might skip lots of values)
+// * usually smaller than 2**32
+//
+// Overall approach (see https://github.com/pdillinger/unique_id for
+// background):
+//
+// First, we have three "structured" values, up to 64 bits each, that we
+// need to fit, without losses, into 128 bits. In practice, the values will
+// be small enough that they should fit. For example, applications generating
+// large SST files (large offsets) will naturally produce fewer files (small
+// file numbers). But we don't know ahead of time what bounds the values will
+// have.
+//
+// Second, we have unstructured inputs that enable distinct RocksDB processes
+// to pick a random point in space, likely very different from others. Xoring
+// the structured with the unstructured give us a cache key that is
+// structurally distinct between related keys (e.g. same file or same RocksDB
+// process) and distinct with high probability between unrelated keys.
+//
+// The problem of packing three structured values into the space for two is
+// complicated by the fact that we want to derive cache keys from SST unique
+// IDs, which have already combined structured and unstructured inputs in a
+// practically inseparable way. And we want a base cache key that works
+// with an offset of any size. So basically, we need to encode these three
+// structured values, each up to 64 bits, into 128 bits without knowing any
+// of their sizes. The DownwardInvolution() function gives us a mechanism to
+// accomplish this. (See its properties in math.h.) Specifically, for inputs
+// a, b, and c:
+// lower64 = DownwardInvolution(a) ^ ReverseBits(b);
+// upper64 = c ^ ReverseBits(a);
+// The 128-bit output is unique assuming there exist some i, j, and k
+// where a < 2**i, b < 2**j, c < 2**k, i <= 64, j <= 64, k <= 64, and
+// i + j + k <= 128. In other words, as long as there exist some bounds
+// that would allow us to pack the bits of a, b, and c into the output
+// if we know the bound, we can generate unique outputs without knowing
+// those bounds. To validate this claim, the inversion function (given
+// the bounds) has been implemented in CacheKeyDecoder in
+// db_block_cache_test.cc.
+//
+// With that in mind, the outputs in terms of the conceptual inputs look
+// like this, using bitwise-xor of the constituent pieces, low bits on left:
+//
+// |------------------------- file_num_etc64 -------------------------|
+// | +++++++++ base_session_id (lower 64 bits, involution) +++++++++ |
+// |-----------------------------------------------------------------|
+// | session_id_counter (involution) ..... | |
+// |-----------------------------------------------------------------|
+// | hash of: ++++++++++++++++++++++++++++++++++++++++++++++++++++++ |
+// | * base_session_id (upper ~39 bits) |
+// | * db_id (~122 bits entropy) |
+// |-----------------------------------------------------------------|
+// | | ..... orig_file_number (reversed) |
+// |-----------------------------------------------------------------|
+//
+//
+// |------------------------- offset_etc64 --------------------------|
+// | ++++++++++ base_session_id (lower 64 bits, reversed) ++++++++++ |
+// |-----------------------------------------------------------------|
+// | | ..... session_id_counter (reversed) |
+// |-----------------------------------------------------------------|
+// | offset_in_file ............... | |
+// |-----------------------------------------------------------------|
+//
+// Some oddities or inconveniences of this layout are due to deriving
+// the "base" cache key (without offset) from the SST unique ID (see
+// GetSstInternalUniqueId). Specifically,
+// * Lower 64 of base_session_id occurs in both output words (ok but
+// weird)
+// * The inclusion of db_id is bad for the conditions under which we
+// can guarantee uniqueness, but could be useful in some cases with
+// few small files per process, to make up for db session id only having
+// ~103 bits of entropy.
+//
+// In fact, if DB ids were not involved, we would be guaranteed unique
+// cache keys for files generated in a single process until total bits for
+// biggest session_id_counter, orig_file_number, and offset_in_file
+// reach 128 bits.
+//
+// With the DB id limitation, we only have nice guaranteed unique cache
+// keys for files generated in a single process until biggest
+// session_id_counter and offset_in_file reach combined 64 bits. This
+// is quite good in practice because we can have millions of DB Opens
+// with terabyte size SST files, or billions of DB Opens with gigabyte
+// size SST files.
+//
+// One of the considerations in the translation between existing SST unique
+// IDs and base cache keys is supporting better SST unique IDs in a future
+// format_version. If we use a process-wide file counter instead of
+// session counter and file numbers, we only need to combine two 64-bit values
+// instead of three. But we don't want to track unique ID versions in the
+// manifest, so we want to keep the same translation layer between SST unique
+// IDs and base cache keys, even with updated SST unique IDs. If the new
+// unique IDs put the file counter where the orig_file_number was, and
+// use no structured field where session_id_counter was, then our translation
+// layer works fine for two structured fields as well as three (for
+// compatibility). The small computation for the translation (one
+// DownwardInvolution(), two ReverseBits(), both ~log(64) instructions deep)
+// is negligible for computing as part of SST file reader open.
+//
+// More on how https://github.com/pdillinger/unique_id applies here:
+// Every bit of output always includes "unstructured" uniqueness bits and
+// often combines with "structured" uniqueness bits. The "unstructured" bits
+// change infrequently: only when we cannot guarantee our state tracking for
+// "structured" uniqueness hasn't been cloned. Using a static
+// SemiStructuredUniqueIdGen for db_session_ids, this means we only get an
+// "all new" session id when a new process uses RocksDB. (Between processes,
+// we don't know if a DB or other persistent storage has been cloned. We
+// assume that if VM hot cloning is used, subsequently generated SST files
+// do not interact.) Within a process, only the session_lower of the
+// db_session_id changes incrementally ("structured" uniqueness).
+//
+// This basically means that our offsets, counters and file numbers allow us
+// to do somewhat "better than random" (birthday paradox) while in the
+// degenerate case of completely new session for each tiny file, we still
+// have strong uniqueness properties from the birthday paradox, with ~103
+// bit session IDs or up to 128 bits entropy with different DB IDs sharing a
+// cache.
+//
+// More collision probability analysis:
+// Suppose a RocksDB host generates (generously) 2 GB/s (10TB data, 17 DWPD)
+// with average process/session lifetime of (pessimistically) 4 minutes.
+// In 180 days (generous allowable data lifespan), we generate 31 million GB
+// of data, or 2^55 bytes, and 2^16 "all new" session IDs.
+//
+// First, suppose this is in a single DB (lifetime 180 days):
+// 128 bits cache key size
+// - 55 <- ideal size for byte offsets + file numbers
+// - 2 <- bits for offsets and file numbers not exactly powers of two
+// + 2 <- bits saved not using byte offsets in BlockBasedTable::GetCacheKey
+// ----
+// 73 <- bits remaining for distinguishing session IDs
+// The probability of a collision in 73 bits of session ID data is less than
+// 1 in 2**(73 - (2 * 16)), or roughly 1 in a trillion. And this assumes all
+// data from the last 180 days is in cache for potential collision, and that
+// cache keys under each session id exhaustively cover the remaining 57 bits
+// while in reality they'll only cover a small fraction of it.
+//
+// Although data could be transferred between hosts, each host has its own
+// cache and we are already assuming a high rate of "all new" session ids.
+// So this doesn't really change the collision calculation. Across a fleet
+// of 1 million, each with <1 in a trillion collision possibility,
+// fleetwide collision probability is <1 in a million.
+//
+// Now suppose we have many DBs per host, say 2**10, with same host-wide write
+// rate and process/session lifetime. File numbers will be ~10 bits smaller
+// and we will have 2**10 times as many session IDs because of simultaneous
+// lifetimes. So now collision chance is less than 1 in 2**(83 - (2 * 26)),
+// or roughly 1 in a billion.
+//
+// Suppose instead we generated random or hashed cache keys for each
+// (compressed) block. For 1KB compressed block size, that is 2^45 cache keys
+// in 180 days. Collision probability is more easily estimated at roughly
+// 1 in 2**(128 - (2 * 45)) or roughly 1 in a trillion (assuming all
+// data from the last 180 days is in cache, but NOT the other assumption
+// for the 1 in a trillion estimate above).
+//
+//
+// Collision probability estimation through simulation:
+// A tool ./cache_bench -stress_cache_key broadly simulates host-wide cache
+// activity over many months, by making some pessimistic simplifying
+// assumptions. See class StressCacheKey in cache_bench_tool.cc for details.
+// Here is some sample output with
+// `./cache_bench -stress_cache_key -sck_keep_bits=43`:
+//
+// Total cache or DBs size: 32TiB Writing 925.926 MiB/s or 76.2939TiB/day
+// Multiply by 1.15292e+18 to correct for simulation losses (but still
+// assume whole file cached)
+//
+// These come from default settings of 2.5M files per day of 32 MB each, and
+// `-sck_keep_bits=43` means that to represent a single file, we are only
+// keeping 43 bits of the 128-bit (base) cache key. With file size of 2**25
+// contiguous keys (pessimistic), our simulation is about 2\*\*(128-43-25) or
+// about 1 billion billion times more prone to collision than reality.
+//
+// More default assumptions, relatively pessimistic:
+// * 100 DBs in same process (doesn't matter much)
+// * Re-open DB in same process (new session ID related to old session ID) on
+// average every 100 files generated
+// * Restart process (all new session IDs unrelated to old) 24 times per day
+//
+// After enough data, we get a result at the end (-sck_keep_bits=43):
+//
+// (keep 43 bits) 18 collisions after 2 x 90 days, est 10 days between
+// (1.15292e+19 corrected)
+//
+// If we believe the (pessimistic) simulation and the mathematical
+// extrapolation, we would need to run a billion machines all for 11 billion
+// days to expect a cache key collision. To help verify that our extrapolation
+// ("corrected") is robust, we can make our simulation more precise by
+// increasing the "keep" bits, which takes more running time to get enough
+// collision data:
+//
+// (keep 44 bits) 16 collisions after 5 x 90 days, est 28.125 days between
+// (1.6213e+19 corrected)
+// (keep 45 bits) 15 collisions after 7 x 90 days, est 42 days between
+// (1.21057e+19 corrected)
+// (keep 46 bits) 15 collisions after 17 x 90 days, est 102 days between
+// (1.46997e+19 corrected)
+// (keep 47 bits) 15 collisions after 49 x 90 days, est 294 days between
+// (2.11849e+19 corrected)
+//
+// The extrapolated prediction seems to be within noise (sampling error).
+//
+// With the `-sck_randomize` option, we can see that typical workloads like
+// above have lower collision probability than "random" cache keys (note:
+// offsets still non-randomized) by a modest amount (roughly 2-3x less
+// collision prone than random), which should make us reasonably comfortable
+// even in "degenerate" cases (e.g. repeatedly launch a process to generate
+// one file with SstFileWriter):
+//
+// (rand 43 bits) 22 collisions after 1 x 90 days, est 4.09091 days between
+// (4.7165e+18 corrected)
+//
+// We can see that with more frequent process restarts,
+// -sck_restarts_per_day=5000, which means more all-new session IDs, we get
+// closer to the "random" cache key performance:
+//
+// 15 collisions after 1 x 90 days, est 6 days between (6.91753e+18 corrected)
+//
+// And with less frequent process restarts and re-opens,
+// -sck_restarts_per_day=1 -sck_reopen_nfiles=1000, we get lower collision
+// probability:
+//
+// 18 collisions after 8 x 90 days, est 40 days between (4.61169e+19 corrected)
+//
+// Other tests have been run to validate other conditions behave as expected,
+// never behaving "worse than random" unless we start chopping off structured
+// data.
+//
+// Conclusion: Even in extreme cases, rapidly burning through "all new" IDs
+// that only arise when a new process is started, the chance of any cache key
+// collisions in a giant fleet of machines is negligible. Especially when
+// processes live for hours or days, the chance of a cache key collision is
+// likely more plausibly due to bad hardware than to bad luck in random
+// session ID data. Software defects are surely more likely to cause corruption
+// than both of those.
+//
+// TODO: Nevertheless / regardless, an efficient way to detect (and thus
+// quantify) block cache corruptions, including collisions, should be added.
+OffsetableCacheKey::OffsetableCacheKey(const std::string &db_id,
+ const std::string &db_session_id,
+ uint64_t file_number) {
+ UniqueId64x2 internal_id;
+ Status s = GetSstInternalUniqueId(db_id, db_session_id, file_number,
+ &internal_id, /*force=*/true);
+ assert(s.ok());
+ *this = FromInternalUniqueId(&internal_id);
+}
+
+OffsetableCacheKey OffsetableCacheKey::FromInternalUniqueId(UniqueIdPtr id) {
+ uint64_t session_lower = id.ptr[0];
+ uint64_t file_num_etc = id.ptr[1];
+
+#ifndef NDEBUG
+ bool is_empty = session_lower == 0 && file_num_etc == 0;
+#endif
+
+ // Although DBImpl guarantees (in recent versions) that session_lower is not
+ // zero, that's not entirely sufficient to guarantee that file_num_etc64_ is
+ // not zero (so that the 0 case can be used by CacheKey::CreateUnique*)
+ // However, if we are given an "empty" id as input, then we should produce
+ // "empty" as output.
+ // As a consequence, this function is only bijective assuming
+ // id[0] == 0 only if id[1] == 0.
+ if (session_lower == 0U) {
+ session_lower = file_num_etc;
+ }
+
+ // See comments above for how DownwardInvolution and ReverseBits
+ // make this function invertible under various assumptions.
+ OffsetableCacheKey rv;
+ rv.file_num_etc64_ =
+ DownwardInvolution(session_lower) ^ ReverseBits(file_num_etc);
+ rv.offset_etc64_ = ReverseBits(session_lower);
+
+ // Because of these transformations and needing to allow arbitrary
+ // offset (thus, second 64 bits of cache key might be 0), we need to
+ // make some correction to ensure the first 64 bits is not 0.
+ // Fortunately, the transformation ensures the second 64 bits is not 0
+ // for non-empty base key, so we can swap in the case one is 0 without
+ // breaking bijectivity (assuming condition above).
+ assert(is_empty || rv.offset_etc64_ > 0);
+ if (rv.file_num_etc64_ == 0) {
+ std::swap(rv.file_num_etc64_, rv.offset_etc64_);
+ }
+ assert(is_empty || rv.file_num_etc64_ > 0);
+ return rv;
+}
+
+// Inverse of FromInternalUniqueId (assuming file_num_etc64 == 0 only if
+// offset_etc64 == 0)
+UniqueId64x2 OffsetableCacheKey::ToInternalUniqueId() {
+ uint64_t a = file_num_etc64_;
+ uint64_t b = offset_etc64_;
+ if (b == 0) {
+ std::swap(a, b);
+ }
+ UniqueId64x2 rv;
+ rv[0] = ReverseBits(b);
+ rv[1] = ReverseBits(a ^ DownwardInvolution(rv[0]));
+ return rv;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_key.h b/src/rocksdb/cache/cache_key.h
new file mode 100644
index 000000000..0b93c6bd9
--- /dev/null
+++ b/src/rocksdb/cache/cache_key.h
@@ -0,0 +1,143 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// 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 <cstdint>
+
+#include "rocksdb/rocksdb_namespace.h"
+#include "rocksdb/slice.h"
+#include "table/unique_id_impl.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Cache;
+
+// A standard holder for fixed-size block cache keys (and for related caches).
+// They are created through one of these, each using its own range of values:
+// * CacheKey::CreateUniqueForCacheLifetime
+// * CacheKey::CreateUniqueForProcessLifetime
+// * Default ctor ("empty" cache key)
+// * OffsetableCacheKey->WithOffset
+//
+// The first two use atomic counters to guarantee uniqueness over the given
+// lifetime and the last uses a form of universally unique identifier for
+// uniqueness with very high probabilty (and guaranteed for files generated
+// during a single process lifetime).
+//
+// CacheKeys are currently used by calling AsSlice() to pass as a key to
+// Cache. For performance, the keys are endianness-dependent (though otherwise
+// portable). (Persistable cache entries are not intended to cross platforms.)
+class CacheKey {
+ public:
+ // For convenience, constructs an "empty" cache key that is never returned
+ // by other means.
+ inline CacheKey() : file_num_etc64_(), offset_etc64_() {}
+
+ inline bool IsEmpty() const {
+ return (file_num_etc64_ == 0) & (offset_etc64_ == 0);
+ }
+
+ // Use this cache key as a Slice (byte order is endianness-dependent)
+ inline Slice AsSlice() const {
+ static_assert(sizeof(*this) == 16, "Standardized on 16-byte cache key");
+ assert(!IsEmpty());
+ return Slice(reinterpret_cast<const char *>(this), sizeof(*this));
+ }
+
+ // Create a CacheKey that is unique among others associated with this Cache
+ // instance. Depends on Cache::NewId. This is useful for block cache
+ // "reservations".
+ static CacheKey CreateUniqueForCacheLifetime(Cache *cache);
+
+ // Create a CacheKey that is unique among others for the lifetime of this
+ // process. This is useful for saving in a static data member so that
+ // different DB instances can agree on a cache key for shared entities,
+ // such as for CacheEntryStatsCollector.
+ static CacheKey CreateUniqueForProcessLifetime();
+
+ protected:
+ friend class OffsetableCacheKey;
+ CacheKey(uint64_t file_num_etc64, uint64_t offset_etc64)
+ : file_num_etc64_(file_num_etc64), offset_etc64_(offset_etc64) {}
+ uint64_t file_num_etc64_;
+ uint64_t offset_etc64_;
+};
+
+constexpr uint8_t kCacheKeySize = static_cast<uint8_t>(sizeof(CacheKey));
+
+// A file-specific generator of cache keys, sometimes referred to as the
+// "base" cache key for a file because all the cache keys for various offsets
+// within the file are computed using simple arithmetic. The basis for the
+// general approach is dicussed here: https://github.com/pdillinger/unique_id
+// Heavily related to GetUniqueIdFromTableProperties.
+//
+// If the db_id, db_session_id, and file_number come from the file's table
+// properties, then the keys will be stable across DB::Open/Close, backup/
+// restore, import/export, etc.
+//
+// This class "is a" CacheKey only privately so that it is not misused as
+// a ready-to-use CacheKey.
+class OffsetableCacheKey : private CacheKey {
+ public:
+ // For convenience, constructs an "empty" cache key that should not be used.
+ inline OffsetableCacheKey() : CacheKey() {}
+
+ // Constructs an OffsetableCacheKey with the given information about a file.
+ // This constructor never generates an "empty" base key.
+ OffsetableCacheKey(const std::string &db_id, const std::string &db_session_id,
+ uint64_t file_number);
+
+ // Creates an OffsetableCacheKey from an SST unique ID, so that cache keys
+ // can be derived from DB manifest data before reading the file from
+ // storage--so that every part of the file can potentially go in a persistent
+ // cache.
+ //
+ // Calling GetSstInternalUniqueId() on a db_id, db_session_id, and
+ // file_number and passing the result to this function produces the same
+ // base cache key as feeding those inputs directly to the constructor.
+ //
+ // This is a bijective transformation assuming either id is empty or
+ // lower 64 bits is non-zero:
+ // * Empty (all zeros) input -> empty (all zeros) output
+ // * Lower 64 input is non-zero -> lower 64 output (file_num_etc64_) is
+ // non-zero
+ static OffsetableCacheKey FromInternalUniqueId(UniqueIdPtr id);
+
+ // This is the inverse transformation to the above, assuming either empty
+ // or lower 64 bits (file_num_etc64_) is non-zero. Perhaps only useful for
+ // testing.
+ UniqueId64x2 ToInternalUniqueId();
+
+ inline bool IsEmpty() const {
+ bool result = file_num_etc64_ == 0;
+ assert(!(offset_etc64_ > 0 && result));
+ return result;
+ }
+
+ // Construct a CacheKey for an offset within a file. An offset is not
+ // necessarily a byte offset if a smaller unique identifier of keyable
+ // offsets is used.
+ //
+ // This class was designed to make this hot code extremely fast.
+ inline CacheKey WithOffset(uint64_t offset) const {
+ assert(!IsEmpty());
+ return CacheKey(file_num_etc64_, offset_etc64_ ^ offset);
+ }
+
+ // The "common prefix" is a shared prefix for all the returned CacheKeys.
+ // It is specific to the file but the same for all offsets within the file.
+ static constexpr size_t kCommonPrefixSize = 8;
+ inline Slice CommonPrefixSlice() const {
+ static_assert(sizeof(file_num_etc64_) == kCommonPrefixSize,
+ "8 byte common prefix expected");
+ assert(!IsEmpty());
+ assert(&this->file_num_etc64_ == static_cast<const void *>(this));
+
+ return Slice(reinterpret_cast<const char *>(this), kCommonPrefixSize);
+ }
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_reservation_manager.cc b/src/rocksdb/cache/cache_reservation_manager.cc
new file mode 100644
index 000000000..53dee5d79
--- /dev/null
+++ b/src/rocksdb/cache/cache_reservation_manager.cc
@@ -0,0 +1,185 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "cache/cache_reservation_manager.h"
+
+#include <cassert>
+#include <cstddef>
+#include <cstring>
+#include <memory>
+
+#include "cache/cache_entry_roles.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "table/block_based/reader_common.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+template <CacheEntryRole R>
+CacheReservationManagerImpl<R>::CacheReservationHandle::CacheReservationHandle(
+ std::size_t incremental_memory_used,
+ std::shared_ptr<CacheReservationManagerImpl> cache_res_mgr)
+ : incremental_memory_used_(incremental_memory_used) {
+ assert(cache_res_mgr);
+ cache_res_mgr_ = cache_res_mgr;
+}
+
+template <CacheEntryRole R>
+CacheReservationManagerImpl<
+ R>::CacheReservationHandle::~CacheReservationHandle() {
+ Status s = cache_res_mgr_->ReleaseCacheReservation(incremental_memory_used_);
+ s.PermitUncheckedError();
+}
+
+template <CacheEntryRole R>
+CacheReservationManagerImpl<R>::CacheReservationManagerImpl(
+ std::shared_ptr<Cache> cache, bool delayed_decrease)
+ : delayed_decrease_(delayed_decrease),
+ cache_allocated_size_(0),
+ memory_used_(0) {
+ assert(cache != nullptr);
+ cache_ = cache;
+}
+
+template <CacheEntryRole R>
+CacheReservationManagerImpl<R>::~CacheReservationManagerImpl() {
+ for (auto* handle : dummy_handles_) {
+ cache_->Release(handle, true);
+ }
+}
+
+template <CacheEntryRole R>
+Status CacheReservationManagerImpl<R>::UpdateCacheReservation(
+ std::size_t new_mem_used) {
+ memory_used_ = new_mem_used;
+ std::size_t cur_cache_allocated_size =
+ cache_allocated_size_.load(std::memory_order_relaxed);
+ if (new_mem_used == cur_cache_allocated_size) {
+ return Status::OK();
+ } else if (new_mem_used > cur_cache_allocated_size) {
+ Status s = IncreaseCacheReservation(new_mem_used);
+ return s;
+ } else {
+ // In delayed decrease mode, we don't decrease cache reservation
+ // untill the memory usage is less than 3/4 of what we reserve
+ // in the cache.
+ // We do this because
+ // (1) Dummy entry insertion is expensive in block cache
+ // (2) Delayed releasing previously inserted dummy entries can save such
+ // expensive dummy entry insertion on memory increase in the near future,
+ // which is likely to happen when the memory usage is greater than or equal
+ // to 3/4 of what we reserve
+ if (delayed_decrease_ && new_mem_used >= cur_cache_allocated_size / 4 * 3) {
+ return Status::OK();
+ } else {
+ Status s = DecreaseCacheReservation(new_mem_used);
+ return s;
+ }
+ }
+}
+
+template <CacheEntryRole R>
+Status CacheReservationManagerImpl<R>::MakeCacheReservation(
+ std::size_t incremental_memory_used,
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>* handle) {
+ assert(handle);
+ Status s =
+ UpdateCacheReservation(GetTotalMemoryUsed() + incremental_memory_used);
+ (*handle).reset(new CacheReservationManagerImpl::CacheReservationHandle(
+ incremental_memory_used,
+ std::enable_shared_from_this<
+ CacheReservationManagerImpl<R>>::shared_from_this()));
+ return s;
+}
+
+template <CacheEntryRole R>
+Status CacheReservationManagerImpl<R>::ReleaseCacheReservation(
+ std::size_t incremental_memory_used) {
+ assert(GetTotalMemoryUsed() >= incremental_memory_used);
+ std::size_t updated_total_mem_used =
+ GetTotalMemoryUsed() - incremental_memory_used;
+ Status s = UpdateCacheReservation(updated_total_mem_used);
+ return s;
+}
+
+template <CacheEntryRole R>
+Status CacheReservationManagerImpl<R>::IncreaseCacheReservation(
+ std::size_t new_mem_used) {
+ Status return_status = Status::OK();
+ while (new_mem_used > cache_allocated_size_.load(std::memory_order_relaxed)) {
+ Cache::Handle* handle = nullptr;
+ return_status = cache_->Insert(GetNextCacheKey(), nullptr, kSizeDummyEntry,
+ GetNoopDeleterForRole<R>(), &handle);
+
+ if (return_status != Status::OK()) {
+ return return_status;
+ }
+
+ dummy_handles_.push_back(handle);
+ cache_allocated_size_ += kSizeDummyEntry;
+ }
+ return return_status;
+}
+
+template <CacheEntryRole R>
+Status CacheReservationManagerImpl<R>::DecreaseCacheReservation(
+ std::size_t new_mem_used) {
+ Status return_status = Status::OK();
+
+ // Decrease to the smallest multiple of kSizeDummyEntry that is greater than
+ // or equal to new_mem_used We do addition instead of new_mem_used <=
+ // cache_allocated_size_.load(std::memory_order_relaxed) - kSizeDummyEntry to
+ // avoid underflow of size_t when cache_allocated_size_ = 0
+ while (new_mem_used + kSizeDummyEntry <=
+ cache_allocated_size_.load(std::memory_order_relaxed)) {
+ assert(!dummy_handles_.empty());
+ auto* handle = dummy_handles_.back();
+ cache_->Release(handle, true);
+ dummy_handles_.pop_back();
+ cache_allocated_size_ -= kSizeDummyEntry;
+ }
+ return return_status;
+}
+
+template <CacheEntryRole R>
+std::size_t CacheReservationManagerImpl<R>::GetTotalReservedCacheSize() {
+ return cache_allocated_size_.load(std::memory_order_relaxed);
+}
+
+template <CacheEntryRole R>
+std::size_t CacheReservationManagerImpl<R>::GetTotalMemoryUsed() {
+ return memory_used_;
+}
+
+template <CacheEntryRole R>
+Slice CacheReservationManagerImpl<R>::GetNextCacheKey() {
+ // Calling this function will have the side-effect of changing the
+ // underlying cache_key_ that is shared among other keys generated from this
+ // fucntion. Therefore please make sure the previous keys are saved/copied
+ // before calling this function.
+ cache_key_ = CacheKey::CreateUniqueForCacheLifetime(cache_.get());
+ return cache_key_.AsSlice();
+}
+
+template <CacheEntryRole R>
+Cache::DeleterFn CacheReservationManagerImpl<R>::TEST_GetNoopDeleterForRole() {
+ return GetNoopDeleterForRole<R>();
+}
+
+template class CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>;
+template class CacheReservationManagerImpl<
+ CacheEntryRole::kCompressionDictionaryBuildingBuffer>;
+template class CacheReservationManagerImpl<CacheEntryRole::kFilterConstruction>;
+template class CacheReservationManagerImpl<CacheEntryRole::kMisc>;
+template class CacheReservationManagerImpl<CacheEntryRole::kWriteBuffer>;
+template class CacheReservationManagerImpl<CacheEntryRole::kFileMetadata>;
+template class CacheReservationManagerImpl<CacheEntryRole::kBlobCache>;
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_reservation_manager.h b/src/rocksdb/cache/cache_reservation_manager.h
new file mode 100644
index 000000000..147aaa915
--- /dev/null
+++ b/src/rocksdb/cache/cache_reservation_manager.h
@@ -0,0 +1,316 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <vector>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_key.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+// CacheReservationManager is an interface for reserving cache space for the
+// memory used
+class CacheReservationManager {
+ public:
+ // CacheReservationHandle is for managing the lifetime of a cache reservation
+ // for an incremental amount of memory used (i.e, incremental_memory_used)
+ class CacheReservationHandle {
+ public:
+ virtual ~CacheReservationHandle() {}
+ };
+ virtual ~CacheReservationManager() {}
+ virtual Status UpdateCacheReservation(std::size_t new_memory_used) = 0;
+ // TODO(hx235): replace the usage of
+ // `UpdateCacheReservation(memory_used_delta, increase)` with
+ // `UpdateCacheReservation(new_memory_used)` so that we only have one
+ // `UpdateCacheReservation` function
+ virtual Status UpdateCacheReservation(std::size_t memory_used_delta,
+ bool increase) = 0;
+ virtual Status MakeCacheReservation(
+ std::size_t incremental_memory_used,
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ *handle) = 0;
+ virtual std::size_t GetTotalReservedCacheSize() = 0;
+ virtual std::size_t GetTotalMemoryUsed() = 0;
+};
+
+// CacheReservationManagerImpl implements interface CacheReservationManager
+// for reserving cache space for the memory used by inserting/releasing dummy
+// entries in the cache.
+//
+// This class is NOT thread-safe, except that GetTotalReservedCacheSize()
+// can be called without external synchronization.
+template <CacheEntryRole R>
+class CacheReservationManagerImpl
+ : public CacheReservationManager,
+ public std::enable_shared_from_this<CacheReservationManagerImpl<R>> {
+ public:
+ class CacheReservationHandle
+ : public CacheReservationManager::CacheReservationHandle {
+ public:
+ CacheReservationHandle(
+ std::size_t incremental_memory_used,
+ std::shared_ptr<CacheReservationManagerImpl> cache_res_mgr);
+ ~CacheReservationHandle() override;
+
+ private:
+ std::size_t incremental_memory_used_;
+ std::shared_ptr<CacheReservationManagerImpl> cache_res_mgr_;
+ };
+
+ // Construct a CacheReservationManagerImpl
+ // @param cache The cache where dummy entries are inserted and released for
+ // reserving cache space
+ // @param delayed_decrease If set true, then dummy entries won't be released
+ // immediately when memory usage decreases.
+ // Instead, it will be released when the memory usage
+ // decreases to 3/4 of what we have reserved so far.
+ // This is for saving some future dummy entry
+ // insertion when memory usage increases are likely to
+ // happen in the near future.
+ //
+ // REQUIRED: cache is not nullptr
+ explicit CacheReservationManagerImpl(std::shared_ptr<Cache> cache,
+ bool delayed_decrease = false);
+
+ // no copy constructor, copy assignment, move constructor, move assignment
+ CacheReservationManagerImpl(const CacheReservationManagerImpl &) = delete;
+ CacheReservationManagerImpl &operator=(const CacheReservationManagerImpl &) =
+ delete;
+ CacheReservationManagerImpl(CacheReservationManagerImpl &&) = delete;
+ CacheReservationManagerImpl &operator=(CacheReservationManagerImpl &&) =
+ delete;
+
+ ~CacheReservationManagerImpl() override;
+
+ // One of the two ways of reserving/releasing cache space,
+ // see MakeCacheReservation() for the other.
+ //
+ // Use ONLY one of these two ways to prevent unexpected behavior.
+ //
+ // Insert and release dummy entries in the cache to
+ // match the size of total dummy entries with the least multiple of
+ // kSizeDummyEntry greater than or equal to new_mem_used
+ //
+ // Insert dummy entries if new_memory_used > cache_allocated_size_;
+ //
+ // Release dummy entries if new_memory_used < cache_allocated_size_
+ // (and new_memory_used < cache_allocated_size_ * 3/4
+ // when delayed_decrease is set true);
+ //
+ // Keey dummy entries the same if (1) new_memory_used == cache_allocated_size_
+ // or (2) new_memory_used is in the interval of
+ // [cache_allocated_size_ * 3/4, cache_allocated_size) when delayed_decrease
+ // is set true.
+ //
+ // @param new_memory_used The number of bytes used by new memory
+ // The most recent new_memoy_used passed in will be returned
+ // in GetTotalMemoryUsed() even when the call return non-ok status.
+ //
+ // Since the class is NOT thread-safe, external synchronization on the
+ // order of calling UpdateCacheReservation() is needed if you want
+ // GetTotalMemoryUsed() indeed returns the latest memory used.
+ //
+ // @return On inserting dummy entries, it returns Status::OK() if all dummy
+ // entry insertions succeed.
+ // Otherwise, it returns the first non-ok status;
+ // On releasing dummy entries, it always returns Status::OK().
+ // On keeping dummy entries the same, it always returns Status::OK().
+ Status UpdateCacheReservation(std::size_t new_memory_used) override;
+
+ Status UpdateCacheReservation(std::size_t /* memory_used_delta */,
+ bool /* increase */) override {
+ return Status::NotSupported();
+ }
+
+ // One of the two ways of reserving cache space and releasing is done through
+ // destruction of CacheReservationHandle.
+ // See UpdateCacheReservation() for the other way.
+ //
+ // Use ONLY one of these two ways to prevent unexpected behavior.
+ //
+ // Insert dummy entries in the cache for the incremental memory usage
+ // to match the size of total dummy entries with the least multiple of
+ // kSizeDummyEntry greater than or equal to the total memory used.
+ //
+ // A CacheReservationHandle is returned as an output parameter.
+ // The reserved dummy entries are automatically released on the destruction of
+ // this handle, which achieves better RAII per cache reservation.
+ //
+ // WARNING: Deallocate all the handles of the CacheReservationManager object
+ // before deallocating the object to prevent unexpected behavior.
+ //
+ // @param incremental_memory_used The number of bytes increased in memory
+ // usage.
+ //
+ // Calling GetTotalMemoryUsed() afterward will return the total memory
+ // increased by this number, even when calling MakeCacheReservation()
+ // returns non-ok status.
+ //
+ // Since the class is NOT thread-safe, external synchronization in
+ // calling MakeCacheReservation() is needed if you want
+ // GetTotalMemoryUsed() indeed returns the latest memory used.
+ //
+ // @param handle An pointer to std::unique_ptr<CacheReservationHandle> that
+ // manages the lifetime of the cache reservation represented by the
+ // handle.
+ //
+ // @return It returns Status::OK() if all dummy
+ // entry insertions succeed.
+ // Otherwise, it returns the first non-ok status;
+ //
+ // REQUIRES: handle != nullptr
+ Status MakeCacheReservation(
+ std::size_t incremental_memory_used,
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle> *handle)
+ override;
+
+ // Return the size of the cache (which is a multiple of kSizeDummyEntry)
+ // successfully reserved by calling UpdateCacheReservation().
+ //
+ // When UpdateCacheReservation() returns non-ok status,
+ // calling GetTotalReservedCacheSize() after that might return a slightly
+ // smaller number than the actual reserved cache size due to
+ // the returned number will always be a multiple of kSizeDummyEntry
+ // and cache full might happen in the middle of inserting a dummy entry.
+ std::size_t GetTotalReservedCacheSize() override;
+
+ // Return the latest total memory used indicated by the most recent call of
+ // UpdateCacheReservation(std::size_t new_memory_used);
+ std::size_t GetTotalMemoryUsed() override;
+
+ static constexpr std::size_t GetDummyEntrySize() { return kSizeDummyEntry; }
+
+ // For testing only - it is to help ensure the NoopDeleterForRole<R>
+ // accessed from CacheReservationManagerImpl and the one accessed from the
+ // test are from the same translation units
+ static Cache::DeleterFn TEST_GetNoopDeleterForRole();
+
+ private:
+ static constexpr std::size_t kSizeDummyEntry = 256 * 1024;
+
+ Slice GetNextCacheKey();
+
+ Status ReleaseCacheReservation(std::size_t incremental_memory_used);
+ Status IncreaseCacheReservation(std::size_t new_mem_used);
+ Status DecreaseCacheReservation(std::size_t new_mem_used);
+
+ std::shared_ptr<Cache> cache_;
+ bool delayed_decrease_;
+ std::atomic<std::size_t> cache_allocated_size_;
+ std::size_t memory_used_;
+ std::vector<Cache::Handle *> dummy_handles_;
+ CacheKey cache_key_;
+};
+
+class ConcurrentCacheReservationManager
+ : public CacheReservationManager,
+ public std::enable_shared_from_this<ConcurrentCacheReservationManager> {
+ public:
+ class CacheReservationHandle
+ : public CacheReservationManager::CacheReservationHandle {
+ public:
+ CacheReservationHandle(
+ std::shared_ptr<ConcurrentCacheReservationManager> cache_res_mgr,
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ cache_res_handle) {
+ assert(cache_res_mgr && cache_res_handle);
+ cache_res_mgr_ = cache_res_mgr;
+ cache_res_handle_ = std::move(cache_res_handle);
+ }
+
+ ~CacheReservationHandle() override {
+ std::lock_guard<std::mutex> lock(cache_res_mgr_->cache_res_mgr_mu_);
+ cache_res_handle_.reset();
+ }
+
+ private:
+ std::shared_ptr<ConcurrentCacheReservationManager> cache_res_mgr_;
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ cache_res_handle_;
+ };
+
+ explicit ConcurrentCacheReservationManager(
+ std::shared_ptr<CacheReservationManager> cache_res_mgr) {
+ cache_res_mgr_ = std::move(cache_res_mgr);
+ }
+ ConcurrentCacheReservationManager(const ConcurrentCacheReservationManager &) =
+ delete;
+ ConcurrentCacheReservationManager &operator=(
+ const ConcurrentCacheReservationManager &) = delete;
+ ConcurrentCacheReservationManager(ConcurrentCacheReservationManager &&) =
+ delete;
+ ConcurrentCacheReservationManager &operator=(
+ ConcurrentCacheReservationManager &&) = delete;
+
+ ~ConcurrentCacheReservationManager() override {}
+
+ inline Status UpdateCacheReservation(std::size_t new_memory_used) override {
+ std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
+ return cache_res_mgr_->UpdateCacheReservation(new_memory_used);
+ }
+
+ inline Status UpdateCacheReservation(std::size_t memory_used_delta,
+ bool increase) override {
+ std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
+ std::size_t total_mem_used = cache_res_mgr_->GetTotalMemoryUsed();
+ Status s;
+ if (!increase) {
+ assert(total_mem_used >= memory_used_delta);
+ s = cache_res_mgr_->UpdateCacheReservation(total_mem_used -
+ memory_used_delta);
+ } else {
+ s = cache_res_mgr_->UpdateCacheReservation(total_mem_used +
+ memory_used_delta);
+ }
+ return s;
+ }
+
+ inline Status MakeCacheReservation(
+ std::size_t incremental_memory_used,
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle> *handle)
+ override {
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ wrapped_handle;
+ Status s;
+ {
+ std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
+ s = cache_res_mgr_->MakeCacheReservation(incremental_memory_used,
+ &wrapped_handle);
+ }
+ (*handle).reset(
+ new ConcurrentCacheReservationManager::CacheReservationHandle(
+ std::enable_shared_from_this<
+ ConcurrentCacheReservationManager>::shared_from_this(),
+ std::move(wrapped_handle)));
+ return s;
+ }
+ inline std::size_t GetTotalReservedCacheSize() override {
+ return cache_res_mgr_->GetTotalReservedCacheSize();
+ }
+ inline std::size_t GetTotalMemoryUsed() override {
+ std::lock_guard<std::mutex> lock(cache_res_mgr_mu_);
+ return cache_res_mgr_->GetTotalMemoryUsed();
+ }
+
+ private:
+ std::mutex cache_res_mgr_mu_;
+ std::shared_ptr<CacheReservationManager> cache_res_mgr_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/cache_reservation_manager_test.cc b/src/rocksdb/cache/cache_reservation_manager_test.cc
new file mode 100644
index 000000000..2a0c318e0
--- /dev/null
+++ b/src/rocksdb/cache/cache_reservation_manager_test.cc
@@ -0,0 +1,469 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "cache/cache_reservation_manager.h"
+
+#include <cstddef>
+#include <cstring>
+#include <memory>
+
+#include "cache/cache_entry_roles.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/slice.h"
+#include "test_util/testharness.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+class CacheReservationManagerTest : public ::testing::Test {
+ protected:
+ static constexpr std::size_t kSizeDummyEntry =
+ CacheReservationManagerImpl<CacheEntryRole::kMisc>::GetDummyEntrySize();
+ static constexpr std::size_t kCacheCapacity = 4096 * kSizeDummyEntry;
+ static constexpr int kNumShardBits = 0; // 2^0 shard
+ static constexpr std::size_t kMetaDataChargeOverhead = 10000;
+
+ std::shared_ptr<Cache> cache = NewLRUCache(kCacheCapacity, kNumShardBits);
+ std::shared_ptr<CacheReservationManager> test_cache_rev_mng;
+
+ CacheReservationManagerTest() {
+ test_cache_rev_mng =
+ std::make_shared<CacheReservationManagerImpl<CacheEntryRole::kMisc>>(
+ cache);
+ }
+};
+
+TEST_F(CacheReservationManagerTest, GenerateCacheKey) {
+ std::size_t new_mem_used = 1 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry);
+ ASSERT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ // Next unique Cache key
+ CacheKey ckey = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+ // Get to the underlying values
+ uint64_t* ckey_data = reinterpret_cast<uint64_t*>(&ckey);
+ // Back it up to the one used by CRM (using CacheKey implementation details)
+ ckey_data[1]--;
+
+ // Specific key (subject to implementation details)
+ EXPECT_EQ(ckey_data[0], 0);
+ EXPECT_EQ(ckey_data[1], 2);
+
+ Cache::Handle* handle = cache->Lookup(ckey.AsSlice());
+ EXPECT_NE(handle, nullptr)
+ << "Failed to generate the cache key for the dummy entry correctly";
+ // Clean up the returned handle from Lookup() to prevent memory leak
+ cache->Release(handle);
+}
+
+TEST_F(CacheReservationManagerTest, KeepCacheReservationTheSame) {
+ std::size_t new_mem_used = 1 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry);
+ ASSERT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used);
+ std::size_t initial_pinned_usage = cache->GetPinnedUsage();
+ ASSERT_GE(initial_pinned_usage, 1 * kSizeDummyEntry);
+ ASSERT_LT(initial_pinned_usage,
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to keep cache reservation the same when new_mem_used equals "
+ "to current cache reservation";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly when new_mem_used equals to current "
+ "cache reservation";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly when new_mem_used "
+ "equals to current cache reservation";
+ EXPECT_EQ(cache->GetPinnedUsage(), initial_pinned_usage)
+ << "Failed to keep underlying dummy entries the same when new_mem_used "
+ "equals to current cache reservation";
+}
+
+TEST_F(CacheReservationManagerTest,
+ IncreaseCacheReservationByMultiplesOfDummyEntrySize) {
+ std::size_t new_mem_used = 2 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to increase cache reservation correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 2 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation increase correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 2 * kSizeDummyEntry)
+ << "Failed to increase underlying dummy entries in cache correctly";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 2 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to increase underlying dummy entries in cache correctly";
+}
+
+TEST_F(CacheReservationManagerTest,
+ IncreaseCacheReservationNotByMultiplesOfDummyEntrySize) {
+ std::size_t new_mem_used = 2 * kSizeDummyEntry + kSizeDummyEntry / 2;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to increase cache reservation correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 3 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation increase correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 3 * kSizeDummyEntry)
+ << "Failed to increase underlying dummy entries in cache correctly";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 3 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to increase underlying dummy entries in cache correctly";
+}
+
+TEST(CacheReservationManagerIncreaseReservcationOnFullCacheTest,
+ IncreaseCacheReservationOnFullCache) {
+ ;
+ constexpr std::size_t kSizeDummyEntry =
+ CacheReservationManagerImpl<CacheEntryRole::kMisc>::GetDummyEntrySize();
+ constexpr std::size_t kSmallCacheCapacity = 4 * kSizeDummyEntry;
+ constexpr std::size_t kBigCacheCapacity = 4096 * kSizeDummyEntry;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+
+ LRUCacheOptions lo;
+ lo.capacity = kSmallCacheCapacity;
+ lo.num_shard_bits = 0; // 2^0 shard
+ lo.strict_capacity_limit = true;
+ std::shared_ptr<Cache> cache = NewLRUCache(lo);
+ std::shared_ptr<CacheReservationManager> test_cache_rev_mng =
+ std::make_shared<CacheReservationManagerImpl<CacheEntryRole::kMisc>>(
+ cache);
+
+ std::size_t new_mem_used = kSmallCacheCapacity + 1;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::MemoryLimit())
+ << "Failed to return status to indicate failure of dummy entry insertion "
+ "during cache reservation on full cache";
+ EXPECT_GE(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly before cache resevation failure happens "
+ "due to full cache";
+ EXPECT_LE(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ kSmallCacheCapacity)
+ << "Failed to bookkeep correctly (i.e, bookkeep only successful dummy "
+ "entry insertions) when encountering cache resevation failure due to "
+ "full cache";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry)
+ << "Failed to insert underlying dummy entries correctly when "
+ "encountering cache resevation failure due to full cache";
+ EXPECT_LE(cache->GetPinnedUsage(), kSmallCacheCapacity)
+ << "Failed to insert underlying dummy entries correctly when "
+ "encountering cache resevation failure due to full cache";
+
+ new_mem_used = kSmallCacheCapacity / 2; // 2 dummy entries
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to decrease cache reservation after encountering cache "
+ "reservation failure due to full cache";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 2 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation decrease correctly after "
+ "encountering cache reservation due to full cache";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 2 * kSizeDummyEntry)
+ << "Failed to release underlying dummy entries correctly on cache "
+ "reservation decrease after encountering cache resevation failure due "
+ "to full cache";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 2 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to release underlying dummy entries correctly on cache "
+ "reservation decrease after encountering cache resevation failure due "
+ "to full cache";
+
+ // Create cache full again for subsequent tests
+ new_mem_used = kSmallCacheCapacity + 1;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::MemoryLimit())
+ << "Failed to return status to indicate failure of dummy entry insertion "
+ "during cache reservation on full cache";
+ EXPECT_GE(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly before cache resevation failure happens "
+ "due to full cache";
+ EXPECT_LE(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ kSmallCacheCapacity)
+ << "Failed to bookkeep correctly (i.e, bookkeep only successful dummy "
+ "entry insertions) when encountering cache resevation failure due to "
+ "full cache";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry)
+ << "Failed to insert underlying dummy entries correctly when "
+ "encountering cache resevation failure due to full cache";
+ EXPECT_LE(cache->GetPinnedUsage(), kSmallCacheCapacity)
+ << "Failed to insert underlying dummy entries correctly when "
+ "encountering cache resevation failure due to full cache";
+
+ // Increase cache capacity so the previously failed insertion can fully
+ // succeed
+ cache->SetCapacity(kBigCacheCapacity);
+ new_mem_used = kSmallCacheCapacity + 1;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to increase cache reservation after increasing cache capacity "
+ "and mitigating cache full error";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 5 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation increase correctly after "
+ "increasing cache capacity and mitigating cache full error";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 5 * kSizeDummyEntry)
+ << "Failed to insert underlying dummy entries correctly after increasing "
+ "cache capacity and mitigating cache full error";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 5 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to insert underlying dummy entries correctly after increasing "
+ "cache capacity and mitigating cache full error";
+}
+
+TEST_F(CacheReservationManagerTest,
+ DecreaseCacheReservationByMultiplesOfDummyEntrySize) {
+ std::size_t new_mem_used = 2 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 2 * kSizeDummyEntry);
+ ASSERT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used);
+ ASSERT_GE(cache->GetPinnedUsage(), 2 * kSizeDummyEntry);
+ ASSERT_LT(cache->GetPinnedUsage(),
+ 2 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ new_mem_used = 1 * kSizeDummyEntry;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to decrease cache reservation correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation decrease correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry)
+ << "Failed to decrease underlying dummy entries in cache correctly";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to decrease underlying dummy entries in cache correctly";
+}
+
+TEST_F(CacheReservationManagerTest,
+ DecreaseCacheReservationNotByMultiplesOfDummyEntrySize) {
+ std::size_t new_mem_used = 2 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 2 * kSizeDummyEntry);
+ ASSERT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used);
+ ASSERT_GE(cache->GetPinnedUsage(), 2 * kSizeDummyEntry);
+ ASSERT_LT(cache->GetPinnedUsage(),
+ 2 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ new_mem_used = kSizeDummyEntry / 2;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to decrease cache reservation correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 1 * kSizeDummyEntry)
+ << "Failed to bookkeep cache reservation decrease correctly";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry)
+ << "Failed to decrease underlying dummy entries in cache correctly";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to decrease underlying dummy entries in cache correctly";
+}
+
+TEST(CacheReservationManagerWithDelayedDecreaseTest,
+ DecreaseCacheReservationWithDelayedDecrease) {
+ constexpr std::size_t kSizeDummyEntry =
+ CacheReservationManagerImpl<CacheEntryRole::kMisc>::GetDummyEntrySize();
+ constexpr std::size_t kCacheCapacity = 4096 * kSizeDummyEntry;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+
+ LRUCacheOptions lo;
+ lo.capacity = kCacheCapacity;
+ lo.num_shard_bits = 0;
+ std::shared_ptr<Cache> cache = NewLRUCache(lo);
+ std::shared_ptr<CacheReservationManager> test_cache_rev_mng =
+ std::make_shared<CacheReservationManagerImpl<CacheEntryRole::kMisc>>(
+ cache, true /* delayed_decrease */);
+
+ std::size_t new_mem_used = 8 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 8 * kSizeDummyEntry);
+ ASSERT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used);
+ std::size_t initial_pinned_usage = cache->GetPinnedUsage();
+ ASSERT_GE(initial_pinned_usage, 8 * kSizeDummyEntry);
+ ASSERT_LT(initial_pinned_usage,
+ 8 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ new_mem_used = 6 * kSizeDummyEntry;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK()) << "Failed to delay decreasing cache reservation";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 8 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly when delaying cache reservation "
+ "decrease";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_EQ(cache->GetPinnedUsage(), initial_pinned_usage)
+ << "Failed to delay decreasing underlying dummy entries in cache";
+
+ new_mem_used = 7 * kSizeDummyEntry;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK()) << "Failed to delay decreasing cache reservation";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 8 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly when delaying cache reservation "
+ "decrease";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_EQ(cache->GetPinnedUsage(), initial_pinned_usage)
+ << "Failed to delay decreasing underlying dummy entries in cache";
+
+ new_mem_used = 6 * kSizeDummyEntry - 1;
+ s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ EXPECT_EQ(s, Status::OK())
+ << "Failed to decrease cache reservation correctly when new_mem_used < "
+ "GetTotalReservedCacheSize() * 3 / 4 on delayed decrease mode";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(),
+ 6 * kSizeDummyEntry)
+ << "Failed to bookkeep correctly when new_mem_used < "
+ "GetTotalReservedCacheSize() * 3 / 4 on delayed decrease mode";
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), new_mem_used)
+ << "Failed to bookkeep the used memory correctly";
+ EXPECT_GE(cache->GetPinnedUsage(), 6 * kSizeDummyEntry)
+ << "Failed to decrease underlying dummy entries in cache when "
+ "new_mem_used < GetTotalReservedCacheSize() * 3 / 4 on delayed "
+ "decrease mode";
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 6 * kSizeDummyEntry + kMetaDataChargeOverhead)
+ << "Failed to decrease underlying dummy entries in cache when "
+ "new_mem_used < GetTotalReservedCacheSize() * 3 / 4 on delayed "
+ "decrease mode";
+}
+
+TEST(CacheReservationManagerDestructorTest,
+ ReleaseRemainingDummyEntriesOnDestruction) {
+ constexpr std::size_t kSizeDummyEntry =
+ CacheReservationManagerImpl<CacheEntryRole::kMisc>::GetDummyEntrySize();
+ constexpr std::size_t kCacheCapacity = 4096 * kSizeDummyEntry;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+
+ LRUCacheOptions lo;
+ lo.capacity = kCacheCapacity;
+ lo.num_shard_bits = 0;
+ std::shared_ptr<Cache> cache = NewLRUCache(lo);
+ {
+ std::shared_ptr<CacheReservationManager> test_cache_rev_mng =
+ std::make_shared<CacheReservationManagerImpl<CacheEntryRole::kMisc>>(
+ cache);
+ std::size_t new_mem_used = 1 * kSizeDummyEntry;
+ Status s = test_cache_rev_mng->UpdateCacheReservation(new_mem_used);
+ ASSERT_EQ(s, Status::OK());
+ ASSERT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry);
+ ASSERT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead);
+ }
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry)
+ << "Failed to release remaining underlying dummy entries in cache in "
+ "CacheReservationManager's destructor";
+}
+
+TEST(CacheReservationHandleTest, HandleTest) {
+ constexpr std::size_t kOneGigabyte = 1024 * 1024 * 1024;
+ constexpr std::size_t kSizeDummyEntry = 256 * 1024;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+
+ LRUCacheOptions lo;
+ lo.capacity = kOneGigabyte;
+ lo.num_shard_bits = 0;
+ std::shared_ptr<Cache> cache = NewLRUCache(lo);
+
+ std::shared_ptr<CacheReservationManager> test_cache_rev_mng(
+ std::make_shared<CacheReservationManagerImpl<CacheEntryRole::kMisc>>(
+ cache));
+
+ std::size_t mem_used = 0;
+ const std::size_t incremental_mem_used_handle_1 = 1 * kSizeDummyEntry;
+ const std::size_t incremental_mem_used_handle_2 = 2 * kSizeDummyEntry;
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle> handle_1,
+ handle_2;
+
+ // To test consecutive CacheReservationManager::MakeCacheReservation works
+ // correctly in terms of returning the handle as well as updating cache
+ // reservation and the latest total memory used
+ Status s = test_cache_rev_mng->MakeCacheReservation(
+ incremental_mem_used_handle_1, &handle_1);
+ mem_used = mem_used + incremental_mem_used_handle_1;
+ ASSERT_EQ(s, Status::OK());
+ EXPECT_TRUE(handle_1 != nullptr);
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(), mem_used);
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), mem_used);
+ EXPECT_GE(cache->GetPinnedUsage(), mem_used);
+ EXPECT_LT(cache->GetPinnedUsage(), mem_used + kMetaDataChargeOverhead);
+
+ s = test_cache_rev_mng->MakeCacheReservation(incremental_mem_used_handle_2,
+ &handle_2);
+ mem_used = mem_used + incremental_mem_used_handle_2;
+ ASSERT_EQ(s, Status::OK());
+ EXPECT_TRUE(handle_2 != nullptr);
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(), mem_used);
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), mem_used);
+ EXPECT_GE(cache->GetPinnedUsage(), mem_used);
+ EXPECT_LT(cache->GetPinnedUsage(), mem_used + kMetaDataChargeOverhead);
+
+ // To test
+ // CacheReservationManager::CacheReservationHandle::~CacheReservationHandle()
+ // works correctly in releasing the cache reserved for the handle
+ handle_1.reset();
+ EXPECT_TRUE(handle_1 == nullptr);
+ mem_used = mem_used - incremental_mem_used_handle_1;
+ EXPECT_EQ(test_cache_rev_mng->GetTotalReservedCacheSize(), mem_used);
+ EXPECT_EQ(test_cache_rev_mng->GetTotalMemoryUsed(), mem_used);
+ EXPECT_GE(cache->GetPinnedUsage(), mem_used);
+ EXPECT_LT(cache->GetPinnedUsage(), mem_used + kMetaDataChargeOverhead);
+
+ // To test the actual CacheReservationManager object won't be deallocated
+ // as long as there remain handles pointing to it.
+ // We strongly recommend deallocating CacheReservationManager object only
+ // after all its handles are deallocated to keep things easy to reasonate
+ test_cache_rev_mng.reset();
+ EXPECT_GE(cache->GetPinnedUsage(), mem_used);
+ EXPECT_LT(cache->GetPinnedUsage(), mem_used + kMetaDataChargeOverhead);
+
+ handle_2.reset();
+ // The CacheReservationManager object is now deallocated since all the handles
+ // and its original pointer is gone
+ mem_used = mem_used - incremental_mem_used_handle_2;
+ EXPECT_EQ(mem_used, 0);
+ EXPECT_EQ(cache->GetPinnedUsage(), mem_used);
+}
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/cache/cache_test.cc b/src/rocksdb/cache/cache_test.cc
new file mode 100644
index 000000000..212d65d96
--- /dev/null
+++ b/src/rocksdb/cache/cache_test.cc
@@ -0,0 +1,1037 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "rocksdb/cache.h"
+
+#include <forward_list>
+#include <functional>
+#include <iostream>
+#include <string>
+#include <vector>
+
+#include "cache/lru_cache.h"
+#include "port/stack_trace.h"
+#include "test_util/testharness.h"
+#include "util/coding.h"
+#include "util/string_util.h"
+
+// HyperClockCache only supports 16-byte keys, so some of the tests
+// originally written for LRUCache do not work on the other caches.
+// Those tests were adapted to use 16-byte keys. We kept the original ones.
+// TODO: Remove the original tests if they ever become unused.
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+// Conversions between numeric keys/values and the types expected by Cache.
+std::string EncodeKey16Bytes(int k) {
+ std::string result;
+ PutFixed32(&result, k);
+ result.append(std::string(12, 'a')); // Because we need a 16B output, we
+ // add a 12-byte padding.
+ return result;
+}
+
+int DecodeKey16Bytes(const Slice& k) {
+ assert(k.size() == 16);
+ return DecodeFixed32(k.data()); // Decodes only the first 4 bytes of k.
+}
+
+std::string EncodeKey32Bits(int k) {
+ std::string result;
+ PutFixed32(&result, k);
+ return result;
+}
+
+int DecodeKey32Bits(const Slice& k) {
+ assert(k.size() == 4);
+ return DecodeFixed32(k.data());
+}
+
+void* EncodeValue(uintptr_t v) { return reinterpret_cast<void*>(v); }
+
+int DecodeValue(void* v) {
+ return static_cast<int>(reinterpret_cast<uintptr_t>(v));
+}
+
+void DumbDeleter(const Slice& /*key*/, void* /*value*/) {}
+
+void EraseDeleter1(const Slice& /*key*/, void* value) {
+ Cache* cache = reinterpret_cast<Cache*>(value);
+ cache->Erase("foo");
+}
+
+void EraseDeleter2(const Slice& /*key*/, void* value) {
+ Cache* cache = reinterpret_cast<Cache*>(value);
+ cache->Erase(EncodeKey16Bytes(1234));
+}
+
+const std::string kLRU = "lru";
+const std::string kHyperClock = "hyper_clock";
+
+} // anonymous namespace
+
+class CacheTest : public testing::TestWithParam<std::string> {
+ public:
+ static CacheTest* current_;
+ static std::string type_;
+
+ static void Deleter(const Slice& key, void* v) {
+ if (type_ == kHyperClock) {
+ current_->deleted_keys_.push_back(DecodeKey16Bytes(key));
+ } else {
+ current_->deleted_keys_.push_back(DecodeKey32Bits(key));
+ }
+ current_->deleted_values_.push_back(DecodeValue(v));
+ }
+
+ static const int kCacheSize = 1000;
+ static const int kNumShardBits = 4;
+
+ static const int kCacheSize2 = 100;
+ static const int kNumShardBits2 = 2;
+
+ std::vector<int> deleted_keys_;
+ std::vector<int> deleted_values_;
+ std::shared_ptr<Cache> cache_;
+ std::shared_ptr<Cache> cache2_;
+
+ size_t estimated_value_size_ = 1;
+
+ CacheTest()
+ : cache_(NewCache(kCacheSize, kNumShardBits, false)),
+ cache2_(NewCache(kCacheSize2, kNumShardBits2, false)) {
+ current_ = this;
+ type_ = GetParam();
+ }
+
+ ~CacheTest() override {}
+
+ std::shared_ptr<Cache> 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<Cache> 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
+ // LRUCache doesn't, so the encoding depends on the cache type.
+ std::string EncodeKey(int k) {
+ auto type = GetParam();
+ if (type == kHyperClock) {
+ return EncodeKey16Bytes(k);
+ } else {
+ return EncodeKey32Bits(k);
+ }
+ }
+
+ int DecodeKey(const Slice& k) {
+ auto type = GetParam();
+ if (type == kHyperClock) {
+ return DecodeKey16Bytes(k);
+ } else {
+ return DecodeKey32Bits(k);
+ }
+ }
+
+ int Lookup(std::shared_ptr<Cache> cache, int key) {
+ Cache::Handle* handle = cache->Lookup(EncodeKey(key));
+ const int r = (handle == nullptr) ? -1 : DecodeValue(cache->Value(handle));
+ if (handle != nullptr) {
+ cache->Release(handle);
+ }
+ return r;
+ }
+
+ void Insert(std::shared_ptr<Cache> cache, int key, int value,
+ int charge = 1) {
+ EXPECT_OK(cache->Insert(EncodeKey(key), EncodeValue(value), charge,
+ &CacheTest::Deleter));
+ }
+
+ void Erase(std::shared_ptr<Cache> cache, int key) {
+ cache->Erase(EncodeKey(key));
+ }
+
+ int Lookup(int key) { return Lookup(cache_, key); }
+
+ void Insert(int key, int value, int charge = 1) {
+ Insert(cache_, key, value, charge);
+ }
+
+ void Erase(int key) { Erase(cache_, key); }
+
+ int Lookup2(int key) { return Lookup(cache2_, key); }
+
+ void Insert2(int key, int value, int charge = 1) {
+ Insert(cache2_, key, value, charge);
+ }
+
+ void Erase2(int key) { Erase(cache2_, key); }
+};
+
+CacheTest* CacheTest::current_;
+std::string CacheTest::type_;
+
+class LRUCacheTest : public CacheTest {};
+
+TEST_P(CacheTest, UsageTest) {
+ auto type = GetParam();
+
+ // cache is std::shared_ptr and will be automatically cleaned up.
+ const size_t kCapacity = 100000;
+ auto cache = NewCache(kCapacity, 8, false, kDontChargeCacheMetadata);
+ auto precise_cache = NewCache(kCapacity, 0, false, kFullChargeCacheMetadata);
+ ASSERT_EQ(0, cache->GetUsage());
+ size_t baseline_meta_usage = precise_cache->GetUsage();
+ if (type != kHyperClock) {
+ ASSERT_EQ(0, baseline_meta_usage);
+ }
+
+ size_t usage = 0;
+ char value[10] = "abcdef";
+ // make sure everything will be cached
+ for (int i = 1; i < 100; ++i) {
+ std::string key;
+ if (type == kLRU) {
+ key = std::string(i, 'a');
+ } else {
+ key = EncodeKey(i);
+ }
+ auto kv_size = key.size() + 5;
+ ASSERT_OK(cache->Insert(key, reinterpret_cast<void*>(value), kv_size,
+ DumbDeleter));
+ ASSERT_OK(precise_cache->Insert(key, reinterpret_cast<void*>(value),
+ kv_size, DumbDeleter));
+ usage += kv_size;
+ ASSERT_EQ(usage, cache->GetUsage());
+ if (type == kHyperClock) {
+ ASSERT_EQ(baseline_meta_usage + usage, precise_cache->GetUsage());
+ } else {
+ ASSERT_LT(usage, precise_cache->GetUsage());
+ }
+ }
+
+ cache->EraseUnRefEntries();
+ precise_cache->EraseUnRefEntries();
+ ASSERT_EQ(0, cache->GetUsage());
+ ASSERT_EQ(baseline_meta_usage, precise_cache->GetUsage());
+
+ // make sure the cache will be overloaded
+ for (size_t i = 1; i < kCapacity; ++i) {
+ std::string key;
+ if (type == kLRU) {
+ key = std::to_string(i);
+ } else {
+ key = EncodeKey(static_cast<int>(1000 + i));
+ }
+ ASSERT_OK(cache->Insert(key, reinterpret_cast<void*>(value), key.size() + 5,
+ DumbDeleter));
+ ASSERT_OK(precise_cache->Insert(key, reinterpret_cast<void*>(value),
+ key.size() + 5, DumbDeleter));
+ }
+
+ // the usage should be close to the capacity
+ ASSERT_GT(kCapacity, cache->GetUsage());
+ ASSERT_GT(kCapacity, precise_cache->GetUsage());
+ ASSERT_LT(kCapacity * 0.95, cache->GetUsage());
+ if (type != kHyperClock) {
+ ASSERT_LT(kCapacity * 0.95, precise_cache->GetUsage());
+ } else {
+ // estimated value size of 1 is weird for clock cache, because
+ // almost all of the capacity will be used for metadata, and due to only
+ // using power of 2 table sizes, we might hit strict occupancy limit
+ // before hitting capacity limit.
+ ASSERT_LT(kCapacity * 0.80, precise_cache->GetUsage());
+ }
+}
+
+// TODO: This test takes longer than expected on ClockCache. This is
+// because the values size estimate at construction is too sloppy.
+// Fix this.
+// Why is it so slow? The cache is constructed with an estimate of 1, but
+// then the charge is claimed to be 21. This will cause the hash table
+// to be extremely sparse, which in turn means clock needs to scan too
+// many slots to find victims.
+TEST_P(CacheTest, PinnedUsageTest) {
+ auto type = GetParam();
+
+ // cache is std::shared_ptr and will be automatically cleaned up.
+ const size_t kCapacity = 200000;
+ auto cache = NewCache(kCapacity, 8, false, kDontChargeCacheMetadata);
+ auto precise_cache = NewCache(kCapacity, 8, false, kFullChargeCacheMetadata);
+ size_t baseline_meta_usage = precise_cache->GetUsage();
+ if (type != kHyperClock) {
+ ASSERT_EQ(0, baseline_meta_usage);
+ }
+
+ size_t pinned_usage = 0;
+ char value[10] = "abcdef";
+
+ std::forward_list<Cache::Handle*> unreleased_handles;
+ std::forward_list<Cache::Handle*> unreleased_handles_in_precise_cache;
+
+ // Add entries. Unpin some of them after insertion. Then, pin some of them
+ // again. Check GetPinnedUsage().
+ for (int i = 1; i < 100; ++i) {
+ std::string key;
+ if (type == kLRU) {
+ key = std::string(i, 'a');
+ } else {
+ key = EncodeKey(i);
+ }
+ auto kv_size = key.size() + 5;
+ Cache::Handle* handle;
+ Cache::Handle* handle_in_precise_cache;
+ ASSERT_OK(cache->Insert(key, reinterpret_cast<void*>(value), kv_size,
+ DumbDeleter, &handle));
+ assert(handle);
+ ASSERT_OK(precise_cache->Insert(key, reinterpret_cast<void*>(value),
+ kv_size, DumbDeleter,
+ &handle_in_precise_cache));
+ assert(handle_in_precise_cache);
+ pinned_usage += kv_size;
+ ASSERT_EQ(pinned_usage, cache->GetPinnedUsage());
+ ASSERT_LT(pinned_usage, precise_cache->GetPinnedUsage());
+ if (i % 2 == 0) {
+ cache->Release(handle);
+ precise_cache->Release(handle_in_precise_cache);
+ pinned_usage -= kv_size;
+ ASSERT_EQ(pinned_usage, cache->GetPinnedUsage());
+ ASSERT_LT(pinned_usage, precise_cache->GetPinnedUsage());
+ } else {
+ unreleased_handles.push_front(handle);
+ unreleased_handles_in_precise_cache.push_front(handle_in_precise_cache);
+ }
+ if (i % 3 == 0) {
+ unreleased_handles.push_front(cache->Lookup(key));
+ auto x = precise_cache->Lookup(key);
+ assert(x);
+ unreleased_handles_in_precise_cache.push_front(x);
+ // If i % 2 == 0, then the entry was unpinned before Lookup, so pinned
+ // usage increased
+ if (i % 2 == 0) {
+ pinned_usage += kv_size;
+ }
+ ASSERT_EQ(pinned_usage, cache->GetPinnedUsage());
+ ASSERT_LT(pinned_usage, precise_cache->GetPinnedUsage());
+ }
+ }
+ auto precise_cache_pinned_usage = precise_cache->GetPinnedUsage();
+ ASSERT_LT(pinned_usage, precise_cache_pinned_usage);
+
+ // check that overloading the cache does not change the pinned usage
+ for (size_t i = 1; i < 2 * kCapacity; ++i) {
+ std::string key;
+ if (type == kLRU) {
+ key = std::to_string(i);
+ } else {
+ key = EncodeKey(static_cast<int>(1000 + i));
+ }
+ ASSERT_OK(cache->Insert(key, reinterpret_cast<void*>(value), key.size() + 5,
+ DumbDeleter));
+ ASSERT_OK(precise_cache->Insert(key, reinterpret_cast<void*>(value),
+ key.size() + 5, DumbDeleter));
+ }
+ ASSERT_EQ(pinned_usage, cache->GetPinnedUsage());
+ ASSERT_EQ(precise_cache_pinned_usage, precise_cache->GetPinnedUsage());
+
+ cache->EraseUnRefEntries();
+ precise_cache->EraseUnRefEntries();
+ ASSERT_EQ(pinned_usage, cache->GetPinnedUsage());
+ ASSERT_EQ(precise_cache_pinned_usage, precise_cache->GetPinnedUsage());
+
+ // release handles for pinned entries to prevent memory leaks
+ for (auto handle : unreleased_handles) {
+ cache->Release(handle);
+ }
+ for (auto handle : unreleased_handles_in_precise_cache) {
+ precise_cache->Release(handle);
+ }
+ ASSERT_EQ(0, cache->GetPinnedUsage());
+ ASSERT_EQ(0, precise_cache->GetPinnedUsage());
+ cache->EraseUnRefEntries();
+ precise_cache->EraseUnRefEntries();
+ ASSERT_EQ(0, cache->GetUsage());
+ ASSERT_EQ(baseline_meta_usage, precise_cache->GetUsage());
+}
+
+TEST_P(CacheTest, HitAndMiss) {
+ ASSERT_EQ(-1, Lookup(100));
+
+ Insert(100, 101);
+ ASSERT_EQ(101, Lookup(100));
+ ASSERT_EQ(-1, Lookup(200));
+ ASSERT_EQ(-1, Lookup(300));
+
+ Insert(200, 201);
+ ASSERT_EQ(101, Lookup(100));
+ ASSERT_EQ(201, Lookup(200));
+ ASSERT_EQ(-1, Lookup(300));
+
+ Insert(100, 102);
+ if (GetParam() == kHyperClock) {
+ // ClockCache usually doesn't overwrite on Insert
+ ASSERT_EQ(101, Lookup(100));
+ } else {
+ ASSERT_EQ(102, Lookup(100));
+ }
+ ASSERT_EQ(201, Lookup(200));
+ ASSERT_EQ(-1, Lookup(300));
+
+ ASSERT_EQ(1U, deleted_keys_.size());
+ ASSERT_EQ(100, deleted_keys_[0]);
+ if (GetParam() == kHyperClock) {
+ ASSERT_EQ(102, deleted_values_[0]);
+ } else {
+ ASSERT_EQ(101, deleted_values_[0]);
+ }
+}
+
+TEST_P(CacheTest, InsertSameKey) {
+ if (GetParam() == kHyperClock) {
+ ROCKSDB_GTEST_BYPASS(
+ "ClockCache doesn't guarantee Insert overwrite same key.");
+ return;
+ }
+ Insert(1, 1);
+ Insert(1, 2);
+ ASSERT_EQ(2, Lookup(1));
+}
+
+TEST_P(CacheTest, Erase) {
+ Erase(200);
+ ASSERT_EQ(0U, deleted_keys_.size());
+
+ Insert(100, 101);
+ Insert(200, 201);
+ Erase(100);
+ ASSERT_EQ(-1, Lookup(100));
+ ASSERT_EQ(201, Lookup(200));
+ ASSERT_EQ(1U, deleted_keys_.size());
+ ASSERT_EQ(100, deleted_keys_[0]);
+ ASSERT_EQ(101, deleted_values_[0]);
+
+ Erase(100);
+ ASSERT_EQ(-1, Lookup(100));
+ ASSERT_EQ(201, Lookup(200));
+ ASSERT_EQ(1U, deleted_keys_.size());
+}
+
+TEST_P(CacheTest, EntriesArePinned) {
+ if (GetParam() == kHyperClock) {
+ ROCKSDB_GTEST_BYPASS(
+ "ClockCache doesn't guarantee Insert overwrite same key.");
+ return;
+ }
+ Insert(100, 101);
+ Cache::Handle* h1 = cache_->Lookup(EncodeKey(100));
+ ASSERT_EQ(101, DecodeValue(cache_->Value(h1)));
+ ASSERT_EQ(1U, cache_->GetUsage());
+
+ Insert(100, 102);
+ Cache::Handle* h2 = cache_->Lookup(EncodeKey(100));
+ ASSERT_EQ(102, DecodeValue(cache_->Value(h2)));
+ ASSERT_EQ(0U, deleted_keys_.size());
+ ASSERT_EQ(2U, cache_->GetUsage());
+
+ cache_->Release(h1);
+ ASSERT_EQ(1U, deleted_keys_.size());
+ ASSERT_EQ(100, deleted_keys_[0]);
+ ASSERT_EQ(101, deleted_values_[0]);
+ ASSERT_EQ(1U, cache_->GetUsage());
+
+ Erase(100);
+ ASSERT_EQ(-1, Lookup(100));
+ ASSERT_EQ(1U, deleted_keys_.size());
+ ASSERT_EQ(1U, cache_->GetUsage());
+
+ cache_->Release(h2);
+ ASSERT_EQ(2U, deleted_keys_.size());
+ ASSERT_EQ(100, deleted_keys_[1]);
+ ASSERT_EQ(102, deleted_values_[1]);
+ ASSERT_EQ(0U, cache_->GetUsage());
+}
+
+TEST_P(CacheTest, EvictionPolicy) {
+ Insert(100, 101);
+ Insert(200, 201);
+ // Frequently used entry must be kept around
+ for (int i = 0; i < 2 * kCacheSize; i++) {
+ Insert(1000 + i, 2000 + i);
+ ASSERT_EQ(101, Lookup(100));
+ }
+ ASSERT_EQ(101, Lookup(100));
+ ASSERT_EQ(-1, Lookup(200));
+}
+
+TEST_P(CacheTest, ExternalRefPinsEntries) {
+ Insert(100, 101);
+ Cache::Handle* h = cache_->Lookup(EncodeKey(100));
+ ASSERT_TRUE(cache_->Ref(h));
+ ASSERT_EQ(101, DecodeValue(cache_->Value(h)));
+ ASSERT_EQ(1U, cache_->GetUsage());
+
+ for (int i = 0; i < 3; ++i) {
+ if (i > 0) {
+ // First release (i == 1) corresponds to Ref(), second release (i == 2)
+ // corresponds to Lookup(). Then, since all external refs are released,
+ // the below insertions should push out the cache entry.
+ cache_->Release(h);
+ }
+ // double cache size because the usage bit in block cache prevents 100 from
+ // being evicted in the first kCacheSize iterations
+ for (int j = 0; j < 2 * kCacheSize + 100; j++) {
+ Insert(1000 + j, 2000 + j);
+ }
+ // Clock cache is even more stateful and needs more churn to evict
+ if (GetParam() == kHyperClock) {
+ for (int j = 0; j < kCacheSize; j++) {
+ Insert(11000 + j, 11000 + j);
+ }
+ }
+ if (i < 2) {
+ ASSERT_EQ(101, Lookup(100));
+ }
+ }
+ ASSERT_EQ(-1, Lookup(100));
+}
+
+TEST_P(CacheTest, EvictionPolicyRef) {
+ Insert(100, 101);
+ Insert(101, 102);
+ Insert(102, 103);
+ Insert(103, 104);
+ Insert(200, 101);
+ Insert(201, 102);
+ Insert(202, 103);
+ Insert(203, 104);
+ Cache::Handle* h201 = cache_->Lookup(EncodeKey(200));
+ Cache::Handle* h202 = cache_->Lookup(EncodeKey(201));
+ Cache::Handle* h203 = cache_->Lookup(EncodeKey(202));
+ Cache::Handle* h204 = cache_->Lookup(EncodeKey(203));
+ Insert(300, 101);
+ Insert(301, 102);
+ Insert(302, 103);
+ Insert(303, 104);
+
+ // Insert entries much more than cache capacity.
+ for (int i = 0; i < 100 * kCacheSize; i++) {
+ Insert(1000 + i, 2000 + i);
+ }
+
+ // Check whether the entries inserted in the beginning
+ // are evicted. Ones without extra ref are evicted and
+ // those with are not.
+ ASSERT_EQ(-1, Lookup(100));
+ ASSERT_EQ(-1, Lookup(101));
+ ASSERT_EQ(-1, Lookup(102));
+ ASSERT_EQ(-1, Lookup(103));
+
+ ASSERT_EQ(-1, Lookup(300));
+ ASSERT_EQ(-1, Lookup(301));
+ ASSERT_EQ(-1, Lookup(302));
+ ASSERT_EQ(-1, Lookup(303));
+
+ ASSERT_EQ(101, Lookup(200));
+ ASSERT_EQ(102, Lookup(201));
+ ASSERT_EQ(103, Lookup(202));
+ ASSERT_EQ(104, Lookup(203));
+
+ // Cleaning up all the handles
+ cache_->Release(h201);
+ cache_->Release(h202);
+ cache_->Release(h203);
+ cache_->Release(h204);
+}
+
+TEST_P(CacheTest, EvictEmptyCache) {
+ auto type = GetParam();
+
+ // Insert item large than capacity to trigger eviction on empty cache.
+ auto cache = NewCache(1, 0, false);
+ if (type == kLRU) {
+ ASSERT_OK(cache->Insert("foo", nullptr, 10, DumbDeleter));
+ } else {
+ ASSERT_OK(cache->Insert(EncodeKey(1000), nullptr, 10, DumbDeleter));
+ }
+}
+
+TEST_P(CacheTest, EraseFromDeleter) {
+ auto type = GetParam();
+
+ // Have deleter which will erase item from cache, which will re-enter
+ // the cache at that point.
+ std::shared_ptr<Cache> cache = NewCache(10, 0, false);
+ std::string foo, bar;
+ Cache::DeleterFn erase_deleter;
+ if (type == kLRU) {
+ foo = "foo";
+ bar = "bar";
+ erase_deleter = EraseDeleter1;
+ } else {
+ foo = EncodeKey(1234);
+ bar = EncodeKey(5678);
+ erase_deleter = EraseDeleter2;
+ }
+
+ ASSERT_OK(cache->Insert(foo, nullptr, 1, DumbDeleter));
+ ASSERT_OK(cache->Insert(bar, cache.get(), 1, erase_deleter));
+
+ cache->Erase(bar);
+ ASSERT_EQ(nullptr, cache->Lookup(foo));
+ ASSERT_EQ(nullptr, cache->Lookup(bar));
+}
+
+TEST_P(CacheTest, ErasedHandleState) {
+ // insert a key and get two handles
+ Insert(100, 1000);
+ Cache::Handle* h1 = cache_->Lookup(EncodeKey(100));
+ Cache::Handle* h2 = cache_->Lookup(EncodeKey(100));
+ ASSERT_EQ(h1, h2);
+ ASSERT_EQ(DecodeValue(cache_->Value(h1)), 1000);
+ ASSERT_EQ(DecodeValue(cache_->Value(h2)), 1000);
+
+ // delete the key from the cache
+ Erase(100);
+ // can no longer find in the cache
+ ASSERT_EQ(-1, Lookup(100));
+
+ // release one handle
+ cache_->Release(h1);
+ // still can't find in cache
+ ASSERT_EQ(-1, Lookup(100));
+
+ cache_->Release(h2);
+}
+
+TEST_P(CacheTest, HeavyEntries) {
+ // Add a bunch of light and heavy entries and then count the combined
+ // size of items still in the cache, which must be approximately the
+ // same as the total capacity.
+ const int kLight = 1;
+ const int kHeavy = 10;
+ int added = 0;
+ int index = 0;
+ while (added < 2 * kCacheSize) {
+ const int weight = (index & 1) ? kLight : kHeavy;
+ Insert(index, 1000 + index, weight);
+ added += weight;
+ index++;
+ }
+
+ int cached_weight = 0;
+ for (int i = 0; i < index; i++) {
+ const int weight = (i & 1 ? kLight : kHeavy);
+ int r = Lookup(i);
+ if (r >= 0) {
+ cached_weight += weight;
+ ASSERT_EQ(1000 + i, r);
+ }
+ }
+ ASSERT_LE(cached_weight, kCacheSize + kCacheSize / 10);
+}
+
+TEST_P(CacheTest, NewId) {
+ uint64_t a = cache_->NewId();
+ uint64_t b = cache_->NewId();
+ ASSERT_NE(a, b);
+}
+
+class Value {
+ public:
+ explicit Value(int v) : v_(v) {}
+
+ int v_;
+};
+
+namespace {
+void deleter(const Slice& /*key*/, void* value) {
+ delete static_cast<Value*>(value);
+}
+} // namespace
+
+TEST_P(CacheTest, ReleaseAndErase) {
+ std::shared_ptr<Cache> cache = NewCache(5, 0, false);
+ Cache::Handle* handle;
+ Status s = cache->Insert(EncodeKey(100), EncodeValue(100), 1,
+ &CacheTest::Deleter, &handle);
+ ASSERT_TRUE(s.ok());
+ ASSERT_EQ(5U, cache->GetCapacity());
+ ASSERT_EQ(1U, cache->GetUsage());
+ ASSERT_EQ(0U, deleted_keys_.size());
+ auto erased = cache->Release(handle, true);
+ ASSERT_TRUE(erased);
+ // This tests that deleter has been called
+ ASSERT_EQ(1U, deleted_keys_.size());
+}
+
+TEST_P(CacheTest, ReleaseWithoutErase) {
+ std::shared_ptr<Cache> cache = NewCache(5, 0, false);
+ Cache::Handle* handle;
+ Status s = cache->Insert(EncodeKey(100), EncodeValue(100), 1,
+ &CacheTest::Deleter, &handle);
+ ASSERT_TRUE(s.ok());
+ ASSERT_EQ(5U, cache->GetCapacity());
+ ASSERT_EQ(1U, cache->GetUsage());
+ ASSERT_EQ(0U, deleted_keys_.size());
+ auto erased = cache->Release(handle);
+ ASSERT_FALSE(erased);
+ // This tests that deleter is not called. When cache has free capacity it is
+ // not expected to immediately erase the released items.
+ ASSERT_EQ(0U, deleted_keys_.size());
+}
+
+TEST_P(CacheTest, SetCapacity) {
+ auto type = GetParam();
+ if (type == kHyperClock) {
+ ROCKSDB_GTEST_BYPASS(
+ "FastLRUCache and HyperClockCache don't support arbitrary capacity "
+ "adjustments.");
+ return;
+ }
+ // test1: increase capacity
+ // lets create a cache with capacity 5,
+ // then, insert 5 elements, then increase capacity
+ // to 10, returned capacity should be 10, usage=5
+ std::shared_ptr<Cache> cache = NewCache(5, 0, false);
+ std::vector<Cache::Handle*> handles(10);
+ // Insert 5 entries, but not releasing.
+ for (int i = 0; i < 5; i++) {
+ std::string key = EncodeKey(i + 1);
+ Status s = cache->Insert(key, new Value(i + 1), 1, &deleter, &handles[i]);
+ ASSERT_TRUE(s.ok());
+ }
+ ASSERT_EQ(5U, cache->GetCapacity());
+ ASSERT_EQ(5U, cache->GetUsage());
+ cache->SetCapacity(10);
+ ASSERT_EQ(10U, cache->GetCapacity());
+ ASSERT_EQ(5U, cache->GetUsage());
+
+ // test2: decrease capacity
+ // insert 5 more elements to cache, then release 5,
+ // then decrease capacity to 7, final capacity should be 7
+ // and usage should be 7
+ for (int i = 5; i < 10; i++) {
+ std::string key = EncodeKey(i + 1);
+ Status s = cache->Insert(key, new Value(i + 1), 1, &deleter, &handles[i]);
+ ASSERT_TRUE(s.ok());
+ }
+ ASSERT_EQ(10U, cache->GetCapacity());
+ ASSERT_EQ(10U, cache->GetUsage());
+ for (int i = 0; i < 5; i++) {
+ cache->Release(handles[i]);
+ }
+ ASSERT_EQ(10U, cache->GetCapacity());
+ ASSERT_EQ(10U, cache->GetUsage());
+ cache->SetCapacity(7);
+ ASSERT_EQ(7, cache->GetCapacity());
+ ASSERT_EQ(7, cache->GetUsage());
+
+ // release remaining 5 to keep valgrind happy
+ for (int i = 5; i < 10; i++) {
+ cache->Release(handles[i]);
+ }
+
+ // Make sure this doesn't crash or upset ASAN/valgrind
+ cache->DisownData();
+}
+
+TEST_P(LRUCacheTest, SetStrictCapacityLimit) {
+ // test1: set the flag to false. Insert more keys than capacity. See if they
+ // all go through.
+ std::shared_ptr<Cache> cache = NewCache(5, 0, false);
+ std::vector<Cache::Handle*> handles(10);
+ Status s;
+ for (int i = 0; i < 10; i++) {
+ std::string key = EncodeKey(i + 1);
+ s = cache->Insert(key, new Value(i + 1), 1, &deleter, &handles[i]);
+ ASSERT_OK(s);
+ ASSERT_NE(nullptr, handles[i]);
+ }
+ ASSERT_EQ(10, cache->GetUsage());
+
+ // test2: set the flag to true. Insert and check if it fails.
+ std::string extra_key = EncodeKey(100);
+ Value* extra_value = new Value(0);
+ cache->SetStrictCapacityLimit(true);
+ Cache::Handle* handle;
+ s = cache->Insert(extra_key, extra_value, 1, &deleter, &handle);
+ ASSERT_TRUE(s.IsMemoryLimit());
+ ASSERT_EQ(nullptr, handle);
+ ASSERT_EQ(10, cache->GetUsage());
+
+ for (int i = 0; i < 10; i++) {
+ cache->Release(handles[i]);
+ }
+
+ // test3: init with flag being true.
+ std::shared_ptr<Cache> cache2 = NewCache(5, 0, true);
+ for (int i = 0; i < 5; i++) {
+ std::string key = EncodeKey(i + 1);
+ s = cache2->Insert(key, new Value(i + 1), 1, &deleter, &handles[i]);
+ ASSERT_OK(s);
+ ASSERT_NE(nullptr, handles[i]);
+ }
+ s = cache2->Insert(extra_key, extra_value, 1, &deleter, &handle);
+ ASSERT_TRUE(s.IsMemoryLimit());
+ ASSERT_EQ(nullptr, handle);
+ // test insert without handle
+ s = cache2->Insert(extra_key, extra_value, 1, &deleter);
+ // AS if the key have been inserted into cache but get evicted immediately.
+ ASSERT_OK(s);
+ ASSERT_EQ(5, cache2->GetUsage());
+ ASSERT_EQ(nullptr, cache2->Lookup(extra_key));
+
+ for (int i = 0; i < 5; i++) {
+ cache2->Release(handles[i]);
+ }
+}
+
+TEST_P(CacheTest, OverCapacity) {
+ size_t n = 10;
+
+ // a LRUCache with n entries and one shard only
+ std::shared_ptr<Cache> cache = NewCache(n, 0, false);
+
+ std::vector<Cache::Handle*> handles(n + 1);
+
+ // Insert n+1 entries, but not releasing.
+ for (int i = 0; i < static_cast<int>(n + 1); i++) {
+ std::string key = EncodeKey(i + 1);
+ Status s = cache->Insert(key, new Value(i + 1), 1, &deleter, &handles[i]);
+ ASSERT_TRUE(s.ok());
+ }
+
+ // Guess what's in the cache now?
+ for (int i = 0; i < static_cast<int>(n + 1); i++) {
+ std::string key = EncodeKey(i + 1);
+ auto h = cache->Lookup(key);
+ ASSERT_TRUE(h != nullptr);
+ if (h) cache->Release(h);
+ }
+
+ // the cache is over capacity since nothing could be evicted
+ ASSERT_EQ(n + 1U, cache->GetUsage());
+ for (int i = 0; i < static_cast<int>(n + 1); i++) {
+ cache->Release(handles[i]);
+ }
+
+ if (GetParam() == kHyperClock) {
+ // Make sure eviction is triggered.
+ ASSERT_OK(cache->Insert(EncodeKey(-1), nullptr, 1, &deleter, &handles[0]));
+
+ // cache is under capacity now since elements were released
+ ASSERT_GE(n, cache->GetUsage());
+
+ // clean up
+ cache->Release(handles[0]);
+ } else {
+ // LRUCache checks for over-capacity in Release.
+
+ // cache is exactly at capacity now with minimal eviction
+ ASSERT_EQ(n, cache->GetUsage());
+
+ // element 0 is evicted and the rest is there
+ // This is consistent with the LRU policy since the element 0
+ // was released first
+ for (int i = 0; i < static_cast<int>(n + 1); i++) {
+ std::string key = EncodeKey(i + 1);
+ auto h = cache->Lookup(key);
+ if (h) {
+ ASSERT_NE(static_cast<size_t>(i), 0U);
+ cache->Release(h);
+ } else {
+ ASSERT_EQ(static_cast<size_t>(i), 0U);
+ }
+ }
+ }
+}
+
+namespace {
+std::vector<std::pair<int, int>> legacy_callback_state;
+void legacy_callback(void* value, size_t charge) {
+ legacy_callback_state.push_back(
+ {DecodeValue(value), static_cast<int>(charge)});
+}
+}; // namespace
+
+TEST_P(CacheTest, ApplyToAllCacheEntriesTest) {
+ std::vector<std::pair<int, int>> inserted;
+ legacy_callback_state.clear();
+
+ for (int i = 0; i < 10; ++i) {
+ Insert(i, i * 2, i + 1);
+ inserted.push_back({i * 2, i + 1});
+ }
+ cache_->ApplyToAllCacheEntries(legacy_callback, true);
+
+ std::sort(inserted.begin(), inserted.end());
+ std::sort(legacy_callback_state.begin(), legacy_callback_state.end());
+ ASSERT_EQ(inserted.size(), legacy_callback_state.size());
+ for (int i = 0; i < static_cast<int>(inserted.size()); ++i) {
+ EXPECT_EQ(inserted[i], legacy_callback_state[i]);
+ }
+}
+
+TEST_P(CacheTest, ApplyToAllEntriesTest) {
+ std::vector<std::string> callback_state;
+ const auto callback = [&](const Slice& key, void* value, size_t charge,
+ Cache::DeleterFn deleter) {
+ callback_state.push_back(std::to_string(DecodeKey(key)) + "," +
+ std::to_string(DecodeValue(value)) + "," +
+ std::to_string(charge));
+ assert(deleter == &CacheTest::Deleter);
+ };
+
+ std::vector<std::string> inserted;
+ callback_state.clear();
+
+ for (int i = 0; i < 10; ++i) {
+ Insert(i, i * 2, i + 1);
+ inserted.push_back(std::to_string(i) + "," + std::to_string(i * 2) + "," +
+ std::to_string(i + 1));
+ }
+ cache_->ApplyToAllEntries(callback, /*opts*/ {});
+
+ std::sort(inserted.begin(), inserted.end());
+ std::sort(callback_state.begin(), callback_state.end());
+ ASSERT_EQ(inserted.size(), callback_state.size());
+ for (int i = 0; i < static_cast<int>(inserted.size()); ++i) {
+ EXPECT_EQ(inserted[i], callback_state[i]);
+ }
+}
+
+TEST_P(CacheTest, ApplyToAllEntriesDuringResize) {
+ // This is a mini-stress test of ApplyToAllEntries, to ensure
+ // items in the cache that are neither added nor removed
+ // during ApplyToAllEntries are counted exactly once.
+
+ // Insert some entries that we expect to be seen exactly once
+ // during iteration.
+ constexpr int kSpecialCharge = 2;
+ constexpr int kNotSpecialCharge = 1;
+ constexpr int kSpecialCount = 100;
+ size_t expected_usage = 0;
+ for (int i = 0; i < kSpecialCount; ++i) {
+ Insert(i, i * 2, kSpecialCharge);
+ expected_usage += kSpecialCharge;
+ }
+
+ // For callback
+ int special_count = 0;
+ const auto callback = [&](const Slice&, void*, size_t charge,
+ Cache::DeleterFn) {
+ if (charge == static_cast<size_t>(kSpecialCharge)) {
+ ++special_count;
+ }
+ };
+
+ // Start counting
+ std::thread apply_thread([&]() {
+ // Use small average_entries_per_lock to make the problem difficult
+ Cache::ApplyToAllEntriesOptions opts;
+ opts.average_entries_per_lock = 2;
+ cache_->ApplyToAllEntries(callback, opts);
+ });
+
+ // In parallel, add more entries, enough to cause resize but not enough
+ // to cause ejections. (Note: if any cache shard is over capacity, there
+ // will be ejections)
+ for (int i = kSpecialCount * 1; i < kSpecialCount * 5; ++i) {
+ Insert(i, i * 2, kNotSpecialCharge);
+ expected_usage += kNotSpecialCharge;
+ }
+
+ apply_thread.join();
+ // verify no evictions
+ ASSERT_EQ(cache_->GetUsage(), expected_usage);
+ // verify everything seen in ApplyToAllEntries
+ ASSERT_EQ(special_count, kSpecialCount);
+}
+
+TEST_P(CacheTest, DefaultShardBits) {
+ // Prevent excessive allocation (to save time & space)
+ estimated_value_size_ = 100000;
+ // Implementations use different minimum shard sizes
+ size_t min_shard_size =
+ (GetParam() == kHyperClock ? 32U * 1024U : 512U) * 1024U;
+
+ std::shared_ptr<Cache> cache = NewCache(32U * min_shard_size);
+ ShardedCacheBase* sc = dynamic_cast<ShardedCacheBase*>(cache.get());
+ ASSERT_EQ(5, sc->GetNumShardBits());
+
+ cache = NewCache(min_shard_size / 1000U * 999U);
+ sc = dynamic_cast<ShardedCacheBase*>(cache.get());
+ ASSERT_EQ(0, sc->GetNumShardBits());
+
+ cache = NewCache(3U * 1024U * 1024U * 1024U);
+ sc = dynamic_cast<ShardedCacheBase*>(cache.get());
+ // current maximum of 6
+ ASSERT_EQ(6, sc->GetNumShardBits());
+
+ if constexpr (sizeof(size_t) > 4) {
+ cache = NewCache(128U * min_shard_size);
+ sc = dynamic_cast<ShardedCacheBase*>(cache.get());
+ // current maximum of 6
+ ASSERT_EQ(6, sc->GetNumShardBits());
+ }
+}
+
+TEST_P(CacheTest, GetChargeAndDeleter) {
+ Insert(1, 2);
+ Cache::Handle* h1 = cache_->Lookup(EncodeKey(1));
+ ASSERT_EQ(2, DecodeValue(cache_->Value(h1)));
+ ASSERT_EQ(1, cache_->GetCharge(h1));
+ ASSERT_EQ(&CacheTest::Deleter, cache_->GetDeleter(h1));
+ cache_->Release(h1);
+}
+
+INSTANTIATE_TEST_CASE_P(CacheTestInstance, CacheTest,
+ testing::Values(kLRU, kHyperClock));
+INSTANTIATE_TEST_CASE_P(CacheTestInstance, LRUCacheTest, testing::Values(kLRU));
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/cache/charged_cache.cc b/src/rocksdb/cache/charged_cache.cc
new file mode 100644
index 000000000..a9ff969b8
--- /dev/null
+++ b/src/rocksdb/cache/charged_cache.cc
@@ -0,0 +1,117 @@
+// 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/charged_cache.h"
+
+#include "cache/cache_reservation_manager.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+ChargedCache::ChargedCache(std::shared_ptr<Cache> cache,
+ std::shared_ptr<Cache> block_cache)
+ : cache_(cache),
+ cache_res_mgr_(std::make_shared<ConcurrentCacheReservationManager>(
+ std::make_shared<
+ CacheReservationManagerImpl<CacheEntryRole::kBlobCache>>(
+ block_cache))) {}
+
+Status ChargedCache::Insert(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter, Handle** handle,
+ Priority priority) {
+ Status s = cache_->Insert(key, value, charge, deleter, handle, priority);
+ if (s.ok()) {
+ // Insert may cause the cache entry eviction if the cache is full. So we
+ // directly call the reservation manager to update the total memory used
+ // in the cache.
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+ }
+ return s;
+}
+
+Status ChargedCache::Insert(const Slice& key, void* value,
+ const CacheItemHelper* helper, size_t charge,
+ Handle** handle, Priority priority) {
+ Status s = cache_->Insert(key, value, helper, charge, handle, priority);
+ if (s.ok()) {
+ // Insert may cause the cache entry eviction if the cache is full. So we
+ // directly call the reservation manager to update the total memory used
+ // in the cache.
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+ }
+ return s;
+}
+
+Cache::Handle* ChargedCache::Lookup(const Slice& key, Statistics* stats) {
+ return cache_->Lookup(key, stats);
+}
+
+Cache::Handle* ChargedCache::Lookup(const Slice& key,
+ const CacheItemHelper* helper,
+ const CreateCallback& create_cb,
+ Priority priority, bool wait,
+ Statistics* stats) {
+ auto handle = cache_->Lookup(key, helper, create_cb, priority, wait, 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.
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+ return handle;
+}
+
+bool ChargedCache::Release(Cache::Handle* handle, bool useful,
+ bool erase_if_last_ref) {
+ size_t memory_used_delta = cache_->GetUsage(handle);
+ bool erased = cache_->Release(handle, useful, erase_if_last_ref);
+ if (erased) {
+ assert(cache_res_mgr_);
+ cache_res_mgr_
+ ->UpdateCacheReservation(memory_used_delta, /* increase */ false)
+ .PermitUncheckedError();
+ }
+ return erased;
+}
+
+bool ChargedCache::Release(Cache::Handle* handle, bool erase_if_last_ref) {
+ size_t memory_used_delta = cache_->GetUsage(handle);
+ bool erased = cache_->Release(handle, erase_if_last_ref);
+ if (erased) {
+ assert(cache_res_mgr_);
+ cache_res_mgr_
+ ->UpdateCacheReservation(memory_used_delta, /* increase */ false)
+ .PermitUncheckedError();
+ }
+ return erased;
+}
+
+void ChargedCache::Erase(const Slice& key) {
+ cache_->Erase(key);
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+}
+
+void ChargedCache::EraseUnRefEntries() {
+ cache_->EraseUnRefEntries();
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+}
+
+void ChargedCache::SetCapacity(size_t capacity) {
+ cache_->SetCapacity(capacity);
+ // SetCapacity can result in evictions when the cache capacity is decreased,
+ // so we would want to update the cache reservation here as well.
+ assert(cache_res_mgr_);
+ cache_res_mgr_->UpdateCacheReservation(cache_->GetUsage())
+ .PermitUncheckedError();
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/charged_cache.h b/src/rocksdb/cache/charged_cache.h
new file mode 100644
index 000000000..1739e4088
--- /dev/null
+++ b/src/rocksdb/cache/charged_cache.h
@@ -0,0 +1,121 @@
+// 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 <string>
+
+#include "port/port.h"
+#include "rocksdb/cache.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class ConcurrentCacheReservationManager;
+
+// A cache interface which wraps around another cache and takes care of
+// reserving space in block cache towards a single global memory limit, and
+// forwards all the calls to the underlying cache.
+class ChargedCache : public Cache {
+ public:
+ ChargedCache(std::shared_ptr<Cache> cache,
+ std::shared_ptr<Cache> block_cache);
+ ~ChargedCache() override = default;
+
+ Status Insert(const Slice& key, void* value, size_t charge, DeleterFn deleter,
+ Handle** handle, Priority priority) override;
+ Status Insert(const Slice& key, void* value, const CacheItemHelper* helper,
+ size_t charge, Handle** handle = nullptr,
+ Priority priority = Priority::LOW) override;
+
+ Cache::Handle* Lookup(const Slice& key, Statistics* stats) override;
+ Cache::Handle* Lookup(const Slice& key, const CacheItemHelper* helper,
+ const CreateCallback& create_cb, Priority priority,
+ bool wait, Statistics* stats = nullptr) 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;
+
+ void Erase(const Slice& key) override;
+ void EraseUnRefEntries() override;
+
+ static const char* kClassName() { return "ChargedCache"; }
+ const char* Name() const override { return kClassName(); }
+
+ uint64_t NewId() override { return cache_->NewId(); }
+
+ void SetCapacity(size_t capacity) override;
+
+ void SetStrictCapacityLimit(bool strict_capacity_limit) override {
+ cache_->SetStrictCapacityLimit(strict_capacity_limit);
+ }
+
+ bool HasStrictCapacityLimit() const override {
+ return cache_->HasStrictCapacityLimit();
+ }
+
+ void* Value(Cache::Handle* handle) override { return cache_->Value(handle); }
+
+ bool IsReady(Cache::Handle* handle) override {
+ return cache_->IsReady(handle);
+ }
+
+ void Wait(Cache::Handle* handle) override { cache_->Wait(handle); }
+
+ void WaitAll(std::vector<Handle*>& handles) override {
+ cache_->WaitAll(handles);
+ }
+
+ bool Ref(Cache::Handle* handle) override { return cache_->Ref(handle); }
+
+ size_t GetCapacity() const override { return cache_->GetCapacity(); }
+
+ size_t GetUsage() const override { return cache_->GetUsage(); }
+
+ size_t GetUsage(Cache::Handle* handle) const override {
+ return cache_->GetUsage(handle);
+ }
+
+ size_t GetPinnedUsage() const override { return cache_->GetPinnedUsage(); }
+
+ size_t GetCharge(Cache::Handle* handle) const override {
+ return cache_->GetCharge(handle);
+ }
+
+ Cache::DeleterFn GetDeleter(Cache::Handle* handle) const override {
+ return cache_->GetDeleter(handle);
+ }
+
+ void ApplyToAllEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ Cache::DeleterFn deleter)>& callback,
+ const Cache::ApplyToAllEntriesOptions& opts) override {
+ cache_->ApplyToAllEntries(callback, opts);
+ }
+
+ void ApplyToAllCacheEntries(void (*callback)(void* value, size_t charge),
+ bool thread_safe) override {
+ cache_->ApplyToAllCacheEntries(callback, thread_safe);
+ }
+
+ std::string GetPrintableOptions() const override {
+ return cache_->GetPrintableOptions();
+ }
+
+ void DisownData() override { return cache_->DisownData(); }
+
+ inline Cache* GetCache() const { return cache_.get(); }
+
+ inline ConcurrentCacheReservationManager* TEST_GetCacheReservationManager()
+ const {
+ return cache_res_mgr_.get();
+ }
+
+ private:
+ std::shared_ptr<Cache> cache_;
+ std::shared_ptr<ConcurrentCacheReservationManager> cache_res_mgr_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/clock_cache.cc b/src/rocksdb/cache/clock_cache.cc
new file mode 100644
index 000000000..6c9f18c2f
--- /dev/null
+++ b/src/rocksdb/cache/clock_cache.cc
@@ -0,0 +1,1404 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "cache/clock_cache.h"
+
+#include <cassert>
+#include <functional>
+#include <numeric>
+
+#include "cache/cache_key.h"
+#include "logging/logging.h"
+#include "monitoring/perf_context_imp.h"
+#include "monitoring/statistics.h"
+#include "port/lang.h"
+#include "util/hash.h"
+#include "util/math.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace clock_cache {
+
+namespace {
+inline uint64_t GetRefcount(uint64_t meta) {
+ return ((meta >> ClockHandle::kAcquireCounterShift) -
+ (meta >> ClockHandle::kReleaseCounterShift)) &
+ ClockHandle::kCounterMask;
+}
+
+inline uint64_t GetInitialCountdown(Cache::Priority priority) {
+ // Set initial clock data from priority
+ // TODO: configuration parameters for priority handling and clock cycle
+ // count?
+ switch (priority) {
+ case Cache::Priority::HIGH:
+ return ClockHandle::kHighCountdown;
+ default:
+ assert(false);
+ FALLTHROUGH_INTENDED;
+ case Cache::Priority::LOW:
+ return ClockHandle::kLowCountdown;
+ case Cache::Priority::BOTTOM:
+ return ClockHandle::kBottomCountdown;
+ }
+}
+
+inline void FreeDataMarkEmpty(ClockHandle& h) {
+ // 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();
+
+#ifndef NDEBUG
+ // Mark slot as empty, with assertion
+ uint64_t meta = h.meta.exchange(0, std::memory_order_release);
+ assert(meta >> ClockHandle::kStateShift == ClockHandle::kStateConstruction);
+#else
+ // Mark slot as empty
+ h.meta.store(0, std::memory_order_release);
+#endif
+}
+
+inline bool ClockUpdate(ClockHandle& h) {
+ uint64_t meta = h.meta.load(std::memory_order_relaxed);
+
+ uint64_t acquire_count =
+ (meta >> ClockHandle::kAcquireCounterShift) & ClockHandle::kCounterMask;
+ uint64_t release_count =
+ (meta >> ClockHandle::kReleaseCounterShift) & ClockHandle::kCounterMask;
+ // fprintf(stderr, "ClockUpdate @ %p: %lu %lu %u\n", &h, acquire_count,
+ // release_count, (unsigned)(meta >> ClockHandle::kStateShift));
+ if (acquire_count != release_count) {
+ // Only clock update entries with no outstanding refs
+ return false;
+ }
+ if (!((meta >> ClockHandle::kStateShift) & ClockHandle::kStateShareableBit)) {
+ // Only clock update Shareable entries
+ return false;
+ }
+ if ((meta >> ClockHandle::kStateShift == ClockHandle::kStateVisible) &&
+ acquire_count > 0) {
+ // Decrement clock
+ uint64_t new_count =
+ std::min(acquire_count - 1, uint64_t{ClockHandle::kMaxCountdown} - 1);
+ // Compare-exchange in the decremented clock info, but
+ // not aggressively
+ uint64_t new_meta =
+ (uint64_t{ClockHandle::kStateVisible} << ClockHandle::kStateShift) |
+ (new_count << ClockHandle::kReleaseCounterShift) |
+ (new_count << ClockHandle::kAcquireCounterShift);
+ h.meta.compare_exchange_strong(meta, new_meta, std::memory_order_relaxed);
+ return false;
+ }
+ // Otherwise, remove entry (either unreferenced invisible or
+ // unreferenced and expired visible).
+ if (h.meta.compare_exchange_strong(
+ meta,
+ uint64_t{ClockHandle::kStateConstruction} << ClockHandle::kStateShift,
+ std::memory_order_acquire)) {
+ // Took ownership.
+ return true;
+ } else {
+ // Compare-exchange failing probably
+ // indicates the entry was used, so skip it in that case.
+ return false;
+ }
+}
+
+} // namespace
+
+void ClockHandleBasicData::FreeData() const {
+ if (deleter) {
+ UniqueId64x2 unhashed;
+ (*deleter)(
+ ClockCacheShard<HyperClockTable>::ReverseHash(hashed_key, &unhashed),
+ value);
+ }
+}
+
+HyperClockTable::HyperClockTable(
+ size_t capacity, bool /*strict_capacity_limit*/,
+ CacheMetadataChargePolicy metadata_charge_policy, 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<size_t>((uint64_t{1} << length_bits_) *
+ kStrictLoadFactor)),
+ array_(new HandleImpl[size_t{1} << length_bits_]) {
+ if (metadata_charge_policy ==
+ CacheMetadataChargePolicy::kFullChargeCacheMetadata) {
+ usage_ += size_t{GetTableSize()} * sizeof(HandleImpl);
+ }
+
+ static_assert(sizeof(HandleImpl) == 64U,
+ "Expecting size / alignment with common cache line size");
+}
+
+HyperClockTable::~HyperClockTable() {
+ // Assumes there are no references or active operations on any slot/element
+ // in the table.
+ for (size_t i = 0; i < GetTableSize(); i++) {
+ HandleImpl& h = array_[i];
+ switch (h.meta >> ClockHandle::kStateShift) {
+ case ClockHandle::kStateEmpty:
+ // noop
+ break;
+ case ClockHandle::kStateInvisible: // rare but possible
+ case ClockHandle::kStateVisible:
+ assert(GetRefcount(h.meta) == 0);
+ h.FreeData();
+#ifndef NDEBUG
+ Rollback(h.hashed_key, &h);
+ ReclaimEntryUsage(h.GetTotalCharge());
+#endif
+ break;
+ // otherwise
+ default:
+ assert(false);
+ break;
+ }
+ }
+
+#ifndef NDEBUG
+ for (size_t i = 0; i < GetTableSize(); i++) {
+ assert(array_[i].displacements.load() == 0);
+ }
+#endif
+
+ assert(usage_.load() == 0 ||
+ usage_.load() == size_t{GetTableSize()} * sizeof(HandleImpl));
+ assert(occupancy_ == 0);
+}
+
+// If an entry doesn't receive clock updates but is repeatedly referenced &
+// released, the acquire and release counters could overflow without some
+// intervention. This is that intervention, which should be inexpensive
+// because it only incurs a simple, very predictable check. (Applying a bit
+// mask in addition to an increment to every Release likely would be
+// relatively expensive, because it's an extra atomic update.)
+//
+// We do have to assume that we never have many millions of simultaneous
+// references to a cache handle, because we cannot represent so many
+// references with the difference in counters, masked to the number of
+// counter bits. Similarly, we assume there aren't millions of threads
+// holding transient references (which might be "undone" rather than
+// released by the way).
+//
+// Consider these possible states for each counter:
+// low: less than kMaxCountdown
+// medium: kMaxCountdown to half way to overflow + kMaxCountdown
+// high: half way to overflow + kMaxCountdown, or greater
+//
+// And these possible states for the combination of counters:
+// acquire / release
+// ------- -------
+// low low - Normal / common, with caveats (see below)
+// medium low - Can happen while holding some refs
+// high low - Violates assumptions (too many refs)
+// low medium - Violates assumptions (refs underflow, etc.)
+// medium medium - Normal (very read heavy cache)
+// high medium - Can happen while holding some refs
+// low high - This function is supposed to prevent
+// medium high - Violates assumptions (refs underflow, etc.)
+// high high - Needs CorrectNearOverflow
+//
+// Basically, this function detects (high, high) state (inferred from
+// release alone being high) and bumps it back down to (medium, medium)
+// state with the same refcount and the same logical countdown counter
+// (everything > kMaxCountdown is logically the same). Note that bumping
+// down to (low, low) would modify the countdown counter, so is "reserved"
+// in a sense.
+//
+// If near-overflow correction is triggered here, there's no guarantee
+// that another thread hasn't freed the entry and replaced it with another.
+// Therefore, it must be the case that the correction does not affect
+// entries unless they are very old (many millions of acquire-release cycles).
+// (Our bit manipulation is indeed idempotent and only affects entries in
+// exceptional cases.) We assume a pre-empted thread will not stall that long.
+// If it did, the state could be corrupted in the (unlikely) case that the top
+// bit of the acquire counter is set but not the release counter, and thus
+// we only clear the top bit of the acquire counter on resumption. It would
+// then appear that there are too many refs and the entry would be permanently
+// pinned (which is not terrible for an exceptionally rare occurrence), unless
+// it is referenced enough (at least kMaxCountdown more times) for the release
+// counter to reach "high" state again and bumped back to "medium." (This
+// motivates only checking for release counter in high state, not both in high
+// state.)
+inline void CorrectNearOverflow(uint64_t old_meta,
+ std::atomic<uint64_t>& meta) {
+ // We clear both top-most counter bits at the same time.
+ constexpr uint64_t kCounterTopBit = uint64_t{1}
+ << (ClockHandle::kCounterNumBits - 1);
+ constexpr uint64_t kClearBits =
+ (kCounterTopBit << ClockHandle::kAcquireCounterShift) |
+ (kCounterTopBit << ClockHandle::kReleaseCounterShift);
+ // A simple check that allows us to initiate clearing the top bits for
+ // a large portion of the "high" state space on release counter.
+ constexpr uint64_t kCheckBits =
+ (kCounterTopBit | (ClockHandle::kMaxCountdown + 1))
+ << ClockHandle::kReleaseCounterShift;
+
+ if (UNLIKELY(old_meta & kCheckBits)) {
+ meta.fetch_and(~kClearBits, std::memory_order_relaxed);
+ }
+}
+
+inline Status HyperClockTable::ChargeUsageMaybeEvictStrict(
+ size_t total_charge, size_t capacity, bool need_evict_for_occupancy) {
+ if (total_charge > capacity) {
+ return Status::MemoryLimit(
+ "Cache entry too large for a single cache shard: " +
+ std::to_string(total_charge) + " > " + std::to_string(capacity));
+ }
+ // Grab any available capacity, and free up any more required.
+ size_t old_usage = usage_.load(std::memory_order_relaxed);
+ size_t new_usage;
+ if (LIKELY(old_usage != capacity)) {
+ do {
+ new_usage = std::min(capacity, old_usage + total_charge);
+ } while (!usage_.compare_exchange_weak(old_usage, new_usage,
+ std::memory_order_relaxed));
+ } else {
+ new_usage = old_usage;
+ }
+ // How much do we need to evict then?
+ size_t need_evict_charge = old_usage + total_charge - new_usage;
+ size_t request_evict_charge = need_evict_charge;
+ if (UNLIKELY(need_evict_for_occupancy) && request_evict_charge == 0) {
+ // Require at least 1 eviction.
+ request_evict_charge = 1;
+ }
+ if (request_evict_charge > 0) {
+ size_t evicted_charge = 0;
+ size_t evicted_count = 0;
+ Evict(request_evict_charge, &evicted_charge, &evicted_count);
+ occupancy_.fetch_sub(evicted_count, std::memory_order_release);
+ if (LIKELY(evicted_charge > need_evict_charge)) {
+ assert(evicted_count > 0);
+ // Evicted more than enough
+ usage_.fetch_sub(evicted_charge - need_evict_charge,
+ std::memory_order_relaxed);
+ } else if (evicted_charge < need_evict_charge ||
+ (UNLIKELY(need_evict_for_occupancy) && evicted_count == 0)) {
+ // Roll back to old usage minus evicted
+ usage_.fetch_sub(evicted_charge + (new_usage - old_usage),
+ std::memory_order_relaxed);
+ if (evicted_charge < need_evict_charge) {
+ return Status::MemoryLimit(
+ "Insert failed because unable to evict entries to stay within "
+ "capacity limit.");
+ } else {
+ return Status::MemoryLimit(
+ "Insert failed because unable to evict entries to stay within "
+ "table occupancy limit.");
+ }
+ }
+ // If we needed to evict something and we are proceeding, we must have
+ // evicted something.
+ assert(evicted_count > 0);
+ }
+ return Status::OK();
+}
+
+inline bool HyperClockTable::ChargeUsageMaybeEvictNonStrict(
+ size_t total_charge, size_t capacity, bool need_evict_for_occupancy) {
+ // For simplicity, we consider that either the cache can accept the insert
+ // with no evictions, or we must evict enough to make (at least) enough
+ // space. It could lead to unnecessary failures or excessive evictions in
+ // some extreme cases, but allows a fast, simple protocol. If we allow a
+ // race to get us over capacity, then we might never get back to capacity
+ // limit if the sizes of entries allow each insertion to evict the minimum
+ // charge. Thus, we should evict some extra if it's not a signifcant
+ // portion of the shard capacity. This can have the side benefit of
+ // involving fewer threads in eviction.
+ size_t old_usage = usage_.load(std::memory_order_relaxed);
+ size_t need_evict_charge;
+ // NOTE: if total_charge > old_usage, there isn't yet enough to evict
+ // `total_charge` amount. Even if we only try to evict `old_usage` amount,
+ // there's likely something referenced and we would eat CPU looking for
+ // enough to evict.
+ if (old_usage + total_charge <= capacity || total_charge > old_usage) {
+ // Good enough for me (might run over with a race)
+ need_evict_charge = 0;
+ } else {
+ // Try to evict enough space, and maybe some extra
+ need_evict_charge = total_charge;
+ if (old_usage > capacity) {
+ // Not too much to avoid thundering herd while avoiding strict
+ // synchronization, such as the compare_exchange used with strict
+ // capacity limit.
+ need_evict_charge += std::min(capacity / 1024, total_charge) + 1;
+ }
+ }
+ if (UNLIKELY(need_evict_for_occupancy) && need_evict_charge == 0) {
+ // Special case: require at least 1 eviction if we only have to
+ // deal with occupancy
+ need_evict_charge = 1;
+ }
+ size_t evicted_charge = 0;
+ size_t evicted_count = 0;
+ if (need_evict_charge > 0) {
+ Evict(need_evict_charge, &evicted_charge, &evicted_count);
+ // Deal with potential occupancy deficit
+ if (UNLIKELY(need_evict_for_occupancy) && evicted_count == 0) {
+ assert(evicted_charge == 0);
+ // Can't meet occupancy requirement
+ return false;
+ } else {
+ // Update occupancy for evictions
+ occupancy_.fetch_sub(evicted_count, std::memory_order_release);
+ }
+ }
+ // Track new usage even if we weren't able to evict enough
+ usage_.fetch_add(total_charge - evicted_charge, std::memory_order_relaxed);
+ // No underflow
+ assert(usage_.load(std::memory_order_relaxed) < SIZE_MAX / 2);
+ // Success
+ return true;
+}
+
+inline HyperClockTable::HandleImpl* HyperClockTable::DetachedInsert(
+ const ClockHandleBasicData& proto) {
+ // Heap allocated separate from table
+ HandleImpl* h = new HandleImpl();
+ ClockHandleBasicData* h_alias = h;
+ *h_alias = proto;
+ h->SetDetached();
+ // Single reference (detached entries only created if returning a refed
+ // Handle back to user)
+ uint64_t meta = uint64_t{ClockHandle::kStateInvisible}
+ << ClockHandle::kStateShift;
+ meta |= uint64_t{1} << ClockHandle::kAcquireCounterShift;
+ h->meta.store(meta, std::memory_order_release);
+ // Keep track of how much of usage is detached
+ detached_usage_.fetch_add(proto.GetTotalCharge(), std::memory_order_relaxed);
+ return h;
+}
+
+Status HyperClockTable::Insert(const ClockHandleBasicData& proto,
+ HandleImpl** handle, Cache::Priority priority,
+ size_t capacity, bool strict_capacity_limit) {
+ // Do we have the available occupancy? Optimistically assume we do
+ // and deal with it if we don't.
+ size_t old_occupancy = occupancy_.fetch_add(1, std::memory_order_acquire);
+ auto revert_occupancy_fn = [&]() {
+ occupancy_.fetch_sub(1, std::memory_order_relaxed);
+ };
+ // Whether we over-committed and need an eviction to make up for it
+ bool need_evict_for_occupancy = old_occupancy >= occupancy_limit_;
+
+ // Usage/capacity handling is somewhat different depending on
+ // strict_capacity_limit, but mostly pessimistic.
+ bool use_detached_insert = false;
+ const size_t total_charge = proto.GetTotalCharge();
+ if (strict_capacity_limit) {
+ Status s = ChargeUsageMaybeEvictStrict(total_charge, capacity,
+ need_evict_for_occupancy);
+ if (!s.ok()) {
+ revert_occupancy_fn();
+ return s;
+ }
+ } else {
+ // Case strict_capacity_limit == false
+ bool success = ChargeUsageMaybeEvictNonStrict(total_charge, capacity,
+ need_evict_for_occupancy);
+ if (!success) {
+ revert_occupancy_fn();
+ if (handle == nullptr) {
+ // Don't insert the entry but still return ok, as if the entry
+ // inserted into cache and evicted immediately.
+ proto.FreeData();
+ return Status::OK();
+ } else {
+ // Need to track usage of fallback detached insert
+ usage_.fetch_add(total_charge, std::memory_order_relaxed);
+ use_detached_insert = true;
+ }
+ }
+ }
+ auto revert_usage_fn = [&]() {
+ usage_.fetch_sub(total_charge, std::memory_order_relaxed);
+ // No underflow
+ assert(usage_.load(std::memory_order_relaxed) < SIZE_MAX / 2);
+ };
+
+ if (!use_detached_insert) {
+ // Attempt a table insert, but abort if we find an existing entry for the
+ // key. If we were to overwrite old entries, we would either
+ // * Have to gain ownership over an existing entry to overwrite it, which
+ // would only work if there are no outstanding (read) references and would
+ // create a small gap in availability of the entry (old or new) to lookups.
+ // * Have to insert into a suboptimal location (more probes) so that the
+ // old entry can be kept around as well.
+
+ uint64_t initial_countdown = GetInitialCountdown(priority);
+ assert(initial_countdown > 0);
+
+ size_t probe = 0;
+ HandleImpl* e = FindSlot(
+ proto.hashed_key,
+ [&](HandleImpl* h) {
+ // Optimistically transition the slot from "empty" to
+ // "under construction" (no effect on other states)
+ uint64_t old_meta =
+ h->meta.fetch_or(uint64_t{ClockHandle::kStateOccupiedBit}
+ << ClockHandle::kStateShift,
+ std::memory_order_acq_rel);
+ uint64_t old_state = old_meta >> ClockHandle::kStateShift;
+
+ if (old_state == ClockHandle::kStateEmpty) {
+ // We've started inserting into an available slot, and taken
+ // ownership Save data fields
+ ClockHandleBasicData* h_alias = h;
+ *h_alias = proto;
+
+ // Transition from "under construction" state to "visible" state
+ uint64_t new_meta = uint64_t{ClockHandle::kStateVisible}
+ << ClockHandle::kStateShift;
+
+ // Maybe with an outstanding reference
+ new_meta |= initial_countdown << ClockHandle::kAcquireCounterShift;
+ new_meta |= (initial_countdown - (handle != nullptr))
+ << ClockHandle::kReleaseCounterShift;
+
+#ifndef NDEBUG
+ // Save the state transition, with assertion
+ old_meta = h->meta.exchange(new_meta, std::memory_order_release);
+ assert(old_meta >> ClockHandle::kStateShift ==
+ ClockHandle::kStateConstruction);
+#else
+ // Save the state transition
+ h->meta.store(new_meta, std::memory_order_release);
+#endif
+ return true;
+ } else if (old_state != ClockHandle::kStateVisible) {
+ // Slot not usable / touchable now
+ return false;
+ }
+ // Existing, visible entry, which might be a match.
+ // But first, we need to acquire a ref to read it. In fact, number of
+ // refs for initial countdown, so that we boost the clock state if
+ // this is a match.
+ old_meta = h->meta.fetch_add(
+ ClockHandle::kAcquireIncrement * initial_countdown,
+ std::memory_order_acq_rel);
+ // Like Lookup
+ if ((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateVisible) {
+ // Acquired a read reference
+ if (h->hashed_key == proto.hashed_key) {
+ // Match. Release in a way that boosts the clock state
+ old_meta = h->meta.fetch_add(
+ ClockHandle::kReleaseIncrement * initial_countdown,
+ std::memory_order_acq_rel);
+ // Correct for possible (but rare) overflow
+ CorrectNearOverflow(old_meta, h->meta);
+ // Insert detached instead (only if return handle needed)
+ use_detached_insert = true;
+ return true;
+ } else {
+ // Mismatch. Pretend we never took the reference
+ old_meta = h->meta.fetch_sub(
+ ClockHandle::kAcquireIncrement * initial_countdown,
+ std::memory_order_acq_rel);
+ }
+ } else if (UNLIKELY((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateInvisible)) {
+ // Pretend we never took the reference
+ // WART: there's a tiny chance we release last ref to invisible
+ // entry here. If that happens, we let eviction take care of it.
+ old_meta = h->meta.fetch_sub(
+ ClockHandle::kAcquireIncrement * initial_countdown,
+ std::memory_order_acq_rel);
+ } else {
+ // For other states, incrementing the acquire counter has no effect
+ // so we don't need to undo it.
+ // Slot not usable / touchable now.
+ }
+ (void)old_meta;
+ return false;
+ },
+ [&](HandleImpl* /*h*/) { return false; },
+ [&](HandleImpl* h) {
+ h->displacements.fetch_add(1, std::memory_order_relaxed);
+ },
+ probe);
+ if (e == nullptr) {
+ // Occupancy check and never abort FindSlot above should generally
+ // prevent this, except it's theoretically possible for other threads
+ // to evict and replace entries in the right order to hit every slot
+ // when it is populated. Assuming random hashing, the chance of that
+ // should be no higher than pow(kStrictLoadFactor, n) for n slots.
+ // That should be infeasible for roughly n >= 256, so if this assertion
+ // fails, that suggests something is going wrong.
+ assert(GetTableSize() < 256);
+ use_detached_insert = true;
+ }
+ if (!use_detached_insert) {
+ // Successfully inserted
+ if (handle) {
+ *handle = e;
+ }
+ return Status::OK();
+ }
+ // Roll back table insertion
+ Rollback(proto.hashed_key, e);
+ revert_occupancy_fn();
+ // Maybe fall back on detached insert
+ if (handle == nullptr) {
+ revert_usage_fn();
+ // As if unrefed entry immdiately evicted
+ proto.FreeData();
+ return Status::OK();
+ }
+ }
+
+ // Run detached insert
+ assert(use_detached_insert);
+
+ *handle = DetachedInsert(proto);
+
+ // The OkOverwritten status is used to count "redundant" insertions into
+ // block cache. This implementation doesn't strictly check for redundant
+ // insertions, but we instead are probably interested in how many insertions
+ // didn't go into the table (instead "detached"), which could be redundant
+ // Insert or some other reason (use_detached_insert reasons above).
+ return Status::OkOverwritten();
+}
+
+HyperClockTable::HandleImpl* HyperClockTable::Lookup(
+ const UniqueId64x2& hashed_key) {
+ size_t probe = 0;
+ HandleImpl* e = FindSlot(
+ hashed_key,
+ [&](HandleImpl* h) {
+ // Mostly branch-free version (similar performance)
+ /*
+ uint64_t old_meta = h->meta.fetch_add(ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+ bool Shareable = (old_meta >> (ClockHandle::kStateShift + 1)) & 1U;
+ bool visible = (old_meta >> ClockHandle::kStateShift) & 1U;
+ bool match = (h->key == key) & visible;
+ h->meta.fetch_sub(static_cast<uint64_t>(Shareable & !match) <<
+ ClockHandle::kAcquireCounterShift, std::memory_order_release); return
+ match;
+ */
+ // Optimistic lookup should pay off when the table is relatively
+ // sparse.
+ constexpr bool kOptimisticLookup = true;
+ uint64_t old_meta;
+ if (!kOptimisticLookup) {
+ old_meta = h->meta.load(std::memory_order_acquire);
+ if ((old_meta >> ClockHandle::kStateShift) !=
+ ClockHandle::kStateVisible) {
+ return false;
+ }
+ }
+ // (Optimistically) increment acquire counter
+ old_meta = h->meta.fetch_add(ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+ // Check if it's an entry visible to lookups
+ if ((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateVisible) {
+ // Acquired a read reference
+ if (h->hashed_key == hashed_key) {
+ // Match
+ return true;
+ } else {
+ // Mismatch. Pretend we never took the reference
+ old_meta = h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ }
+ } else if (UNLIKELY((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateInvisible)) {
+ // Pretend we never took the reference
+ // WART: there's a tiny chance we release last ref to invisible
+ // entry here. If that happens, we let eviction take care of it.
+ old_meta = h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ } else {
+ // For other states, incrementing the acquire counter has no effect
+ // so we don't need to undo it. Furthermore, we cannot safely undo
+ // it because we did not acquire a read reference to lock the
+ // entry in a Shareable state.
+ }
+ (void)old_meta;
+ return false;
+ },
+ [&](HandleImpl* h) {
+ return h->displacements.load(std::memory_order_relaxed) == 0;
+ },
+ [&](HandleImpl* /*h*/) {}, probe);
+
+ return e;
+}
+
+bool HyperClockTable::Release(HandleImpl* h, bool useful,
+ bool erase_if_last_ref) {
+ // In contrast with LRUCache's Release, this function won't delete the handle
+ // when the cache is above capacity and the reference is the last one. Space
+ // is only freed up by EvictFromClock (called by Insert when space is needed)
+ // and Erase. We do this to avoid an extra atomic read of the variable usage_.
+
+ uint64_t old_meta;
+ if (useful) {
+ // Increment release counter to indicate was used
+ old_meta = h->meta.fetch_add(ClockHandle::kReleaseIncrement,
+ std::memory_order_release);
+ } else {
+ // Decrement acquire counter to pretend it never happened
+ old_meta = h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ }
+
+ assert((old_meta >> ClockHandle::kStateShift) &
+ ClockHandle::kStateShareableBit);
+ // No underflow
+ assert(((old_meta >> ClockHandle::kAcquireCounterShift) &
+ ClockHandle::kCounterMask) !=
+ ((old_meta >> ClockHandle::kReleaseCounterShift) &
+ ClockHandle::kCounterMask));
+
+ if (erase_if_last_ref || UNLIKELY(old_meta >> ClockHandle::kStateShift ==
+ ClockHandle::kStateInvisible)) {
+ // Update for last fetch_add op
+ if (useful) {
+ old_meta += ClockHandle::kReleaseIncrement;
+ } else {
+ old_meta -= ClockHandle::kAcquireIncrement;
+ }
+ // Take ownership if no refs
+ do {
+ if (GetRefcount(old_meta) != 0) {
+ // Not last ref at some point in time during this Release call
+ // Correct for possible (but rare) overflow
+ CorrectNearOverflow(old_meta, h->meta);
+ return false;
+ }
+ if ((old_meta & (uint64_t{ClockHandle::kStateShareableBit}
+ << ClockHandle::kStateShift)) == 0) {
+ // Someone else took ownership
+ return false;
+ }
+ // Note that there's a small chance that we release, another thread
+ // replaces this entry with another, reaches zero refs, and then we end
+ // up erasing that other entry. That's an acceptable risk / imprecision.
+ } while (!h->meta.compare_exchange_weak(
+ old_meta,
+ uint64_t{ClockHandle::kStateConstruction} << ClockHandle::kStateShift,
+ std::memory_order_acquire));
+ // Took ownership
+ size_t total_charge = h->GetTotalCharge();
+ if (UNLIKELY(h->IsDetached())) {
+ h->FreeData();
+ // Delete detached handle
+ delete h;
+ detached_usage_.fetch_sub(total_charge, std::memory_order_relaxed);
+ usage_.fetch_sub(total_charge, std::memory_order_relaxed);
+ } else {
+ Rollback(h->hashed_key, h);
+ FreeDataMarkEmpty(*h);
+ ReclaimEntryUsage(total_charge);
+ }
+ return true;
+ } else {
+ // Correct for possible (but rare) overflow
+ CorrectNearOverflow(old_meta, h->meta);
+ return false;
+ }
+}
+
+void HyperClockTable::Ref(HandleImpl& h) {
+ // Increment acquire counter
+ uint64_t old_meta = h.meta.fetch_add(ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+
+ assert((old_meta >> ClockHandle::kStateShift) &
+ ClockHandle::kStateShareableBit);
+ // Must have already had a reference
+ assert(GetRefcount(old_meta) > 0);
+ (void)old_meta;
+}
+
+void HyperClockTable::TEST_RefN(HandleImpl& h, size_t n) {
+ // Increment acquire counter
+ uint64_t old_meta = h.meta.fetch_add(n * ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+
+ assert((old_meta >> ClockHandle::kStateShift) &
+ ClockHandle::kStateShareableBit);
+ (void)old_meta;
+}
+
+void HyperClockTable::TEST_ReleaseN(HandleImpl* h, size_t n) {
+ if (n > 0) {
+ // Split into n - 1 and 1 steps.
+ uint64_t old_meta = h->meta.fetch_add(
+ (n - 1) * ClockHandle::kReleaseIncrement, std::memory_order_acquire);
+ assert((old_meta >> ClockHandle::kStateShift) &
+ ClockHandle::kStateShareableBit);
+ (void)old_meta;
+
+ Release(h, /*useful*/ true, /*erase_if_last_ref*/ false);
+ }
+}
+
+void HyperClockTable::Erase(const UniqueId64x2& hashed_key) {
+ size_t probe = 0;
+ (void)FindSlot(
+ hashed_key,
+ [&](HandleImpl* h) {
+ // Could be multiple entries in rare cases. Erase them all.
+ // Optimistically increment acquire counter
+ uint64_t old_meta = h->meta.fetch_add(ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+ // Check if it's an entry visible to lookups
+ if ((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateVisible) {
+ // Acquired a read reference
+ if (h->hashed_key == hashed_key) {
+ // Match. Set invisible.
+ old_meta =
+ h->meta.fetch_and(~(uint64_t{ClockHandle::kStateVisibleBit}
+ << ClockHandle::kStateShift),
+ std::memory_order_acq_rel);
+ // Apply update to local copy
+ old_meta &= ~(uint64_t{ClockHandle::kStateVisibleBit}
+ << ClockHandle::kStateShift);
+ for (;;) {
+ uint64_t refcount = GetRefcount(old_meta);
+ assert(refcount > 0);
+ if (refcount > 1) {
+ // Not last ref at some point in time during this Erase call
+ // Pretend we never took the reference
+ h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ break;
+ } else if (h->meta.compare_exchange_weak(
+ old_meta,
+ uint64_t{ClockHandle::kStateConstruction}
+ << ClockHandle::kStateShift,
+ std::memory_order_acq_rel)) {
+ // Took ownership
+ assert(hashed_key == h->hashed_key);
+ size_t total_charge = h->GetTotalCharge();
+ FreeDataMarkEmpty(*h);
+ ReclaimEntryUsage(total_charge);
+ // We already have a copy of hashed_key in this case, so OK to
+ // delay Rollback until after releasing the entry
+ Rollback(hashed_key, h);
+ break;
+ }
+ }
+ } else {
+ // Mismatch. Pretend we never took the reference
+ h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ }
+ } else if (UNLIKELY((old_meta >> ClockHandle::kStateShift) ==
+ ClockHandle::kStateInvisible)) {
+ // Pretend we never took the reference
+ // WART: there's a tiny chance we release last ref to invisible
+ // entry here. If that happens, we let eviction take care of it.
+ h->meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ } else {
+ // For other states, incrementing the acquire counter has no effect
+ // so we don't need to undo it.
+ }
+ return false;
+ },
+ [&](HandleImpl* h) {
+ return h->displacements.load(std::memory_order_relaxed) == 0;
+ },
+ [&](HandleImpl* /*h*/) {}, probe);
+}
+
+void HyperClockTable::ConstApplyToEntriesRange(
+ std::function<void(const HandleImpl&)> func, size_t index_begin,
+ size_t index_end, bool apply_if_will_be_deleted) const {
+ uint64_t check_state_mask = ClockHandle::kStateShareableBit;
+ if (!apply_if_will_be_deleted) {
+ check_state_mask |= ClockHandle::kStateVisibleBit;
+ }
+
+ for (size_t i = index_begin; i < index_end; i++) {
+ HandleImpl& h = array_[i];
+
+ // Note: to avoid using compare_exchange, we have to be extra careful.
+ uint64_t old_meta = h.meta.load(std::memory_order_relaxed);
+ // Check if it's an entry visible to lookups
+ if ((old_meta >> ClockHandle::kStateShift) & check_state_mask) {
+ // Increment acquire counter. Note: it's possible that the entry has
+ // completely changed since we loaded old_meta, but incrementing acquire
+ // count is always safe. (Similar to optimistic Lookup here.)
+ old_meta = h.meta.fetch_add(ClockHandle::kAcquireIncrement,
+ std::memory_order_acquire);
+ // Check whether we actually acquired a reference.
+ if ((old_meta >> ClockHandle::kStateShift) &
+ ClockHandle::kStateShareableBit) {
+ // Apply func if appropriate
+ if ((old_meta >> ClockHandle::kStateShift) & check_state_mask) {
+ func(h);
+ }
+ // Pretend we never took the reference
+ h.meta.fetch_sub(ClockHandle::kAcquireIncrement,
+ std::memory_order_release);
+ // No net change, so don't need to check for overflow
+ } else {
+ // For other states, incrementing the acquire counter has no effect
+ // so we don't need to undo it. Furthermore, we cannot safely undo
+ // it because we did not acquire a read reference to lock the
+ // entry in a Shareable state.
+ }
+ }
+ }
+}
+
+void HyperClockTable::EraseUnRefEntries() {
+ for (size_t i = 0; i <= this->length_bits_mask_; i++) {
+ HandleImpl& h = array_[i];
+
+ uint64_t old_meta = h.meta.load(std::memory_order_relaxed);
+ if (old_meta & (uint64_t{ClockHandle::kStateShareableBit}
+ << ClockHandle::kStateShift) &&
+ GetRefcount(old_meta) == 0 &&
+ h.meta.compare_exchange_strong(old_meta,
+ uint64_t{ClockHandle::kStateConstruction}
+ << ClockHandle::kStateShift,
+ std::memory_order_acquire)) {
+ // Took ownership
+ size_t total_charge = h.GetTotalCharge();
+ Rollback(h.hashed_key, &h);
+ FreeDataMarkEmpty(h);
+ ReclaimEntryUsage(total_charge);
+ }
+ }
+}
+
+inline HyperClockTable::HandleImpl* HyperClockTable::FindSlot(
+ const UniqueId64x2& hashed_key, std::function<bool(HandleImpl*)> match_fn,
+ std::function<bool(HandleImpl*)> abort_fn,
+ std::function<void(HandleImpl*)> update_fn, size_t& probe) {
+ // NOTE: upper 32 bits of hashed_key[0] is used for sharding
+ //
+ // We use double-hashing probing. Every probe in the sequence is a
+ // pseudorandom integer, computed as a linear function of two random hashes,
+ // which we call base and increment. Specifically, the i-th probe is base + i
+ // * increment modulo the table size.
+ size_t base = static_cast<size_t>(hashed_key[1]);
+ // We use an odd increment, which is relatively prime with the power-of-two
+ // table size. This implies that we cycle back to the first probe only
+ // after probing every slot exactly once.
+ // TODO: we could also reconsider linear probing, though locality benefits
+ // are limited because each slot is a full cache line
+ size_t increment = static_cast<size_t>(hashed_key[0]) | 1U;
+ size_t current = ModTableSize(base + probe * increment);
+ while (probe <= length_bits_mask_) {
+ HandleImpl* h = &array_[current];
+ if (match_fn(h)) {
+ probe++;
+ return h;
+ }
+ if (abort_fn(h)) {
+ return nullptr;
+ }
+ probe++;
+ update_fn(h);
+ current = ModTableSize(current + increment);
+ }
+ // We looped back.
+ return nullptr;
+}
+
+inline void HyperClockTable::Rollback(const UniqueId64x2& hashed_key,
+ const HandleImpl* h) {
+ size_t current = ModTableSize(hashed_key[1]);
+ size_t increment = static_cast<size_t>(hashed_key[0]) | 1U;
+ while (&array_[current] != h) {
+ array_[current].displacements.fetch_sub(1, std::memory_order_relaxed);
+ current = ModTableSize(current + increment);
+ }
+}
+
+inline void HyperClockTable::ReclaimEntryUsage(size_t total_charge) {
+ auto old_occupancy = occupancy_.fetch_sub(1U, std::memory_order_release);
+ (void)old_occupancy;
+ // No underflow
+ assert(old_occupancy > 0);
+ auto old_usage = usage_.fetch_sub(total_charge, std::memory_order_relaxed);
+ (void)old_usage;
+ // No underflow
+ assert(old_usage >= total_charge);
+}
+
+inline void HyperClockTable::Evict(size_t requested_charge,
+ size_t* freed_charge, size_t* freed_count) {
+ // precondition
+ assert(requested_charge > 0);
+
+ // TODO: make a tuning parameter?
+ constexpr size_t step_size = 4;
+
+ // First (concurrent) increment clock pointer
+ uint64_t old_clock_pointer =
+ clock_pointer_.fetch_add(step_size, std::memory_order_relaxed);
+
+ // Cap the eviction effort at this thread (along with those operating in
+ // parallel) circling through the whole structure kMaxCountdown times.
+ // In other words, this eviction run must find something/anything that is
+ // unreferenced at start of and during the eviction run that isn't reclaimed
+ // by a concurrent eviction run.
+ uint64_t max_clock_pointer =
+ old_clock_pointer + (ClockHandle::kMaxCountdown << length_bits_);
+
+ for (;;) {
+ for (size_t i = 0; i < step_size; i++) {
+ HandleImpl& h = array_[ModTableSize(Lower32of64(old_clock_pointer + i))];
+ bool evicting = ClockUpdate(h);
+ if (evicting) {
+ Rollback(h.hashed_key, &h);
+ *freed_charge += h.GetTotalCharge();
+ *freed_count += 1;
+ FreeDataMarkEmpty(h);
+ }
+ }
+
+ // Loop exit condition
+ if (*freed_charge >= requested_charge) {
+ return;
+ }
+ if (old_clock_pointer >= max_clock_pointer) {
+ return;
+ }
+
+ // Advance clock pointer (concurrently)
+ old_clock_pointer =
+ clock_pointer_.fetch_add(step_size, std::memory_order_relaxed);
+ }
+}
+
+template <class Table>
+ClockCacheShard<Table>::ClockCacheShard(
+ size_t capacity, bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ const typename Table::Opts& opts)
+ : CacheShardBase(metadata_charge_policy),
+ table_(capacity, strict_capacity_limit, metadata_charge_policy, opts),
+ capacity_(capacity),
+ strict_capacity_limit_(strict_capacity_limit) {
+ // Initial charge metadata should not exceed capacity
+ assert(table_.GetUsage() <= capacity_ || capacity_ < sizeof(HandleImpl));
+}
+
+template <class Table>
+void ClockCacheShard<Table>::EraseUnRefEntries() {
+ table_.EraseUnRefEntries();
+}
+
+template <class Table>
+void ClockCacheShard<Table>::ApplyToSomeEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ size_t average_entries_per_lock, size_t* state) {
+ // The state is essentially going to be the starting hash, which works
+ // nicely even if we resize between calls because we use upper-most
+ // hash bits for table indexes.
+ size_t length_bits = table_.GetLengthBits();
+ size_t length = table_.GetTableSize();
+
+ assert(average_entries_per_lock > 0);
+ // Assuming we are called with same average_entries_per_lock repeatedly,
+ // this simplifies some logic (index_end will not overflow).
+ assert(average_entries_per_lock < length || *state == 0);
+
+ size_t index_begin = *state >> (sizeof(size_t) * 8u - length_bits);
+ size_t index_end = index_begin + average_entries_per_lock;
+ if (index_end >= length) {
+ // Going to end.
+ index_end = length;
+ *state = SIZE_MAX;
+ } else {
+ *state = index_end << (sizeof(size_t) * 8u - length_bits);
+ }
+
+ table_.ConstApplyToEntriesRange(
+ [callback](const HandleImpl& h) {
+ UniqueId64x2 unhashed;
+ callback(ReverseHash(h.hashed_key, &unhashed), h.value,
+ h.GetTotalCharge(), h.deleter);
+ },
+ index_begin, index_end, false);
+}
+
+int HyperClockTable::CalcHashBits(
+ size_t capacity, size_t estimated_value_size,
+ CacheMetadataChargePolicy metadata_charge_policy) {
+ double average_slot_charge = estimated_value_size * kLoadFactor;
+ if (metadata_charge_policy == kFullChargeCacheMetadata) {
+ average_slot_charge += sizeof(HandleImpl);
+ }
+ assert(average_slot_charge > 0.0);
+ uint64_t num_slots =
+ static_cast<uint64_t>(capacity / average_slot_charge + 0.999999);
+
+ int hash_bits = FloorLog2((num_slots << 1) - 1);
+ if (metadata_charge_policy == kFullChargeCacheMetadata) {
+ // For very small estimated value sizes, it's possible to overshoot
+ while (hash_bits > 0 &&
+ uint64_t{sizeof(HandleImpl)} << hash_bits > capacity) {
+ hash_bits--;
+ }
+ }
+ return hash_bits;
+}
+
+template <class Table>
+void ClockCacheShard<Table>::SetCapacity(size_t capacity) {
+ capacity_.store(capacity, std::memory_order_relaxed);
+ // next Insert will take care of any necessary evictions
+}
+
+template <class Table>
+void ClockCacheShard<Table>::SetStrictCapacityLimit(
+ bool strict_capacity_limit) {
+ strict_capacity_limit_.store(strict_capacity_limit,
+ std::memory_order_relaxed);
+ // next Insert will take care of any necessary evictions
+}
+
+template <class Table>
+Status ClockCacheShard<Table>::Insert(const Slice& key,
+ const UniqueId64x2& hashed_key,
+ void* value, size_t charge,
+ Cache::DeleterFn deleter,
+ HandleImpl** handle,
+ Cache::Priority priority) {
+ if (UNLIKELY(key.size() != kCacheKeySize)) {
+ return Status::NotSupported("ClockCache only supports key size " +
+ std::to_string(kCacheKeySize) + "B");
+ }
+ ClockHandleBasicData proto;
+ proto.hashed_key = hashed_key;
+ proto.value = value;
+ proto.deleter = deleter;
+ 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;
+}
+
+template <class Table>
+typename ClockCacheShard<Table>::HandleImpl* ClockCacheShard<Table>::Lookup(
+ const Slice& key, const UniqueId64x2& hashed_key) {
+ if (UNLIKELY(key.size() != kCacheKeySize)) {
+ return nullptr;
+ }
+ return table_.Lookup(hashed_key);
+}
+
+template <class Table>
+bool ClockCacheShard<Table>::Ref(HandleImpl* h) {
+ if (h == nullptr) {
+ return false;
+ }
+ table_.Ref(*h);
+ return true;
+}
+
+template <class Table>
+bool ClockCacheShard<Table>::Release(HandleImpl* handle, bool useful,
+ bool erase_if_last_ref) {
+ if (handle == nullptr) {
+ return false;
+ }
+ return table_.Release(handle, useful, erase_if_last_ref);
+}
+
+template <class Table>
+void ClockCacheShard<Table>::TEST_RefN(HandleImpl* h, size_t n) {
+ table_.TEST_RefN(*h, n);
+}
+
+template <class Table>
+void ClockCacheShard<Table>::TEST_ReleaseN(HandleImpl* h, size_t n) {
+ table_.TEST_ReleaseN(h, n);
+}
+
+template <class Table>
+bool ClockCacheShard<Table>::Release(HandleImpl* handle,
+ bool erase_if_last_ref) {
+ return Release(handle, /*useful=*/true, erase_if_last_ref);
+}
+
+template <class Table>
+void ClockCacheShard<Table>::Erase(const Slice& key,
+ const UniqueId64x2& hashed_key) {
+ if (UNLIKELY(key.size() != kCacheKeySize)) {
+ return;
+ }
+ table_.Erase(hashed_key);
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetUsage() const {
+ return table_.GetUsage();
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetDetachedUsage() const {
+ return table_.GetDetachedUsage();
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetCapacity() const {
+ return capacity_;
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetPinnedUsage() const {
+ // Computes the pinned usage by scanning the whole hash table. This
+ // is slow, but avoids keeping an exact counter on the clock usage,
+ // i.e., the number of not externally referenced elements.
+ // Why avoid this counter? Because Lookup removes elements from the clock
+ // list, so it would need to update the pinned usage every time,
+ // which creates additional synchronization costs.
+ size_t table_pinned_usage = 0;
+ const bool charge_metadata =
+ metadata_charge_policy_ == kFullChargeCacheMetadata;
+ table_.ConstApplyToEntriesRange(
+ [&table_pinned_usage, charge_metadata](const HandleImpl& h) {
+ uint64_t meta = h.meta.load(std::memory_order_relaxed);
+ uint64_t refcount = GetRefcount(meta);
+ // Holding one ref for ConstApplyToEntriesRange
+ assert(refcount > 0);
+ if (refcount > 1) {
+ table_pinned_usage += h.GetTotalCharge();
+ if (charge_metadata) {
+ table_pinned_usage += sizeof(HandleImpl);
+ }
+ }
+ },
+ 0, table_.GetTableSize(), true);
+
+ return table_pinned_usage + table_.GetDetachedUsage();
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetOccupancyCount() const {
+ return table_.GetOccupancy();
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetOccupancyLimit() const {
+ return table_.GetOccupancyLimit();
+}
+
+template <class Table>
+size_t ClockCacheShard<Table>::GetTableAddressCount() const {
+ return table_.GetTableSize();
+}
+
+// Explicit instantiation
+template class ClockCacheShard<HyperClockTable>;
+
+HyperClockCache::HyperClockCache(
+ size_t capacity, size_t estimated_value_size, int num_shard_bits,
+ bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ std::shared_ptr<MemoryAllocator> memory_allocator)
+ : ShardedCache(capacity, num_shard_bits, strict_capacity_limit,
+ std::move(memory_allocator)) {
+ assert(estimated_value_size > 0 ||
+ metadata_charge_policy != kDontChargeCacheMetadata);
+ // TODO: should not need to go through two levels of pointer indirection to
+ // get to table entries
+ size_t per_shard = GetPerShardCapacity();
+ InitShards([=](Shard* cs) {
+ HyperClockTable::Opts opts;
+ opts.estimated_value_size = estimated_value_size;
+ new (cs)
+ Shard(per_shard, strict_capacity_limit, metadata_charge_policy, opts);
+ });
+}
+
+void* HyperClockCache::Value(Handle* handle) {
+ return reinterpret_cast<const HandleImpl*>(handle)->value;
+}
+
+size_t HyperClockCache::GetCharge(Handle* handle) const {
+ return reinterpret_cast<const HandleImpl*>(handle)->GetTotalCharge();
+}
+
+Cache::DeleterFn HyperClockCache::GetDeleter(Handle* handle) const {
+ auto h = reinterpret_cast<const HandleImpl*>(handle);
+ return h->deleter;
+}
+
+namespace {
+
+// For each cache shard, estimate what the table load factor would be if
+// cache filled to capacity with average entries. This is considered
+// indicative of a potential problem if the shard is essentially operating
+// "at limit", which we define as high actual usage (>80% of capacity)
+// or actual occupancy very close to limit (>95% of limit).
+// Also, for each shard compute the recommended estimated_entry_charge,
+// and keep the minimum one for use as overall recommendation.
+void AddShardEvaluation(const HyperClockCache::Shard& shard,
+ std::vector<double>& predicted_load_factors,
+ size_t& min_recommendation) {
+ size_t usage = shard.GetUsage() - shard.GetDetachedUsage();
+ size_t capacity = shard.GetCapacity();
+ double usage_ratio = 1.0 * usage / capacity;
+
+ size_t occupancy = shard.GetOccupancyCount();
+ size_t occ_limit = shard.GetOccupancyLimit();
+ double occ_ratio = 1.0 * occupancy / occ_limit;
+ if (usage == 0 || occupancy == 0 || (usage_ratio < 0.8 && occ_ratio < 0.95)) {
+ // Skip as described above
+ return;
+ }
+
+ // If filled to capacity, what would the occupancy ratio be?
+ double ratio = occ_ratio / usage_ratio;
+ // Given max load factor, what that load factor be?
+ double lf = ratio * kStrictLoadFactor;
+ predicted_load_factors.push_back(lf);
+
+ // Update min_recommendation also
+ size_t recommendation = usage / occupancy;
+ min_recommendation = std::min(min_recommendation, recommendation);
+}
+
+} // namespace
+
+void HyperClockCache::ReportProblems(
+ const std::shared_ptr<Logger>& info_log) const {
+ uint32_t shard_count = GetNumShards();
+ std::vector<double> predicted_load_factors;
+ size_t min_recommendation = SIZE_MAX;
+ const_cast<HyperClockCache*>(this)->ForEachShard(
+ [&](HyperClockCache::Shard* shard) {
+ AddShardEvaluation(*shard, predicted_load_factors, min_recommendation);
+ });
+
+ if (predicted_load_factors.empty()) {
+ // None operating "at limit" -> nothing to report
+ return;
+ }
+ std::sort(predicted_load_factors.begin(), predicted_load_factors.end());
+
+ // First, if the average load factor is within spec, we aren't going to
+ // complain about a few shards being out of spec.
+ // NOTE: this is only the average among cache shards operating "at limit,"
+ // which should be representative of what we care about. It it normal, even
+ // desirable, for a cache to operate "at limit" so this should not create
+ // selection bias. See AddShardEvaluation().
+ // TODO: Consider detecting cases where decreasing the number of shards
+ // would be good, e.g. serious imbalance among shards.
+ double average_load_factor =
+ std::accumulate(predicted_load_factors.begin(),
+ predicted_load_factors.end(), 0.0) /
+ shard_count;
+
+ constexpr double kLowSpecLoadFactor = kLoadFactor / 2;
+ constexpr double kMidSpecLoadFactor = kLoadFactor / 1.414;
+ if (average_load_factor > kLoadFactor) {
+ // Out of spec => Consider reporting load factor too high
+ // Estimate effective overall capacity loss due to enforcing occupancy limit
+ double lost_portion = 0.0;
+ int over_count = 0;
+ for (double lf : predicted_load_factors) {
+ if (lf > kStrictLoadFactor) {
+ ++over_count;
+ lost_portion += (lf - kStrictLoadFactor) / lf / shard_count;
+ }
+ }
+ // >= 20% loss -> error
+ // >= 10% loss -> consistent warning
+ // >= 1% loss -> intermittent warning
+ InfoLogLevel level = InfoLogLevel::INFO_LEVEL;
+ bool report = true;
+ if (lost_portion > 0.2) {
+ level = InfoLogLevel::ERROR_LEVEL;
+ } else if (lost_portion > 0.1) {
+ level = InfoLogLevel::WARN_LEVEL;
+ } else if (lost_portion > 0.01) {
+ int report_percent = static_cast<int>(lost_portion * 100.0);
+ if (Random::GetTLSInstance()->PercentTrue(report_percent)) {
+ level = InfoLogLevel::WARN_LEVEL;
+ }
+ } else {
+ // don't report
+ report = false;
+ }
+ if (report) {
+ ROCKS_LOG_AT_LEVEL(
+ info_log, level,
+ "HyperClockCache@%p unable to use estimated %.1f%% capacity because "
+ "of "
+ "full occupancy in %d/%u cache shards (estimated_entry_charge too "
+ "high). Recommend estimated_entry_charge=%zu",
+ this, lost_portion * 100.0, over_count, (unsigned)shard_count,
+ min_recommendation);
+ }
+ } else if (average_load_factor < kLowSpecLoadFactor) {
+ // Out of spec => Consider reporting load factor too low
+ // But cautiously because low is not as big of a problem.
+
+ // Only report if highest occupancy shard is also below
+ // spec and only if average is substantially out of spec
+ if (predicted_load_factors.back() < kLowSpecLoadFactor &&
+ average_load_factor < kLowSpecLoadFactor / 1.414) {
+ InfoLogLevel level = InfoLogLevel::INFO_LEVEL;
+ if (average_load_factor < kLowSpecLoadFactor / 2) {
+ level = InfoLogLevel::WARN_LEVEL;
+ }
+ ROCKS_LOG_AT_LEVEL(
+ info_log, level,
+ "HyperClockCache@%p table has low occupancy at full capacity. Higher "
+ "estimated_entry_charge (about %.1fx) would likely improve "
+ "performance. Recommend estimated_entry_charge=%zu",
+ this, kMidSpecLoadFactor / average_load_factor, min_recommendation);
+ }
+ }
+}
+
+} // namespace clock_cache
+
+// DEPRECATED (see public API)
+std::shared_ptr<Cache> NewClockCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy) {
+ return NewLRUCache(capacity, num_shard_bits, strict_capacity_limit,
+ /* high_pri_pool_ratio */ 0.5, nullptr,
+ kDefaultToAdaptiveMutex, metadata_charge_policy,
+ /* low_pri_pool_ratio */ 0.0);
+}
+
+std::shared_ptr<Cache> HyperClockCacheOptions::MakeSharedCache() const {
+ auto my_num_shard_bits = num_shard_bits;
+ if (my_num_shard_bits >= 20) {
+ return nullptr; // The cache cannot be sharded into too many fine pieces.
+ }
+ if (my_num_shard_bits < 0) {
+ // Use larger shard size to reduce risk of large entries clustering
+ // or skewing individual shards.
+ constexpr size_t min_shard_size = 32U * 1024U * 1024U;
+ my_num_shard_bits = GetDefaultCacheShardBits(capacity, min_shard_size);
+ }
+ return std::make_shared<clock_cache::HyperClockCache>(
+ capacity, estimated_entry_charge, my_num_shard_bits,
+ strict_capacity_limit, metadata_charge_policy, memory_allocator);
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/clock_cache.h b/src/rocksdb/cache/clock_cache.h
new file mode 100644
index 000000000..ef1b0ccb7
--- /dev/null
+++ b/src/rocksdb/cache/clock_cache.h
@@ -0,0 +1,701 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <array>
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "cache/cache_key.h"
+#include "cache/sharded_cache.h"
+#include "port/lang.h"
+#include "port/malloc.h"
+#include "port/port.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/secondary_cache.h"
+#include "util/autovector.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace clock_cache {
+
+// Forward declaration of friend class.
+class ClockCacheTest;
+
+// HyperClockCache is an alternative to LRUCache specifically tailored for
+// use as BlockBasedTableOptions::block_cache
+//
+// Benefits
+// --------
+// * Fully lock free (no waits or spins) for efficiency under high concurrency
+// * Optimized for hot path reads. For concurrency control, most Lookup() and
+// essentially all Release() are a single atomic add operation.
+// * Eviction on insertion is fully parallel and lock-free.
+// * Uses a generalized + aging variant of CLOCK eviction that might outperform
+// LRU in some cases. (For background, see
+// https://en.wikipedia.org/wiki/Page_replacement_algorithm)
+//
+// Costs
+// -----
+// * Hash table is not resizable (for lock-free efficiency) so capacity is not
+// dynamically changeable. Rely on an estimated average value (block) size for
+// space+time efficiency. (See estimated_entry_charge option details.)
+// * Insert usually does not (but might) overwrite a previous entry associated
+// with a cache key. This is OK for RocksDB uses of Cache.
+// * Only supports keys of exactly 16 bytes, which is what RocksDB uses for
+// block cache (not row cache or table cache).
+// * SecondaryCache is not supported.
+// * Cache priorities are less aggressively enforced. Unlike LRUCache, enough
+// transient LOW or BOTTOM priority items can evict HIGH priority entries that
+// are not referenced recently (or often) enough.
+// * If pinned entries leave little or nothing eligible for eviction,
+// performance can degrade substantially, because of clock eviction eating
+// CPU looking for evictable entries and because Release does not
+// pro-actively delete unreferenced entries when the cache is over-full.
+// Specifically, this makes this implementation more susceptible to the
+// following combination:
+// * num_shard_bits is high (e.g. 6)
+// * capacity small (e.g. some MBs)
+// * some large individual entries (e.g. non-partitioned filters)
+// where individual entries occupy a large portion of their shard capacity.
+// This should be mostly mitigated by the implementation picking a lower
+// number of cache shards than LRUCache for a given capacity (when
+// num_shard_bits is not overridden; see calls to GetDefaultCacheShardBits()).
+// * With strict_capacity_limit=false, respecting the capacity limit is not as
+// aggressive as LRUCache. The limit might be transiently exceeded by a very
+// small number of entries even when not strictly necessary, and slower to
+// recover after pinning forces limit to be substantially exceeded. (Even with
+// strict_capacity_limit=true, RocksDB will nevertheless transiently allocate
+// memory before discovering it is over the block cache capacity, so this
+// should not be a detectable regression in respecting memory limits, except
+// on exceptionally small caches.)
+// * In some cases, erased or duplicated entries might not be freed
+// immediately. They will eventually be freed by eviction from further Inserts.
+// * Internal metadata can overflow if the number of simultaneous references
+// to a cache handle reaches many millions.
+//
+// High-level eviction algorithm
+// -----------------------------
+// A score (or "countdown") is maintained for each entry, initially determined
+// by priority. The score is incremented on each Lookup, up to a max of 3,
+// though is easily returned to previous state if useful=false with Release.
+// During CLOCK-style eviction iteration, entries with score > 0 are
+// decremented if currently unreferenced and entries with score == 0 are
+// evicted if currently unreferenced. Note that scoring might not be perfect
+// because entries can be referenced transiently within the cache even when
+// there are no outside references to the entry.
+//
+// Cache sharding like LRUCache is used to reduce contention on usage+eviction
+// state, though here the performance improvement from more shards is small,
+// and (as noted above) potentially detrimental if shard capacity is too close
+// to largest entry size. Here cache sharding mostly only affects cache update
+// (Insert / Erase) performance, not read performance.
+//
+// Read efficiency (hot path)
+// --------------------------
+// Mostly to minimize the cost of accessing metadata blocks with
+// cache_index_and_filter_blocks=true, we focus on optimizing Lookup and
+// Release. In terms of concurrency, at a minimum, these operations have
+// to do reference counting (and Lookup has to compare full keys in a safe
+// way). Can we fold in all the other metadata tracking *for free* with
+// Lookup and Release doing a simple atomic fetch_add/fetch_sub? (Assume
+// for the moment that Lookup succeeds on the first probe.)
+//
+// We have a clever way of encoding an entry's reference count and countdown
+// clock so that Lookup and Release are each usually a single atomic addition.
+// In a single metadata word we have both an "acquire" count, incremented by
+// Lookup, and a "release" count, incremented by Release. If useful=false,
+// Release can instead decrement the acquire count. Thus the current ref
+// count is (acquires - releases), and the countdown clock is min(3, acquires).
+// Note that only unreferenced entries (acquires == releases) are eligible
+// for CLOCK manipulation and eviction. We tolerate use of more expensive
+// compare_exchange operations for cache writes (insertions and erasures).
+//
+// In a cache receiving many reads and little or no writes, it is possible
+// for the acquire and release counters to overflow. Assuming the *current*
+// refcount never reaches to many millions, we only have to correct for
+// overflow in both counters in Release, not in Lookup. The overflow check
+// should be only 1-2 CPU cycles per Release because it is a predictable
+// branch on a simple condition on data already in registers.
+//
+// Slot states
+// -----------
+// We encode a state indicator into the same metadata word with the
+// acquire and release counters. This allows bigger state transitions to
+// be atomic. States:
+//
+// * Empty - slot is not in use and unowned. All other metadata and data is
+// in an undefined state.
+// * Construction - slot is exclusively owned by one thread, the thread
+// successfully entering this state, for populating or freeing data.
+// * Shareable (group) - slot holds an entry with counted references for
+// pinning and reading, including
+// * Visible - slot holds an entry that can be returned by Lookup
+// * Invisible - slot holds an entry that is not visible to Lookup
+// (erased by user) but can be read by existing references, and ref count
+// changed by Ref and Release.
+//
+// A special case is "detached" entries, which are heap-allocated handles
+// not in the table. They are always Invisible and freed on zero refs.
+//
+// State transitions:
+// Empty -> Construction (in Insert): The encoding of state enables Insert to
+// perform an optimistic atomic bitwise-or to take ownership if a slot is
+// empty, or otherwise make no state change.
+//
+// Construction -> Visible (in Insert): This can be a simple assignment to the
+// metadata word because the current thread has exclusive ownership and other
+// metadata is meaningless.
+//
+// Visible -> Invisible (in Erase): This can be a bitwise-and while holding
+// a shared reference, which is safe because the change is idempotent (in case
+// of parallel Erase). By the way, we never go Invisible->Visible.
+//
+// Shareable -> Construction (in Evict part of Insert, in Erase, and in
+// Release if Invisible): This is for starting to freeing/deleting an
+// unreferenced entry. We have to use compare_exchange to ensure we only make
+// this transition when there are zero refs.
+//
+// Construction -> Empty (in same places): This is for completing free/delete
+// of an entry. A "release" atomic store suffices, as we have exclusive
+// ownership of the slot but have to ensure none of the data member reads are
+// re-ordered after committing the state transition.
+//
+// Insert
+// ------
+// If Insert were to guarantee replacing an existing entry for a key, there
+// would be complications for concurrency and efficiency. First, consider how
+// many probes to get to an entry. To ensure Lookup never waits and
+// availability of a key is uninterrupted, we would need to use a different
+// slot for a new entry for the same key. This means it is most likely in a
+// later probing position than the old version, which should soon be removed.
+// (Also, an entry is too big to replace atomically, even if no current refs.)
+//
+// However, overwrite capability is not really needed by RocksDB. Also, we
+// know from our "redundant" stats that overwrites are very rare for the block
+// cache, so we should not spend much to make them effective.
+//
+// So instead we Insert as soon as we find an empty slot in the probing
+// sequence without seeing an existing (visible) entry for the same key. This
+// way we only insert if we can improve the probing performance, and we don't
+// need to probe beyond our insert position, assuming we are willing to let
+// the previous entry for the same key die of old age (eventual eviction from
+// not being used). We can reach a similar state with concurrent insertions,
+// where one will pass over the other while it is "under construction."
+// This temporary duplication is acceptable for RocksDB block cache because
+// we know redundant insertion is rare.
+//
+// Another problem to solve is what to return to the caller when we find an
+// existing entry whose probing position we cannot improve on, or when the
+// table occupancy limit has been reached. If strict_capacity_limit=false,
+// we must never fail Insert, and if a Handle* is provided, we have to return
+// a usable Cache handle on success. The solution to this (typically rare)
+// problem is "detached" handles, which are usable by the caller but not
+// actually available for Lookup in the Cache. Detached handles are allocated
+// independently on the heap and specially marked so that they are freed on
+// the heap when their last reference is released.
+//
+// Usage on capacity
+// -----------------
+// Insert takes different approaches to usage tracking depending on
+// strict_capacity_limit setting. If true, we enforce a kind of strong
+// consistency where compare-exchange is used to ensure the usage number never
+// exceeds its limit, and provide threads with an authoritative signal on how
+// much "usage" they have taken ownership of. With strict_capacity_limit=false,
+// we use a kind of "eventual consistency" where all threads Inserting to the
+// same cache shard might race on reserving the same space, but the
+// over-commitment will be worked out in later insertions. It is kind of a
+// dance because we don't want threads racing each other too much on paying
+// down the over-commitment (with eviction) either.
+//
+// Eviction
+// --------
+// A key part of Insert is evicting some entries currently unreferenced to
+// make room for new entries. The high-level eviction algorithm is described
+// above, but the details are also interesting. A key part is parallelizing
+// eviction with a single CLOCK pointer. This works by each thread working on
+// eviction pre-emptively incrementing the CLOCK pointer, and then CLOCK-
+// updating or evicting the incremented-over slot(s). To reduce contention at
+// the cost of possibly evicting too much, each thread increments the clock
+// pointer by 4, so commits to updating at least 4 slots per batch. As
+// described above, a CLOCK update will decrement the "countdown" of
+// unreferenced entries, or evict unreferenced entries with zero countdown.
+// Referenced entries are not updated, because we (presumably) don't want
+// long-referenced entries to age while referenced. Note however that we
+// cannot distinguish transiently referenced entries from cache user
+// references, so some CLOCK updates might be somewhat arbitrarily skipped.
+// This is OK as long as it is rare enough that eviction order is still
+// pretty good.
+//
+// There is no synchronization on the completion of the CLOCK updates, so it
+// is theoretically possible for another thread to cycle back around and have
+// two threads racing on CLOCK updates to the same slot. Thus, we cannot rely
+// on any implied exclusivity to make the updates or eviction more efficient.
+// These updates use an opportunistic compare-exchange (no loop), where a
+// racing thread might cause the update to be skipped without retry, but in
+// such case the update is likely not needed because the most likely update
+// to an entry is that it has become referenced. (TODO: test efficiency of
+// avoiding compare-exchange loop)
+//
+// Release
+// -------
+// In the common case, Release is a simple atomic increment of the release
+// counter. There is a simple overflow check that only does another atomic
+// update in extremely rare cases, so costs almost nothing.
+//
+// If the Release specifies "not useful", we can instead decrement the
+// acquire counter, which returns to the same CLOCK state as before Lookup
+// or Ref.
+//
+// Adding a check for over-full cache on every release to zero-refs would
+// likely be somewhat expensive, increasing read contention on cache shard
+// metadata. Instead we are less aggressive about deleting entries right
+// away in those cases.
+//
+// However Release tries to immediately delete entries reaching zero refs
+// if (a) erase_if_last_ref is set by the caller, or (b) the entry is already
+// marked invisible. Both of these are checks on values already in CPU
+// registers so do not increase cross-CPU contention when not applicable.
+// When applicable, they use a compare-exchange loop to take exclusive
+// ownership of the slot for freeing the entry. These are rare cases
+// that should not usually affect performance.
+//
+// Erase
+// -----
+// Searches for an entry like Lookup but moves it to Invisible state if found.
+// This state transition is with bit operations so is idempotent and safely
+// done while only holding a shared "read" reference. Like Release, it makes
+// a best effort to immediately release an Invisible entry that reaches zero
+// refs, but there are some corner cases where it will only be freed by the
+// clock eviction process.
+
+// ----------------------------------------------------------------------- //
+
+// The load factor p is a real number in (0, 1) such that at all
+// times at most a fraction p of all slots, without counting tombstones,
+// are occupied by elements. This means that the probability that a random
+// probe hits an occupied slot is at most p, and thus at most 1/p probes
+// are required on average. For example, p = 70% implies that between 1 and 2
+// probes are needed on average (bear in mind that this reasoning doesn't
+// consider the effects of clustering over time, which should be negligible
+// with double hashing).
+// Because the size of the hash table is always rounded up to the next
+// power of 2, p is really an upper bound on the actual load factor---the
+// actual load factor is anywhere between p/2 and p. This is a bit wasteful,
+// but bear in mind that slots only hold metadata, not actual values.
+// Since space cost is dominated by the values (the LSM blocks),
+// overprovisioning the table with metadata only increases the total cache space
+// usage by a tiny fraction.
+constexpr double kLoadFactor = 0.7;
+
+// The user can exceed kLoadFactor if the sizes of the inserted values don't
+// match estimated_value_size, or in some rare cases with
+// strict_capacity_limit == false. To avoid degenerate performance, we set a
+// strict upper bound on the load factor.
+constexpr double kStrictLoadFactor = 0.84;
+
+struct ClockHandleBasicData {
+ void* value = nullptr;
+ Cache::DeleterFn deleter = nullptr;
+ // A lossless, reversible hash of the fixed-size (16 byte) cache key. This
+ // eliminates the need to store a hash separately.
+ UniqueId64x2 hashed_key = kNullUniqueId64x2;
+ size_t total_charge = 0;
+
+ // For total_charge_and_flags
+ // "Detached" means the handle is allocated separately from hash table.
+ static constexpr uint64_t kFlagDetached = uint64_t{1} << 63;
+ // Extract just the total charge
+ static constexpr uint64_t kTotalChargeMask = kFlagDetached - 1;
+
+ inline size_t GetTotalCharge() const { return total_charge; }
+
+ // Calls deleter (if non-null) on cache key and value
+ void FreeData() const;
+
+ // Required by concept HandleImpl
+ const UniqueId64x2& GetHash() const { return hashed_key; }
+};
+
+struct ClockHandle : public ClockHandleBasicData {
+ // Constants for handling the atomic `meta` word, which tracks most of the
+ // state of the handle. The meta word looks like this:
+ // low bits high bits
+ // -----------------------------------------------------------------------
+ // | acquire counter | release counter | state marker |
+ // -----------------------------------------------------------------------
+
+ // For reading or updating counters in meta word.
+ static constexpr uint8_t kCounterNumBits = 30;
+ static constexpr uint64_t kCounterMask = (uint64_t{1} << kCounterNumBits) - 1;
+
+ static constexpr uint8_t kAcquireCounterShift = 0;
+ static constexpr uint64_t kAcquireIncrement = uint64_t{1}
+ << kAcquireCounterShift;
+ static constexpr uint8_t kReleaseCounterShift = kCounterNumBits;
+ static constexpr uint64_t kReleaseIncrement = uint64_t{1}
+ << kReleaseCounterShift;
+
+ // For reading or updating the state marker in meta word
+ static constexpr uint8_t kStateShift = 2U * kCounterNumBits;
+
+ // Bits contribution to state marker.
+ // Occupied means any state other than empty
+ static constexpr uint8_t kStateOccupiedBit = 0b100;
+ // Shareable means the entry is reference counted (visible or invisible)
+ // (only set if also occupied)
+ static constexpr uint8_t kStateShareableBit = 0b010;
+ // Visible is only set if also shareable
+ static constexpr uint8_t kStateVisibleBit = 0b001;
+
+ // Complete state markers (not shifted into full word)
+ static constexpr uint8_t kStateEmpty = 0b000;
+ static constexpr uint8_t kStateConstruction = kStateOccupiedBit;
+ static constexpr uint8_t kStateInvisible =
+ kStateOccupiedBit | kStateShareableBit;
+ static constexpr uint8_t kStateVisible =
+ kStateOccupiedBit | kStateShareableBit | kStateVisibleBit;
+
+ // Constants for initializing the countdown clock. (Countdown clock is only
+ // in effect with zero refs, acquire counter == release counter, and in that
+ // case the countdown clock == both of those counters.)
+ static constexpr uint8_t kHighCountdown = 3;
+ static constexpr uint8_t kLowCountdown = 2;
+ static constexpr uint8_t kBottomCountdown = 1;
+ // During clock update, treat any countdown clock value greater than this
+ // value the same as this value.
+ static constexpr uint8_t kMaxCountdown = kHighCountdown;
+ // TODO: make these coundown values tuning parameters for eviction?
+
+ // See above
+ std::atomic<uint64_t> meta{};
+
+ // Anticipating use for SecondaryCache support
+ void* reserved_for_future_use = nullptr;
+}; // struct ClockHandle
+
+class HyperClockTable {
+ public:
+ // Target size to be exactly a common cache line size (see static_assert in
+ // clock_cache.cc)
+ struct ALIGN_AS(64U) HandleImpl : public ClockHandle {
+ // The number of elements that hash to this slot or a lower one, but wind
+ // up in this slot or a higher one.
+ std::atomic<uint32_t> displacements{};
+
+ // Whether this is a "deteched" handle that is independently allocated
+ // with `new` (so must be deleted with `delete`).
+ // TODO: ideally this would be packed into some other data field, such
+ // as upper bits of total_charge, but that incurs a measurable performance
+ // regression.
+ bool detached = false;
+
+ inline bool IsDetached() const { return detached; }
+
+ inline void SetDetached() { detached = true; }
+ }; // struct HandleImpl
+
+ struct Opts {
+ size_t estimated_value_size;
+ };
+
+ HyperClockTable(size_t capacity, bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ const Opts& opts);
+ ~HyperClockTable();
+
+ Status Insert(const ClockHandleBasicData& proto, HandleImpl** handle,
+ Cache::Priority priority, size_t capacity,
+ bool strict_capacity_limit);
+
+ HandleImpl* Lookup(const UniqueId64x2& hashed_key);
+
+ bool Release(HandleImpl* handle, bool useful, bool erase_if_last_ref);
+
+ void Ref(HandleImpl& handle);
+
+ void Erase(const UniqueId64x2& hashed_key);
+
+ void ConstApplyToEntriesRange(std::function<void(const HandleImpl&)> func,
+ size_t index_begin, size_t index_end,
+ bool apply_if_will_be_deleted) const;
+
+ void EraseUnRefEntries();
+
+ size_t GetTableSize() const { return size_t{1} << length_bits_; }
+
+ int GetLengthBits() const { return length_bits_; }
+
+ size_t GetOccupancy() const {
+ return occupancy_.load(std::memory_order_relaxed);
+ }
+
+ size_t GetOccupancyLimit() const { return occupancy_limit_; }
+
+ size_t GetUsage() const { return usage_.load(std::memory_order_relaxed); }
+
+ size_t GetDetachedUsage() const {
+ return detached_usage_.load(std::memory_order_relaxed);
+ }
+
+ // Acquire/release N references
+ void TEST_RefN(HandleImpl& handle, size_t n);
+ void TEST_ReleaseN(HandleImpl* handle, size_t n);
+
+ private: // functions
+ // Returns x mod 2^{length_bits_}.
+ inline size_t ModTableSize(uint64_t x) {
+ return static_cast<size_t>(x) & length_bits_mask_;
+ }
+
+ // Runs the clock eviction algorithm trying to reclaim at least
+ // requested_charge. Returns how much is evicted, which could be less
+ // if it appears impossible to evict the requested amount without blocking.
+ inline void Evict(size_t requested_charge, size_t* freed_charge,
+ size_t* freed_count);
+
+ // Returns the first slot in the probe sequence, starting from the given
+ // probe number, with a handle e such that match(e) is true. At every
+ // step, the function first tests whether match(e) holds. If this is false,
+ // it evaluates abort(e) to decide whether the search should be aborted,
+ // and in the affirmative returns -1. For every handle e probed except
+ // the last one, the function runs update(e).
+ // The probe parameter is modified as follows. We say a probe to a handle
+ // e is aborting if match(e) is false and abort(e) is true. Then the final
+ // value of probe is one more than the last non-aborting probe during the
+ // call. This is so that that the variable can be used to keep track of
+ // progress across consecutive calls to FindSlot.
+ inline HandleImpl* FindSlot(const UniqueId64x2& hashed_key,
+ std::function<bool(HandleImpl*)> match,
+ std::function<bool(HandleImpl*)> stop,
+ std::function<void(HandleImpl*)> update,
+ size_t& probe);
+
+ // Re-decrement all displacements in probe path starting from beginning
+ // until (not including) the given handle
+ inline void Rollback(const UniqueId64x2& hashed_key, const HandleImpl* h);
+
+ // Subtracts `total_charge` from `usage_` and 1 from `occupancy_`.
+ // Ideally this comes after releasing the entry itself so that we
+ // actually have the available occupancy/usage that is claimed.
+ // However, that means total_charge has to be saved from the handle
+ // before releasing it so that it can be provided to this function.
+ inline void ReclaimEntryUsage(size_t total_charge);
+
+ // Helper for updating `usage_` for new entry with given `total_charge`
+ // and evicting if needed under strict_capacity_limit=true rules. This
+ // means the operation might fail with Status::MemoryLimit. If
+ // `need_evict_for_occupancy`, then eviction of at least one entry is
+ // required, and the operation should fail if not possible.
+ // NOTE: Otherwise, occupancy_ is not managed in this function
+ inline Status ChargeUsageMaybeEvictStrict(size_t total_charge,
+ size_t capacity,
+ bool need_evict_for_occupancy);
+
+ // Helper for updating `usage_` for new entry with given `total_charge`
+ // and evicting if needed under strict_capacity_limit=false rules. This
+ // means that updating `usage_` always succeeds even if forced to exceed
+ // capacity. If `need_evict_for_occupancy`, then eviction of at least one
+ // entry is required, and the operation should return false if such eviction
+ // is not possible. `usage_` is not updated in that case. Otherwise, returns
+ // true, indicating success.
+ // NOTE: occupancy_ is not managed in this function
+ inline bool ChargeUsageMaybeEvictNonStrict(size_t total_charge,
+ size_t capacity,
+ bool need_evict_for_occupancy);
+
+ // Creates a "detached" handle for returning from an Insert operation that
+ // cannot be completed by actually inserting into the table.
+ // Updates `detached_usage_` but not `usage_` nor `occupancy_`.
+ inline HandleImpl* DetachedInsert(const ClockHandleBasicData& proto);
+
+ // Returns the number of bits used to hash an element in the hash
+ // table.
+ static int CalcHashBits(size_t capacity, size_t estimated_value_size,
+ CacheMetadataChargePolicy metadata_charge_policy);
+
+ private: // data
+ // Number of hash bits used for table index.
+ // The size of the table is 1 << length_bits_.
+ const int length_bits_;
+
+ // For faster computation of ModTableSize.
+ const size_t length_bits_mask_;
+
+ // Maximum number of elements the user can store in the table.
+ const size_t occupancy_limit_;
+
+ // Array of slots comprising the hash table.
+ const std::unique_ptr<HandleImpl[]> array_;
+
+ // We partition the following members into different cache lines
+ // to avoid false sharing among Lookup, Release, Erase and Insert
+ // operations in ClockCacheShard.
+
+ ALIGN_AS(CACHE_LINE_SIZE)
+ // Clock algorithm sweep pointer.
+ std::atomic<uint64_t> clock_pointer_{};
+
+ ALIGN_AS(CACHE_LINE_SIZE)
+ // Number of elements in the table.
+ std::atomic<size_t> occupancy_{};
+
+ // Memory usage by entries tracked by the cache (including detached)
+ std::atomic<size_t> usage_{};
+
+ // Part of usage by detached entries (not in table)
+ std::atomic<size_t> detached_usage_{};
+}; // class HyperClockTable
+
+// A single shard of sharded cache.
+template <class Table>
+class ALIGN_AS(CACHE_LINE_SIZE) ClockCacheShard final : public CacheShardBase {
+ public:
+ ClockCacheShard(size_t capacity, bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ const typename Table::Opts& opts);
+
+ // For CacheShard concept
+ using HandleImpl = typename Table::HandleImpl;
+ // Hash is lossless hash of 128-bit key
+ using HashVal = UniqueId64x2;
+ using HashCref = const HashVal&;
+ static inline uint32_t HashPieceForSharding(HashCref hash) {
+ return Upper32of64(hash[0]);
+ }
+ static inline HashVal ComputeHash(const Slice& key) {
+ assert(key.size() == kCacheKeySize);
+ HashVal in;
+ HashVal out;
+ // NOTE: endian dependence
+ // TODO: use GetUnaligned?
+ std::memcpy(&in, key.data(), kCacheKeySize);
+ BijectiveHash2x64(in[1], in[0], &out[1], &out[0]);
+ return out;
+ }
+
+ // For reconstructing key from hashed_key. Requires the caller to provide
+ // backing storage for the Slice in `unhashed`
+ static inline Slice ReverseHash(const UniqueId64x2& hashed,
+ UniqueId64x2* unhashed) {
+ BijectiveUnhash2x64(hashed[1], hashed[0], &(*unhashed)[1], &(*unhashed)[0]);
+ // NOTE: endian dependence
+ return Slice(reinterpret_cast<const char*>(unhashed), kCacheKeySize);
+ }
+
+ // Although capacity is dynamically changeable, the number of table slots is
+ // not, so growing capacity substantially could lead to hitting occupancy
+ // limit.
+ void SetCapacity(size_t capacity);
+
+ void SetStrictCapacityLimit(bool strict_capacity_limit);
+
+ Status Insert(const Slice& key, const UniqueId64x2& hashed_key, void* value,
+ size_t charge, Cache::DeleterFn deleter, HandleImpl** handle,
+ Cache::Priority priority);
+
+ HandleImpl* Lookup(const Slice& key, const UniqueId64x2& hashed_key);
+
+ bool Release(HandleImpl* handle, bool useful, bool erase_if_last_ref);
+
+ bool Release(HandleImpl* handle, bool erase_if_last_ref = false);
+
+ bool Ref(HandleImpl* handle);
+
+ void Erase(const Slice& key, const UniqueId64x2& hashed_key);
+
+ size_t GetCapacity() const;
+
+ size_t GetUsage() const;
+
+ size_t GetDetachedUsage() const;
+
+ size_t GetPinnedUsage() const;
+
+ size_t GetOccupancyCount() const;
+
+ size_t GetOccupancyLimit() const;
+
+ size_t GetTableAddressCount() const;
+
+ void ApplyToSomeEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ size_t average_entries_per_lock, size_t* state);
+
+ void EraseUnRefEntries();
+
+ std::string GetPrintableOptions() const { return std::string{}; }
+
+ // SecondaryCache not yet supported
+ Status Insert(const Slice& key, const UniqueId64x2& hashed_key, void* value,
+ const Cache::CacheItemHelper* helper, size_t charge,
+ HandleImpl** handle, Cache::Priority priority) {
+ return Insert(key, hashed_key, value, charge, helper->del_cb, handle,
+ priority);
+ }
+
+ HandleImpl* Lookup(const Slice& key, const UniqueId64x2& hashed_key,
+ const Cache::CacheItemHelper* /*helper*/,
+ const Cache::CreateCallback& /*create_cb*/,
+ Cache::Priority /*priority*/, bool /*wait*/,
+ 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);
+
+ private: // data
+ Table table_;
+
+ // Maximum total charge of all elements stored in the table.
+ std::atomic<size_t> capacity_;
+
+ // Whether to reject insertion if cache reaches its full capacity.
+ std::atomic<bool> strict_capacity_limit_;
+}; // class ClockCacheShard
+
+class HyperClockCache
+#ifdef NDEBUG
+ final
+#endif
+ : public ShardedCache<ClockCacheShard<HyperClockTable>> {
+ public:
+ using Shard = ClockCacheShard<HyperClockTable>;
+
+ HyperClockCache(size_t capacity, size_t estimated_value_size,
+ int num_shard_bits, bool strict_capacity_limit,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ std::shared_ptr<MemoryAllocator> memory_allocator);
+
+ const char* Name() const override { return "HyperClockCache"; }
+
+ void* Value(Handle* handle) override;
+
+ size_t GetCharge(Handle* handle) const override;
+
+ DeleterFn GetDeleter(Handle* handle) const override;
+
+ void ReportProblems(
+ const std::shared_ptr<Logger>& /*info_log*/) const override;
+}; // class HyperClockCache
+
+} // namespace clock_cache
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/compressed_secondary_cache.cc b/src/rocksdb/cache/compressed_secondary_cache.cc
new file mode 100644
index 000000000..7d1bdc789
--- /dev/null
+++ b/src/rocksdb/cache/compressed_secondary_cache.cc
@@ -0,0 +1,325 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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/compressed_secondary_cache.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <memory>
+
+#include "memory/memory_allocator.h"
+#include "monitoring/perf_context_imp.h"
+#include "util/compression.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+CompressedSecondaryCache::CompressedSecondaryCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio, double low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator, bool use_adaptive_mutex,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ CompressionType compression_type, uint32_t compress_format_version,
+ bool enable_custom_split_merge)
+ : cache_options_(capacity, num_shard_bits, strict_capacity_limit,
+ high_pri_pool_ratio, low_pri_pool_ratio, memory_allocator,
+ use_adaptive_mutex, metadata_charge_policy,
+ compression_type, compress_format_version,
+ enable_custom_split_merge) {
+ cache_ =
+ NewLRUCache(capacity, num_shard_bits, strict_capacity_limit,
+ high_pri_pool_ratio, memory_allocator, use_adaptive_mutex,
+ metadata_charge_policy, low_pri_pool_ratio);
+}
+
+CompressedSecondaryCache::~CompressedSecondaryCache() { cache_.reset(); }
+
+std::unique_ptr<SecondaryCacheResultHandle> CompressedSecondaryCache::Lookup(
+ const Slice& key, const Cache::CreateCallback& create_cb, bool /*wait*/,
+ bool advise_erase, bool& is_in_sec_cache) {
+ std::unique_ptr<SecondaryCacheResultHandle> handle;
+ is_in_sec_cache = false;
+ Cache::Handle* lru_handle = cache_->Lookup(key);
+ if (lru_handle == nullptr) {
+ return nullptr;
+ }
+
+ void* handle_value = cache_->Value(lru_handle);
+ if (handle_value == nullptr) {
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/false);
+ return nullptr;
+ }
+
+ CacheAllocationPtr* ptr{nullptr};
+ CacheAllocationPtr merged_value;
+ size_t handle_value_charge{0};
+ if (cache_options_.enable_custom_split_merge) {
+ CacheValueChunk* value_chunk_ptr =
+ reinterpret_cast<CacheValueChunk*>(handle_value);
+ merged_value = MergeChunksIntoValue(value_chunk_ptr, handle_value_charge);
+ ptr = &merged_value;
+ } else {
+ ptr = reinterpret_cast<CacheAllocationPtr*>(handle_value);
+ handle_value_charge = cache_->GetCharge(lru_handle);
+ }
+
+ Status s;
+ void* value{nullptr};
+ size_t charge{0};
+ if (cache_options_.compression_type == kNoCompression) {
+ s = create_cb(ptr->get(), handle_value_charge, &value, &charge);
+ } else {
+ UncompressionContext uncompression_context(cache_options_.compression_type);
+ UncompressionInfo uncompression_info(uncompression_context,
+ UncompressionDict::GetEmptyDict(),
+ cache_options_.compression_type);
+
+ size_t uncompressed_size{0};
+ CacheAllocationPtr uncompressed = UncompressData(
+ uncompression_info, (char*)ptr->get(), handle_value_charge,
+ &uncompressed_size, cache_options_.compress_format_version,
+ cache_options_.memory_allocator.get());
+
+ if (!uncompressed) {
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/true);
+ return nullptr;
+ }
+ s = create_cb(uncompressed.get(), uncompressed_size, &value, &charge);
+ }
+
+ if (!s.ok()) {
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/true);
+ return nullptr;
+ }
+
+ if (advise_erase) {
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/true);
+ // Insert a dummy handle.
+ cache_
+ ->Insert(key, /*value=*/nullptr, /*charge=*/0,
+ GetDeletionCallback(cache_options_.enable_custom_split_merge))
+ .PermitUncheckedError();
+ } else {
+ is_in_sec_cache = true;
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/false);
+ }
+ handle.reset(new CompressedSecondaryCacheResultHandle(value, charge));
+ return handle;
+}
+
+Status CompressedSecondaryCache::Insert(const Slice& key, void* value,
+ const Cache::CacheItemHelper* helper) {
+ if (value == nullptr) {
+ return Status::InvalidArgument();
+ }
+
+ Cache::Handle* lru_handle = cache_->Lookup(key);
+ Cache::DeleterFn del_cb =
+ GetDeletionCallback(cache_options_.enable_custom_split_merge);
+ if (lru_handle == nullptr) {
+ PERF_COUNTER_ADD(compressed_sec_cache_insert_dummy_count, 1);
+ // Insert a dummy handle if the handle is evicted for the first time.
+ return cache_->Insert(key, /*value=*/nullptr, /*charge=*/0, del_cb);
+ } else {
+ cache_->Release(lru_handle, /*erase_if_last_ref=*/false);
+ }
+
+ size_t size = (*helper->size_cb)(value);
+ CacheAllocationPtr ptr =
+ AllocateBlock(size, cache_options_.memory_allocator.get());
+
+ Status s = (*helper->saveto_cb)(value, 0, size, ptr.get());
+ if (!s.ok()) {
+ return s;
+ }
+ Slice val(ptr.get(), size);
+
+ std::string compressed_val;
+ if (cache_options_.compression_type != kNoCompression) {
+ PERF_COUNTER_ADD(compressed_sec_cache_uncompressed_bytes, size);
+ CompressionOptions compression_opts;
+ CompressionContext compression_context(cache_options_.compression_type);
+ uint64_t sample_for_compression{0};
+ CompressionInfo compression_info(
+ compression_opts, compression_context, CompressionDict::GetEmptyDict(),
+ cache_options_.compression_type, sample_for_compression);
+
+ bool success =
+ CompressData(val, compression_info,
+ cache_options_.compress_format_version, &compressed_val);
+
+ if (!success) {
+ return Status::Corruption("Error compressing value.");
+ }
+
+ val = Slice(compressed_val);
+ size = compressed_val.size();
+ PERF_COUNTER_ADD(compressed_sec_cache_compressed_bytes, size);
+
+ if (!cache_options_.enable_custom_split_merge) {
+ ptr = AllocateBlock(size, cache_options_.memory_allocator.get());
+ memcpy(ptr.get(), compressed_val.data(), size);
+ }
+ }
+
+ PERF_COUNTER_ADD(compressed_sec_cache_insert_real_count, 1);
+ if (cache_options_.enable_custom_split_merge) {
+ size_t charge{0};
+ CacheValueChunk* value_chunks_head =
+ SplitValueIntoChunks(val, cache_options_.compression_type, charge);
+ return cache_->Insert(key, value_chunks_head, charge, del_cb);
+ } else {
+ CacheAllocationPtr* buf = new CacheAllocationPtr(std::move(ptr));
+ return cache_->Insert(key, buf, size, del_cb);
+ }
+}
+
+void CompressedSecondaryCache::Erase(const Slice& key) { cache_->Erase(key); }
+
+Status CompressedSecondaryCache::SetCapacity(size_t capacity) {
+ MutexLock l(&capacity_mutex_);
+ cache_options_.capacity = capacity;
+ cache_->SetCapacity(capacity);
+ return Status::OK();
+}
+
+Status CompressedSecondaryCache::GetCapacity(size_t& capacity) {
+ MutexLock l(&capacity_mutex_);
+ capacity = cache_options_.capacity;
+ return Status::OK();
+}
+
+std::string CompressedSecondaryCache::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(20000);
+ const int kBufferSize{200};
+ char buffer[kBufferSize];
+ ret.append(cache_->GetPrintableOptions());
+ snprintf(buffer, kBufferSize, " compression_type : %s\n",
+ CompressionTypeToString(cache_options_.compression_type).c_str());
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " compress_format_version : %d\n",
+ cache_options_.compress_format_version);
+ ret.append(buffer);
+ return ret;
+}
+
+CompressedSecondaryCache::CacheValueChunk*
+CompressedSecondaryCache::SplitValueIntoChunks(const Slice& value,
+ CompressionType compression_type,
+ size_t& charge) {
+ assert(!value.empty());
+ const char* src_ptr = value.data();
+ size_t src_size{value.size()};
+
+ CacheValueChunk dummy_head = CacheValueChunk();
+ CacheValueChunk* current_chunk = &dummy_head;
+ // Do not split when value size is large or there is no compression.
+ size_t predicted_chunk_size{0};
+ size_t actual_chunk_size{0};
+ size_t tmp_size{0};
+ while (src_size > 0) {
+ predicted_chunk_size = sizeof(CacheValueChunk) - 1 + src_size;
+ auto upper =
+ std::upper_bound(malloc_bin_sizes_.begin(), malloc_bin_sizes_.end(),
+ predicted_chunk_size);
+ // Do not split when value size is too small, too large, close to a bin
+ // size, or there is no compression.
+ if (upper == malloc_bin_sizes_.begin() ||
+ upper == malloc_bin_sizes_.end() ||
+ *upper - predicted_chunk_size < malloc_bin_sizes_.front() ||
+ compression_type == kNoCompression) {
+ tmp_size = predicted_chunk_size;
+ } else {
+ tmp_size = *(--upper);
+ }
+
+ CacheValueChunk* new_chunk =
+ reinterpret_cast<CacheValueChunk*>(new char[tmp_size]);
+ current_chunk->next = new_chunk;
+ current_chunk = current_chunk->next;
+ actual_chunk_size = tmp_size - sizeof(CacheValueChunk) + 1;
+ memcpy(current_chunk->data, src_ptr, actual_chunk_size);
+ current_chunk->size = actual_chunk_size;
+ src_ptr += actual_chunk_size;
+ src_size -= actual_chunk_size;
+ charge += tmp_size;
+ }
+ current_chunk->next = nullptr;
+
+ return dummy_head.next;
+}
+
+CacheAllocationPtr CompressedSecondaryCache::MergeChunksIntoValue(
+ const void* chunks_head, size_t& charge) {
+ const CacheValueChunk* head =
+ reinterpret_cast<const CacheValueChunk*>(chunks_head);
+ const CacheValueChunk* current_chunk = head;
+ charge = 0;
+ while (current_chunk != nullptr) {
+ charge += current_chunk->size;
+ current_chunk = current_chunk->next;
+ }
+
+ CacheAllocationPtr ptr =
+ AllocateBlock(charge, cache_options_.memory_allocator.get());
+ current_chunk = head;
+ size_t pos{0};
+ while (current_chunk != nullptr) {
+ memcpy(ptr.get() + pos, current_chunk->data, current_chunk->size);
+ pos += current_chunk->size;
+ current_chunk = current_chunk->next;
+ }
+
+ return ptr;
+}
+
+Cache::DeleterFn CompressedSecondaryCache::GetDeletionCallback(
+ bool enable_custom_split_merge) {
+ if (enable_custom_split_merge) {
+ return [](const Slice& /*key*/, void* obj) {
+ CacheValueChunk* chunks_head = reinterpret_cast<CacheValueChunk*>(obj);
+ while (chunks_head != nullptr) {
+ CacheValueChunk* tmp_chunk = chunks_head;
+ chunks_head = chunks_head->next;
+ tmp_chunk->Free();
+ obj = nullptr;
+ };
+ };
+ } else {
+ return [](const Slice& /*key*/, void* obj) {
+ delete reinterpret_cast<CacheAllocationPtr*>(obj);
+ obj = nullptr;
+ };
+ }
+}
+
+std::shared_ptr<SecondaryCache> NewCompressedSecondaryCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio, double low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator, bool use_adaptive_mutex,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ CompressionType compression_type, uint32_t compress_format_version,
+ bool enable_custom_split_merge) {
+ return std::make_shared<CompressedSecondaryCache>(
+ capacity, num_shard_bits, strict_capacity_limit, high_pri_pool_ratio,
+ low_pri_pool_ratio, memory_allocator, use_adaptive_mutex,
+ metadata_charge_policy, compression_type, compress_format_version,
+ enable_custom_split_merge);
+}
+
+std::shared_ptr<SecondaryCache> NewCompressedSecondaryCache(
+ const CompressedSecondaryCacheOptions& opts) {
+ // The secondary_cache is disabled for this LRUCache instance.
+ assert(opts.secondary_cache == nullptr);
+ return NewCompressedSecondaryCache(
+ opts.capacity, opts.num_shard_bits, opts.strict_capacity_limit,
+ opts.high_pri_pool_ratio, opts.low_pri_pool_ratio, opts.memory_allocator,
+ opts.use_adaptive_mutex, opts.metadata_charge_policy,
+ opts.compression_type, opts.compress_format_version,
+ opts.enable_custom_split_merge);
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/compressed_secondary_cache.h b/src/rocksdb/cache/compressed_secondary_cache.h
new file mode 100644
index 000000000..4dee38802
--- /dev/null
+++ b/src/rocksdb/cache/compressed_secondary_cache.h
@@ -0,0 +1,139 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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 <array>
+#include <cstddef>
+#include <memory>
+
+#include "cache/lru_cache.h"
+#include "memory/memory_allocator.h"
+#include "rocksdb/secondary_cache.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "util/compression.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class CompressedSecondaryCacheResultHandle : public SecondaryCacheResultHandle {
+ public:
+ CompressedSecondaryCacheResultHandle(void* value, size_t size)
+ : value_(value), size_(size) {}
+ ~CompressedSecondaryCacheResultHandle() override = default;
+
+ CompressedSecondaryCacheResultHandle(
+ const CompressedSecondaryCacheResultHandle&) = delete;
+ CompressedSecondaryCacheResultHandle& operator=(
+ const CompressedSecondaryCacheResultHandle&) = delete;
+
+ bool IsReady() override { return true; }
+
+ void Wait() override {}
+
+ void* Value() override { return value_; }
+
+ size_t Size() override { return size_; }
+
+ private:
+ void* value_;
+ size_t size_;
+};
+
+// The CompressedSecondaryCache is a concrete implementation of
+// rocksdb::SecondaryCache.
+//
+// When a block is found from CompressedSecondaryCache::Lookup, we check whether
+// there is a dummy block with the same key in the primary cache.
+// 1. If the dummy block exits, we erase the block from
+// CompressedSecondaryCache and insert it into the primary cache.
+// 2. If not, we just insert a dummy block into the primary cache
+// (charging the actual size of the block) and don not erase the block from
+// CompressedSecondaryCache. A standalone handle is returned to the caller.
+//
+// When a block is evicted from the primary cache, we check whether
+// there is a dummy block with the same key in CompressedSecondaryCache.
+// 1. If the dummy block exits, the block is inserted into
+// CompressedSecondaryCache.
+// 2. If not, we just insert a dummy block (size 0) in CompressedSecondaryCache.
+//
+// Users can also cast a pointer to CompressedSecondaryCache and call methods on
+// it directly, especially custom methods that may be added
+// in the future. For example -
+// std::unique_ptr<rocksdb::SecondaryCache> cache =
+// NewCompressedSecondaryCache(opts);
+// static_cast<CompressedSecondaryCache*>(cache.get())->Erase(key);
+
+class CompressedSecondaryCache : public SecondaryCache {
+ public:
+ CompressedSecondaryCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio, double low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator = nullptr,
+ bool use_adaptive_mutex = kDefaultToAdaptiveMutex,
+ CacheMetadataChargePolicy metadata_charge_policy =
+ kDefaultCacheMetadataChargePolicy,
+ CompressionType compression_type = CompressionType::kLZ4Compression,
+ uint32_t compress_format_version = 2,
+ bool enable_custom_split_merge = false);
+ ~CompressedSecondaryCache() override;
+
+ const char* Name() const override { return "CompressedSecondaryCache"; }
+
+ Status Insert(const Slice& key, void* value,
+ const Cache::CacheItemHelper* helper) override;
+
+ std::unique_ptr<SecondaryCacheResultHandle> Lookup(
+ const Slice& key, const Cache::CreateCallback& create_cb, bool /*wait*/,
+ bool advise_erase, bool& is_in_sec_cache) override;
+
+ bool SupportForceErase() const override { return true; }
+
+ void Erase(const Slice& key) override;
+
+ void WaitAll(std::vector<SecondaryCacheResultHandle*> /*handles*/) override {}
+
+ Status SetCapacity(size_t capacity) override;
+
+ Status GetCapacity(size_t& capacity) override;
+
+ std::string GetPrintableOptions() const override;
+
+ private:
+ friend class CompressedSecondaryCacheTest;
+ static constexpr std::array<uint16_t, 8> malloc_bin_sizes_{
+ 128, 256, 512, 1024, 2048, 4096, 8192, 16384};
+
+ struct CacheValueChunk {
+ // TODO try "CacheAllocationPtr next;".
+ CacheValueChunk* next;
+ size_t size;
+ // Beginning of the chunk data (MUST BE THE LAST FIELD IN THIS STRUCT!)
+ char data[1];
+
+ void Free() { delete[] reinterpret_cast<char*>(this); }
+ };
+
+ // Split value into chunks to better fit into jemalloc bins. The chunks
+ // are stored in CacheValueChunk and extra charge is needed for each chunk,
+ // so the cache charge is recalculated here.
+ CacheValueChunk* SplitValueIntoChunks(const Slice& value,
+ CompressionType compression_type,
+ size_t& charge);
+
+ // After merging chunks, the extra charge for each chunk is removed, so
+ // the charge is recalculated.
+ CacheAllocationPtr MergeChunksIntoValue(const void* chunks_head,
+ size_t& charge);
+
+ // An implementation of Cache::DeleterFn.
+ static Cache::DeleterFn GetDeletionCallback(bool enable_custom_split_merge);
+ std::shared_ptr<Cache> cache_;
+ CompressedSecondaryCacheOptions cache_options_;
+ mutable port::Mutex capacity_mutex_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/compressed_secondary_cache_test.cc b/src/rocksdb/cache/compressed_secondary_cache_test.cc
new file mode 100644
index 000000000..574c257a7
--- /dev/null
+++ b/src/rocksdb/cache/compressed_secondary_cache_test.cc
@@ -0,0 +1,1005 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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/compressed_secondary_cache.h"
+
+#include <iterator>
+#include <memory>
+#include <tuple>
+
+#include "memory/jemalloc_nodump_allocator.h"
+#include "rocksdb/convenience.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class CompressedSecondaryCacheTest : public testing::Test {
+ public:
+ CompressedSecondaryCacheTest() : fail_create_(false) {}
+ ~CompressedSecondaryCacheTest() override = default;
+
+ protected:
+ 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_; }
+
+ private:
+ std::unique_ptr<char[]> buf_;
+ size_t size_;
+ };
+
+ static size_t SizeCallback(void* obj) {
+ return reinterpret_cast<TestItem*>(obj)->Size();
+ }
+
+ static Status SaveToCallback(void* from_obj, size_t from_offset,
+ size_t length, void* out) {
+ auto item = reinterpret_cast<TestItem*>(from_obj);
+ const char* buf = item->Buf();
+ EXPECT_EQ(length, item->Size());
+ EXPECT_EQ(from_offset, 0);
+ memcpy(out, buf, length);
+ return Status::OK();
+ }
+
+ static void DeletionCallback(const Slice& /*key*/, void* obj) {
+ delete reinterpret_cast<TestItem*>(obj);
+ obj = nullptr;
+ }
+
+ static Cache::CacheItemHelper helper_;
+
+ static Status SaveToCallbackFail(void* /*obj*/, size_t /*offset*/,
+ size_t /*size*/, void* /*out*/) {
+ return Status::NotSupported();
+ }
+
+ static Cache::CacheItemHelper helper_fail_;
+
+ Cache::CreateCallback test_item_creator = [&](const void* buf, size_t size,
+ void** out_obj,
+ size_t* charge) -> Status {
+ if (fail_create_) {
+ return Status::NotSupported();
+ }
+ *out_obj = reinterpret_cast<void*>(new TestItem((char*)buf, size));
+ *charge = size;
+ return Status::OK();
+ };
+
+ void SetFailCreate(bool fail) { fail_create_ = fail; }
+
+ void BasicTestHelper(std::shared_ptr<SecondaryCache> sec_cache,
+ bool sec_cache_is_compressed) {
+ get_perf_context()->Reset();
+ bool is_in_sec_cache{true};
+ // Lookup an non-existent key.
+ std::unique_ptr<SecondaryCacheResultHandle> handle0 = sec_cache->Lookup(
+ "k0", test_item_creator, true, /*advise_erase=*/true, is_in_sec_cache);
+ ASSERT_EQ(handle0, nullptr);
+
+ Random rnd(301);
+ // Insert and Lookup the item k1 for the first time.
+ 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,
+ &CompressedSecondaryCacheTest::helper_));
+ 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<SecondaryCacheResultHandle> handle1_1 = sec_cache->Lookup(
+ "k1", test_item_creator, true, /*advise_erase=*/false, is_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,
+ &CompressedSecondaryCacheTest::helper_));
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 1);
+
+ std::unique_ptr<SecondaryCacheResultHandle> handle1_2 = sec_cache->Lookup(
+ "k1", test_item_creator, true, /*advise_erase=*/true, is_in_sec_cache);
+ ASSERT_NE(handle1_2, nullptr);
+ ASSERT_FALSE(is_in_sec_cache);
+ if (sec_cache_is_compressed) {
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes,
+ 1000);
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes,
+ 1007);
+ } else {
+ 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<TestItem> val1 =
+ std::unique_ptr<TestItem>(static_cast<TestItem*>(handle1_2->Value()));
+ ASSERT_NE(val1, nullptr);
+ ASSERT_EQ(memcmp(val1->Buf(), item1.Buf(), item1.Size()), 0);
+
+ // Lookup the item k1 again.
+ std::unique_ptr<SecondaryCacheResultHandle> handle1_3 = sec_cache->Lookup(
+ "k1", test_item_creator, true, /*advise_erase=*/true, is_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,
+ &CompressedSecondaryCacheTest::helper_));
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_dummy_count, 2);
+ std::unique_ptr<SecondaryCacheResultHandle> handle2_1 = sec_cache->Lookup(
+ "k2", test_item_creator, true, /*advise_erase=*/false, is_in_sec_cache);
+ ASSERT_EQ(handle2_1, nullptr);
+
+ ASSERT_OK(sec_cache->Insert("k2", &item2,
+ &CompressedSecondaryCacheTest::helper_));
+ 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,
+ 2000);
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes,
+ 2014);
+ } else {
+ 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<SecondaryCacheResultHandle> handle2_2 = sec_cache->Lookup(
+ "k2", test_item_creator, true, /*advise_erase=*/false, is_in_sec_cache);
+ ASSERT_NE(handle2_2, nullptr);
+ std::unique_ptr<TestItem> val2 =
+ std::unique_ptr<TestItem>(static_cast<TestItem*>(handle2_2->Value()));
+ ASSERT_NE(val2, nullptr);
+ ASSERT_EQ(memcmp(val2->Buf(), item2.Buf(), item2.Size()), 0);
+
+ std::vector<SecondaryCacheResultHandle*> handles = {handle1_2.get(),
+ handle2_2.get()};
+ sec_cache->WaitAll(handles);
+
+ sec_cache.reset();
+ }
+
+ void BasicTest(bool sec_cache_is_compressed, bool use_jemalloc) {
+ CompressedSecondaryCacheOptions opts;
+ opts.capacity = 2048;
+ opts.num_shard_bits = 0;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ opts.compression_type = CompressionType::kNoCompression;
+ sec_cache_is_compressed = false;
+ }
+ } else {
+ opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ if (use_jemalloc) {
+ JemallocAllocatorOptions jopts;
+ std::shared_ptr<MemoryAllocator> allocator;
+ std::string msg;
+ if (JemallocNodumpAllocator::IsSupported(&msg)) {
+ Status s = NewJemallocNodumpAllocator(jopts, &allocator);
+ if (s.ok()) {
+ opts.memory_allocator = allocator;
+ }
+ } else {
+ ROCKSDB_GTEST_BYPASS("JEMALLOC not supported");
+ }
+ }
+ std::shared_ptr<SecondaryCache> sec_cache =
+ NewCompressedSecondaryCache(opts);
+
+ BasicTestHelper(sec_cache, sec_cache_is_compressed);
+ }
+
+ void FailsTest(bool sec_cache_is_compressed) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 1100;
+ secondary_cache_opts.num_shard_bits = 0;
+ std::shared_ptr<SecondaryCache> sec_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+
+ // Insert and Lookup the first item.
+ Random rnd(301);
+ std::string str1(rnd.RandomString(1000));
+ TestItem item1(str1.data(), str1.length());
+ // Insert a dummy handle.
+ ASSERT_OK(sec_cache->Insert("k1", &item1,
+ &CompressedSecondaryCacheTest::helper_));
+ // Insert k1.
+ ASSERT_OK(sec_cache->Insert("k1", &item1,
+ &CompressedSecondaryCacheTest::helper_));
+
+ // 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,
+ &CompressedSecondaryCacheTest::helper_));
+ bool is_in_sec_cache{false};
+ std::unique_ptr<SecondaryCacheResultHandle> handle1 = sec_cache->Lookup(
+ "k1", test_item_creator, true, /*advise_erase=*/false, is_in_sec_cache);
+ ASSERT_EQ(handle1, nullptr);
+
+ // Insert k2 and k1 is evicted.
+ ASSERT_OK(sec_cache->Insert("k2", &item2,
+ &CompressedSecondaryCacheTest::helper_));
+ std::unique_ptr<SecondaryCacheResultHandle> handle2 = sec_cache->Lookup(
+ "k2", test_item_creator, true, /*advise_erase=*/false, is_in_sec_cache);
+ ASSERT_NE(handle2, nullptr);
+ std::unique_ptr<TestItem> val2 =
+ std::unique_ptr<TestItem>(static_cast<TestItem*>(handle2->Value()));
+ ASSERT_NE(val2, nullptr);
+ 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,
+ &CompressedSecondaryCacheTest::helper_));
+
+ std::unique_ptr<SecondaryCacheResultHandle> handle1_1 = sec_cache->Lookup(
+ "k1", test_item_creator, true, /*advise_erase=*/false, is_in_sec_cache);
+ ASSERT_EQ(handle1_1, nullptr);
+
+ // Create Fails.
+ SetFailCreate(true);
+ std::unique_ptr<SecondaryCacheResultHandle> handle2_1 = sec_cache->Lookup(
+ "k2", test_item_creator, true, /*advise_erase=*/true, is_in_sec_cache);
+ ASSERT_EQ(handle2_1, nullptr);
+
+ // Save Fails.
+ 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,
+ &CompressedSecondaryCacheTest::helper_fail_));
+ ASSERT_NOK(sec_cache->Insert("k3", &item3,
+ &CompressedSecondaryCacheTest::helper_fail_));
+
+ sec_cache.reset();
+ }
+
+ void BasicIntegrationTest(bool sec_cache_is_compressed,
+ bool enable_custom_split_merge) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ sec_cache_is_compressed = false;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 6000;
+ secondary_cache_opts.num_shard_bits = 0;
+ secondary_cache_opts.enable_custom_split_merge = enable_custom_split_merge;
+ std::shared_ptr<SecondaryCache> secondary_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+ LRUCacheOptions lru_cache_opts(
+ /*_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> cache = NewLRUCache(lru_cache_opts);
+ std::shared_ptr<Statistics> 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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);
+
+ std::string str3 = rnd.RandomString(1024);
+ 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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,
+ str1.length());
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes,
+ 1008);
+ } else {
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, 0);
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes, 0);
+ }
+
+ // 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, &CompressedSecondaryCacheTest::helper_, 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,
+ str1.length() + str2.length());
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes,
+ 2027);
+ } else {
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_uncompressed_bytes, 0);
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_compressed_bytes, 0);
+ }
+
+ Cache::Handle* handle;
+ handle = cache->Lookup("k3", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+ ASSERT_NE(handle, nullptr);
+ auto val3 = static_cast<TestItem*>(cache->Value(handle));
+ ASSERT_NE(val3, nullptr);
+ ASSERT_EQ(memcmp(val3->Buf(), item3_2->Buf(), item3_2->Size()), 0);
+ cache->Release(handle);
+
+ // Lookup an non-existent key.
+ handle = cache->Lookup("k0", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ 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", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 1);
+ auto val1_1 = static_cast<TestItem*>(cache->Value(handle));
+ ASSERT_NE(val1_1, nullptr);
+ ASSERT_EQ(memcmp(val1_1->Buf(), str1.data(), str1.size()), 0);
+ cache->Release(handle);
+
+ // 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", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 1);
+ ASSERT_EQ(get_perf_context()->compressed_sec_cache_insert_real_count, 3);
+ cache->Release(handle);
+
+ // k2 is still in secondary cache.
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 2);
+ cache->Release(handle);
+
+ // Testing SetCapacity().
+ ASSERT_OK(secondary_cache->SetCapacity(0));
+ handle = cache->Lookup("k3", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+ ASSERT_EQ(handle, nullptr);
+
+ ASSERT_OK(secondary_cache->SetCapacity(7000));
+ size_t capacity;
+ ASSERT_OK(secondary_cache->GetCapacity(capacity));
+ 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, 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", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true,
+ stats.get());
+
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+ ASSERT_EQ(get_perf_context()->block_cache_standalone_handle_count, 3);
+
+ cache.reset();
+ secondary_cache.reset();
+ }
+
+ void BasicIntegrationFailTest(bool sec_cache_is_compressed) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 6000;
+ secondary_cache_opts.num_shard_bits = 0;
+ std::shared_ptr<SecondaryCache> secondary_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+
+ LRUCacheOptions opts(
+ /*_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> cache = NewLRUCache(opts);
+
+ Random rnd(301);
+ std::string str1 = rnd.RandomString(1001);
+ auto item1 = std::make_unique<TestItem>(str1.data(), str1.length());
+ ASSERT_NOK(cache->Insert("k1", item1.get(), nullptr, str1.length()));
+ ASSERT_OK(cache->Insert("k1", item1.get(),
+ &CompressedSecondaryCacheTest::helper_,
+ str1.length()));
+ item1.release(); // Appease clang-analyze "potential memory leak"
+
+ Cache::Handle* handle;
+ handle = cache->Lookup("k2", nullptr, test_item_creator,
+ Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, false);
+ ASSERT_EQ(handle, nullptr);
+
+ cache.reset();
+ secondary_cache.reset();
+ }
+
+ void IntegrationSaveFailTest(bool sec_cache_is_compressed) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 6000;
+ secondary_cache_opts.num_shard_bits = 0;
+
+ std::shared_ptr<SecondaryCache> secondary_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+
+ LRUCacheOptions opts(
+ /*_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> cache = NewLRUCache(opts);
+
+ Random rnd(301);
+ std::string str1 = rnd.RandomString(1001);
+ auto item1 = new TestItem(str1.data(), str1.length());
+ ASSERT_OK(cache->Insert("k1", item1,
+ &CompressedSecondaryCacheTest::helper_fail_,
+ 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,
+ &CompressedSecondaryCacheTest::helper_fail_,
+ str2.length()));
+
+ Cache::Handle* handle;
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_fail_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+ // This lookup should fail, since k1 demotion would have failed.
+ handle = cache->Lookup("k1", &CompressedSecondaryCacheTest::helper_fail_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ // Since k1 was not promoted, k2 should still be in cache.
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_fail_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+
+ cache.reset();
+ secondary_cache.reset();
+ }
+
+ void IntegrationCreateFailTest(bool sec_cache_is_compressed) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 6000;
+ secondary_cache_opts.num_shard_bits = 0;
+
+ std::shared_ptr<SecondaryCache> secondary_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+
+ LRUCacheOptions opts(
+ /*_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> cache = NewLRUCache(opts);
+
+ Random rnd(301);
+ std::string str1 = rnd.RandomString(1001);
+ auto item1 = new TestItem(str1.data(), str1.length());
+ ASSERT_OK(cache->Insert("k1", item1, &CompressedSecondaryCacheTest::helper_,
+ 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, &CompressedSecondaryCacheTest::helper_,
+ str2.length()));
+
+ Cache::Handle* handle;
+ SetFailCreate(true);
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+ // This lookup should fail, since k1 creation would have failed
+ handle = cache->Lookup("k1", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ // Since k1 didn't get promoted, k2 should still be in cache
+ handle = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+
+ cache.reset();
+ secondary_cache.reset();
+ }
+
+ void IntegrationFullCapacityTest(bool sec_cache_is_compressed) {
+ CompressedSecondaryCacheOptions secondary_cache_opts;
+
+ if (sec_cache_is_compressed) {
+ if (!LZ4_Supported()) {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+ } else {
+ secondary_cache_opts.compression_type = CompressionType::kNoCompression;
+ }
+
+ secondary_cache_opts.capacity = 6000;
+ secondary_cache_opts.num_shard_bits = 0;
+
+ std::shared_ptr<SecondaryCache> secondary_cache =
+ NewCompressedSecondaryCache(secondary_cache_opts);
+
+ LRUCacheOptions opts(
+ /*_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> cache = NewLRUCache(opts);
+
+ 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_,
+ 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, &CompressedSecondaryCacheTest::helper_, 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, &CompressedSecondaryCacheTest::helper_, str2.length()));
+
+ Cache::Handle* handle2;
+ handle2 = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle2, nullptr);
+ cache->Release(handle2);
+
+ // k1 promotion should fail because cache is at capacity and
+ // 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", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle1, nullptr);
+ cache->Release(handle1);
+
+ // Since k1 didn't get inserted, k2 should still be in cache
+ handle2 = cache->Lookup("k2", &CompressedSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle2, nullptr);
+ cache->Release(handle2);
+
+ cache.reset();
+ secondary_cache.reset();
+ }
+
+ void SplitValueIntoChunksTest() {
+ JemallocAllocatorOptions jopts;
+ std::shared_ptr<MemoryAllocator> allocator;
+ std::string msg;
+ if (JemallocNodumpAllocator::IsSupported(&msg)) {
+ Status s = NewJemallocNodumpAllocator(jopts, &allocator);
+ if (!s.ok()) {
+ ROCKSDB_GTEST_BYPASS("JEMALLOC not supported");
+ }
+ } else {
+ ROCKSDB_GTEST_BYPASS("JEMALLOC not supported");
+ }
+
+ using CacheValueChunk = CompressedSecondaryCache::CacheValueChunk;
+ std::unique_ptr<CompressedSecondaryCache> sec_cache =
+ std::make_unique<CompressedSecondaryCache>(1000, 0, true, 0.5, 0.0,
+ allocator);
+ Random rnd(301);
+ // 8500 = 8169 + 233 + 98, so there should be 3 chunks after split.
+ size_t str_size{8500};
+ std::string str = rnd.RandomString(static_cast<int>(str_size));
+ size_t charge{0};
+ CacheValueChunk* chunks_head =
+ sec_cache->SplitValueIntoChunks(str, kLZ4Compression, charge);
+ ASSERT_EQ(charge, str_size + 3 * (sizeof(CacheValueChunk) - 1));
+
+ CacheValueChunk* current_chunk = chunks_head;
+ ASSERT_EQ(current_chunk->size, 8192 - sizeof(CacheValueChunk) + 1);
+ current_chunk = current_chunk->next;
+ ASSERT_EQ(current_chunk->size, 256 - sizeof(CacheValueChunk) + 1);
+ current_chunk = current_chunk->next;
+ ASSERT_EQ(current_chunk->size, 98);
+
+ sec_cache->GetDeletionCallback(true)("dummy", chunks_head);
+ }
+
+ void MergeChunksIntoValueTest() {
+ using CacheValueChunk = CompressedSecondaryCache::CacheValueChunk;
+ Random rnd(301);
+ size_t size1{2048};
+ std::string str1 = rnd.RandomString(static_cast<int>(size1));
+ CacheValueChunk* current_chunk = reinterpret_cast<CacheValueChunk*>(
+ new char[sizeof(CacheValueChunk) - 1 + size1]);
+ CacheValueChunk* chunks_head = current_chunk;
+ memcpy(current_chunk->data, str1.data(), size1);
+ current_chunk->size = size1;
+
+ size_t size2{256};
+ std::string str2 = rnd.RandomString(static_cast<int>(size2));
+ current_chunk->next = reinterpret_cast<CacheValueChunk*>(
+ new char[sizeof(CacheValueChunk) - 1 + size2]);
+ current_chunk = current_chunk->next;
+ memcpy(current_chunk->data, str2.data(), size2);
+ current_chunk->size = size2;
+
+ size_t size3{31};
+ std::string str3 = rnd.RandomString(static_cast<int>(size3));
+ current_chunk->next = reinterpret_cast<CacheValueChunk*>(
+ new char[sizeof(CacheValueChunk) - 1 + size3]);
+ current_chunk = current_chunk->next;
+ memcpy(current_chunk->data, str3.data(), size3);
+ current_chunk->size = size3;
+ current_chunk->next = nullptr;
+
+ std::string str = str1 + str2 + str3;
+
+ std::unique_ptr<CompressedSecondaryCache> sec_cache =
+ std::make_unique<CompressedSecondaryCache>(1000, 0, true, 0.5, 0.0);
+ size_t charge{0};
+ CacheAllocationPtr value =
+ sec_cache->MergeChunksIntoValue(chunks_head, charge);
+ ASSERT_EQ(charge, size1 + size2 + size3);
+ std::string value_str{value.get(), charge};
+ ASSERT_EQ(strcmp(value_str.data(), str.data()), 0);
+
+ while (chunks_head != nullptr) {
+ CacheValueChunk* tmp_chunk = chunks_head;
+ chunks_head = chunks_head->next;
+ tmp_chunk->Free();
+ }
+ }
+
+ void SplictValueAndMergeChunksTest() {
+ JemallocAllocatorOptions jopts;
+ std::shared_ptr<MemoryAllocator> allocator;
+ std::string msg;
+ if (JemallocNodumpAllocator::IsSupported(&msg)) {
+ Status s = NewJemallocNodumpAllocator(jopts, &allocator);
+ if (!s.ok()) {
+ ROCKSDB_GTEST_BYPASS("JEMALLOC not supported");
+ }
+ } else {
+ ROCKSDB_GTEST_BYPASS("JEMALLOC not supported");
+ }
+
+ using CacheValueChunk = CompressedSecondaryCache::CacheValueChunk;
+ std::unique_ptr<CompressedSecondaryCache> sec_cache =
+ std::make_unique<CompressedSecondaryCache>(1000, 0, true, 0.5, 0.0,
+ allocator);
+ Random rnd(301);
+ // 8500 = 8169 + 233 + 98, so there should be 3 chunks after split.
+ size_t str_size{8500};
+ std::string str = rnd.RandomString(static_cast<int>(str_size));
+ size_t charge{0};
+ CacheValueChunk* chunks_head =
+ sec_cache->SplitValueIntoChunks(str, kLZ4Compression, charge);
+ ASSERT_EQ(charge, str_size + 3 * (sizeof(CacheValueChunk) - 1));
+
+ CacheAllocationPtr value =
+ sec_cache->MergeChunksIntoValue(chunks_head, charge);
+ ASSERT_EQ(charge, str_size);
+ std::string value_str{value.get(), charge};
+ ASSERT_EQ(strcmp(value_str.data(), str.data()), 0);
+
+ sec_cache->GetDeletionCallback(true)("dummy", chunks_head);
+ }
+
+ private:
+ bool fail_create_;
+};
+
+Cache::CacheItemHelper CompressedSecondaryCacheTest::helper_(
+ CompressedSecondaryCacheTest::SizeCallback,
+ CompressedSecondaryCacheTest::SaveToCallback,
+ CompressedSecondaryCacheTest::DeletionCallback);
+
+Cache::CacheItemHelper CompressedSecondaryCacheTest::helper_fail_(
+ CompressedSecondaryCacheTest::SizeCallback,
+ CompressedSecondaryCacheTest::SaveToCallbackFail,
+ CompressedSecondaryCacheTest::DeletionCallback);
+
+class CompressedSecCacheTestWithCompressAndAllocatorParam
+ : public CompressedSecondaryCacheTest,
+ public ::testing::WithParamInterface<std::tuple<bool, bool>> {
+ public:
+ CompressedSecCacheTestWithCompressAndAllocatorParam() {
+ sec_cache_is_compressed_ = std::get<0>(GetParam());
+ use_jemalloc_ = std::get<1>(GetParam());
+ }
+ bool sec_cache_is_compressed_;
+ bool use_jemalloc_;
+};
+
+TEST_P(CompressedSecCacheTestWithCompressAndAllocatorParam, BasicTes) {
+ BasicTest(sec_cache_is_compressed_, use_jemalloc_);
+}
+
+INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests,
+ CompressedSecCacheTestWithCompressAndAllocatorParam,
+ ::testing::Combine(testing::Bool(), testing::Bool()));
+
+class CompressedSecondaryCacheTestWithCompressionParam
+ : public CompressedSecondaryCacheTest,
+ public ::testing::WithParamInterface<bool> {
+ public:
+ CompressedSecondaryCacheTestWithCompressionParam() {
+ sec_cache_is_compressed_ = GetParam();
+ }
+ bool sec_cache_is_compressed_;
+};
+
+#ifndef ROCKSDB_LITE
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam, BasicTestFromString) {
+ std::shared_ptr<SecondaryCache> sec_cache{nullptr};
+ std::string sec_cache_uri;
+ if (sec_cache_is_compressed_) {
+ if (LZ4_Supported()) {
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kLZ4Compression;"
+ "compress_format_version=2";
+ } else {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kNoCompression";
+ sec_cache_is_compressed_ = false;
+ }
+ Status s = SecondaryCache::CreateFromString(ConfigOptions(), sec_cache_uri,
+ &sec_cache);
+ EXPECT_OK(s);
+ } else {
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kNoCompression";
+ Status s = SecondaryCache::CreateFromString(ConfigOptions(), sec_cache_uri,
+ &sec_cache);
+ EXPECT_OK(s);
+ }
+ BasicTestHelper(sec_cache, sec_cache_is_compressed_);
+}
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam,
+ BasicTestFromStringWithSplit) {
+ std::shared_ptr<SecondaryCache> sec_cache{nullptr};
+ std::string sec_cache_uri;
+ if (sec_cache_is_compressed_) {
+ if (LZ4_Supported()) {
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kLZ4Compression;"
+ "compress_format_version=2;enable_custom_split_merge=true";
+ } else {
+ ROCKSDB_GTEST_SKIP("This test requires LZ4 support.");
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kNoCompression;"
+ "enable_custom_split_merge=true";
+ sec_cache_is_compressed_ = false;
+ }
+ Status s = SecondaryCache::CreateFromString(ConfigOptions(), sec_cache_uri,
+ &sec_cache);
+ EXPECT_OK(s);
+ } else {
+ sec_cache_uri =
+ "compressed_secondary_cache://"
+ "capacity=2048;num_shard_bits=0;compression_type=kNoCompression;"
+ "enable_custom_split_merge=true";
+ Status s = SecondaryCache::CreateFromString(ConfigOptions(), sec_cache_uri,
+ &sec_cache);
+ EXPECT_OK(s);
+ }
+ BasicTestHelper(sec_cache, sec_cache_is_compressed_);
+}
+
+#endif // ROCKSDB_LITE
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam, FailsTest) {
+ FailsTest(sec_cache_is_compressed_);
+}
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam,
+ BasicIntegrationFailTest) {
+ BasicIntegrationFailTest(sec_cache_is_compressed_);
+}
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam,
+ IntegrationSaveFailTest) {
+ IntegrationSaveFailTest(sec_cache_is_compressed_);
+}
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam,
+ IntegrationCreateFailTest) {
+ IntegrationCreateFailTest(sec_cache_is_compressed_);
+}
+
+TEST_P(CompressedSecondaryCacheTestWithCompressionParam,
+ IntegrationFullCapacityTest) {
+ IntegrationFullCapacityTest(sec_cache_is_compressed_);
+}
+
+INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests,
+ CompressedSecondaryCacheTestWithCompressionParam,
+ testing::Bool());
+
+class CompressedSecCacheTestWithCompressAndSplitParam
+ : public CompressedSecondaryCacheTest,
+ public ::testing::WithParamInterface<std::tuple<bool, bool>> {
+ public:
+ CompressedSecCacheTestWithCompressAndSplitParam() {
+ sec_cache_is_compressed_ = std::get<0>(GetParam());
+ enable_custom_split_merge_ = std::get<1>(GetParam());
+ }
+ bool sec_cache_is_compressed_;
+ bool enable_custom_split_merge_;
+};
+
+TEST_P(CompressedSecCacheTestWithCompressAndSplitParam, BasicIntegrationTest) {
+ BasicIntegrationTest(sec_cache_is_compressed_, enable_custom_split_merge_);
+}
+
+INSTANTIATE_TEST_CASE_P(CompressedSecCacheTests,
+ CompressedSecCacheTestWithCompressAndSplitParam,
+ ::testing::Combine(testing::Bool(), testing::Bool()));
+
+TEST_F(CompressedSecondaryCacheTest, SplitValueIntoChunksTest) {
+ SplitValueIntoChunksTest();
+}
+
+TEST_F(CompressedSecondaryCacheTest, MergeChunksIntoValueTest) {
+ MergeChunksIntoValueTest();
+}
+
+TEST_F(CompressedSecondaryCacheTest, SplictValueAndMergeChunksTest) {
+ SplictValueAndMergeChunksTest();
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/cache/lru_cache.cc b/src/rocksdb/cache/lru_cache.cc
new file mode 100644
index 000000000..c8e4d29ba
--- /dev/null
+++ b/src/rocksdb/cache/lru_cache.cc
@@ -0,0 +1,921 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "cache/lru_cache.h"
+
+#include <cassert>
+#include <cstdint>
+#include <cstdio>
+#include <cstdlib>
+
+#include "monitoring/perf_context_imp.h"
+#include "monitoring/statistics.h"
+#include "port/lang.h"
+#include "util/distributed_mutex.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace lru_cache {
+
+// A distinct pointer value for marking "dummy" cache entries
+void* const kDummyValueMarker = const_cast<char*>("kDummyValueMarker");
+
+LRUHandleTable::LRUHandleTable(int max_upper_hash_bits)
+ : length_bits_(/* historical starting size*/ 4),
+ list_(new LRUHandle* [size_t{1} << length_bits_] {}),
+ elems_(0),
+ max_length_bits_(max_upper_hash_bits) {}
+
+LRUHandleTable::~LRUHandleTable() {
+ ApplyToEntriesRange(
+ [](LRUHandle* h) {
+ if (!h->HasRefs()) {
+ h->Free();
+ }
+ },
+ 0, size_t{1} << length_bits_);
+}
+
+LRUHandle* LRUHandleTable::Lookup(const Slice& key, uint32_t hash) {
+ return *FindPointer(key, hash);
+}
+
+LRUHandle* LRUHandleTable::Insert(LRUHandle* h) {
+ LRUHandle** ptr = FindPointer(h->key(), h->hash);
+ LRUHandle* old = *ptr;
+ h->next_hash = (old == nullptr ? nullptr : old->next_hash);
+ *ptr = h;
+ if (old == nullptr) {
+ ++elems_;
+ if ((elems_ >> length_bits_) > 0) { // elems_ >= length
+ // Since each cache entry is fairly large, we aim for a small
+ // average linked list length (<= 1).
+ Resize();
+ }
+ }
+ return old;
+}
+
+LRUHandle* LRUHandleTable::Remove(const Slice& key, uint32_t hash) {
+ LRUHandle** ptr = FindPointer(key, hash);
+ LRUHandle* result = *ptr;
+ if (result != nullptr) {
+ *ptr = result->next_hash;
+ --elems_;
+ }
+ return result;
+}
+
+LRUHandle** LRUHandleTable::FindPointer(const Slice& key, uint32_t hash) {
+ LRUHandle** ptr = &list_[hash >> (32 - length_bits_)];
+ while (*ptr != nullptr && ((*ptr)->hash != hash || key != (*ptr)->key())) {
+ ptr = &(*ptr)->next_hash;
+ }
+ return ptr;
+}
+
+void LRUHandleTable::Resize() {
+ if (length_bits_ >= max_length_bits_) {
+ // Due to reaching limit of hash information, if we made the table bigger,
+ // we would allocate more addresses but only the same number would be used.
+ return;
+ }
+ if (length_bits_ >= 31) {
+ // Avoid undefined behavior shifting uint32_t by 32.
+ return;
+ }
+
+ uint32_t old_length = uint32_t{1} << length_bits_;
+ int new_length_bits = length_bits_ + 1;
+ std::unique_ptr<LRUHandle* []> new_list {
+ new LRUHandle* [size_t{1} << new_length_bits] {}
+ };
+ uint32_t count = 0;
+ for (uint32_t i = 0; i < old_length; i++) {
+ LRUHandle* h = list_[i];
+ while (h != nullptr) {
+ LRUHandle* next = h->next_hash;
+ uint32_t hash = h->hash;
+ LRUHandle** ptr = &new_list[hash >> (32 - new_length_bits)];
+ h->next_hash = *ptr;
+ *ptr = h;
+ h = next;
+ count++;
+ }
+ }
+ assert(elems_ == count);
+ list_ = std::move(new_list);
+ length_bits_ = new_length_bits;
+}
+
+LRUCacheShard::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,
+ SecondaryCache* secondary_cache)
+ : CacheShardBase(metadata_charge_policy),
+ capacity_(0),
+ high_pri_pool_usage_(0),
+ low_pri_pool_usage_(0),
+ strict_capacity_limit_(strict_capacity_limit),
+ high_pri_pool_ratio_(high_pri_pool_ratio),
+ high_pri_pool_capacity_(0),
+ low_pri_pool_ratio_(low_pri_pool_ratio),
+ low_pri_pool_capacity_(0),
+ table_(max_upper_hash_bits),
+ usage_(0),
+ lru_usage_(0),
+ mutex_(use_adaptive_mutex),
+ secondary_cache_(secondary_cache) {
+ // Make empty circular linked list.
+ lru_.next = &lru_;
+ lru_.prev = &lru_;
+ lru_low_pri_ = &lru_;
+ lru_bottom_pri_ = &lru_;
+ SetCapacity(capacity);
+}
+
+void LRUCacheShard::EraseUnRefEntries() {
+ autovector<LRUHandle*> last_reference_list;
+ {
+ DMutexLock l(mutex_);
+ while (lru_.next != &lru_) {
+ LRUHandle* old = lru_.next;
+ // LRU list contains only elements which can be evicted.
+ assert(old->InCache() && !old->HasRefs());
+ LRU_Remove(old);
+ table_.Remove(old->key(), old->hash);
+ old->SetInCache(false);
+ assert(usage_ >= old->total_charge);
+ usage_ -= old->total_charge;
+ last_reference_list.push_back(old);
+ }
+ }
+
+ for (auto entry : last_reference_list) {
+ entry->Free();
+ }
+}
+
+void LRUCacheShard::ApplyToSomeEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ size_t average_entries_per_lock, size_t* state) {
+ // The state is essentially going to be the starting hash, which works
+ // nicely even if we resize between calls because we use upper-most
+ // hash bits for table indexes.
+ DMutexLock l(mutex_);
+ int length_bits = table_.GetLengthBits();
+ size_t length = size_t{1} << length_bits;
+
+ assert(average_entries_per_lock > 0);
+ // Assuming we are called with same average_entries_per_lock repeatedly,
+ // this simplifies some logic (index_end will not overflow).
+ assert(average_entries_per_lock < length || *state == 0);
+
+ size_t index_begin = *state >> (sizeof(size_t) * 8u - length_bits);
+ size_t index_end = index_begin + average_entries_per_lock;
+ if (index_end >= length) {
+ // Going to end
+ index_end = length;
+ *state = SIZE_MAX;
+ } else {
+ *state = index_end << (sizeof(size_t) * 8u - length_bits);
+ }
+
+ table_.ApplyToEntriesRange(
+ [callback,
+ metadata_charge_policy = metadata_charge_policy_](LRUHandle* h) {
+ DeleterFn deleter = h->IsSecondaryCacheCompatible()
+ ? h->info_.helper->del_cb
+ : h->info_.deleter;
+ callback(h->key(), h->value, h->GetCharge(metadata_charge_policy),
+ deleter);
+ },
+ index_begin, index_end);
+}
+
+void LRUCacheShard::TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri,
+ LRUHandle** lru_bottom_pri) {
+ DMutexLock l(mutex_);
+ *lru = &lru_;
+ *lru_low_pri = lru_low_pri_;
+ *lru_bottom_pri = lru_bottom_pri_;
+}
+
+size_t LRUCacheShard::TEST_GetLRUSize() {
+ DMutexLock l(mutex_);
+ LRUHandle* lru_handle = lru_.next;
+ size_t lru_size = 0;
+ while (lru_handle != &lru_) {
+ lru_size++;
+ lru_handle = lru_handle->next;
+ }
+ return lru_size;
+}
+
+double LRUCacheShard::GetHighPriPoolRatio() {
+ DMutexLock l(mutex_);
+ return high_pri_pool_ratio_;
+}
+
+double LRUCacheShard::GetLowPriPoolRatio() {
+ DMutexLock l(mutex_);
+ return low_pri_pool_ratio_;
+}
+
+void LRUCacheShard::LRU_Remove(LRUHandle* e) {
+ assert(e->next != nullptr);
+ assert(e->prev != nullptr);
+ if (lru_low_pri_ == e) {
+ lru_low_pri_ = e->prev;
+ }
+ if (lru_bottom_pri_ == e) {
+ lru_bottom_pri_ = e->prev;
+ }
+ e->next->prev = e->prev;
+ e->prev->next = e->next;
+ e->prev = e->next = nullptr;
+ assert(lru_usage_ >= e->total_charge);
+ lru_usage_ -= e->total_charge;
+ assert(!e->InHighPriPool() || !e->InLowPriPool());
+ if (e->InHighPriPool()) {
+ assert(high_pri_pool_usage_ >= e->total_charge);
+ high_pri_pool_usage_ -= e->total_charge;
+ } else if (e->InLowPriPool()) {
+ assert(low_pri_pool_usage_ >= e->total_charge);
+ low_pri_pool_usage_ -= e->total_charge;
+ }
+}
+
+void LRUCacheShard::LRU_Insert(LRUHandle* e) {
+ assert(e->next == nullptr);
+ assert(e->prev == nullptr);
+ if (high_pri_pool_ratio_ > 0 && (e->IsHighPri() || e->HasHit())) {
+ // Inset "e" to head of LRU list.
+ e->next = &lru_;
+ e->prev = lru_.prev;
+ e->prev->next = e;
+ e->next->prev = e;
+ e->SetInHighPriPool(true);
+ e->SetInLowPriPool(false);
+ high_pri_pool_usage_ += e->total_charge;
+ MaintainPoolSize();
+ } else if (low_pri_pool_ratio_ > 0 &&
+ (e->IsHighPri() || e->IsLowPri() || e->HasHit())) {
+ // Insert "e" to the head of low-pri pool.
+ e->next = lru_low_pri_->next;
+ e->prev = lru_low_pri_;
+ e->prev->next = e;
+ e->next->prev = e;
+ e->SetInHighPriPool(false);
+ e->SetInLowPriPool(true);
+ low_pri_pool_usage_ += e->total_charge;
+ MaintainPoolSize();
+ lru_low_pri_ = e;
+ } else {
+ // Insert "e" to the head of bottom-pri pool.
+ e->next = lru_bottom_pri_->next;
+ e->prev = lru_bottom_pri_;
+ e->prev->next = e;
+ e->next->prev = e;
+ e->SetInHighPriPool(false);
+ e->SetInLowPriPool(false);
+ // if the low-pri pool is empty, lru_low_pri_ also needs to be updated.
+ if (lru_bottom_pri_ == lru_low_pri_) {
+ lru_low_pri_ = e;
+ }
+ lru_bottom_pri_ = e;
+ }
+ lru_usage_ += e->total_charge;
+}
+
+void LRUCacheShard::MaintainPoolSize() {
+ while (high_pri_pool_usage_ > high_pri_pool_capacity_) {
+ // Overflow last entry in high-pri pool to low-pri pool.
+ lru_low_pri_ = lru_low_pri_->next;
+ assert(lru_low_pri_ != &lru_);
+ lru_low_pri_->SetInHighPriPool(false);
+ lru_low_pri_->SetInLowPriPool(true);
+ assert(high_pri_pool_usage_ >= lru_low_pri_->total_charge);
+ high_pri_pool_usage_ -= lru_low_pri_->total_charge;
+ low_pri_pool_usage_ += lru_low_pri_->total_charge;
+ }
+
+ while (low_pri_pool_usage_ > low_pri_pool_capacity_) {
+ // Overflow last entry in low-pri pool to bottom-pri pool.
+ lru_bottom_pri_ = lru_bottom_pri_->next;
+ assert(lru_bottom_pri_ != &lru_);
+ lru_bottom_pri_->SetInHighPriPool(false);
+ lru_bottom_pri_->SetInLowPriPool(false);
+ assert(low_pri_pool_usage_ >= lru_bottom_pri_->total_charge);
+ low_pri_pool_usage_ -= lru_bottom_pri_->total_charge;
+ }
+}
+
+void LRUCacheShard::EvictFromLRU(size_t charge,
+ autovector<LRUHandle*>* deleted) {
+ while ((usage_ + charge) > capacity_ && lru_.next != &lru_) {
+ LRUHandle* old = lru_.next;
+ // LRU list contains only elements which can be evicted.
+ assert(old->InCache() && !old->HasRefs());
+ LRU_Remove(old);
+ table_.Remove(old->key(), old->hash);
+ old->SetInCache(false);
+ assert(usage_ >= old->total_charge);
+ usage_ -= old->total_charge;
+ deleted->push_back(old);
+ }
+}
+
+void LRUCacheShard::TryInsertIntoSecondaryCache(
+ autovector<LRUHandle*> evicted_handles) {
+ for (auto entry : evicted_handles) {
+ if (secondary_cache_ && entry->IsSecondaryCacheCompatible() &&
+ !entry->IsInSecondaryCache()) {
+ secondary_cache_->Insert(entry->key(), entry->value, entry->info_.helper)
+ .PermitUncheckedError();
+ }
+ // Free the entries here outside of mutex for performance reasons.
+ entry->Free();
+ }
+}
+
+void LRUCacheShard::SetCapacity(size_t capacity) {
+ autovector<LRUHandle*> last_reference_list;
+ {
+ DMutexLock l(mutex_);
+ capacity_ = capacity;
+ high_pri_pool_capacity_ = capacity_ * high_pri_pool_ratio_;
+ low_pri_pool_capacity_ = capacity_ * low_pri_pool_ratio_;
+ EvictFromLRU(0, &last_reference_list);
+ }
+
+ TryInsertIntoSecondaryCache(last_reference_list);
+}
+
+void LRUCacheShard::SetStrictCapacityLimit(bool strict_capacity_limit) {
+ DMutexLock l(mutex_);
+ strict_capacity_limit_ = strict_capacity_limit;
+}
+
+Status LRUCacheShard::InsertItem(LRUHandle* e, LRUHandle** handle,
+ bool free_handle_on_fail) {
+ Status s = Status::OK();
+ autovector<LRUHandle*> last_reference_list;
+
+ {
+ 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_ || handle == nullptr)) {
+ e->SetInCache(false);
+ if (handle == nullptr) {
+ // Don't insert the entry but still return ok, as if the entry inserted
+ // into cache and get evicted immediately.
+ last_reference_list.push_back(e);
+ } else {
+ if (free_handle_on_fail) {
+ free(e);
+ *handle = nullptr;
+ }
+ s = Status::MemoryLimit("Insert failed due to LRU cache being full.");
+ }
+ } else {
+ // Insert into the cache. Note that the cache might get larger than its
+ // capacity if not enough space was freed up.
+ LRUHandle* old = table_.Insert(e);
+ usage_ += e->total_charge;
+ if (old != nullptr) {
+ s = Status::OkOverwritten();
+ assert(old->InCache());
+ old->SetInCache(false);
+ if (!old->HasRefs()) {
+ // old is on LRU because it's in cache and its reference count is 0.
+ LRU_Remove(old);
+ assert(usage_ >= old->total_charge);
+ usage_ -= old->total_charge;
+ last_reference_list.push_back(old);
+ }
+ }
+ if (handle == nullptr) {
+ LRU_Insert(e);
+ } else {
+ // If caller already holds a ref, no need to take one here.
+ if (!e->HasRefs()) {
+ e->Ref();
+ }
+ *handle = e;
+ }
+ }
+ }
+
+ TryInsertIntoSecondaryCache(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<LRUHandle*> 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();
+ 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, kDummyValueMarker, /*charge=*/0,
+ /*deleter=*/nullptr, /*helper=*/nullptr, /*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,
+ const Cache::CreateCallback& create_cb,
+ 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 == kDummyValueMarker) {
+ // 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->saveto_cb) {
+ // For objects from the secondary cache, we expect the caller to provide
+ // a way to create/delete the primary cache object. The only case where
+ // a deleter would not be required is for dummy entries inserted for
+ // accounting purposes, which we won't demote to the secondary cache
+ // anyway.
+ assert(create_cb && helper->del_cb);
+ bool is_in_sec_cache{false};
+ std::unique_ptr<SecondaryCacheResultHandle> secondary_handle =
+ secondary_cache_->Lookup(key, create_cb, wait, found_dummy_entry,
+ is_in_sec_cache);
+ if (secondary_handle != nullptr) {
+ e = static_cast<LRUHandle*>(malloc(sizeof(LRUHandle) - 1 + key.size()));
+
+ e->m_flags = 0;
+ e->im_flags = 0;
+ e->SetSecondaryCacheCompatible(true);
+ e->info_.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->SetIsInSecondaryCache(is_in_sec_cache);
+ 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();
+ e = nullptr;
+ } else {
+ PERF_COUNTER_ADD(secondary_cache_hit_count, 1);
+ RecordTick(stats, SECONDARY_CACHE_HITS);
+ }
+ }
+ } 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);
+ // This may be slightly inaccurate, if the lookup eventually fails.
+ // But the probability is very low.
+ 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);
+ }
+ }
+ return e;
+}
+
+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;
+}
+
+void LRUCacheShard::SetHighPriorityPoolRatio(double high_pri_pool_ratio) {
+ DMutexLock l(mutex_);
+ high_pri_pool_ratio_ = high_pri_pool_ratio;
+ high_pri_pool_capacity_ = capacity_ * high_pri_pool_ratio_;
+ MaintainPoolSize();
+}
+
+void LRUCacheShard::SetLowPriorityPoolRatio(double low_pri_pool_ratio) {
+ DMutexLock l(mutex_);
+ low_pri_pool_ratio_ = low_pri_pool_ratio;
+ low_pri_pool_capacity_ = capacity_ * low_pri_pool_ratio_;
+ MaintainPoolSize();
+}
+
+bool LRUCacheShard::Release(LRUHandle* e, bool /*useful*/,
+ bool erase_if_last_ref) {
+ 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());
+ {
+ DMutexLock l(mutex_);
+ last_reference = e->Unref();
+ if (last_reference && e->InCache()) {
+ // 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.
+ assert(lru_.next == &lru_ || erase_if_last_ref);
+ // Take this opportunity and remove the item.
+ table_.Remove(e->key(), e->hash);
+ e->SetInCache(false);
+ } else {
+ // Put the item back on the LRU list, and don't free it.
+ LRU_Insert(e);
+ last_reference = false;
+ }
+ }
+ // If it was the last reference, then decrement the cache usage.
+ if (last_reference) {
+ assert(usage_ >= e->total_charge);
+ usage_ -= e->total_charge;
+ }
+ }
+
+ // Free the entry here outside of mutex for performance reasons.
+ if (last_reference) {
+ e->Free();
+ }
+ return last_reference;
+}
+
+Status LRUCacheShard::Insert(const Slice& key, uint32_t hash, void* value,
+ size_t charge,
+ void (*deleter)(const Slice& key, void* value),
+ const Cache::CacheItemHelper* helper,
+ LRUHandle** handle, Cache::Priority priority) {
+ // Allocate the memory here outside of the mutex.
+ // If the cache is full, we'll have to release it.
+ // It shouldn't happen very often though.
+ LRUHandle* e =
+ static_cast<LRUHandle*>(malloc(sizeof(LRUHandle) - 1 + key.size()));
+
+ e->value = value;
+ e->m_flags = 0;
+ e->im_flags = 0;
+ if (helper) {
+ // Use only one of the two parameters
+ assert(deleter == nullptr);
+ // value == nullptr is reserved for indicating failure for when secondary
+ // cache compatible
+ assert(value != nullptr);
+ e->SetSecondaryCacheCompatible(true);
+ e->info_.helper = helper;
+ } else {
+ e->info_.deleter = deleter;
+ }
+ e->key_length = key.size();
+ 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_);
+
+ return InsertItem(e, handle, /* free_handle_on_fail */ true);
+}
+
+void LRUCacheShard::Erase(const Slice& key, uint32_t hash) {
+ LRUHandle* e;
+ bool last_reference = false;
+ {
+ DMutexLock l(mutex_);
+ e = table_.Remove(key, hash);
+ if (e != nullptr) {
+ assert(e->InCache());
+ e->SetInCache(false);
+ if (!e->HasRefs()) {
+ // The entry is in LRU since it's in hash and has no external references
+ LRU_Remove(e);
+ assert(usage_ >= e->total_charge);
+ usage_ -= e->total_charge;
+ last_reference = true;
+ }
+ }
+ }
+
+ // Free the entry here outside of mutex for performance reasons.
+ // last_reference will only be true if e != nullptr.
+ if (last_reference) {
+ e->Free();
+ }
+}
+
+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_;
+}
+
+size_t LRUCacheShard::GetPinnedUsage() const {
+ DMutexLock l(mutex_);
+ assert(usage_ >= lru_usage_);
+ return usage_ - lru_usage_;
+}
+
+size_t LRUCacheShard::GetOccupancyCount() const {
+ DMutexLock l(mutex_);
+ return table_.GetOccupancyCount();
+}
+
+size_t LRUCacheShard::GetTableAddressCount() const {
+ DMutexLock l(mutex_);
+ return size_t{1} << table_.GetLengthBits();
+}
+
+void LRUCacheShard::AppendPrintableOptions(std::string& str) const {
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+ {
+ DMutexLock l(mutex_);
+ snprintf(buffer, kBufferSize, " high_pri_pool_ratio: %.3lf\n",
+ high_pri_pool_ratio_);
+ snprintf(buffer + strlen(buffer), kBufferSize - strlen(buffer),
+ " low_pri_pool_ratio: %.3lf\n", low_pri_pool_ratio_);
+ }
+ str.append(buffer);
+}
+
+LRUCache::LRUCache(size_t capacity, int num_shard_bits,
+ bool strict_capacity_limit, double high_pri_pool_ratio,
+ double low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> allocator,
+ bool use_adaptive_mutex,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ std::shared_ptr<SecondaryCache> _secondary_cache)
+ : ShardedCache(capacity, num_shard_bits, strict_capacity_limit,
+ std::move(allocator)),
+ secondary_cache_(std::move(_secondary_cache)) {
+ size_t per_shard = GetPerShardCapacity();
+ SecondaryCache* secondary_cache = secondary_cache_.get();
+ 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, secondary_cache);
+ });
+}
+
+void* LRUCache::Value(Handle* handle) {
+ auto h = reinterpret_cast<const LRUHandle*>(handle);
+ assert(!h->IsPending() || h->value == nullptr);
+ assert(h->value != kDummyValueMarker);
+ return h->value;
+}
+
+size_t LRUCache::GetCharge(Handle* handle) const {
+ return reinterpret_cast<const LRUHandle*>(handle)->GetCharge(
+ GetShard(0).metadata_charge_policy_);
+}
+
+Cache::DeleterFn LRUCache::GetDeleter(Handle* handle) const {
+ auto h = reinterpret_cast<const LRUHandle*>(handle);
+ if (h->IsSecondaryCacheCompatible()) {
+ return h->info_.helper->del_cb;
+ } else {
+ return h->info_.deleter;
+ }
+}
+
+size_t LRUCache::TEST_GetLRUSize() {
+ return SumOverShards([](LRUCacheShard& cs) { return cs.TEST_GetLRUSize(); });
+}
+
+double LRUCache::GetHighPriPoolRatio() {
+ return GetShard(0).GetHighPriPoolRatio();
+}
+
+void LRUCache::WaitAll(std::vector<Handle*>& handles) {
+ if (secondary_cache_) {
+ std::vector<SecondaryCacheResultHandle*> sec_handles;
+ sec_handles.reserve(handles.size());
+ for (Handle* handle : handles) {
+ if (!handle) {
+ continue;
+ }
+ LRUHandle* lru_handle = reinterpret_cast<LRUHandle*>(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<LRUHandle*>(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<Cache> NewLRUCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator, bool use_adaptive_mutex,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ const std::shared_ptr<SecondaryCache>& secondary_cache,
+ double low_pri_pool_ratio) {
+ if (num_shard_bits >= 20) {
+ return nullptr; // The cache cannot be sharded into too many fine pieces.
+ }
+ if (high_pri_pool_ratio < 0.0 || high_pri_pool_ratio > 1.0) {
+ // Invalid high_pri_pool_ratio
+ return nullptr;
+ }
+ if (low_pri_pool_ratio < 0.0 || low_pri_pool_ratio > 1.0) {
+ // Invalid low_pri_pool_ratio
+ return nullptr;
+ }
+ if (low_pri_pool_ratio + high_pri_pool_ratio > 1.0) {
+ // Invalid high_pri_pool_ratio and low_pri_pool_ratio combination
+ return nullptr;
+ }
+ if (num_shard_bits < 0) {
+ num_shard_bits = GetDefaultCacheShardBits(capacity);
+ }
+ return std::make_shared<LRUCache>(
+ 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);
+}
+
+std::shared_ptr<Cache> NewLRUCache(const LRUCacheOptions& cache_opts) {
+ return NewLRUCache(cache_opts.capacity, cache_opts.num_shard_bits,
+ cache_opts.strict_capacity_limit,
+ cache_opts.high_pri_pool_ratio,
+ cache_opts.memory_allocator, cache_opts.use_adaptive_mutex,
+ cache_opts.metadata_charge_policy,
+ cache_opts.secondary_cache, cache_opts.low_pri_pool_ratio);
+}
+
+std::shared_ptr<Cache> NewLRUCache(
+ size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator, bool use_adaptive_mutex,
+ CacheMetadataChargePolicy metadata_charge_policy,
+ double low_pri_pool_ratio) {
+ return NewLRUCache(capacity, num_shard_bits, strict_capacity_limit,
+ high_pri_pool_ratio, memory_allocator, use_adaptive_mutex,
+ metadata_charge_policy, nullptr, low_pri_pool_ratio);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/lru_cache.h b/src/rocksdb/cache/lru_cache.h
new file mode 100644
index 000000000..99b2f2b20
--- /dev/null
+++ b/src/rocksdb/cache/lru_cache.h
@@ -0,0 +1,546 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "cache/sharded_cache.h"
+#include "port/lang.h"
+#include "port/malloc.h"
+#include "port/port.h"
+#include "rocksdb/secondary_cache.h"
+#include "util/autovector.h"
+#include "util/distributed_mutex.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace lru_cache {
+
+// LRU cache implementation. This class is not thread-safe.
+
+// An entry is a variable length heap-allocated structure.
+// Entries are referenced by cache and/or by any external entity.
+// The cache keeps all its entries in a hash table. Some elements
+// are also stored on LRU list.
+//
+// LRUHandle can be in these states:
+// 1. Referenced externally AND in hash table.
+// In that case the entry is *not* in the LRU list
+// (refs >= 1 && in_cache == true)
+// 2. Not referenced externally AND in hash table.
+// In that case the entry is in the LRU list and can be freed.
+// (refs == 0 && in_cache == true)
+// 3. Referenced externally AND not in hash table.
+// In that case the entry is not in the LRU list and not in hash table.
+// The entry must be freed if refs becomes 0 in this state.
+// (refs >= 1 && in_cache == false)
+// If you call LRUCacheShard::Release enough times on an entry in state 1, it
+// will go into state 2. To move from state 1 to state 3, either call
+// LRUCacheShard::Erase or LRUCacheShard::Insert with the same key (but
+// possibly different value). To move from state 2 to state 1, use
+// LRUCacheShard::Lookup.
+// While refs > 0, public properties like value and deleter must not change.
+
+struct LRUHandle {
+ void* value;
+ union Info {
+ Info() {}
+ ~Info() {}
+ Cache::DeleterFn deleter;
+ const Cache::CacheItemHelper* helper;
+ } info_;
+ // 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;
+ LRUHandle* prev;
+ size_t total_charge; // TODO(opt): Only allow uint32_t?
+ size_t key_length;
+ // The hash of key(). Used for fast sharding and comparisons.
+ uint32_t hash;
+ // The number of external refs to this entry. The cache itself is not counted.
+ uint32_t refs;
+
+ // Mutable flags - access controlled by mutex
+ // The m_ and M_ prefixes (and im_ and IM_ later) are to hopefully avoid
+ // checking an M_ flag on im_flags or an IM_ flag on m_flags.
+ uint8_t m_flags;
+ enum MFlags : uint8_t {
+ // Whether this entry is referenced by the hash table.
+ M_IN_CACHE = (1 << 0),
+ // Whether this entry has had any lookups (hits).
+ M_HAS_HIT = (1 << 1),
+ // Whether this entry is in high-pri pool.
+ M_IN_HIGH_PRI_POOL = (1 << 2),
+ // Whether this entry is in low-pri pool.
+ M_IN_LOW_PRI_POOL = (1 << 3),
+ };
+
+ // "Immutable" flags - only set in single-threaded context and then
+ // can be accessed without mutex
+ uint8_t im_flags;
+ enum ImFlags : uint8_t {
+ // Whether this entry is high priority entry.
+ IM_IS_HIGH_PRI = (1 << 0),
+ // Whether this entry is low priority entry.
+ IM_IS_LOW_PRI = (1 << 1),
+ // Can this be inserted into the secondary cache.
+ IM_IS_SECONDARY_CACHE_COMPATIBLE = (1 << 2),
+ // Is the handle still being read from a lower tier.
+ IM_IS_PENDING = (1 << 3),
+ // Whether this handle is still in a lower tier
+ IM_IS_IN_SECONDARY_CACHE = (1 << 4),
+ // Marks result handles that should not be inserted into cache
+ IM_IS_STANDALONE = (1 << 5),
+ };
+
+ // Beginning of the key (MUST BE THE LAST FIELD IN THIS STRUCT!)
+ char key_data[1];
+
+ Slice key() const { return Slice(key_data, key_length); }
+
+ // For HandleImpl concept
+ uint32_t GetHash() const { return hash; }
+
+ // Increase the reference count by 1.
+ void Ref() { refs++; }
+
+ // Just reduce the reference count by 1. Return true if it was last reference.
+ bool Unref() {
+ assert(refs > 0);
+ refs--;
+ return refs == 0;
+ }
+
+ // Return true if there are external refs, false otherwise.
+ bool HasRefs() const { return refs > 0; }
+
+ bool InCache() const { return m_flags & M_IN_CACHE; }
+ bool IsHighPri() const { return im_flags & IM_IS_HIGH_PRI; }
+ bool InHighPriPool() const { return m_flags & M_IN_HIGH_PRI_POOL; }
+ 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 im_flags & IM_IS_SECONDARY_CACHE_COMPATIBLE;
+ }
+ bool IsPending() const { return im_flags & IM_IS_PENDING; }
+ bool IsInSecondaryCache() const {
+ return im_flags & IM_IS_IN_SECONDARY_CACHE;
+ }
+ bool IsStandalone() const { return im_flags & IM_IS_STANDALONE; }
+
+ void SetInCache(bool in_cache) {
+ if (in_cache) {
+ m_flags |= M_IN_CACHE;
+ } else {
+ m_flags &= ~M_IN_CACHE;
+ }
+ }
+
+ void SetPriority(Cache::Priority priority) {
+ if (priority == Cache::Priority::HIGH) {
+ im_flags |= IM_IS_HIGH_PRI;
+ im_flags &= ~IM_IS_LOW_PRI;
+ } else if (priority == Cache::Priority::LOW) {
+ im_flags &= ~IM_IS_HIGH_PRI;
+ im_flags |= IM_IS_LOW_PRI;
+ } else {
+ im_flags &= ~IM_IS_HIGH_PRI;
+ im_flags &= ~IM_IS_LOW_PRI;
+ }
+ }
+
+ void SetInHighPriPool(bool in_high_pri_pool) {
+ if (in_high_pri_pool) {
+ m_flags |= M_IN_HIGH_PRI_POOL;
+ } else {
+ m_flags &= ~M_IN_HIGH_PRI_POOL;
+ }
+ }
+
+ void SetInLowPriPool(bool in_low_pri_pool) {
+ if (in_low_pri_pool) {
+ m_flags |= M_IN_LOW_PRI_POOL;
+ } else {
+ m_flags &= ~M_IN_LOW_PRI_POOL;
+ }
+ }
+
+ void SetHit() { m_flags |= M_HAS_HIT; }
+
+ void SetSecondaryCacheCompatible(bool compat) {
+ if (compat) {
+ im_flags |= IM_IS_SECONDARY_CACHE_COMPATIBLE;
+ } else {
+ im_flags &= ~IM_IS_SECONDARY_CACHE_COMPATIBLE;
+ }
+ }
+
+ void SetIsPending(bool pending) {
+ if (pending) {
+ im_flags |= IM_IS_PENDING;
+ } else {
+ im_flags &= ~IM_IS_PENDING;
+ }
+ }
+
+ void SetIsInSecondaryCache(bool is_in_secondary_cache) {
+ if (is_in_secondary_cache) {
+ im_flags |= IM_IS_IN_SECONDARY_CACHE;
+ } else {
+ im_flags &= ~IM_IS_IN_SECONDARY_CACHE;
+ }
+ }
+
+ void SetIsStandalone(bool is_standalone) {
+ if (is_standalone) {
+ im_flags |= IM_IS_STANDALONE;
+ } else {
+ im_flags &= ~IM_IS_STANDALONE;
+ }
+ }
+
+ void Free() {
+ assert(refs == 0);
+
+ if (!IsSecondaryCacheCompatible() && info_.deleter) {
+ (*info_.deleter)(key(), value);
+ } else if (IsSecondaryCacheCompatible()) {
+ if (IsPending()) {
+ assert(sec_handle != nullptr);
+ SecondaryCacheResultHandle* tmp_sec_handle = sec_handle;
+ tmp_sec_handle->Wait();
+ value = tmp_sec_handle->Value();
+ delete tmp_sec_handle;
+ }
+ if (value) {
+ (*info_.helper->del_cb)(key(), value);
+ }
+ }
+
+ free(this);
+ }
+
+ inline size_t CalcuMetaCharge(
+ CacheMetadataChargePolicy metadata_charge_policy) const {
+ if (metadata_charge_policy != kFullChargeCacheMetadata) {
+ return 0;
+ } else {
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ return malloc_usable_size(
+ const_cast<void*>(static_cast<const void*>(this)));
+#else
+ // This is the size that is used when a new handle is created.
+ return sizeof(LRUHandle) - 1 + key_length;
+#endif
+ }
+ }
+
+ // Calculate the memory usage by metadata.
+ inline void CalcTotalCharge(
+ size_t charge, CacheMetadataChargePolicy metadata_charge_policy) {
+ total_charge = charge + CalcuMetaCharge(metadata_charge_policy);
+ }
+
+ inline size_t GetCharge(
+ CacheMetadataChargePolicy metadata_charge_policy) const {
+ size_t meta_charge = CalcuMetaCharge(metadata_charge_policy);
+ assert(total_charge >= meta_charge);
+ return total_charge - meta_charge;
+ }
+};
+
+// We provide our own simple hash table since it removes a whole bunch
+// of porting hacks and is also faster than some of the built-in hash
+// table implementations in some of the compiler/runtime combinations
+// we have tested. E.g., readrandom speeds up by ~5% over the g++
+// 4.4.3's builtin hashtable.
+class LRUHandleTable {
+ public:
+ explicit LRUHandleTable(int max_upper_hash_bits);
+ ~LRUHandleTable();
+
+ LRUHandle* Lookup(const Slice& key, uint32_t hash);
+ LRUHandle* Insert(LRUHandle* h);
+ LRUHandle* Remove(const Slice& key, uint32_t hash);
+
+ template <typename T>
+ void ApplyToEntriesRange(T func, size_t index_begin, size_t index_end) {
+ for (size_t i = index_begin; i < index_end; i++) {
+ LRUHandle* h = list_[i];
+ while (h != nullptr) {
+ auto n = h->next_hash;
+ assert(h->InCache());
+ func(h);
+ h = n;
+ }
+ }
+ }
+
+ int GetLengthBits() const { return length_bits_; }
+
+ size_t GetOccupancyCount() const { return elems_; }
+
+ private:
+ // Return a pointer to slot that points to a cache entry that
+ // matches key/hash. If there is no such cache entry, return a
+ // pointer to the trailing slot in the corresponding linked list.
+ LRUHandle** FindPointer(const Slice& key, uint32_t hash);
+
+ void Resize();
+
+ // Number of hash bits (upper because lower bits used for sharding)
+ // used for table index. Length == 1 << length_bits_
+ int length_bits_;
+
+ // The table consists of an array of buckets where each bucket is
+ // a linked list of cache entries that hash into the bucket.
+ std::unique_ptr<LRUHandle*[]> list_;
+
+ // Number of elements currently in the table.
+ uint32_t elems_;
+
+ // Set from max_upper_hash_bits (see constructor).
+ const int max_length_bits_;
+};
+
+// A single shard of sharded cache.
+class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShardBase {
+ public:
+ 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, SecondaryCache* secondary_cache);
+
+ public: // Type definitions expected as parameter to ShardedCache
+ using HandleImpl = LRUHandle;
+ using HashVal = uint32_t;
+ using HashCref = uint32_t;
+
+ public: // Function definitions expected as parameter to ShardedCache
+ static inline HashVal ComputeHash(const Slice& key) {
+ return Lower32of64(GetSliceNPHash64(key));
+ }
+
+ // Separate from constructor so caller can easily make an array of LRUCache
+ // if current usage is more than new capacity, the function will attempt to
+ // free the needed space.
+ void SetCapacity(size_t capacity);
+
+ // Set the flag to reject insertion if cache if full.
+ void SetStrictCapacityLimit(bool strict_capacity_limit);
+
+ // Set percentage of capacity reserved for high-pri cache entries.
+ void SetHighPriorityPoolRatio(double high_pri_pool_ratio);
+
+ // Set percentage of capacity reserved for low-pri cache entries.
+ void SetLowPriorityPoolRatio(double low_pri_pool_ratio);
+
+ // Like Cache methods, but with an extra "hash" parameter.
+ inline Status Insert(const Slice& key, uint32_t hash, void* value,
+ size_t charge, Cache::DeleterFn deleter,
+ LRUHandle** handle, Cache::Priority priority) {
+ return Insert(key, hash, value, charge, deleter, nullptr, handle, priority);
+ }
+ inline Status Insert(const Slice& key, uint32_t hash, void* value,
+ const Cache::CacheItemHelper* helper, size_t charge,
+ LRUHandle** handle, Cache::Priority priority) {
+ assert(helper);
+ return Insert(key, hash, value, charge, nullptr, helper, handle, priority);
+ }
+ // If helper_cb is null, the values of the following arguments don't matter.
+ LRUHandle* Lookup(const Slice& key, uint32_t hash,
+ const Cache::CacheItemHelper* helper,
+ const Cache::CreateCallback& create_cb,
+ Cache::Priority priority, bool wait, Statistics* stats);
+ inline LRUHandle* Lookup(const Slice& key, uint32_t hash) {
+ return Lookup(key, hash, nullptr, nullptr, Cache::Priority::LOW, true,
+ nullptr);
+ }
+ 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);
+
+ // Although in some platforms the update of size_t is atomic, to make sure
+ // GetUsage() and GetPinnedUsage() work correctly under any platform, we'll
+ // protect them with mutex_.
+
+ size_t GetUsage() const;
+ size_t GetPinnedUsage() const;
+ size_t GetOccupancyCount() const;
+ size_t GetTableAddressCount() const;
+
+ void ApplyToSomeEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ size_t average_entries_per_lock, size_t* state);
+
+ void EraseUnRefEntries();
+
+ public: // other function definitions
+ void TEST_GetLRUList(LRUHandle** lru, LRUHandle** lru_low_pri,
+ LRUHandle** lru_bottom_pri);
+
+ // Retrieves number of elements in LRU, for unit test purpose only.
+ // Not threadsafe.
+ size_t TEST_GetLRUSize();
+
+ // Retrieves high pri pool ratio
+ double GetHighPriPoolRatio();
+
+ // Retrieves low pri pool ratio
+ double GetLowPriPoolRatio();
+
+ void AppendPrintableOptions(std::string& /*str*/) const;
+
+ 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);
+ Status Insert(const Slice& key, uint32_t hash, void* value, size_t charge,
+ DeleterFn deleter, const Cache::CacheItemHelper* helper,
+ LRUHandle** handle, Cache::Priority priority);
+ // 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);
+ void LRU_Remove(LRUHandle* e);
+ void LRU_Insert(LRUHandle* e);
+
+ // Overflow the last entry in high-pri pool to low-pri pool until size of
+ // high-pri pool is no larger than the size specify by high_pri_pool_pct.
+ void MaintainPoolSize();
+
+ // Free some space following strict LRU policy until enough space
+ // to hold (usage_ + charge) is freed or the lru list is empty
+ // This function is not thread safe - it needs to be executed while
+ // holding the mutex_.
+ void EvictFromLRU(size_t charge, autovector<LRUHandle*>* deleted);
+
+ // Try to insert the evicted handles into the secondary cache.
+ void TryInsertIntoSecondaryCache(autovector<LRUHandle*> evicted_handles);
+
+ // Initialized before use.
+ size_t capacity_;
+
+ // Memory size for entries in high-pri pool.
+ size_t high_pri_pool_usage_;
+
+ // Memory size for entries in low-pri pool.
+ size_t low_pri_pool_usage_;
+
+ // Whether to reject insertion if cache reaches its full capacity.
+ bool strict_capacity_limit_;
+
+ // Ratio of capacity reserved for high priority cache entries.
+ double high_pri_pool_ratio_;
+
+ // High-pri pool size, equals to capacity * high_pri_pool_ratio.
+ // Remember the value to avoid recomputing each time.
+ double high_pri_pool_capacity_;
+
+ // Ratio of capacity reserved for low priority cache entries.
+ double low_pri_pool_ratio_;
+
+ // Low-pri pool size, equals to capacity * low_pri_pool_ratio.
+ // Remember the value to avoid recomputing each time.
+ double low_pri_pool_capacity_;
+
+ // Dummy head of LRU list.
+ // lru.prev is newest entry, lru.next is oldest entry.
+ // LRU contains items which can be evicted, ie reference only by cache
+ LRUHandle lru_;
+
+ // Pointer to head of low-pri pool in LRU list.
+ LRUHandle* lru_low_pri_;
+
+ // Pointer to head of bottom-pri pool in LRU list.
+ LRUHandle* lru_bottom_pri_;
+
+ // ------------^^^^^^^^^^^^^-----------
+ // Not frequently modified data members
+ // ------------------------------------
+ //
+ // We separate data members that are updated frequently from the ones that
+ // are not frequently updated so that they don't share the same cache line
+ // which will lead into false cache sharing
+ //
+ // ------------------------------------
+ // Frequently modified data members
+ // ------------vvvvvvvvvvvvv-----------
+ LRUHandleTable table_;
+
+ // Memory size for entries residing in the cache.
+ size_t usage_;
+
+ // Memory size for entries residing only in the LRU list.
+ size_t lru_usage_;
+
+ // mutex_ protects the following state.
+ // We don't count mutex_ as the cache's internal state so semantically we
+ // don't mind mutex_ invoking the non-const actions.
+ mutable DMutex mutex_;
+
+ // Owned by LRUCache
+ SecondaryCache* secondary_cache_;
+};
+
+class LRUCache
+#ifdef NDEBUG
+ final
+#endif
+ : public ShardedCache<LRUCacheShard> {
+ public:
+ LRUCache(size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ double high_pri_pool_ratio, double low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> memory_allocator = nullptr,
+ bool use_adaptive_mutex = kDefaultToAdaptiveMutex,
+ CacheMetadataChargePolicy metadata_charge_policy =
+ kDontChargeCacheMetadata,
+ std::shared_ptr<SecondaryCache> secondary_cache = nullptr);
+ const char* Name() const override { return "LRUCache"; }
+ void* Value(Handle* handle) override;
+ size_t GetCharge(Handle* handle) const override;
+ DeleterFn GetDeleter(Handle* handle) const override;
+ void WaitAll(std::vector<Handle*>& 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<SecondaryCache> secondary_cache_;
+};
+
+} // namespace lru_cache
+
+using LRUCache = lru_cache::LRUCache;
+using LRUHandle = lru_cache::LRUHandle;
+using LRUCacheShard = lru_cache::LRUCacheShard;
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/lru_cache_test.cc b/src/rocksdb/cache/lru_cache_test.cc
new file mode 100644
index 000000000..7904a196d
--- /dev/null
+++ b/src/rocksdb/cache/lru_cache_test.cc
@@ -0,0 +1,2624 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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/lru_cache.h"
+
+#include <string>
+#include <vector>
+
+#include "cache/cache_key.h"
+#include "cache/clock_cache.h"
+#include "db/db_test_util.h"
+#include "file/sst_file_manager_impl.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/io_status.h"
+#include "rocksdb/sst_file_manager.h"
+#include "rocksdb/utilities/cache_dump_load.h"
+#include "test_util/testharness.h"
+#include "util/coding.h"
+#include "util/random.h"
+#include "utilities/cache_dump_load_impl.h"
+#include "utilities/fault_injection_fs.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class LRUCacheTest : public testing::Test {
+ public:
+ LRUCacheTest() {}
+ ~LRUCacheTest() override { DeleteCache(); }
+
+ void DeleteCache() {
+ if (cache_ != nullptr) {
+ cache_->~LRUCacheShard();
+ port::cacheline_aligned_free(cache_);
+ cache_ = nullptr;
+ }
+ }
+
+ void NewCache(size_t capacity, double high_pri_pool_ratio = 0.0,
+ double low_pri_pool_ratio = 1.0,
+ bool use_adaptive_mutex = kDefaultToAdaptiveMutex) {
+ DeleteCache();
+ cache_ = reinterpret_cast<LRUCacheShard*>(
+ port::cacheline_aligned_alloc(sizeof(LRUCacheShard)));
+ new (cache_) LRUCacheShard(capacity, /*strict_capacity_limit=*/false,
+ high_pri_pool_ratio, low_pri_pool_ratio,
+ use_adaptive_mutex, kDontChargeCacheMetadata,
+ /*max_upper_hash_bits=*/24,
+ /*secondary_cache=*/nullptr);
+ }
+
+ void Insert(const std::string& key,
+ Cache::Priority priority = Cache::Priority::LOW) {
+ EXPECT_OK(cache_->Insert(key, 0 /*hash*/, nullptr /*value*/, 1 /*charge*/,
+ nullptr /*deleter*/, nullptr /*handle*/,
+ priority));
+ }
+
+ void Insert(char key, Cache::Priority priority = Cache::Priority::LOW) {
+ Insert(std::string(1, key), priority);
+ }
+
+ bool Lookup(const std::string& key) {
+ auto handle = cache_->Lookup(key, 0 /*hash*/);
+ if (handle) {
+ cache_->Release(handle, true /*useful*/, false /*erase*/);
+ return true;
+ }
+ return false;
+ }
+
+ bool Lookup(char key) { return Lookup(std::string(1, key)); }
+
+ void Erase(const std::string& key) { cache_->Erase(key, 0 /*hash*/); }
+
+ void ValidateLRUList(std::vector<std::string> keys,
+ size_t num_high_pri_pool_keys = 0,
+ size_t num_low_pri_pool_keys = 0,
+ size_t num_bottom_pri_pool_keys = 0) {
+ LRUHandle* lru;
+ LRUHandle* lru_low_pri;
+ LRUHandle* lru_bottom_pri;
+ cache_->TEST_GetLRUList(&lru, &lru_low_pri, &lru_bottom_pri);
+
+ LRUHandle* iter = lru;
+
+ bool in_low_pri_pool = false;
+ bool in_high_pri_pool = false;
+
+ size_t high_pri_pool_keys = 0;
+ size_t low_pri_pool_keys = 0;
+ size_t bottom_pri_pool_keys = 0;
+
+ if (iter == lru_bottom_pri) {
+ in_low_pri_pool = true;
+ in_high_pri_pool = false;
+ }
+ if (iter == lru_low_pri) {
+ in_low_pri_pool = false;
+ in_high_pri_pool = true;
+ }
+
+ for (const auto& key : keys) {
+ iter = iter->next;
+ ASSERT_NE(lru, iter);
+ ASSERT_EQ(key, iter->key().ToString());
+ ASSERT_EQ(in_high_pri_pool, iter->InHighPriPool());
+ ASSERT_EQ(in_low_pri_pool, iter->InLowPriPool());
+ if (in_high_pri_pool) {
+ ASSERT_FALSE(iter->InLowPriPool());
+ high_pri_pool_keys++;
+ } else if (in_low_pri_pool) {
+ ASSERT_FALSE(iter->InHighPriPool());
+ low_pri_pool_keys++;
+ } else {
+ bottom_pri_pool_keys++;
+ }
+ if (iter == lru_bottom_pri) {
+ ASSERT_FALSE(in_low_pri_pool);
+ ASSERT_FALSE(in_high_pri_pool);
+ in_low_pri_pool = true;
+ in_high_pri_pool = false;
+ }
+ if (iter == lru_low_pri) {
+ ASSERT_TRUE(in_low_pri_pool);
+ ASSERT_FALSE(in_high_pri_pool);
+ in_low_pri_pool = false;
+ in_high_pri_pool = true;
+ }
+ }
+ ASSERT_EQ(lru, iter->next);
+ ASSERT_FALSE(in_low_pri_pool);
+ ASSERT_TRUE(in_high_pri_pool);
+ ASSERT_EQ(num_high_pri_pool_keys, high_pri_pool_keys);
+ ASSERT_EQ(num_low_pri_pool_keys, low_pri_pool_keys);
+ ASSERT_EQ(num_bottom_pri_pool_keys, bottom_pri_pool_keys);
+ }
+
+ private:
+ LRUCacheShard* cache_ = nullptr;
+};
+
+TEST_F(LRUCacheTest, BasicLRU) {
+ NewCache(5);
+ for (char ch = 'a'; ch <= 'e'; ch++) {
+ Insert(ch);
+ }
+ ValidateLRUList({"a", "b", "c", "d", "e"}, 0, 5);
+ for (char ch = 'x'; ch <= 'z'; ch++) {
+ Insert(ch);
+ }
+ ValidateLRUList({"d", "e", "x", "y", "z"}, 0, 5);
+ ASSERT_FALSE(Lookup("b"));
+ ValidateLRUList({"d", "e", "x", "y", "z"}, 0, 5);
+ ASSERT_TRUE(Lookup("e"));
+ ValidateLRUList({"d", "x", "y", "z", "e"}, 0, 5);
+ ASSERT_TRUE(Lookup("z"));
+ ValidateLRUList({"d", "x", "y", "e", "z"}, 0, 5);
+ Erase("x");
+ ValidateLRUList({"d", "y", "e", "z"}, 0, 4);
+ ASSERT_TRUE(Lookup("d"));
+ ValidateLRUList({"y", "e", "z", "d"}, 0, 4);
+ Insert("u");
+ ValidateLRUList({"y", "e", "z", "d", "u"}, 0, 5);
+ Insert("v");
+ ValidateLRUList({"e", "z", "d", "u", "v"}, 0, 5);
+}
+
+TEST_F(LRUCacheTest, LowPriorityMidpointInsertion) {
+ // Allocate 2 cache entries to high-pri pool and 3 to low-pri pool.
+ NewCache(5, /* high_pri_pool_ratio */ 0.40, /* low_pri_pool_ratio */ 0.60);
+
+ Insert("a", Cache::Priority::LOW);
+ Insert("b", Cache::Priority::LOW);
+ Insert("c", Cache::Priority::LOW);
+ Insert("x", Cache::Priority::HIGH);
+ Insert("y", Cache::Priority::HIGH);
+ ValidateLRUList({"a", "b", "c", "x", "y"}, 2, 3);
+
+ // Low-pri entries inserted to the tail of low-pri list (the midpoint).
+ // After lookup, it will move to the tail of the full list.
+ Insert("d", Cache::Priority::LOW);
+ ValidateLRUList({"b", "c", "d", "x", "y"}, 2, 3);
+ ASSERT_TRUE(Lookup("d"));
+ ValidateLRUList({"b", "c", "x", "y", "d"}, 2, 3);
+
+ // High-pri entries will be inserted to the tail of full list.
+ Insert("z", Cache::Priority::HIGH);
+ ValidateLRUList({"c", "x", "y", "d", "z"}, 2, 3);
+}
+
+TEST_F(LRUCacheTest, BottomPriorityMidpointInsertion) {
+ // Allocate 2 cache entries to high-pri pool and 2 to low-pri pool.
+ NewCache(6, /* high_pri_pool_ratio */ 0.35, /* low_pri_pool_ratio */ 0.35);
+
+ Insert("a", Cache::Priority::BOTTOM);
+ Insert("b", Cache::Priority::BOTTOM);
+ Insert("i", Cache::Priority::LOW);
+ Insert("j", Cache::Priority::LOW);
+ Insert("x", Cache::Priority::HIGH);
+ Insert("y", Cache::Priority::HIGH);
+ ValidateLRUList({"a", "b", "i", "j", "x", "y"}, 2, 2, 2);
+
+ // Low-pri entries will be inserted to the tail of low-pri list (the
+ // midpoint). After lookup, 'k' will move to the tail of the full list, and
+ // 'x' will spill over to the low-pri pool.
+ Insert("k", Cache::Priority::LOW);
+ ValidateLRUList({"b", "i", "j", "k", "x", "y"}, 2, 2, 2);
+ ASSERT_TRUE(Lookup("k"));
+ ValidateLRUList({"b", "i", "j", "x", "y", "k"}, 2, 2, 2);
+
+ // High-pri entries will be inserted to the tail of full list. Although y was
+ // inserted with high priority, it got spilled over to the low-pri pool. As
+ // a result, j also got spilled over to the bottom-pri pool.
+ Insert("z", Cache::Priority::HIGH);
+ ValidateLRUList({"i", "j", "x", "y", "k", "z"}, 2, 2, 2);
+ Erase("x");
+ ValidateLRUList({"i", "j", "y", "k", "z"}, 2, 1, 2);
+ Erase("y");
+ ValidateLRUList({"i", "j", "k", "z"}, 2, 0, 2);
+
+ // Bottom-pri entries will be inserted to the tail of bottom-pri list.
+ Insert("c", Cache::Priority::BOTTOM);
+ ValidateLRUList({"i", "j", "c", "k", "z"}, 2, 0, 3);
+ Insert("d", Cache::Priority::BOTTOM);
+ ValidateLRUList({"i", "j", "c", "d", "k", "z"}, 2, 0, 4);
+ Insert("e", Cache::Priority::BOTTOM);
+ ValidateLRUList({"j", "c", "d", "e", "k", "z"}, 2, 0, 4);
+
+ // Low-pri entries will be inserted to the tail of low-pri list (the
+ // midpoint).
+ Insert("l", Cache::Priority::LOW);
+ ValidateLRUList({"c", "d", "e", "l", "k", "z"}, 2, 1, 3);
+ Insert("m", Cache::Priority::LOW);
+ ValidateLRUList({"d", "e", "l", "m", "k", "z"}, 2, 2, 2);
+
+ Erase("k");
+ ValidateLRUList({"d", "e", "l", "m", "z"}, 1, 2, 2);
+ Erase("z");
+ ValidateLRUList({"d", "e", "l", "m"}, 0, 2, 2);
+
+ // Bottom-pri entries will be inserted to the tail of bottom-pri list.
+ Insert("f", Cache::Priority::BOTTOM);
+ ValidateLRUList({"d", "e", "f", "l", "m"}, 0, 2, 3);
+ Insert("g", Cache::Priority::BOTTOM);
+ ValidateLRUList({"d", "e", "f", "g", "l", "m"}, 0, 2, 4);
+
+ // High-pri entries will be inserted to the tail of full list.
+ Insert("o", Cache::Priority::HIGH);
+ ValidateLRUList({"e", "f", "g", "l", "m", "o"}, 1, 2, 3);
+ Insert("p", Cache::Priority::HIGH);
+ ValidateLRUList({"f", "g", "l", "m", "o", "p"}, 2, 2, 2);
+}
+
+TEST_F(LRUCacheTest, EntriesWithPriority) {
+ // Allocate 2 cache entries to high-pri pool and 2 to low-pri pool.
+ NewCache(6, /* high_pri_pool_ratio */ 0.35, /* low_pri_pool_ratio */ 0.35);
+
+ Insert("a", Cache::Priority::LOW);
+ Insert("b", Cache::Priority::LOW);
+ ValidateLRUList({"a", "b"}, 0, 2, 0);
+ // Low-pri entries can overflow to bottom-pri pool.
+ Insert("c", Cache::Priority::LOW);
+ ValidateLRUList({"a", "b", "c"}, 0, 2, 1);
+
+ // Bottom-pri entries can take high-pri pool capacity if available
+ Insert("t", Cache::Priority::LOW);
+ Insert("u", Cache::Priority::LOW);
+ ValidateLRUList({"a", "b", "c", "t", "u"}, 0, 2, 3);
+ Insert("v", Cache::Priority::LOW);
+ ValidateLRUList({"a", "b", "c", "t", "u", "v"}, 0, 2, 4);
+ Insert("w", Cache::Priority::LOW);
+ ValidateLRUList({"b", "c", "t", "u", "v", "w"}, 0, 2, 4);
+
+ Insert("X", Cache::Priority::HIGH);
+ Insert("Y", Cache::Priority::HIGH);
+ ValidateLRUList({"t", "u", "v", "w", "X", "Y"}, 2, 2, 2);
+
+ // After lookup, the high-pri entry 'X' got spilled over to the low-pri pool.
+ // The low-pri entry 'v' got spilled over to the bottom-pri pool.
+ Insert("Z", Cache::Priority::HIGH);
+ ValidateLRUList({"u", "v", "w", "X", "Y", "Z"}, 2, 2, 2);
+
+ // Low-pri entries will be inserted to head of low-pri pool.
+ Insert("a", Cache::Priority::LOW);
+ ValidateLRUList({"v", "w", "X", "a", "Y", "Z"}, 2, 2, 2);
+
+ // After lookup, the high-pri entry 'Y' got spilled over to the low-pri pool.
+ // The low-pri entry 'X' got spilled over to the bottom-pri pool.
+ ASSERT_TRUE(Lookup("v"));
+ ValidateLRUList({"w", "X", "a", "Y", "Z", "v"}, 2, 2, 2);
+
+ // After lookup, the high-pri entry 'Z' got spilled over to the low-pri pool.
+ // The low-pri entry 'a' got spilled over to the bottom-pri pool.
+ ASSERT_TRUE(Lookup("X"));
+ ValidateLRUList({"w", "a", "Y", "Z", "v", "X"}, 2, 2, 2);
+
+ // After lookup, the low pri entry 'Z' got promoted back to high-pri pool. The
+ // high-pri entry 'v' got spilled over to the low-pri pool.
+ ASSERT_TRUE(Lookup("Z"));
+ ValidateLRUList({"w", "a", "Y", "v", "X", "Z"}, 2, 2, 2);
+
+ Erase("Y");
+ ValidateLRUList({"w", "a", "v", "X", "Z"}, 2, 1, 2);
+ Erase("X");
+ ValidateLRUList({"w", "a", "v", "Z"}, 1, 1, 2);
+
+ Insert("d", Cache::Priority::LOW);
+ Insert("e", Cache::Priority::LOW);
+ ValidateLRUList({"w", "a", "v", "d", "e", "Z"}, 1, 2, 3);
+
+ Insert("f", Cache::Priority::LOW);
+ Insert("g", Cache::Priority::LOW);
+ ValidateLRUList({"v", "d", "e", "f", "g", "Z"}, 1, 2, 3);
+ ASSERT_TRUE(Lookup("d"));
+ ValidateLRUList({"v", "e", "f", "g", "Z", "d"}, 2, 2, 2);
+
+ // Erase some entries.
+ Erase("e");
+ Erase("f");
+ Erase("Z");
+ ValidateLRUList({"v", "g", "d"}, 1, 1, 1);
+
+ // Bottom-pri entries can take low- and high-pri pool capacity if available
+ Insert("o", Cache::Priority::BOTTOM);
+ ValidateLRUList({"v", "o", "g", "d"}, 1, 1, 2);
+ Insert("p", Cache::Priority::BOTTOM);
+ ValidateLRUList({"v", "o", "p", "g", "d"}, 1, 1, 3);
+ Insert("q", Cache::Priority::BOTTOM);
+ ValidateLRUList({"v", "o", "p", "q", "g", "d"}, 1, 1, 4);
+
+ // High-pri entries can overflow to low-pri pool, and bottom-pri entries will
+ // be evicted.
+ Insert("x", Cache::Priority::HIGH);
+ ValidateLRUList({"o", "p", "q", "g", "d", "x"}, 2, 1, 3);
+ Insert("y", Cache::Priority::HIGH);
+ ValidateLRUList({"p", "q", "g", "d", "x", "y"}, 2, 2, 2);
+ Insert("z", Cache::Priority::HIGH);
+ ValidateLRUList({"q", "g", "d", "x", "y", "z"}, 2, 2, 2);
+
+ // 'g' is bottom-pri before this lookup, it will be inserted to head of
+ // high-pri pool after lookup.
+ ASSERT_TRUE(Lookup("g"));
+ ValidateLRUList({"q", "d", "x", "y", "z", "g"}, 2, 2, 2);
+
+ // High-pri entries will be inserted to head of high-pri pool after lookup.
+ ASSERT_TRUE(Lookup("z"));
+ ValidateLRUList({"q", "d", "x", "y", "g", "z"}, 2, 2, 2);
+
+ // Bottom-pri entries will be inserted to head of high-pri pool after lookup.
+ ASSERT_TRUE(Lookup("d"));
+ ValidateLRUList({"q", "x", "y", "g", "z", "d"}, 2, 2, 2);
+
+ // Bottom-pri entries will be inserted to the tail of bottom-pri list.
+ Insert("m", Cache::Priority::BOTTOM);
+ ValidateLRUList({"x", "m", "y", "g", "z", "d"}, 2, 2, 2);
+
+ // Bottom-pri entries will be inserted to head of high-pri pool after lookup.
+ ASSERT_TRUE(Lookup("m"));
+ ValidateLRUList({"x", "y", "g", "z", "d", "m"}, 2, 2, 2);
+}
+
+namespace clock_cache {
+
+class ClockCacheTest : public testing::Test {
+ public:
+ using Shard = HyperClockCache::Shard;
+ using Table = HyperClockTable;
+ using HandleImpl = Shard::HandleImpl;
+
+ ClockCacheTest() {}
+ ~ClockCacheTest() override { DeleteShard(); }
+
+ void DeleteShard() {
+ if (shard_ != nullptr) {
+ shard_->~ClockCacheShard();
+ port::cacheline_aligned_free(shard_);
+ shard_ = nullptr;
+ }
+ }
+
+ void NewShard(size_t capacity, bool strict_capacity_limit = true) {
+ DeleteShard();
+ shard_ =
+ reinterpret_cast<Shard*>(port::cacheline_aligned_alloc(sizeof(Shard)));
+
+ Table::Opts opts;
+ opts.estimated_value_size = 1;
+ new (shard_)
+ Shard(capacity, strict_capacity_limit, kDontChargeCacheMetadata, opts);
+ }
+
+ Status Insert(const UniqueId64x2& hashed_key,
+ Cache::Priority priority = Cache::Priority::LOW) {
+ return shard_->Insert(TestKey(hashed_key), hashed_key, nullptr /*value*/,
+ 1 /*charge*/, nullptr /*deleter*/, nullptr /*handle*/,
+ priority);
+ }
+
+ Status Insert(char key, Cache::Priority priority = Cache::Priority::LOW) {
+ return Insert(TestHashedKey(key), priority);
+ }
+
+ Status InsertWithLen(char key, size_t len) {
+ std::string skey(len, key);
+ return shard_->Insert(skey, TestHashedKey(key), nullptr /*value*/,
+ 1 /*charge*/, nullptr /*deleter*/, nullptr /*handle*/,
+ Cache::Priority::LOW);
+ }
+
+ bool Lookup(const Slice& key, const UniqueId64x2& hashed_key,
+ bool useful = true) {
+ auto handle = shard_->Lookup(key, hashed_key);
+ if (handle) {
+ shard_->Release(handle, useful, /*erase_if_last_ref=*/false);
+ return true;
+ }
+ return false;
+ }
+
+ bool Lookup(const UniqueId64x2& hashed_key, bool useful = true) {
+ return Lookup(TestKey(hashed_key), hashed_key, useful);
+ }
+
+ bool Lookup(char key, bool useful = true) {
+ return Lookup(TestHashedKey(key), useful);
+ }
+
+ void Erase(char key) {
+ UniqueId64x2 hashed_key = TestHashedKey(key);
+ shard_->Erase(TestKey(hashed_key), hashed_key);
+ }
+
+ static inline Slice TestKey(const UniqueId64x2& hashed_key) {
+ return Slice(reinterpret_cast<const char*>(&hashed_key), 16U);
+ }
+
+ static inline UniqueId64x2 TestHashedKey(char key) {
+ // For testing hash near-collision behavior, put the variance in
+ // hashed_key in bits that are unlikely to be used as hash bits.
+ return {(static_cast<uint64_t>(key) << 56) + 1234U, 5678U};
+ }
+
+ Shard* shard_ = nullptr;
+};
+
+TEST_F(ClockCacheTest, Misc) {
+ NewShard(3);
+
+ // Key size stuff
+ EXPECT_OK(InsertWithLen('a', 16));
+ EXPECT_NOK(InsertWithLen('b', 15));
+ EXPECT_OK(InsertWithLen('b', 16));
+ EXPECT_NOK(InsertWithLen('c', 17));
+ EXPECT_NOK(InsertWithLen('d', 1000));
+ EXPECT_NOK(InsertWithLen('e', 11));
+ EXPECT_NOK(InsertWithLen('f', 0));
+
+ // Some of this is motivated by code coverage
+ std::string wrong_size_key(15, 'x');
+ EXPECT_FALSE(Lookup(wrong_size_key, TestHashedKey('x')));
+ EXPECT_FALSE(shard_->Ref(nullptr));
+ EXPECT_FALSE(shard_->Release(nullptr));
+ shard_->Erase(wrong_size_key, TestHashedKey('x')); // no-op
+}
+
+TEST_F(ClockCacheTest, Limits) {
+ constexpr size_t kCapacity = 3;
+ NewShard(kCapacity, false /*strict_capacity_limit*/);
+ for (bool strict_capacity_limit : {false, true, false}) {
+ SCOPED_TRACE("strict_capacity_limit = " +
+ std::to_string(strict_capacity_limit));
+
+ // Also tests switching between strict limit and not
+ shard_->SetStrictCapacityLimit(strict_capacity_limit);
+
+ UniqueId64x2 hkey = TestHashedKey('x');
+
+ // Single entry charge beyond capacity
+ {
+ Status s = shard_->Insert(TestKey(hkey), hkey, nullptr /*value*/,
+ 5 /*charge*/, nullptr /*deleter*/,
+ nullptr /*handle*/, Cache::Priority::LOW);
+ if (strict_capacity_limit) {
+ EXPECT_TRUE(s.IsMemoryLimit());
+ } else {
+ EXPECT_OK(s);
+ }
+ }
+
+ // Single entry fills capacity
+ {
+ HandleImpl* h;
+ ASSERT_OK(shard_->Insert(TestKey(hkey), hkey, nullptr /*value*/,
+ 3 /*charge*/, nullptr /*deleter*/, &h,
+ Cache::Priority::LOW));
+ // Try to insert more
+ Status s = Insert('a');
+ if (strict_capacity_limit) {
+ EXPECT_TRUE(s.IsMemoryLimit());
+ } else {
+ EXPECT_OK(s);
+ }
+ // Release entry filling capacity.
+ // Cover useful = false case.
+ shard_->Release(h, false /*useful*/, false /*erase_if_last_ref*/);
+ }
+
+ // Insert more than table size can handle to exceed occupancy limit.
+ // (Cleverly using mostly zero-charge entries, but some non-zero to
+ // verify usage tracking on detached entries.)
+ {
+ size_t n = shard_->GetTableAddressCount() + 1;
+ std::unique_ptr<HandleImpl* []> ha { new HandleImpl* [n] {} };
+ Status s;
+ for (size_t i = 0; i < n && s.ok(); ++i) {
+ hkey[1] = i;
+ s = shard_->Insert(TestKey(hkey), hkey, nullptr /*value*/,
+ (i + kCapacity < n) ? 0 : 1 /*charge*/,
+ nullptr /*deleter*/, &ha[i], Cache::Priority::LOW);
+ if (i == 0) {
+ EXPECT_OK(s);
+ }
+ }
+ if (strict_capacity_limit) {
+ EXPECT_TRUE(s.IsMemoryLimit());
+ } else {
+ EXPECT_OK(s);
+ }
+ // Same result if not keeping a reference
+ s = Insert('a');
+ if (strict_capacity_limit) {
+ EXPECT_TRUE(s.IsMemoryLimit());
+ } else {
+ EXPECT_OK(s);
+ }
+
+ // Regardless, we didn't allow table to actually get full
+ EXPECT_LT(shard_->GetOccupancyCount(), shard_->GetTableAddressCount());
+
+ // Release handles
+ for (size_t i = 0; i < n; ++i) {
+ if (ha[i]) {
+ shard_->Release(ha[i]);
+ }
+ }
+ }
+ }
+}
+
+TEST_F(ClockCacheTest, ClockEvictionTest) {
+ for (bool strict_capacity_limit : {false, true}) {
+ SCOPED_TRACE("strict_capacity_limit = " +
+ std::to_string(strict_capacity_limit));
+
+ NewShard(6, strict_capacity_limit);
+ EXPECT_OK(Insert('a', Cache::Priority::BOTTOM));
+ EXPECT_OK(Insert('b', Cache::Priority::LOW));
+ EXPECT_OK(Insert('c', Cache::Priority::HIGH));
+ EXPECT_OK(Insert('d', Cache::Priority::BOTTOM));
+ EXPECT_OK(Insert('e', Cache::Priority::LOW));
+ EXPECT_OK(Insert('f', Cache::Priority::HIGH));
+
+ EXPECT_TRUE(Lookup('a', /*use*/ false));
+ EXPECT_TRUE(Lookup('b', /*use*/ false));
+ EXPECT_TRUE(Lookup('c', /*use*/ false));
+ EXPECT_TRUE(Lookup('d', /*use*/ false));
+ EXPECT_TRUE(Lookup('e', /*use*/ false));
+ EXPECT_TRUE(Lookup('f', /*use*/ false));
+
+ // Ensure bottom are evicted first, even if new entries are low
+ EXPECT_OK(Insert('g', Cache::Priority::LOW));
+ EXPECT_OK(Insert('h', Cache::Priority::LOW));
+
+ EXPECT_FALSE(Lookup('a', /*use*/ false));
+ EXPECT_TRUE(Lookup('b', /*use*/ false));
+ EXPECT_TRUE(Lookup('c', /*use*/ false));
+ EXPECT_FALSE(Lookup('d', /*use*/ false));
+ EXPECT_TRUE(Lookup('e', /*use*/ false));
+ EXPECT_TRUE(Lookup('f', /*use*/ false));
+ // Mark g & h useful
+ EXPECT_TRUE(Lookup('g', /*use*/ true));
+ EXPECT_TRUE(Lookup('h', /*use*/ true));
+
+ // Then old LOW entries
+ EXPECT_OK(Insert('i', Cache::Priority::LOW));
+ EXPECT_OK(Insert('j', Cache::Priority::LOW));
+
+ EXPECT_FALSE(Lookup('b', /*use*/ false));
+ EXPECT_TRUE(Lookup('c', /*use*/ false));
+ EXPECT_FALSE(Lookup('e', /*use*/ false));
+ EXPECT_TRUE(Lookup('f', /*use*/ false));
+ // Mark g & h useful once again
+ EXPECT_TRUE(Lookup('g', /*use*/ true));
+ EXPECT_TRUE(Lookup('h', /*use*/ true));
+ EXPECT_TRUE(Lookup('i', /*use*/ false));
+ EXPECT_TRUE(Lookup('j', /*use*/ false));
+
+ // Then old HIGH entries
+ EXPECT_OK(Insert('k', Cache::Priority::LOW));
+ EXPECT_OK(Insert('l', Cache::Priority::LOW));
+
+ EXPECT_FALSE(Lookup('c', /*use*/ false));
+ EXPECT_FALSE(Lookup('f', /*use*/ false));
+ EXPECT_TRUE(Lookup('g', /*use*/ false));
+ EXPECT_TRUE(Lookup('h', /*use*/ false));
+ EXPECT_TRUE(Lookup('i', /*use*/ false));
+ EXPECT_TRUE(Lookup('j', /*use*/ false));
+ EXPECT_TRUE(Lookup('k', /*use*/ false));
+ EXPECT_TRUE(Lookup('l', /*use*/ false));
+
+ // Then the (roughly) least recently useful
+ EXPECT_OK(Insert('m', Cache::Priority::HIGH));
+ EXPECT_OK(Insert('n', Cache::Priority::HIGH));
+
+ EXPECT_TRUE(Lookup('g', /*use*/ false));
+ EXPECT_TRUE(Lookup('h', /*use*/ false));
+ EXPECT_FALSE(Lookup('i', /*use*/ false));
+ EXPECT_FALSE(Lookup('j', /*use*/ false));
+ EXPECT_TRUE(Lookup('k', /*use*/ false));
+ EXPECT_TRUE(Lookup('l', /*use*/ false));
+
+ // Now try changing capacity down
+ shard_->SetCapacity(4);
+ // Insert to ensure evictions happen
+ EXPECT_OK(Insert('o', Cache::Priority::LOW));
+ EXPECT_OK(Insert('p', Cache::Priority::LOW));
+
+ EXPECT_FALSE(Lookup('g', /*use*/ false));
+ EXPECT_FALSE(Lookup('h', /*use*/ false));
+ EXPECT_FALSE(Lookup('k', /*use*/ false));
+ EXPECT_FALSE(Lookup('l', /*use*/ false));
+ EXPECT_TRUE(Lookup('m', /*use*/ false));
+ EXPECT_TRUE(Lookup('n', /*use*/ false));
+ EXPECT_TRUE(Lookup('o', /*use*/ false));
+ EXPECT_TRUE(Lookup('p', /*use*/ false));
+
+ // Now try changing capacity up
+ EXPECT_TRUE(Lookup('m', /*use*/ true));
+ EXPECT_TRUE(Lookup('n', /*use*/ true));
+ shard_->SetCapacity(6);
+ EXPECT_OK(Insert('q', Cache::Priority::HIGH));
+ EXPECT_OK(Insert('r', Cache::Priority::HIGH));
+ EXPECT_OK(Insert('s', Cache::Priority::HIGH));
+ EXPECT_OK(Insert('t', Cache::Priority::HIGH));
+
+ EXPECT_FALSE(Lookup('o', /*use*/ false));
+ EXPECT_FALSE(Lookup('p', /*use*/ false));
+ EXPECT_TRUE(Lookup('m', /*use*/ false));
+ EXPECT_TRUE(Lookup('n', /*use*/ false));
+ EXPECT_TRUE(Lookup('q', /*use*/ false));
+ EXPECT_TRUE(Lookup('r', /*use*/ false));
+ EXPECT_TRUE(Lookup('s', /*use*/ false));
+ EXPECT_TRUE(Lookup('t', /*use*/ false));
+ }
+}
+
+void IncrementIntDeleter(const Slice& /*key*/, void* value) {
+ *reinterpret_cast<int*>(value) += 1;
+}
+
+// Testing calls to CorrectNearOverflow in Release
+TEST_F(ClockCacheTest, ClockCounterOverflowTest) {
+ NewShard(6, /*strict_capacity_limit*/ false);
+ HandleImpl* h;
+ int deleted = 0;
+ UniqueId64x2 hkey = TestHashedKey('x');
+ ASSERT_OK(shard_->Insert(TestKey(hkey), hkey, &deleted, 1,
+ IncrementIntDeleter, &h, Cache::Priority::HIGH));
+
+ // Some large number outstanding
+ shard_->TEST_RefN(h, 123456789);
+ // Simulate many lookup/ref + release, plenty to overflow counters
+ for (int i = 0; i < 10000; ++i) {
+ shard_->TEST_RefN(h, 1234567);
+ shard_->TEST_ReleaseN(h, 1234567);
+ }
+ // Mark it invisible (to reach a different CorrectNearOverflow() in Release)
+ shard_->Erase(TestKey(hkey), hkey);
+ // Simulate many more lookup/ref + release (one-by-one would be too
+ // expensive for unit test)
+ for (int i = 0; i < 10000; ++i) {
+ shard_->TEST_RefN(h, 1234567);
+ shard_->TEST_ReleaseN(h, 1234567);
+ }
+ // Free all but last 1
+ shard_->TEST_ReleaseN(h, 123456789);
+ // Still alive
+ ASSERT_EQ(deleted, 0);
+ // Free last ref, which will finalize erasure
+ shard_->Release(h);
+ // Deleted
+ ASSERT_EQ(deleted, 1);
+}
+
+// This test is mostly to exercise some corner case logic, by forcing two
+// keys to have the same hash, and more
+TEST_F(ClockCacheTest, CollidingInsertEraseTest) {
+ NewShard(6, /*strict_capacity_limit*/ false);
+ int deleted = 0;
+ UniqueId64x2 hkey1 = TestHashedKey('x');
+ Slice key1 = TestKey(hkey1);
+ UniqueId64x2 hkey2 = TestHashedKey('y');
+ Slice key2 = TestKey(hkey2);
+ UniqueId64x2 hkey3 = TestHashedKey('z');
+ Slice key3 = TestKey(hkey3);
+ HandleImpl* h1;
+ ASSERT_OK(shard_->Insert(key1, hkey1, &deleted, 1, IncrementIntDeleter, &h1,
+ Cache::Priority::HIGH));
+ HandleImpl* h2;
+ ASSERT_OK(shard_->Insert(key2, hkey2, &deleted, 1, IncrementIntDeleter, &h2,
+ Cache::Priority::HIGH));
+ HandleImpl* h3;
+ ASSERT_OK(shard_->Insert(key3, hkey3, &deleted, 1, IncrementIntDeleter, &h3,
+ Cache::Priority::HIGH));
+
+ // Can repeatedly lookup+release despite the hash collision
+ HandleImpl* tmp_h;
+ for (bool erase_if_last_ref : {true, false}) { // but not last ref
+ tmp_h = shard_->Lookup(key1, hkey1);
+ ASSERT_EQ(h1, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+
+ tmp_h = shard_->Lookup(key2, hkey2);
+ ASSERT_EQ(h2, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+
+ tmp_h = shard_->Lookup(key3, hkey3);
+ ASSERT_EQ(h3, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+ }
+
+ // Make h1 invisible
+ shard_->Erase(key1, hkey1);
+ // Redundant erase
+ shard_->Erase(key1, hkey1);
+
+ // All still alive
+ ASSERT_EQ(deleted, 0);
+
+ // Invisible to Lookup
+ tmp_h = shard_->Lookup(key1, hkey1);
+ ASSERT_EQ(nullptr, tmp_h);
+
+ // Can still find h2, h3
+ for (bool erase_if_last_ref : {true, false}) { // but not last ref
+ tmp_h = shard_->Lookup(key2, hkey2);
+ ASSERT_EQ(h2, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+
+ tmp_h = shard_->Lookup(key3, hkey3);
+ ASSERT_EQ(h3, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+ }
+
+ // Also Insert with invisible entry there
+ ASSERT_OK(shard_->Insert(key1, hkey1, &deleted, 1, IncrementIntDeleter,
+ nullptr, Cache::Priority::HIGH));
+ tmp_h = shard_->Lookup(key1, hkey1);
+ // Found but distinct handle
+ ASSERT_NE(nullptr, tmp_h);
+ ASSERT_NE(h1, tmp_h);
+ ASSERT_TRUE(shard_->Release(tmp_h, /*erase_if_last_ref*/ true));
+
+ // tmp_h deleted
+ ASSERT_EQ(deleted--, 1);
+
+ // Release last ref on h1 (already invisible)
+ ASSERT_TRUE(shard_->Release(h1, /*erase_if_last_ref*/ false));
+
+ // h1 deleted
+ ASSERT_EQ(deleted--, 1);
+ h1 = nullptr;
+
+ // Can still find h2, h3
+ for (bool erase_if_last_ref : {true, false}) { // but not last ref
+ tmp_h = shard_->Lookup(key2, hkey2);
+ ASSERT_EQ(h2, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+
+ tmp_h = shard_->Lookup(key3, hkey3);
+ ASSERT_EQ(h3, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+ }
+
+ // Release last ref on h2
+ ASSERT_FALSE(shard_->Release(h2, /*erase_if_last_ref*/ false));
+
+ // h2 still not deleted (unreferenced in cache)
+ ASSERT_EQ(deleted, 0);
+
+ // Can still find it
+ tmp_h = shard_->Lookup(key2, hkey2);
+ ASSERT_EQ(h2, tmp_h);
+
+ // Release last ref on h2, with erase
+ ASSERT_TRUE(shard_->Release(h2, /*erase_if_last_ref*/ true));
+
+ // h2 deleted
+ ASSERT_EQ(deleted--, 1);
+ tmp_h = shard_->Lookup(key2, hkey2);
+ ASSERT_EQ(nullptr, tmp_h);
+
+ // Can still find h3
+ for (bool erase_if_last_ref : {true, false}) { // but not last ref
+ tmp_h = shard_->Lookup(key3, hkey3);
+ ASSERT_EQ(h3, tmp_h);
+ ASSERT_FALSE(shard_->Release(tmp_h, erase_if_last_ref));
+ }
+
+ // Release last ref on h3, without erase
+ ASSERT_FALSE(shard_->Release(h3, /*erase_if_last_ref*/ false));
+
+ // h3 still not deleted (unreferenced in cache)
+ ASSERT_EQ(deleted, 0);
+
+ // Explicit erase
+ shard_->Erase(key3, hkey3);
+
+ // h3 deleted
+ ASSERT_EQ(deleted--, 1);
+ tmp_h = shard_->Lookup(key3, hkey3);
+ ASSERT_EQ(nullptr, tmp_h);
+}
+
+// This uses the public API to effectively test CalcHashBits etc.
+TEST_F(ClockCacheTest, TableSizesTest) {
+ for (size_t est_val_size : {1U, 5U, 123U, 2345U, 345678U}) {
+ SCOPED_TRACE("est_val_size = " + std::to_string(est_val_size));
+ for (double est_count : {1.1, 2.2, 511.9, 512.1, 2345.0}) {
+ SCOPED_TRACE("est_count = " + std::to_string(est_count));
+ size_t capacity = static_cast<size_t>(est_val_size * est_count);
+ // kDontChargeCacheMetadata
+ auto cache = HyperClockCacheOptions(
+ capacity, est_val_size, /*num shard_bits*/ -1,
+ /*strict_capacity_limit*/ false,
+ /*memory_allocator*/ nullptr, kDontChargeCacheMetadata)
+ .MakeSharedCache();
+ // Table sizes are currently only powers of two
+ EXPECT_GE(cache->GetTableAddressCount(), est_count / kLoadFactor);
+ EXPECT_LE(cache->GetTableAddressCount(), est_count / kLoadFactor * 2.0);
+ EXPECT_EQ(cache->GetUsage(), 0);
+
+ // kFullChargeMetaData
+ // Because table sizes are currently only powers of two, sizes get
+ // really weird when metadata is a huge portion of capacity. For example,
+ // doubling the table size could cut by 90% the space available to
+ // values. Therefore, we omit those weird cases for now.
+ if (est_val_size >= 512) {
+ cache = HyperClockCacheOptions(
+ capacity, est_val_size, /*num shard_bits*/ -1,
+ /*strict_capacity_limit*/ false,
+ /*memory_allocator*/ nullptr, kFullChargeCacheMetadata)
+ .MakeSharedCache();
+ double est_count_after_meta =
+ (capacity - cache->GetUsage()) * 1.0 / est_val_size;
+ EXPECT_GE(cache->GetTableAddressCount(),
+ est_count_after_meta / kLoadFactor);
+ EXPECT_LE(cache->GetTableAddressCount(),
+ est_count_after_meta / kLoadFactor * 2.0);
+ }
+ }
+ }
+}
+
+} // namespace clock_cache
+
+class TestSecondaryCache : public SecondaryCache {
+ public:
+ // Specifies what action to take on a lookup for a particular key
+ enum ResultType {
+ SUCCESS,
+ // Fail lookup immediately
+ FAIL,
+ // Defer the result. It will returned after Wait/WaitAll is called
+ DEFER,
+ // Defer the result and eventually return failure
+ DEFER_AND_FAIL
+ };
+
+ using ResultMap = std::unordered_map<std::string, ResultType>;
+
+ explicit TestSecondaryCache(size_t capacity)
+ : num_inserts_(0), num_lookups_(0), inject_failure_(false) {
+ cache_ =
+ NewLRUCache(capacity, 0, false, 0.5 /* high_pri_pool_ratio */, nullptr,
+ kDefaultToAdaptiveMutex, kDontChargeCacheMetadata);
+ }
+ ~TestSecondaryCache() override { cache_.reset(); }
+
+ const char* Name() const override { return "TestSecondaryCache"; }
+
+ void InjectFailure() { inject_failure_ = true; }
+
+ void ResetInjectFailure() { inject_failure_ = false; }
+
+ Status Insert(const Slice& key, void* value,
+ const Cache::CacheItemHelper* helper) override {
+ if (inject_failure_) {
+ return Status::Corruption("Insertion Data Corrupted");
+ }
+ CheckCacheKeyCommonPrefix(key);
+ size_t size;
+ char* buf;
+ Status s;
+
+ num_inserts_++;
+ size = (*helper->size_cb)(value);
+ buf = new char[size + sizeof(uint64_t)];
+ EncodeFixed64(buf, size);
+ s = (*helper->saveto_cb)(value, 0, size, buf + sizeof(uint64_t));
+ if (!s.ok()) {
+ delete[] buf;
+ return s;
+ }
+ return cache_->Insert(key, buf, size,
+ [](const Slice& /*key*/, void* val) -> void {
+ delete[] static_cast<char*>(val);
+ });
+ }
+
+ std::unique_ptr<SecondaryCacheResultHandle> Lookup(
+ const Slice& key, const Cache::CreateCallback& create_cb, bool /*wait*/,
+ bool /*advise_erase*/, bool& is_in_sec_cache) override {
+ std::string key_str = key.ToString();
+ TEST_SYNC_POINT_CALLBACK("TestSecondaryCache::Lookup", &key_str);
+
+ std::unique_ptr<SecondaryCacheResultHandle> secondary_handle;
+ is_in_sec_cache = false;
+ ResultType type = ResultType::SUCCESS;
+ auto iter = result_map_.find(key.ToString());
+ if (iter != result_map_.end()) {
+ type = iter->second;
+ }
+ if (type == ResultType::FAIL) {
+ return secondary_handle;
+ }
+
+ Cache::Handle* handle = cache_->Lookup(key);
+ num_lookups_++;
+ if (handle) {
+ void* value = nullptr;
+ size_t charge = 0;
+ Status s;
+ if (type != ResultType::DEFER_AND_FAIL) {
+ char* ptr = (char*)cache_->Value(handle);
+ size_t size = DecodeFixed64(ptr);
+ ptr += sizeof(uint64_t);
+ s = create_cb(ptr, size, &value, &charge);
+ }
+ if (s.ok()) {
+ secondary_handle.reset(new TestSecondaryCacheResultHandle(
+ cache_.get(), handle, value, charge, type));
+ is_in_sec_cache = true;
+ } else {
+ cache_->Release(handle);
+ }
+ }
+ return secondary_handle;
+ }
+
+ bool SupportForceErase() const override { return false; }
+
+ void Erase(const Slice& /*key*/) override {}
+
+ void WaitAll(std::vector<SecondaryCacheResultHandle*> handles) override {
+ for (SecondaryCacheResultHandle* handle : handles) {
+ TestSecondaryCacheResultHandle* sec_handle =
+ static_cast<TestSecondaryCacheResultHandle*>(handle);
+ sec_handle->SetReady();
+ }
+ }
+
+ std::string GetPrintableOptions() const override { return ""; }
+
+ void SetResultMap(ResultMap&& map) { result_map_ = std::move(map); }
+
+ uint32_t num_inserts() { return num_inserts_; }
+
+ uint32_t num_lookups() { return num_lookups_; }
+
+ void CheckCacheKeyCommonPrefix(const Slice& key) {
+ Slice current_prefix(key.data(), OffsetableCacheKey::kCommonPrefixSize);
+ if (ckey_prefix_.empty()) {
+ ckey_prefix_ = current_prefix.ToString();
+ } else {
+ EXPECT_EQ(ckey_prefix_, current_prefix.ToString());
+ }
+ }
+
+ private:
+ class TestSecondaryCacheResultHandle : public SecondaryCacheResultHandle {
+ public:
+ TestSecondaryCacheResultHandle(Cache* cache, Cache::Handle* handle,
+ void* value, size_t size, ResultType type)
+ : cache_(cache),
+ handle_(handle),
+ value_(value),
+ size_(size),
+ is_ready_(true) {
+ if (type != ResultType::SUCCESS) {
+ is_ready_ = false;
+ }
+ }
+
+ ~TestSecondaryCacheResultHandle() override { cache_->Release(handle_); }
+
+ bool IsReady() override { return is_ready_; }
+
+ void Wait() override {}
+
+ void* Value() override {
+ assert(is_ready_);
+ return value_;
+ }
+
+ size_t Size() override { return Value() ? size_ : 0; }
+
+ void SetReady() { is_ready_ = true; }
+
+ private:
+ Cache* cache_;
+ Cache::Handle* handle_;
+ void* value_;
+ size_t size_;
+ bool is_ready_;
+ };
+
+ std::shared_ptr<Cache> cache_;
+ uint32_t num_inserts_;
+ uint32_t num_lookups_;
+ bool inject_failure_;
+ std::string ckey_prefix_;
+ ResultMap result_map_;
+};
+
+class DBSecondaryCacheTest : public DBTestBase {
+ public:
+ DBSecondaryCacheTest()
+ : DBTestBase("db_secondary_cache_test", /*env_do_fsync=*/true) {
+ fault_fs_.reset(new FaultInjectionTestFS(env_->GetFileSystem()));
+ fault_env_.reset(new CompositeEnvWrapper(env_, fault_fs_));
+ }
+
+ std::shared_ptr<FaultInjectionTestFS> fault_fs_;
+ std::unique_ptr<Env> fault_env_;
+};
+
+class LRUCacheSecondaryCacheTest : public LRUCacheTest {
+ public:
+ LRUCacheSecondaryCacheTest() : fail_create_(false) {}
+ ~LRUCacheSecondaryCacheTest() {}
+
+ protected:
+ class TestItem {
+ public:
+ TestItem(const char* buf, size_t size) : buf_(new char[size]), size_(size) {
+ memcpy(buf_.get(), buf, size);
+ }
+ ~TestItem() {}
+
+ char* Buf() { return buf_.get(); }
+ size_t Size() { return size_; }
+ std::string ToString() { return std::string(Buf(), Size()); }
+
+ private:
+ std::unique_ptr<char[]> buf_;
+ size_t size_;
+ };
+
+ static size_t SizeCallback(void* obj) {
+ return reinterpret_cast<TestItem*>(obj)->Size();
+ }
+
+ static Status SaveToCallback(void* from_obj, size_t from_offset,
+ size_t length, void* out) {
+ TestItem* item = reinterpret_cast<TestItem*>(from_obj);
+ char* buf = item->Buf();
+ EXPECT_EQ(length, item->Size());
+ EXPECT_EQ(from_offset, 0);
+ memcpy(out, buf, length);
+ return Status::OK();
+ }
+
+ static void DeletionCallback(const Slice& /*key*/, void* obj) {
+ delete reinterpret_cast<TestItem*>(obj);
+ }
+
+ static Cache::CacheItemHelper helper_;
+
+ static Status SaveToCallbackFail(void* /*obj*/, size_t /*offset*/,
+ size_t /*size*/, void* /*out*/) {
+ return Status::NotSupported();
+ }
+
+ static Cache::CacheItemHelper helper_fail_;
+
+ Cache::CreateCallback test_item_creator = [&](const void* buf, size_t size,
+ void** out_obj,
+ size_t* charge) -> Status {
+ if (fail_create_) {
+ return Status::NotSupported();
+ }
+ *out_obj = reinterpret_cast<void*>(new TestItem((char*)buf, size));
+ *charge = size;
+ return Status::OK();
+ };
+
+ void SetFailCreate(bool fail) { fail_create_ = fail; }
+
+ private:
+ bool fail_create_;
+};
+
+Cache::CacheItemHelper LRUCacheSecondaryCacheTest::helper_(
+ LRUCacheSecondaryCacheTest::SizeCallback,
+ LRUCacheSecondaryCacheTest::SaveToCallback,
+ LRUCacheSecondaryCacheTest::DeletionCallback);
+
+Cache::CacheItemHelper LRUCacheSecondaryCacheTest::helper_fail_(
+ LRUCacheSecondaryCacheTest::SizeCallback,
+ LRUCacheSecondaryCacheTest::SaveToCallbackFail,
+ LRUCacheSecondaryCacheTest::DeletionCallback);
+
+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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(4096);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ std::shared_ptr<Statistics> stats = CreateDBStatistics();
+ CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+ CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+ CacheKey k3 = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+
+ Random rnd(301);
+ // Start with warming k3
+ std::string str3 = rnd.RandomString(1021);
+ ASSERT_OK(secondary_cache->InsertSaved(k3.AsSlice(), str3));
+
+ std::string str1 = rnd.RandomString(1020);
+ TestItem* item1 = new TestItem(str1.data(), str1.length());
+ ASSERT_OK(cache->Insert(k1.AsSlice(), item1,
+ &LRUCacheSecondaryCacheTest::helper_, str1.length()));
+ std::string str2 = rnd.RandomString(1021);
+ TestItem* item2 = new TestItem(str2.data(), str2.length());
+ // k1 should be demoted to NVM
+ ASSERT_OK(cache->Insert(k2.AsSlice(), item2,
+ &LRUCacheSecondaryCacheTest::helper_, str2.length()));
+
+ get_perf_context()->Reset();
+ Cache::Handle* handle;
+ handle =
+ cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true, stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(static_cast<TestItem*>(cache->Value(handle))->Size(), str2.size());
+ cache->Release(handle);
+
+ // This lookup should promote k1 and demote k2
+ handle =
+ cache->Lookup(k1.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true, stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(static_cast<TestItem*>(cache->Value(handle))->Size(), str1.size());
+ cache->Release(handle);
+
+ // This lookup should promote k3 and demote k1
+ handle =
+ cache->Lookup(k3.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true, stats.get());
+ ASSERT_NE(handle, nullptr);
+ ASSERT_EQ(static_cast<TestItem*>(cache->Value(handle))->Size(), str3.size());
+ cache->Release(handle);
+
+ ASSERT_EQ(secondary_cache->num_inserts(), 3u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+ ASSERT_EQ(stats->getTickerCount(SECONDARY_CACHE_HITS),
+ secondary_cache->num_lookups());
+ PerfContext perf_ctx = *get_perf_context();
+ ASSERT_EQ(perf_ctx.secondary_cache_hit_count, secondary_cache->num_lookups());
+
+ cache.reset();
+ 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ CacheKey k1 = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+ CacheKey k2 = CacheKey::CreateUniqueForCacheLifetime(cache.get());
+
+ Random rnd(301);
+ std::string str1 = rnd.RandomString(1020);
+ auto item1 = std::make_unique<TestItem>(str1.data(), str1.length());
+ ASSERT_TRUE(cache->Insert(k1.AsSlice(), item1.get(), nullptr, str1.length())
+ .IsInvalidArgument());
+ ASSERT_OK(cache->Insert(k1.AsSlice(), item1.get(),
+ &LRUCacheSecondaryCacheTest::helper_, str1.length()));
+ item1.release(); // Appease clang-analyze "potential memory leak"
+
+ Cache::Handle* handle;
+ handle = cache->Lookup(k2.AsSlice(), nullptr, test_item_creator,
+ Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ handle = cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, false);
+ 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ 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,
+ &LRUCacheSecondaryCacheTest::helper_fail_,
+ str1.length()));
+ std::string str2 = rnd.RandomString(1020);
+ TestItem* item2 = new TestItem(str2.data(), str2.length());
+ // k1 should be demoted to NVM
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_OK(cache->Insert(k2.AsSlice(), item2,
+ &LRUCacheSecondaryCacheTest::helper_fail_,
+ str2.length()));
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+
+ Cache::Handle* handle;
+ handle =
+ cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_fail_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+ // This lookup should fail, since k1 demotion would have failed
+ handle =
+ cache->Lookup(k1.AsSlice(), &LRUCacheSecondaryCacheTest::helper_fail_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ // Since k1 didn't get promoted, k2 should still be in cache
+ handle =
+ cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_fail_,
+ test_item_creator, 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.reset();
+ 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ 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,
+ &LRUCacheSecondaryCacheTest::helper_, 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,
+ &LRUCacheSecondaryCacheTest::helper_, str2.length()));
+
+ Cache::Handle* handle;
+ SetFailCreate(true);
+ handle = cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_NE(handle, nullptr);
+ cache->Release(handle);
+ // This lookup should fail, since k1 creation would have failed
+ handle = cache->Lookup(k1.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, true);
+ ASSERT_EQ(handle, nullptr);
+ // Since k1 didn't get promoted, k2 should still be in cache
+ handle = cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, 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.reset();
+ 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<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ 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,
+ &LRUCacheSecondaryCacheTest::helper_, 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,
+ &LRUCacheSecondaryCacheTest::helper_, str2.length()));
+
+ Cache::Handle* handle;
+ handle = cache->Lookup(k2.AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, 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(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, 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(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, 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.reset();
+ secondary_cache.reset();
+}
+
+// In this test, the block cache size is set to 4096, after insert 6 KV-pairs
+// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta
+// blocks. block_1 size is 4096 and block_2 size is 2056. The total size
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+
+ // Set the file paranoid check, so after flush, the file will be read
+ // all the blocks will be accessed.
+ options.paranoid_file_checks = true;
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+ // After Flush is successful, RocksDB will do the paranoid check for the new
+ // SST file. Meta blocks are always cached in the block cache and they
+ // will not be evicted. When block_2 is cache miss and read out, it is
+ // inserted to the block cache. Note that, block_1 is never successfully
+ // inserted to the block cache. Here are 2 lookups in the secondary cache
+ // for block_1 and block_2
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ Compact("a", "z");
+ // Compaction will create the iterator to scan the whole file. So all the
+ // blocks are needed. Meta blocks are always cached. When block_1 is read
+ // out, block_2 is evicted from block cache and inserted to secondary
+ // cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 3u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // The first data block is not in the cache, similarly, trigger the block
+ // cache Lookup and secondary cache lookup for block_1. But block_1 will not
+ // be inserted successfully due to the size. Currently, cache only has
+ // the meta blocks.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // The second data block is not in the cache, similarly, trigger the block
+ // cache Lookup and secondary cache lookup for block_2 and block_2 is found
+ // in the secondary cache. Now block cache has block_2
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // block_2 is in the block cache. There is a block cache hit. No need to
+ // lookup or insert the secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // Lookup the first data block, not in the block cache, so lookup the
+ // secondary cache. Also not in the secondary cache. After Get, still
+ // block_1 is will not be cached.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 6u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // Lookup the first data block, not in the block cache, so lookup the
+ // secondary cache. Also not in the secondary cache. After Get, still
+ // block_1 is will not be cached.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 7u);
+
+ Destroy(options);
+}
+
+// In this test, the block cache size is set to 6100, after insert 6 KV-pairs
+// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta
+// blocks. block_1 size is 4096 and block_2 size is 2056. The total size
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.paranoid_file_checks = true;
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+ // After Flush is successful, RocksDB will do the paranoid check for the new
+ // SST file. Meta blocks are always cached in the block cache and they
+ // will not be evicted. When block_2 is cache miss and read out, it is
+ // inserted to the block cache. Thefore, block_1 is evicted from block
+ // cache and successfully inserted to the secondary cache. Here are 2
+ // lookups in the secondary cache for block_1 and block_2.
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ Compact("a", "z");
+ // Compaction will create the iterator to scan the whole file. So all the
+ // blocks are needed. After Flush, only block_2 is cached in block cache
+ // and block_1 is in the secondary cache. So when read block_1, it is
+ // read out from secondary cache and inserted to block cache. At the same
+ // time, block_2 is inserted to secondary cache. Now, secondary cache has
+ // both block_1 and block_2. After compaction, block_1 is in the cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 3u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // This Get needs to access block_1, since block_1 is cached in block cache
+ // there is no secondary cache lookup.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 3u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // This Get needs to access block_2 which is not in the block cache. So
+ // it will lookup the secondary cache for block_2 and cache it in the
+ // block_cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // This Get needs to access block_2 which is already in the block cache.
+ // No need to lookup secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // This Get needs to access block_1, since block_1 is not in block cache
+ // there is one econdary cache lookup. Then, block_1 is cached in the
+ // block cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // This Get needs to access block_1, since block_1 is cached in block cache
+ // there is no secondary cache lookup.
+ ASSERT_EQ(secondary_cache->num_inserts(), 2u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ Destroy(options);
+}
+
+// The block cache size is set to 1024*1024, after insert 6 KV-pairs
+// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta
+// blocks. block_1 size is 4096 and block_2 size is 2056. The total size
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.paranoid_file_checks = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1000);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+ // After Flush is successful, RocksDB will do the paranoid check for the new
+ // SST file. Meta blocks are always cached in the block cache and they
+ // will not be evicted. Now, block cache is large enough, it cache
+ // both block_1 and block_2. When first time read block_1 and block_2
+ // there are cache misses. So 2 secondary cache lookups are needed for
+ // the 2 blocks
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ Compact("a", "z");
+ // Compaction will iterate the whole SST file. Since all the data blocks
+ // are in the block cache. No need to lookup the secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1000, v.size());
+ // Since the block cache is large enough, all the blocks are cached. we
+ // do not need to lookup the seondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 256;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1000);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+ ASSERT_OK(Flush());
+ Compact("a", "z");
+
+ Random r_index(47);
+ std::string v;
+ for (int i = 0; i < 1000; i++) {
+ uint32_t key_i = r_index.Next() % N;
+ v = Get(Key(key_i));
+ }
+
+ // We have over 200 data blocks there will be multiple insertion
+ // and lookups.
+ ASSERT_GE(secondary_cache->num_inserts(), 1u);
+ ASSERT_GE(secondary_cache->num_lookups(), 1u);
+
+ Destroy(options);
+}
+
+// In this test, the block cache size is set to 4096, after insert 6 KV-pairs
+// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta
+// blocks. block_1 size is 4096 and block_2 size is 2056. The total size
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.paranoid_file_checks = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+ // After Flush is successful, RocksDB will do the paranoid check for the new
+ // SST file. Meta blocks are always cached in the block cache and they
+ // will not be evicted. When block_2 is cache miss and read out, it is
+ // inserted to the block cache. Note that, block_1 is never successfully
+ // inserted to the block cache. Here are 2 lookups in the secondary cache
+ // for block_1 and block_2
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ // Fail the insertion, in LRU cache, the secondary insertion returned status
+ // is not checked, therefore, the DB will not be influenced.
+ secondary_cache->InjectFailure();
+ Compact("a", "z");
+ // Compaction will create the iterator to scan the whole file. So all the
+ // blocks are needed. Meta blocks are always cached. When block_1 is read
+ // out, block_2 is evicted from block cache and inserted to secondary
+ // cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 3u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // The first data block is not in the cache, similarly, trigger the block
+ // cache Lookup and secondary cache lookup for block_1. But block_1 will not
+ // be inserted successfully due to the size. Currently, cache only has
+ // the meta blocks.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // The second data block is not in the cache, similarly, trigger the block
+ // cache Lookup and secondary cache lookup for block_2 and block_2 is found
+ // in the secondary cache. Now block cache has block_2
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+ // block_2 is in the block cache. There is a block cache hit. No need to
+ // lookup or insert the secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 5u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // Lookup the first data block, not in the block cache, so lookup the
+ // secondary cache. Also not in the secondary cache. After Get, still
+ // block_1 is will not be cached.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 6u);
+
+ v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+ // Lookup the first data block, not in the block cache, so lookup the
+ // secondary cache. Also not in the secondary cache. After Get, still
+ // block_1 is will not be cached.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 7u);
+ secondary_cache->ResetInjectFailure();
+
+ Destroy(options);
+}
+
+TEST_F(DBSecondaryCacheTest, TestSecondaryWithCompressedCache) {
+ if (!Snappy_Supported()) {
+ ROCKSDB_GTEST_SKIP("Compressed cache test requires snappy support");
+ return;
+ }
+ LRUCacheOptions opts(2000 /* capacity */, 0 /* num_shard_bits */,
+ false /* strict_capacity_limit */,
+ 0.5 /* high_pri_pool_ratio */,
+ nullptr /* memory_allocator */, kDefaultToAdaptiveMutex,
+ kDontChargeCacheMetadata);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache_compressed = cache;
+ table_options.no_block_cache = true;
+ table_options.block_size = 1234;
+ Options options = GetDefaultOptions();
+ options.compression = kSnappyCompression;
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ // Partly compressible
+ std::string p_v = rnd.RandomString(507) + std::string(500, ' ');
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+ ASSERT_OK(Flush());
+ for (int i = 0; i < 2 * N; i++) {
+ std::string v = Get(Key(i % N));
+ ASSERT_EQ(1007, v.size());
+ }
+}
+
+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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(32 * 1024);
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ const int num_keys = 32;
+ OffsetableCacheKey ock{"foo", "bar", 1};
+
+ Random rnd(301);
+ std::vector<std::string> values;
+ for (int i = 0; i < num_keys; ++i) {
+ std::string str = rnd.RandomString(1020);
+ values.emplace_back(str);
+ TestItem* item = new TestItem(str.data(), str.length());
+ ASSERT_OK(cache->Insert(ock.WithOffset(i).AsSlice(), item,
+ &LRUCacheSecondaryCacheTest::helper_,
+ str.length()));
+ }
+ // Force all entries to be evicted to the secondary cache
+ cache->SetCapacity(0);
+ ASSERT_EQ(secondary_cache->num_inserts(), 32u);
+ cache->SetCapacity(32 * 1024);
+
+ secondary_cache->SetResultMap(
+ {{ock.WithOffset(3).AsSlice().ToString(),
+ TestSecondaryCache::ResultType::DEFER},
+ {ock.WithOffset(4).AsSlice().ToString(),
+ TestSecondaryCache::ResultType::DEFER_AND_FAIL},
+ {ock.WithOffset(5).AsSlice().ToString(),
+ TestSecondaryCache::ResultType::FAIL}});
+ std::vector<Cache::Handle*> results;
+ for (int i = 0; i < 6; ++i) {
+ results.emplace_back(cache->Lookup(
+ ock.WithOffset(i).AsSlice(), &LRUCacheSecondaryCacheTest::helper_,
+ test_item_creator, Cache::Priority::LOW, false));
+ }
+ 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);
+ continue;
+ } else {
+ TestItem* item = static_cast<TestItem*>(cache->Value(results[i]));
+ ASSERT_EQ(item->ToString(), values[i]);
+ }
+ cache->Release(results[i]);
+ }
+
+ cache.reset();
+ secondary_cache.reset();
+}
+
+// In this test, we have one KV pair per data block. We indirectly determine
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ table_options.cache_index_and_filter_blocks = false;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.paranoid_file_checks = true;
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 8;
+ std::vector<std::string> keys;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(4000);
+ keys.emplace_back(p_v);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+ // After Flush is successful, RocksDB does the paranoid check for the new
+ // SST file. This will try to lookup all data blocks in the secondary
+ // cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 8u);
+
+ cache->SetCapacity(0);
+ ASSERT_EQ(secondary_cache->num_inserts(), 8u);
+ cache->SetCapacity(1 << 20);
+
+ std::vector<std::string> cache_keys;
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
+ "TestSecondaryCache::Lookup", [&cache_keys](void* key) -> void {
+ cache_keys.emplace_back(*(static_cast<std::string*>(key)));
+ });
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
+ for (int i = 0; i < N; ++i) {
+ std::string v = Get(Key(i));
+ ASSERT_EQ(4000, v.size());
+ ASSERT_EQ(v, keys[i]);
+ }
+ ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
+ ASSERT_EQ(secondary_cache->num_lookups(), 16u);
+ cache->SetCapacity(0);
+ cache->SetCapacity(1 << 20);
+
+ ASSERT_EQ(Get(Key(2)), keys[2]);
+ ASSERT_EQ(Get(Key(7)), keys[7]);
+ secondary_cache->SetResultMap(
+ {{cache_keys[3], TestSecondaryCache::ResultType::DEFER},
+ {cache_keys[4], TestSecondaryCache::ResultType::DEFER_AND_FAIL},
+ {cache_keys[5], TestSecondaryCache::ResultType::FAIL}});
+
+ std::vector<std::string> mget_keys(
+ {Key(0), Key(1), Key(2), Key(3), Key(4), Key(5), Key(6), Key(7)});
+ std::vector<PinnableSlice> values(mget_keys.size());
+ std::vector<Status> s(keys.size());
+ std::vector<Slice> key_slices;
+ for (const std::string& key : mget_keys) {
+ key_slices.emplace_back(key);
+ }
+ uint32_t num_lookups = secondary_cache->num_lookups();
+ dbfull()->MultiGet(ReadOptions(), dbfull()->DefaultColumnFamily(),
+ key_slices.size(), key_slices.data(), values.data(),
+ s.data(), false);
+ ASSERT_EQ(secondary_cache->num_lookups(), num_lookups + 5);
+ for (int i = 0; i < N; ++i) {
+ ASSERT_OK(s[i]);
+ ASSERT_EQ(values[i].ToString(), keys[i]);
+ values[i].Reset();
+ }
+ Destroy(options);
+}
+
+class LRUCacheWithStat : public LRUCache {
+ public:
+ LRUCacheWithStat(
+ size_t _capacity, int _num_shard_bits, bool _strict_capacity_limit,
+ double _high_pri_pool_ratio, double _low_pri_pool_ratio,
+ std::shared_ptr<MemoryAllocator> _memory_allocator = nullptr,
+ bool _use_adaptive_mutex = kDefaultToAdaptiveMutex,
+ CacheMetadataChargePolicy _metadata_charge_policy =
+ kDontChargeCacheMetadata,
+ const std::shared_ptr<SecondaryCache>& _secondary_cache = nullptr)
+ : LRUCache(_capacity, _num_shard_bits, _strict_capacity_limit,
+ _high_pri_pool_ratio, _low_pri_pool_ratio, _memory_allocator,
+ _use_adaptive_mutex, _metadata_charge_policy,
+ _secondary_cache) {
+ insert_count_ = 0;
+ lookup_count_ = 0;
+ }
+ ~LRUCacheWithStat() {}
+
+ Status Insert(const Slice& key, void* value, size_t charge, DeleterFn deleter,
+ Handle** handle, Priority priority) override {
+ insert_count_++;
+ return LRUCache::Insert(key, value, charge, deleter, handle, priority);
+ }
+ Status Insert(const Slice& key, void* value, const CacheItemHelper* helper,
+ size_t charge, Handle** handle = nullptr,
+ Priority priority = Priority::LOW) override {
+ insert_count_++;
+ return LRUCache::Insert(key, value, helper, charge, handle, priority);
+ }
+ Handle* Lookup(const Slice& key, Statistics* stats) override {
+ lookup_count_++;
+ return LRUCache::Lookup(key, stats);
+ }
+ Handle* Lookup(const Slice& key, const CacheItemHelper* helper,
+ const CreateCallback& create_cb, Priority priority, bool wait,
+ Statistics* stats = nullptr) override {
+ lookup_count_++;
+ return LRUCache::Lookup(key, helper, create_cb, priority, wait, stats);
+ }
+
+ uint32_t GetInsertCount() { return insert_count_; }
+ uint32_t GetLookupcount() { return lookup_count_; }
+ void ResetCount() {
+ insert_count_ = 0;
+ lookup_count_ = 0;
+ }
+
+ private:
+ uint32_t insert_count_;
+ uint32_t lookup_count_;
+};
+
+#ifndef ROCKSDB_LITE
+
+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);
+ LRUCacheWithStat* tmp_cache = new LRUCacheWithStat(
+ cache_opts.capacity, cache_opts.num_shard_bits,
+ cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio,
+ cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator,
+ cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy,
+ cache_opts.secondary_cache);
+ std::shared_ptr<Cache> cache(tmp_cache);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ DestroyAndReopen(options);
+ fault_fs_->SetFailGetUniqueId(true);
+
+ Random rnd(301);
+ const int N = 256;
+ std::vector<std::string> value;
+ char buf[1000];
+ memset(buf, 'a', 1000);
+ value.resize(N);
+ for (int i = 0; i < N; i++) {
+ // std::string p_v = rnd.RandomString(1000);
+ std::string p_v(buf, 1000);
+ value[i] = p_v;
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+ ASSERT_OK(Flush());
+ Compact("a", "z");
+
+ // do th eread for all the key value pairs, so all the blocks should be in
+ // cache
+ uint32_t start_insert = tmp_cache->GetInsertCount();
+ uint32_t start_lookup = tmp_cache->GetLookupcount();
+ std::string v;
+ for (int i = 0; i < N; i++) {
+ v = Get(Key(i));
+ ASSERT_EQ(v, value[i]);
+ }
+ uint32_t dump_insert = tmp_cache->GetInsertCount() - start_insert;
+ uint32_t dump_lookup = tmp_cache->GetLookupcount() - start_lookup;
+ ASSERT_EQ(63,
+ static_cast<int>(dump_insert)); // the insert in the block cache
+ ASSERT_EQ(256,
+ static_cast<int>(dump_lookup)); // the lookup in the block cache
+ // We have enough blocks in the block cache
+
+ CacheDumpOptions cd_options;
+ cd_options.clock = fault_env_->GetSystemClock().get();
+ std::string dump_path = db_->GetName() + "/cache_dump";
+ std::unique_ptr<CacheDumpWriter> dump_writer;
+ Status s = NewToFileCacheDumpWriter(fault_fs_, FileOptions(), dump_path,
+ &dump_writer);
+ ASSERT_OK(s);
+ std::unique_ptr<CacheDumper> cache_dumper;
+ s = NewDefaultCacheDumper(cd_options, cache, std::move(dump_writer),
+ &cache_dumper);
+ ASSERT_OK(s);
+ std::vector<DB*> db_list;
+ db_list.push_back(db_);
+ s = cache_dumper->SetDumpFilter(db_list);
+ ASSERT_OK(s);
+ s = cache_dumper->DumpCacheEntriesToWriter();
+ ASSERT_OK(s);
+ cache_dumper.reset();
+
+ // we have a new cache it is empty, then, before we do the Get, we do the
+ // dumpload
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048 * 1024);
+ cache_opts.secondary_cache = secondary_cache;
+ tmp_cache = new LRUCacheWithStat(
+ cache_opts.capacity, cache_opts.num_shard_bits,
+ cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio,
+ cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator,
+ cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy,
+ cache_opts.secondary_cache);
+ std::shared_ptr<Cache> cache_new(tmp_cache);
+ table_options.block_cache = cache_new;
+ table_options.block_size = 4 * 1024;
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+
+ // start to load the data to new block cache
+ start_insert = secondary_cache->num_inserts();
+ start_lookup = secondary_cache->num_lookups();
+ std::unique_ptr<CacheDumpReader> dump_reader;
+ s = NewFromFileCacheDumpReader(fault_fs_, FileOptions(), dump_path,
+ &dump_reader);
+ ASSERT_OK(s);
+ std::unique_ptr<CacheDumpedLoader> cache_loader;
+ s = NewDefaultCacheDumpedLoader(cd_options, table_options, secondary_cache,
+ std::move(dump_reader), &cache_loader);
+ ASSERT_OK(s);
+ s = cache_loader->RestoreCacheEntriesToSecondaryCache();
+ ASSERT_OK(s);
+ uint32_t load_insert = secondary_cache->num_inserts() - start_insert;
+ uint32_t load_lookup = secondary_cache->num_lookups() - start_lookup;
+ // check the number we inserted
+ ASSERT_EQ(64, static_cast<int>(load_insert));
+ ASSERT_EQ(0, static_cast<int>(load_lookup));
+ ASSERT_OK(s);
+
+ Reopen(options);
+
+ // After load, we do the Get again
+ start_insert = secondary_cache->num_inserts();
+ start_lookup = secondary_cache->num_lookups();
+ uint32_t cache_insert = tmp_cache->GetInsertCount();
+ uint32_t cache_lookup = tmp_cache->GetLookupcount();
+ for (int i = 0; i < N; i++) {
+ v = Get(Key(i));
+ ASSERT_EQ(v, value[i]);
+ }
+ uint32_t final_insert = secondary_cache->num_inserts() - start_insert;
+ uint32_t final_lookup = secondary_cache->num_lookups() - start_lookup;
+ // no insert to secondary cache
+ ASSERT_EQ(0, static_cast<int>(final_insert));
+ // lookup the secondary to get all blocks
+ ASSERT_EQ(64, static_cast<int>(final_lookup));
+ uint32_t block_insert = tmp_cache->GetInsertCount() - cache_insert;
+ uint32_t block_lookup = tmp_cache->GetLookupcount() - cache_lookup;
+ // Check the new block cache insert and lookup, should be no insert since all
+ // blocks are from the secondary cache.
+ ASSERT_EQ(0, static_cast<int>(block_insert));
+ ASSERT_EQ(256, static_cast<int>(block_lookup));
+
+ fault_fs_->SetFailGetUniqueId(false);
+ 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);
+ LRUCacheWithStat* tmp_cache = new LRUCacheWithStat(
+ cache_opts.capacity, cache_opts.num_shard_bits,
+ cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio,
+ cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator,
+ cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy,
+ cache_opts.secondary_cache);
+ std::shared_ptr<Cache> cache(tmp_cache);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ std::string dbname1 = test::PerThreadDBPath("db_1");
+ ASSERT_OK(DestroyDB(dbname1, options));
+ DB* db1 = nullptr;
+ ASSERT_OK(DB::Open(options, dbname1, &db1));
+ std::string dbname2 = test::PerThreadDBPath("db_2");
+ ASSERT_OK(DestroyDB(dbname2, options));
+ DB* db2 = nullptr;
+ ASSERT_OK(DB::Open(options, dbname2, &db2));
+ fault_fs_->SetFailGetUniqueId(true);
+
+ // write the KVs to db1
+ Random rnd(301);
+ const int N = 256;
+ std::vector<std::string> value1;
+ WriteOptions wo;
+ char buf[1000];
+ memset(buf, 'a', 1000);
+ value1.resize(N);
+ for (int i = 0; i < N; i++) {
+ std::string p_v(buf, 1000);
+ value1[i] = p_v;
+ ASSERT_OK(db1->Put(wo, Key(i), p_v));
+ }
+ ASSERT_OK(db1->Flush(FlushOptions()));
+ Slice bg("a");
+ Slice ed("b");
+ ASSERT_OK(db1->CompactRange(CompactRangeOptions(), &bg, &ed));
+
+ // Write the KVs to DB2
+ std::vector<std::string> value2;
+ memset(buf, 'b', 1000);
+ value2.resize(N);
+ for (int i = 0; i < N; i++) {
+ std::string p_v(buf, 1000);
+ value2[i] = p_v;
+ ASSERT_OK(db2->Put(wo, Key(i), p_v));
+ }
+ ASSERT_OK(db2->Flush(FlushOptions()));
+ ASSERT_OK(db2->CompactRange(CompactRangeOptions(), &bg, &ed));
+
+ // do th eread for all the key value pairs, so all the blocks should be in
+ // cache
+ uint32_t start_insert = tmp_cache->GetInsertCount();
+ uint32_t start_lookup = tmp_cache->GetLookupcount();
+ ReadOptions ro;
+ std::string v;
+ for (int i = 0; i < N; i++) {
+ ASSERT_OK(db1->Get(ro, Key(i), &v));
+ ASSERT_EQ(v, value1[i]);
+ }
+ for (int i = 0; i < N; i++) {
+ ASSERT_OK(db2->Get(ro, Key(i), &v));
+ ASSERT_EQ(v, value2[i]);
+ }
+ uint32_t dump_insert = tmp_cache->GetInsertCount() - start_insert;
+ uint32_t dump_lookup = tmp_cache->GetLookupcount() - start_lookup;
+ ASSERT_EQ(128,
+ static_cast<int>(dump_insert)); // the insert in the block cache
+ ASSERT_EQ(512,
+ static_cast<int>(dump_lookup)); // the lookup in the block cache
+ // We have enough blocks in the block cache
+
+ CacheDumpOptions cd_options;
+ cd_options.clock = fault_env_->GetSystemClock().get();
+ std::string dump_path = db1->GetName() + "/cache_dump";
+ std::unique_ptr<CacheDumpWriter> dump_writer;
+ Status s = NewToFileCacheDumpWriter(fault_fs_, FileOptions(), dump_path,
+ &dump_writer);
+ ASSERT_OK(s);
+ std::unique_ptr<CacheDumper> cache_dumper;
+ s = NewDefaultCacheDumper(cd_options, cache, std::move(dump_writer),
+ &cache_dumper);
+ ASSERT_OK(s);
+ std::vector<DB*> db_list;
+ db_list.push_back(db1);
+ s = cache_dumper->SetDumpFilter(db_list);
+ ASSERT_OK(s);
+ s = cache_dumper->DumpCacheEntriesToWriter();
+ ASSERT_OK(s);
+ cache_dumper.reset();
+
+ // we have a new cache it is empty, then, before we do the Get, we do the
+ // dumpload
+ std::shared_ptr<TestSecondaryCache> secondary_cache =
+ std::make_shared<TestSecondaryCache>(2048 * 1024);
+ cache_opts.secondary_cache = secondary_cache;
+ tmp_cache = new LRUCacheWithStat(
+ cache_opts.capacity, cache_opts.num_shard_bits,
+ cache_opts.strict_capacity_limit, cache_opts.high_pri_pool_ratio,
+ cache_opts.low_pri_pool_ratio, cache_opts.memory_allocator,
+ cache_opts.use_adaptive_mutex, cache_opts.metadata_charge_policy,
+ cache_opts.secondary_cache);
+ std::shared_ptr<Cache> cache_new(tmp_cache);
+ table_options.block_cache = cache_new;
+ table_options.block_size = 4 * 1024;
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+
+ // Start the cache loading process
+ start_insert = secondary_cache->num_inserts();
+ start_lookup = secondary_cache->num_lookups();
+ std::unique_ptr<CacheDumpReader> dump_reader;
+ s = NewFromFileCacheDumpReader(fault_fs_, FileOptions(), dump_path,
+ &dump_reader);
+ ASSERT_OK(s);
+ std::unique_ptr<CacheDumpedLoader> cache_loader;
+ s = NewDefaultCacheDumpedLoader(cd_options, table_options, secondary_cache,
+ std::move(dump_reader), &cache_loader);
+ ASSERT_OK(s);
+ s = cache_loader->RestoreCacheEntriesToSecondaryCache();
+ ASSERT_OK(s);
+ uint32_t load_insert = secondary_cache->num_inserts() - start_insert;
+ uint32_t load_lookup = secondary_cache->num_lookups() - start_lookup;
+ // check the number we inserted
+ ASSERT_EQ(64, static_cast<int>(load_insert));
+ ASSERT_EQ(0, static_cast<int>(load_lookup));
+ ASSERT_OK(s);
+
+ ASSERT_OK(db1->Close());
+ delete db1;
+ ASSERT_OK(DB::Open(options, dbname1, &db1));
+
+ // After load, we do the Get again. To validate the cache, we do not allow any
+ // I/O, so we set the file system to false.
+ IOStatus error_msg = IOStatus::IOError("Retryable IO Error");
+ fault_fs_->SetFilesystemActive(false, error_msg);
+ start_insert = secondary_cache->num_inserts();
+ start_lookup = secondary_cache->num_lookups();
+ uint32_t cache_insert = tmp_cache->GetInsertCount();
+ uint32_t cache_lookup = tmp_cache->GetLookupcount();
+ for (int i = 0; i < N; i++) {
+ ASSERT_OK(db1->Get(ro, Key(i), &v));
+ ASSERT_EQ(v, value1[i]);
+ }
+ uint32_t final_insert = secondary_cache->num_inserts() - start_insert;
+ uint32_t final_lookup = secondary_cache->num_lookups() - start_lookup;
+ // no insert to secondary cache
+ ASSERT_EQ(0, static_cast<int>(final_insert));
+ // lookup the secondary to get all blocks
+ ASSERT_EQ(64, static_cast<int>(final_lookup));
+ uint32_t block_insert = tmp_cache->GetInsertCount() - cache_insert;
+ uint32_t block_lookup = tmp_cache->GetLookupcount() - cache_lookup;
+ // Check the new block cache insert and lookup, should be no insert since all
+ // blocks are from the secondary cache.
+ ASSERT_EQ(0, static_cast<int>(block_insert));
+ ASSERT_EQ(256, static_cast<int>(block_lookup));
+ fault_fs_->SetFailGetUniqueId(false);
+ fault_fs_->SetFilesystemActive(true);
+ delete db1;
+ delete db2;
+ ASSERT_OK(DestroyDB(dbname1, options));
+ ASSERT_OK(DestroyDB(dbname2, options));
+}
+
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+ options.lowest_used_cache_tier = CacheTier::kVolatileTier;
+
+ // Set the file paranoid check, so after flush, the file will be read
+ // all the blocks will be accessed.
+ options.paranoid_file_checks = true;
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i + 70), p_v));
+ }
+
+ ASSERT_OK(Flush());
+
+ // Flush will trigger the paranoid check and read blocks. But only block cache
+ // will be read. No operations for secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ Compact("a", "z");
+
+ // Compaction will also insert and evict blocks, no operations to the block
+ // cache. No operations for secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the data in first block. Cache miss, direclty read from SST file.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the second block.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+
+ // block cache hit
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(70));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the first block in the second SST file. Cache miss and trigger SST
+ // file read. No operations for secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(75));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the second block in the second SST file. Cache miss and trigger SST
+ // file read. No operations for secondary cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ Destroy(options);
+}
+
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ fault_fs_->SetFailGetUniqueId(true);
+ options.lowest_used_cache_tier = CacheTier::kVolatileTier;
+
+ // Set the file paranoid check, so after flush, the file will be read
+ // all the blocks will be accessed.
+ options.paranoid_file_checks = true;
+ DestroyAndReopen(options);
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i), p_v));
+ }
+
+ ASSERT_OK(Flush());
+
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(Put(Key(i + 70), p_v));
+ }
+
+ ASSERT_OK(Flush());
+
+ // Flush will trigger the paranoid check and read blocks. But only block cache
+ // will be read.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ Compact("a", "z");
+
+ // Compaction will also insert and evict blocks, no operations to the block
+ // cache.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ std::string v = Get(Key(0));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the data in first block. Cache miss, direclty read from SST file.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+
+ // Check the second block.
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ v = Get(Key(5));
+ ASSERT_EQ(1007, v.size());
+
+ // block cache hit
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+
+ // Change the option to enable secondary cache after we Reopen the DB
+ options.lowest_used_cache_tier = CacheTier::kNonVolatileBlockTier;
+ Reopen(options);
+
+ v = Get(Key(70));
+ ASSERT_EQ(1007, v.size());
+
+ // Enable the secondary cache, trigger lookup of the first block in second SST
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 1u);
+
+ v = Get(Key(75));
+ ASSERT_EQ(1007, v.size());
+
+ // trigger lookup of the second block in second SST
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+ Destroy(options);
+}
+
+// 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);
+ std::shared_ptr<TestSecondaryCache> secondary_cache(
+ new TestSecondaryCache(2048 * 1024));
+ opts.secondary_cache = secondary_cache;
+ std::shared_ptr<Cache> cache = NewLRUCache(opts);
+ BlockBasedTableOptions table_options;
+ table_options.block_cache = cache;
+ table_options.block_size = 4 * 1024;
+ Options options = GetDefaultOptions();
+ options.create_if_missing = true;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.env = fault_env_.get();
+ options.paranoid_file_checks = true;
+ std::string dbname1 = test::PerThreadDBPath("db_t_1");
+ ASSERT_OK(DestroyDB(dbname1, options));
+ DB* db1 = nullptr;
+ ASSERT_OK(DB::Open(options, dbname1, &db1));
+ std::string dbname2 = test::PerThreadDBPath("db_t_2");
+ ASSERT_OK(DestroyDB(dbname2, options));
+ DB* db2 = nullptr;
+ Options options2 = options;
+ options2.lowest_used_cache_tier = CacheTier::kVolatileTier;
+ ASSERT_OK(DB::Open(options2, dbname2, &db2));
+ fault_fs_->SetFailGetUniqueId(true);
+
+ WriteOptions wo;
+ Random rnd(301);
+ const int N = 6;
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(db1->Put(wo, Key(i), p_v));
+ }
+
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 0u);
+ ASSERT_OK(db1->Flush(FlushOptions()));
+
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ for (int i = 0; i < N; i++) {
+ std::string p_v = rnd.RandomString(1007);
+ ASSERT_OK(db2->Put(wo, Key(i), p_v));
+ }
+
+ // No change in the secondary cache, since it is disabled in DB2
+ ASSERT_EQ(secondary_cache->num_inserts(), 0u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+ ASSERT_OK(db2->Flush(FlushOptions()));
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ Slice bg("a");
+ Slice ed("b");
+ ASSERT_OK(db1->CompactRange(CompactRangeOptions(), &bg, &ed));
+ ASSERT_OK(db2->CompactRange(CompactRangeOptions(), &bg, &ed));
+
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 2u);
+
+ ReadOptions ro;
+ std::string v;
+ ASSERT_OK(db1->Get(ro, Key(0), &v));
+ ASSERT_EQ(1007, v.size());
+
+ // DB 1 has lookup block 1 and it is miss in block cache, trigger secondary
+ // cache lookup
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 3u);
+
+ ASSERT_OK(db1->Get(ro, Key(5), &v));
+ ASSERT_EQ(1007, v.size());
+
+ // DB 1 lookup the second block and it is miss in block cache, trigger
+ // secondary cache lookup
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ ASSERT_OK(db2->Get(ro, Key(0), &v));
+ ASSERT_EQ(1007, v.size());
+
+ // For db2, it is not enabled with secondary cache, so no search in the
+ // secondary cache
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ ASSERT_OK(db2->Get(ro, Key(5), &v));
+ ASSERT_EQ(1007, v.size());
+
+ // For db2, it is not enabled with secondary cache, so no search in the
+ // secondary cache
+ ASSERT_EQ(secondary_cache->num_inserts(), 1u);
+ ASSERT_EQ(secondary_cache->num_lookups(), 4u);
+
+ fault_fs_->SetFailGetUniqueId(false);
+ fault_fs_->SetFilesystemActive(true);
+ delete db1;
+ delete db2;
+ ASSERT_OK(DestroyDB(dbname1, options));
+ ASSERT_OK(DestroyDB(dbname2, options));
+}
+
+#endif // ROCKSDB_LITE
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/cache/secondary_cache.cc b/src/rocksdb/cache/secondary_cache.cc
new file mode 100644
index 000000000..84352db71
--- /dev/null
+++ b/src/rocksdb/cache/secondary_cache.cc
@@ -0,0 +1,32 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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 "rocksdb/secondary_cache.h"
+
+#include "cache/cache_entry_roles.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+size_t SliceSize(void* obj) { return static_cast<Slice*>(obj)->size(); }
+
+Status SliceSaveTo(void* from_obj, size_t from_offset, size_t length,
+ void* out) {
+ const Slice& slice = *static_cast<Slice*>(from_obj);
+ std::memcpy(out, slice.data() + from_offset, length);
+ return Status::OK();
+}
+
+} // namespace
+
+Status SecondaryCache::InsertSaved(const Slice& key, const Slice& saved) {
+ static Cache::CacheItemHelper helper{
+ &SliceSize, &SliceSaveTo, GetNoopDeleterForRole<CacheEntryRole::kMisc>()};
+ // NOTE: depends on Insert() being synchronous, not keeping pointer `&saved`
+ return Insert(key, const_cast<Slice*>(&saved), &helper);
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/sharded_cache.cc b/src/rocksdb/cache/sharded_cache.cc
new file mode 100644
index 000000000..9ebca3ba8
--- /dev/null
+++ b/src/rocksdb/cache/sharded_cache.cc
@@ -0,0 +1,100 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "cache/sharded_cache.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <memory>
+
+#include "util/hash.h"
+#include "util/math.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+ShardedCacheBase::ShardedCacheBase(size_t capacity, int num_shard_bits,
+ bool strict_capacity_limit,
+ std::shared_ptr<MemoryAllocator> allocator)
+ : Cache(std::move(allocator)),
+ last_id_(1),
+ shard_mask_((uint32_t{1} << num_shard_bits) - 1),
+ strict_capacity_limit_(strict_capacity_limit),
+ capacity_(capacity) {}
+
+size_t ShardedCacheBase::ComputePerShardCapacity(size_t capacity) const {
+ uint32_t num_shards = GetNumShards();
+ return (capacity + (num_shards - 1)) / num_shards;
+}
+
+size_t ShardedCacheBase::GetPerShardCapacity() const {
+ return ComputePerShardCapacity(GetCapacity());
+}
+
+uint64_t ShardedCacheBase::NewId() {
+ return last_id_.fetch_add(1, std::memory_order_relaxed);
+}
+
+size_t ShardedCacheBase::GetCapacity() const {
+ MutexLock l(&config_mutex_);
+ return capacity_;
+}
+
+bool ShardedCacheBase::HasStrictCapacityLimit() const {
+ MutexLock l(&config_mutex_);
+ return strict_capacity_limit_;
+}
+
+size_t ShardedCacheBase::GetUsage(Handle* handle) const {
+ return GetCharge(handle);
+}
+
+std::string ShardedCacheBase::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(20000);
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+ {
+ MutexLock l(&config_mutex_);
+ snprintf(buffer, kBufferSize, " capacity : %" ROCKSDB_PRIszt "\n",
+ capacity_);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " num_shard_bits : %d\n",
+ GetNumShardBits());
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " strict_capacity_limit : %d\n",
+ strict_capacity_limit_);
+ ret.append(buffer);
+ }
+ snprintf(buffer, kBufferSize, " memory_allocator : %s\n",
+ memory_allocator() ? memory_allocator()->Name() : "None");
+ ret.append(buffer);
+ AppendPrintableOptions(ret);
+ return ret;
+}
+
+int GetDefaultCacheShardBits(size_t capacity, size_t min_shard_size) {
+ int num_shard_bits = 0;
+ size_t num_shards = capacity / min_shard_size;
+ while (num_shards >>= 1) {
+ if (++num_shard_bits >= 6) {
+ // No more than 6.
+ return num_shard_bits;
+ }
+ }
+ return num_shard_bits;
+}
+
+int ShardedCacheBase::GetNumShardBits() const {
+ return BitsSetToOne(shard_mask_);
+}
+
+uint32_t ShardedCacheBase::GetNumShards() const { return shard_mask_ + 1; }
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/cache/sharded_cache.h b/src/rocksdb/cache/sharded_cache.h
new file mode 100644
index 000000000..e3271cc7b
--- /dev/null
+++ b/src/rocksdb/cache/sharded_cache.h
@@ -0,0 +1,322 @@
+// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
+// 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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <atomic>
+#include <cstdint>
+#include <string>
+
+#include "port/lang.h"
+#include "port/port.h"
+#include "rocksdb/cache.h"
+#include "util/hash.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Optional base class for classes implementing the CacheShard concept
+class CacheShardBase {
+ public:
+ explicit CacheShardBase(CacheMetadataChargePolicy metadata_charge_policy)
+ : metadata_charge_policy_(metadata_charge_policy) {}
+
+ using DeleterFn = Cache::DeleterFn;
+
+ // Expected by concept CacheShard (TODO with C++20 support)
+ // Some Defaults
+ std::string GetPrintableOptions() const { return ""; }
+ using HashVal = uint64_t;
+ using HashCref = uint64_t;
+ static inline HashVal ComputeHash(const Slice& key) {
+ return GetSliceNPHash64(key);
+ }
+ static inline uint32_t HashPieceForSharding(HashCref hash) {
+ return Lower32of64(hash);
+ }
+ void AppendPrintableOptions(std::string& /*str*/) const {}
+
+ // Must be provided for concept CacheShard (TODO with C++20 support)
+ /*
+ struct HandleImpl { // for concept HandleImpl
+ HashVal hash;
+ HashCref GetHash() const;
+ ...
+ };
+ Status Insert(const Slice& key, HashCref hash, void* value, size_t charge,
+ DeleterFn deleter, HandleImpl** handle,
+ Cache::Priority priority) = 0;
+ Status Insert(const Slice& key, HashCref hash, void* value,
+ const Cache::CacheItemHelper* helper, size_t charge,
+ HandleImpl** handle, Cache::Priority priority) = 0;
+ HandleImpl* Lookup(const Slice& key, HashCref hash) = 0;
+ HandleImpl* Lookup(const Slice& key, HashCref hash,
+ const Cache::CacheItemHelper* helper,
+ const Cache::CreateCallback& create_cb,
+ Cache::Priority priority, bool wait,
+ 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;
+ void SetStrictCapacityLimit(bool strict_capacity_limit) = 0;
+ size_t GetUsage() const = 0;
+ size_t GetPinnedUsage() const = 0;
+ size_t GetOccupancyCount() const = 0;
+ size_t GetTableAddressCount() const = 0;
+ // Handles iterating over roughly `average_entries_per_lock` entries, using
+ // `state` to somehow record where it last ended up. Caller initially uses
+ // *state == 0 and implementation sets *state = SIZE_MAX to indicate
+ // completion.
+ void ApplyToSomeEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ size_t average_entries_per_lock, size_t* state) = 0;
+ void EraseUnRefEntries() = 0;
+ */
+
+ protected:
+ const CacheMetadataChargePolicy metadata_charge_policy_;
+};
+
+// Portions of ShardedCache that do not depend on the template parameter
+class ShardedCacheBase : public Cache {
+ public:
+ ShardedCacheBase(size_t capacity, int num_shard_bits,
+ bool strict_capacity_limit,
+ std::shared_ptr<MemoryAllocator> memory_allocator);
+ virtual ~ShardedCacheBase() = default;
+
+ int GetNumShardBits() const;
+ uint32_t GetNumShards() const;
+
+ uint64_t NewId() override;
+
+ bool HasStrictCapacityLimit() const override;
+ size_t GetCapacity() const override;
+
+ using Cache::GetUsage;
+ size_t GetUsage(Handle* handle) const override;
+ std::string GetPrintableOptions() const override;
+
+ protected: // fns
+ virtual void AppendPrintableOptions(std::string& str) const = 0;
+ size_t GetPerShardCapacity() const;
+ size_t ComputePerShardCapacity(size_t capacity) const;
+
+ protected: // data
+ std::atomic<uint64_t> last_id_; // For NewId
+ const uint32_t shard_mask_;
+
+ // Dynamic configuration parameters, guarded by config_mutex_
+ bool strict_capacity_limit_;
+ size_t capacity_;
+ mutable port::Mutex config_mutex_;
+};
+
+// Generic cache interface that shards cache by hash of keys. 2^num_shard_bits
+// shards will be created, with capacity split evenly to each of the shards.
+// Keys are typically sharded by the lowest num_shard_bits bits of hash value
+// so that the upper bits of the hash value can keep a stable ordering of
+// table entries even as the table grows (using more upper hash bits).
+// See CacheShardBase above for what is expected of the CacheShard parameter.
+template <class CacheShard>
+class ShardedCache : public ShardedCacheBase {
+ public:
+ using HashVal = typename CacheShard::HashVal;
+ using HashCref = typename CacheShard::HashCref;
+ using HandleImpl = typename CacheShard::HandleImpl;
+
+ ShardedCache(size_t capacity, int num_shard_bits, bool strict_capacity_limit,
+ std::shared_ptr<MemoryAllocator> allocator)
+ : ShardedCacheBase(capacity, num_shard_bits, strict_capacity_limit,
+ allocator),
+ shards_(reinterpret_cast<CacheShard*>(port::cacheline_aligned_alloc(
+ sizeof(CacheShard) * GetNumShards()))),
+ destroy_shards_in_dtor_(false) {}
+
+ virtual ~ShardedCache() {
+ if (destroy_shards_in_dtor_) {
+ ForEachShard([](CacheShard* cs) { cs->~CacheShard(); });
+ }
+ port::cacheline_aligned_free(shards_);
+ }
+
+ CacheShard& GetShard(HashCref hash) {
+ return shards_[CacheShard::HashPieceForSharding(hash) & shard_mask_];
+ }
+
+ const CacheShard& GetShard(HashCref hash) const {
+ return shards_[CacheShard::HashPieceForSharding(hash) & shard_mask_];
+ }
+
+ void SetCapacity(size_t capacity) override {
+ MutexLock l(&config_mutex_);
+ capacity_ = capacity;
+ auto per_shard = ComputePerShardCapacity(capacity);
+ ForEachShard([=](CacheShard* cs) { cs->SetCapacity(per_shard); });
+ }
+
+ void SetStrictCapacityLimit(bool s_c_l) override {
+ MutexLock l(&config_mutex_);
+ strict_capacity_limit_ = s_c_l;
+ ForEachShard(
+ [s_c_l](CacheShard* cs) { cs->SetStrictCapacityLimit(s_c_l); });
+ }
+
+ Status Insert(const Slice& key, void* value, size_t charge, DeleterFn deleter,
+ Handle** handle, Priority priority) override {
+ HashVal hash = CacheShard::ComputeHash(key);
+ auto h_out = reinterpret_cast<HandleImpl**>(handle);
+ return GetShard(hash).Insert(key, hash, value, charge, deleter, h_out,
+ priority);
+ }
+ Status Insert(const Slice& key, void* value, const CacheItemHelper* helper,
+ size_t charge, Handle** handle = nullptr,
+ Priority priority = Priority::LOW) override {
+ if (!helper) {
+ return Status::InvalidArgument();
+ }
+ HashVal hash = CacheShard::ComputeHash(key);
+ auto h_out = reinterpret_cast<HandleImpl**>(handle);
+ return GetShard(hash).Insert(key, hash, value, helper, charge, h_out,
+ priority);
+ }
+
+ Handle* Lookup(const Slice& key, Statistics* /*stats*/) override {
+ HashVal hash = CacheShard::ComputeHash(key);
+ HandleImpl* result = GetShard(hash).Lookup(key, hash);
+ return reinterpret_cast<Handle*>(result);
+ }
+ Handle* Lookup(const Slice& key, const CacheItemHelper* helper,
+ const CreateCallback& create_cb, Priority priority, bool wait,
+ Statistics* stats = nullptr) override {
+ HashVal hash = CacheShard::ComputeHash(key);
+ HandleImpl* result = GetShard(hash).Lookup(key, hash, helper, create_cb,
+ priority, wait, stats);
+ return reinterpret_cast<Handle*>(result);
+ }
+
+ void Erase(const Slice& key) override {
+ HashVal hash = CacheShard::ComputeHash(key);
+ GetShard(hash).Erase(key, hash);
+ }
+
+ bool Release(Handle* handle, bool useful,
+ bool erase_if_last_ref = false) override {
+ auto h = reinterpret_cast<HandleImpl*>(handle);
+ return GetShard(h->GetHash()).Release(h, useful, erase_if_last_ref);
+ }
+ bool IsReady(Handle* handle) override {
+ auto h = reinterpret_cast<HandleImpl*>(handle);
+ return GetShard(h->GetHash()).IsReady(h);
+ }
+ void Wait(Handle* handle) override {
+ auto h = reinterpret_cast<HandleImpl*>(handle);
+ GetShard(h->GetHash()).Wait(h);
+ }
+ bool Ref(Handle* handle) override {
+ auto h = reinterpret_cast<HandleImpl*>(handle);
+ return GetShard(h->GetHash()).Ref(h);
+ }
+ bool Release(Handle* handle, bool erase_if_last_ref = false) override {
+ return Release(handle, true /*useful*/, erase_if_last_ref);
+ }
+ using ShardedCacheBase::GetUsage;
+ size_t GetUsage() const override {
+ return SumOverShards2(&CacheShard::GetUsage);
+ }
+ size_t GetPinnedUsage() const override {
+ return SumOverShards2(&CacheShard::GetPinnedUsage);
+ }
+ size_t GetOccupancyCount() const override {
+ return SumOverShards2(&CacheShard::GetPinnedUsage);
+ }
+ size_t GetTableAddressCount() const override {
+ return SumOverShards2(&CacheShard::GetTableAddressCount);
+ }
+ void ApplyToAllEntries(
+ const std::function<void(const Slice& key, void* value, size_t charge,
+ DeleterFn deleter)>& callback,
+ const ApplyToAllEntriesOptions& opts) override {
+ uint32_t num_shards = GetNumShards();
+ // Iterate over part of each shard, rotating between shards, to
+ // minimize impact on latency of concurrent operations.
+ std::unique_ptr<size_t[]> states(new size_t[num_shards]{});
+
+ size_t aepl = opts.average_entries_per_lock;
+ aepl = std::min(aepl, size_t{1});
+
+ bool remaining_work;
+ do {
+ remaining_work = false;
+ for (uint32_t i = 0; i < num_shards; i++) {
+ if (states[i] != SIZE_MAX) {
+ shards_[i].ApplyToSomeEntries(callback, aepl, &states[i]);
+ remaining_work |= states[i] != SIZE_MAX;
+ }
+ }
+ } while (remaining_work);
+ }
+
+ virtual void EraseUnRefEntries() override {
+ ForEachShard([](CacheShard* cs) { cs->EraseUnRefEntries(); });
+ }
+
+ void DisownData() override {
+ // Leak data only if that won't generate an ASAN/valgrind warning.
+ if (!kMustFreeHeapAllocations) {
+ destroy_shards_in_dtor_ = false;
+ }
+ }
+
+ protected:
+ inline void ForEachShard(const std::function<void(CacheShard*)>& fn) {
+ uint32_t num_shards = GetNumShards();
+ for (uint32_t i = 0; i < num_shards; i++) {
+ fn(shards_ + i);
+ }
+ }
+
+ inline size_t SumOverShards(
+ const std::function<size_t(CacheShard&)>& fn) const {
+ uint32_t num_shards = GetNumShards();
+ size_t result = 0;
+ for (uint32_t i = 0; i < num_shards; i++) {
+ result += fn(shards_[i]);
+ }
+ return result;
+ }
+
+ inline size_t SumOverShards2(size_t (CacheShard::*fn)() const) const {
+ return SumOverShards([fn](CacheShard& cs) { return (cs.*fn)(); });
+ }
+
+ // Must be called exactly once by derived class constructor
+ void InitShards(const std::function<void(CacheShard*)>& placement_new) {
+ ForEachShard(placement_new);
+ destroy_shards_in_dtor_ = true;
+ }
+
+ void AppendPrintableOptions(std::string& str) const override {
+ shards_[0].AppendPrintableOptions(str);
+ }
+
+ private:
+ CacheShard* const shards_;
+ bool destroy_shards_in_dtor_;
+};
+
+// 512KB is traditional minimum shard size.
+int GetDefaultCacheShardBits(size_t capacity,
+ size_t min_shard_size = 512U * 1024U);
+
+} // namespace ROCKSDB_NAMESPACE