summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/utilities/simulator_cache
diff options
context:
space:
mode:
Diffstat (limited to '')
-rw-r--r--src/rocksdb/utilities/simulator_cache/cache_simulator.cc274
-rw-r--r--src/rocksdb/utilities/simulator_cache/cache_simulator.h231
-rw-r--r--src/rocksdb/utilities/simulator_cache/cache_simulator_test.cc494
-rw-r--r--src/rocksdb/utilities/simulator_cache/sim_cache.cc354
-rw-r--r--src/rocksdb/utilities/simulator_cache/sim_cache_test.cc225
5 files changed, 1578 insertions, 0 deletions
diff --git a/src/rocksdb/utilities/simulator_cache/cache_simulator.cc b/src/rocksdb/utilities/simulator_cache/cache_simulator.cc
new file mode 100644
index 000000000..16a78ea71
--- /dev/null
+++ b/src/rocksdb/utilities/simulator_cache/cache_simulator.cc
@@ -0,0 +1,274 @@
+// 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 "utilities/simulator_cache/cache_simulator.h"
+#include <algorithm>
+#include "db/dbformat.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+const std::string kGhostCachePrefix = "ghost_";
+} // namespace
+
+GhostCache::GhostCache(std::shared_ptr<Cache> sim_cache)
+ : sim_cache_(sim_cache) {}
+
+bool GhostCache::Admit(const Slice& lookup_key) {
+ auto handle = sim_cache_->Lookup(lookup_key);
+ if (handle != nullptr) {
+ sim_cache_->Release(handle);
+ return true;
+ }
+ sim_cache_->Insert(lookup_key, /*value=*/nullptr, lookup_key.size(),
+ /*deleter=*/nullptr);
+ return false;
+}
+
+CacheSimulator::CacheSimulator(std::unique_ptr<GhostCache>&& ghost_cache,
+ std::shared_ptr<Cache> sim_cache)
+ : ghost_cache_(std::move(ghost_cache)), sim_cache_(sim_cache) {}
+
+void CacheSimulator::Access(const BlockCacheTraceRecord& access) {
+ bool admit = true;
+ const bool is_user_access =
+ BlockCacheTraceHelper::IsUserAccess(access.caller);
+ bool is_cache_miss = true;
+ if (ghost_cache_ && access.no_insert == Boolean::kFalse) {
+ admit = ghost_cache_->Admit(access.block_key);
+ }
+ auto handle = sim_cache_->Lookup(access.block_key);
+ if (handle != nullptr) {
+ sim_cache_->Release(handle);
+ is_cache_miss = false;
+ } else {
+ if (access.no_insert == Boolean::kFalse && admit && access.block_size > 0) {
+ sim_cache_->Insert(access.block_key, /*value=*/nullptr, access.block_size,
+ /*deleter=*/nullptr);
+ }
+ }
+ miss_ratio_stats_.UpdateMetrics(access.access_timestamp, is_user_access,
+ is_cache_miss);
+}
+
+void MissRatioStats::UpdateMetrics(uint64_t timestamp_in_ms,
+ bool is_user_access, bool is_cache_miss) {
+ uint64_t timestamp_in_seconds = timestamp_in_ms / kMicrosInSecond;
+ num_accesses_timeline_[timestamp_in_seconds] += 1;
+ num_accesses_ += 1;
+ if (num_misses_timeline_.find(timestamp_in_seconds) ==
+ num_misses_timeline_.end()) {
+ num_misses_timeline_[timestamp_in_seconds] = 0;
+ }
+ if (is_cache_miss) {
+ num_misses_ += 1;
+ num_misses_timeline_[timestamp_in_seconds] += 1;
+ }
+ if (is_user_access) {
+ user_accesses_ += 1;
+ if (is_cache_miss) {
+ user_misses_ += 1;
+ }
+ }
+}
+
+Cache::Priority PrioritizedCacheSimulator::ComputeBlockPriority(
+ const BlockCacheTraceRecord& access) const {
+ if (access.block_type == TraceType::kBlockTraceFilterBlock ||
+ access.block_type == TraceType::kBlockTraceIndexBlock ||
+ access.block_type == TraceType::kBlockTraceUncompressionDictBlock) {
+ return Cache::Priority::HIGH;
+ }
+ return Cache::Priority::LOW;
+}
+
+void PrioritizedCacheSimulator::AccessKVPair(
+ const Slice& key, uint64_t value_size, Cache::Priority priority,
+ const BlockCacheTraceRecord& access, bool no_insert, bool is_user_access,
+ bool* is_cache_miss, bool* admitted, bool update_metrics) {
+ assert(is_cache_miss);
+ assert(admitted);
+ *is_cache_miss = true;
+ *admitted = true;
+ if (ghost_cache_ && !no_insert) {
+ *admitted = ghost_cache_->Admit(key);
+ }
+ auto handle = sim_cache_->Lookup(key);
+ if (handle != nullptr) {
+ sim_cache_->Release(handle);
+ *is_cache_miss = false;
+ } else if (!no_insert && *admitted && value_size > 0) {
+ sim_cache_->Insert(key, /*value=*/nullptr, value_size, /*deleter=*/nullptr,
+ /*handle=*/nullptr, priority);
+ }
+ if (update_metrics) {
+ miss_ratio_stats_.UpdateMetrics(access.access_timestamp, is_user_access,
+ *is_cache_miss);
+ }
+}
+
+void PrioritizedCacheSimulator::Access(const BlockCacheTraceRecord& access) {
+ bool is_cache_miss = true;
+ bool admitted = true;
+ AccessKVPair(access.block_key, access.block_size,
+ ComputeBlockPriority(access), access, access.no_insert,
+ BlockCacheTraceHelper::IsUserAccess(access.caller),
+ &is_cache_miss, &admitted, /*update_metrics=*/true);
+}
+
+void HybridRowBlockCacheSimulator::Access(const BlockCacheTraceRecord& access) {
+ // TODO (haoyu): We only support Get for now. We need to extend the tracing
+ // for MultiGet, i.e., non-data block accesses must log all keys in a
+ // MultiGet.
+ bool is_cache_miss = true;
+ bool admitted = false;
+ if (access.caller == TableReaderCaller::kUserGet &&
+ access.get_id != BlockCacheTraceHelper::kReservedGetId) {
+ // This is a Get request.
+ const std::string& row_key = BlockCacheTraceHelper::ComputeRowKey(access);
+ GetRequestStatus& status = getid_status_map_[access.get_id];
+ if (status.is_complete) {
+ // This Get request completes.
+ // Skip future accesses to its index/filter/data
+ // blocks. These block lookups are unnecessary if we observe a hit for the
+ // referenced key-value pair already. Thus, we treat these lookups as
+ // hits. This is also to ensure the total number of accesses are the same
+ // when comparing to other policies.
+ miss_ratio_stats_.UpdateMetrics(access.access_timestamp,
+ /*is_user_access=*/true,
+ /*is_cache_miss=*/false);
+ return;
+ }
+ if (status.row_key_status.find(row_key) == status.row_key_status.end()) {
+ // This is the first time that this key is accessed. Look up the key-value
+ // pair first. Do not update the miss/accesses metrics here since it will
+ // be updated later.
+ AccessKVPair(row_key, access.referenced_data_size, Cache::Priority::HIGH,
+ access,
+ /*no_insert=*/false,
+ /*is_user_access=*/true, &is_cache_miss, &admitted,
+ /*update_metrics=*/false);
+ InsertResult result = InsertResult::NO_INSERT;
+ if (admitted && access.referenced_data_size > 0) {
+ result = InsertResult::INSERTED;
+ } else if (admitted) {
+ result = InsertResult::ADMITTED;
+ }
+ status.row_key_status[row_key] = result;
+ }
+ if (!is_cache_miss) {
+ // A cache hit.
+ status.is_complete = true;
+ miss_ratio_stats_.UpdateMetrics(access.access_timestamp,
+ /*is_user_access=*/true,
+ /*is_cache_miss=*/false);
+ return;
+ }
+ // The row key-value pair observes a cache miss. We need to access its
+ // index/filter/data blocks.
+ InsertResult inserted = status.row_key_status[row_key];
+ AccessKVPair(
+ access.block_key, access.block_size, ComputeBlockPriority(access),
+ access,
+ /*no_insert=*/!insert_blocks_upon_row_kvpair_miss_ || access.no_insert,
+ /*is_user_access=*/true, &is_cache_miss, &admitted,
+ /*update_metrics=*/true);
+ if (access.referenced_data_size > 0 && inserted == InsertResult::ADMITTED) {
+ sim_cache_->Insert(row_key, /*value=*/nullptr,
+ access.referenced_data_size, /*deleter=*/nullptr,
+ /*handle=*/nullptr, Cache::Priority::HIGH);
+ status.row_key_status[row_key] = InsertResult::INSERTED;
+ }
+ return;
+ }
+ AccessKVPair(access.block_key, access.block_size,
+ ComputeBlockPriority(access), access, access.no_insert,
+ BlockCacheTraceHelper::IsUserAccess(access.caller),
+ &is_cache_miss, &admitted, /*update_metrics=*/true);
+}
+
+BlockCacheTraceSimulator::BlockCacheTraceSimulator(
+ uint64_t warmup_seconds, uint32_t downsample_ratio,
+ const std::vector<CacheConfiguration>& cache_configurations)
+ : warmup_seconds_(warmup_seconds),
+ downsample_ratio_(downsample_ratio),
+ cache_configurations_(cache_configurations) {}
+
+Status BlockCacheTraceSimulator::InitializeCaches() {
+ for (auto const& config : cache_configurations_) {
+ for (auto cache_capacity : config.cache_capacities) {
+ // Scale down the cache capacity since the trace contains accesses on
+ // 1/'downsample_ratio' blocks.
+ uint64_t simulate_cache_capacity = cache_capacity / downsample_ratio_;
+ std::shared_ptr<CacheSimulator> sim_cache;
+ std::unique_ptr<GhostCache> ghost_cache;
+ std::string cache_name = config.cache_name;
+ if (cache_name.find(kGhostCachePrefix) != std::string::npos) {
+ ghost_cache.reset(new GhostCache(
+ NewLRUCache(config.ghost_cache_capacity, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ cache_name = cache_name.substr(kGhostCachePrefix.size());
+ }
+ if (cache_name == "lru") {
+ sim_cache = std::make_shared<CacheSimulator>(
+ std::move(ghost_cache),
+ NewLRUCache(simulate_cache_capacity, config.num_shard_bits,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0));
+ } else if (cache_name == "lru_priority") {
+ sim_cache = std::make_shared<PrioritizedCacheSimulator>(
+ std::move(ghost_cache),
+ NewLRUCache(simulate_cache_capacity, config.num_shard_bits,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0.5));
+ } else if (cache_name == "lru_hybrid") {
+ sim_cache = std::make_shared<HybridRowBlockCacheSimulator>(
+ std::move(ghost_cache),
+ NewLRUCache(simulate_cache_capacity, config.num_shard_bits,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0.5),
+ /*insert_blocks_upon_row_kvpair_miss=*/true);
+ } else if (cache_name == "lru_hybrid_no_insert_on_row_miss") {
+ sim_cache = std::make_shared<HybridRowBlockCacheSimulator>(
+ std::move(ghost_cache),
+ NewLRUCache(simulate_cache_capacity, config.num_shard_bits,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0.5),
+ /*insert_blocks_upon_row_kvpair_miss=*/false);
+ } else {
+ // Not supported.
+ return Status::InvalidArgument("Unknown cache name " +
+ config.cache_name);
+ }
+ sim_caches_[config].push_back(sim_cache);
+ }
+ }
+ return Status::OK();
+}
+
+void BlockCacheTraceSimulator::Access(const BlockCacheTraceRecord& access) {
+ if (trace_start_time_ == 0) {
+ trace_start_time_ = access.access_timestamp;
+ }
+ // access.access_timestamp is in microseconds.
+ if (!warmup_complete_ &&
+ trace_start_time_ + warmup_seconds_ * kMicrosInSecond <=
+ access.access_timestamp) {
+ for (auto& config_caches : sim_caches_) {
+ for (auto& sim_cache : config_caches.second) {
+ sim_cache->reset_counter();
+ }
+ }
+ warmup_complete_ = true;
+ }
+ for (auto& config_caches : sim_caches_) {
+ for (auto& sim_cache : config_caches.second) {
+ sim_cache->Access(access);
+ }
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/simulator_cache/cache_simulator.h b/src/rocksdb/utilities/simulator_cache/cache_simulator.h
new file mode 100644
index 000000000..6d4979013
--- /dev/null
+++ b/src/rocksdb/utilities/simulator_cache/cache_simulator.h
@@ -0,0 +1,231 @@
+// 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 <unordered_map>
+
+#include "cache/lru_cache.h"
+#include "trace_replay/block_cache_tracer.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// A cache configuration provided by user.
+struct CacheConfiguration {
+ std::string cache_name; // LRU.
+ uint32_t num_shard_bits;
+ uint64_t ghost_cache_capacity; // ghost cache capacity in bytes.
+ std::vector<uint64_t>
+ cache_capacities; // simulate cache capacities in bytes.
+
+ bool operator==(const CacheConfiguration& o) const {
+ return cache_name == o.cache_name && num_shard_bits == o.num_shard_bits &&
+ ghost_cache_capacity == o.ghost_cache_capacity;
+ }
+ bool operator<(const CacheConfiguration& o) const {
+ return cache_name < o.cache_name ||
+ (cache_name == o.cache_name && num_shard_bits < o.num_shard_bits) ||
+ (cache_name == o.cache_name && num_shard_bits == o.num_shard_bits &&
+ ghost_cache_capacity < o.ghost_cache_capacity);
+ }
+};
+
+class MissRatioStats {
+ public:
+ void reset_counter() {
+ num_misses_ = 0;
+ num_accesses_ = 0;
+ user_accesses_ = 0;
+ user_misses_ = 0;
+ }
+ double miss_ratio() const {
+ if (num_accesses_ == 0) {
+ return -1;
+ }
+ return static_cast<double>(num_misses_ * 100.0 / num_accesses_);
+ }
+ uint64_t total_accesses() const { return num_accesses_; }
+ uint64_t total_misses() const { return num_misses_; }
+
+ const std::map<uint64_t, uint64_t>& num_accesses_timeline() const {
+ return num_accesses_timeline_;
+ }
+
+ const std::map<uint64_t, uint64_t>& num_misses_timeline() const {
+ return num_misses_timeline_;
+ }
+
+ double user_miss_ratio() const {
+ if (user_accesses_ == 0) {
+ return -1;
+ }
+ return static_cast<double>(user_misses_ * 100.0 / user_accesses_);
+ }
+ uint64_t user_accesses() const { return user_accesses_; }
+ uint64_t user_misses() const { return user_misses_; }
+
+ void UpdateMetrics(uint64_t timestamp_in_ms, bool is_user_access,
+ bool is_cache_miss);
+
+ private:
+ uint64_t num_accesses_ = 0;
+ uint64_t num_misses_ = 0;
+ uint64_t user_accesses_ = 0;
+ uint64_t user_misses_ = 0;
+
+ std::map<uint64_t, uint64_t> num_accesses_timeline_;
+ std::map<uint64_t, uint64_t> num_misses_timeline_;
+};
+
+// A ghost cache admits an entry on its second access.
+class GhostCache {
+ public:
+ explicit GhostCache(std::shared_ptr<Cache> sim_cache);
+ ~GhostCache() = default;
+ // No copy and move.
+ GhostCache(const GhostCache&) = delete;
+ GhostCache& operator=(const GhostCache&) = delete;
+ GhostCache(GhostCache&&) = delete;
+ GhostCache& operator=(GhostCache&&) = delete;
+
+ // Returns true if the lookup_key is in the ghost cache.
+ // Returns false otherwise.
+ bool Admit(const Slice& lookup_key);
+
+ private:
+ std::shared_ptr<Cache> sim_cache_;
+};
+
+// A cache simulator that runs against a block cache trace.
+class CacheSimulator {
+ public:
+ CacheSimulator(std::unique_ptr<GhostCache>&& ghost_cache,
+ std::shared_ptr<Cache> sim_cache);
+ virtual ~CacheSimulator() = default;
+ // No copy and move.
+ CacheSimulator(const CacheSimulator&) = delete;
+ CacheSimulator& operator=(const CacheSimulator&) = delete;
+ CacheSimulator(CacheSimulator&&) = delete;
+ CacheSimulator& operator=(CacheSimulator&&) = delete;
+
+ virtual void Access(const BlockCacheTraceRecord& access);
+
+ void reset_counter() { miss_ratio_stats_.reset_counter(); }
+
+ const MissRatioStats& miss_ratio_stats() const { return miss_ratio_stats_; }
+
+ protected:
+ MissRatioStats miss_ratio_stats_;
+ std::unique_ptr<GhostCache> ghost_cache_;
+ std::shared_ptr<Cache> sim_cache_;
+};
+
+// A prioritized cache simulator that runs against a block cache trace.
+// It inserts missing index/filter/uncompression-dictionary blocks with high
+// priority in the cache.
+class PrioritizedCacheSimulator : public CacheSimulator {
+ public:
+ PrioritizedCacheSimulator(std::unique_ptr<GhostCache>&& ghost_cache,
+ std::shared_ptr<Cache> sim_cache)
+ : CacheSimulator(std::move(ghost_cache), sim_cache) {}
+ void Access(const BlockCacheTraceRecord& access) override;
+
+ protected:
+ // Access the key-value pair and returns true upon a cache miss.
+ void AccessKVPair(const Slice& key, uint64_t value_size,
+ Cache::Priority priority,
+ const BlockCacheTraceRecord& access, bool no_insert,
+ bool is_user_access, bool* is_cache_miss, bool* admitted,
+ bool update_metrics);
+
+ Cache::Priority ComputeBlockPriority(
+ const BlockCacheTraceRecord& access) const;
+};
+
+// A hybrid row and block cache simulator. It looks up/inserts key-value pairs
+// referenced by Get/MultiGet requests, and not their accessed index/filter/data
+// blocks.
+//
+// Upon a Get/MultiGet request, it looks up the referenced key first.
+// If it observes a cache hit, future block accesses on this key-value pair is
+// skipped since the request is served already. Otherwise, it continues to look
+// up/insert its index/filter/data blocks. It also inserts the referenced
+// key-value pair in the cache for future lookups.
+class HybridRowBlockCacheSimulator : public PrioritizedCacheSimulator {
+ public:
+ HybridRowBlockCacheSimulator(std::unique_ptr<GhostCache>&& ghost_cache,
+ std::shared_ptr<Cache> sim_cache,
+ bool insert_blocks_upon_row_kvpair_miss)
+ : PrioritizedCacheSimulator(std::move(ghost_cache), sim_cache),
+ insert_blocks_upon_row_kvpair_miss_(
+ insert_blocks_upon_row_kvpair_miss) {}
+ void Access(const BlockCacheTraceRecord& access) override;
+
+ private:
+ enum InsertResult : char {
+ INSERTED,
+ ADMITTED,
+ NO_INSERT,
+ };
+
+ // We set is_complete to true when the referenced row-key of a get request
+ // hits the cache. If is_complete is true, we treat future accesses of this
+ // get request as hits.
+ //
+ // For each row key, it stores an enum. It is INSERTED when the
+ // kv-pair has been inserted into the cache, ADMITTED if it should be inserted
+ // but haven't been, NO_INSERT if it should not be inserted.
+ //
+ // A kv-pair is in ADMITTED state when we encounter this kv-pair but do not
+ // know its size. This may happen if the first access on the referenced key is
+ // an index/filter block.
+ struct GetRequestStatus {
+ bool is_complete = false;
+ std::map<std::string, InsertResult> row_key_status;
+ };
+
+ // A map stores get_id to a map of row keys.
+ std::map<uint64_t, GetRequestStatus> getid_status_map_;
+ bool insert_blocks_upon_row_kvpair_miss_;
+};
+
+// A block cache simulator that reports miss ratio curves given a set of cache
+// configurations.
+class BlockCacheTraceSimulator {
+ public:
+ // warmup_seconds: The number of seconds to warmup simulated caches. The
+ // hit/miss counters are reset after the warmup completes.
+ BlockCacheTraceSimulator(
+ uint64_t warmup_seconds, uint32_t downsample_ratio,
+ const std::vector<CacheConfiguration>& cache_configurations);
+ ~BlockCacheTraceSimulator() = default;
+ // No copy and move.
+ BlockCacheTraceSimulator(const BlockCacheTraceSimulator&) = delete;
+ BlockCacheTraceSimulator& operator=(const BlockCacheTraceSimulator&) = delete;
+ BlockCacheTraceSimulator(BlockCacheTraceSimulator&&) = delete;
+ BlockCacheTraceSimulator& operator=(BlockCacheTraceSimulator&&) = delete;
+
+ Status InitializeCaches();
+
+ void Access(const BlockCacheTraceRecord& access);
+
+ const std::map<CacheConfiguration,
+ std::vector<std::shared_ptr<CacheSimulator>>>&
+ sim_caches() const {
+ return sim_caches_;
+ }
+
+ private:
+ const uint64_t warmup_seconds_;
+ const uint32_t downsample_ratio_;
+ const std::vector<CacheConfiguration> cache_configurations_;
+
+ bool warmup_complete_ = false;
+ std::map<CacheConfiguration, std::vector<std::shared_ptr<CacheSimulator>>>
+ sim_caches_;
+ uint64_t trace_start_time_ = 0;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/simulator_cache/cache_simulator_test.cc b/src/rocksdb/utilities/simulator_cache/cache_simulator_test.cc
new file mode 100644
index 000000000..a205315cc
--- /dev/null
+++ b/src/rocksdb/utilities/simulator_cache/cache_simulator_test.cc
@@ -0,0 +1,494 @@
+// 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 "utilities/simulator_cache/cache_simulator.h"
+
+#include <cstdlib>
+#include "rocksdb/env.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace {
+const std::string kBlockKeyPrefix = "test-block-";
+const std::string kRefKeyPrefix = "test-get-";
+const std::string kRefKeySequenceNumber = std::string(8, 'c');
+const uint64_t kGetId = 1;
+const uint64_t kGetBlockId = 100;
+const uint64_t kCompactionBlockId = 1000;
+const uint64_t kCacheSize = 1024 * 1024 * 1024;
+const uint64_t kGhostCacheSize = 1024 * 1024;
+} // namespace
+
+class CacheSimulatorTest : public testing::Test {
+ public:
+ const size_t kNumBlocks = 5;
+ const size_t kValueSize = 1000;
+
+ CacheSimulatorTest() { env_ = ROCKSDB_NAMESPACE::Env::Default(); }
+
+ BlockCacheTraceRecord GenerateGetRecord(uint64_t getid) {
+ BlockCacheTraceRecord record;
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ record.block_size = 4096;
+ record.block_key = kBlockKeyPrefix + std::to_string(kGetBlockId);
+ record.access_timestamp = env_->NowMicros();
+ record.cf_id = 0;
+ record.cf_name = "test";
+ record.caller = TableReaderCaller::kUserGet;
+ record.level = 6;
+ record.sst_fd_number = 0;
+ record.get_id = getid;
+ record.is_cache_hit = Boolean::kFalse;
+ record.no_insert = Boolean::kFalse;
+ record.referenced_key =
+ kRefKeyPrefix + std::to_string(kGetId) + kRefKeySequenceNumber;
+ record.referenced_key_exist_in_block = Boolean::kTrue;
+ record.referenced_data_size = 100;
+ record.num_keys_in_block = 300;
+ return record;
+ }
+
+ BlockCacheTraceRecord GenerateCompactionRecord() {
+ BlockCacheTraceRecord record;
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ record.block_size = 4096;
+ record.block_key = kBlockKeyPrefix + std::to_string(kCompactionBlockId);
+ record.access_timestamp = env_->NowMicros();
+ record.cf_id = 0;
+ record.cf_name = "test";
+ record.caller = TableReaderCaller::kCompaction;
+ record.level = 6;
+ record.sst_fd_number = kCompactionBlockId;
+ record.is_cache_hit = Boolean::kFalse;
+ record.no_insert = Boolean::kTrue;
+ return record;
+ }
+
+ void AssertCache(std::shared_ptr<Cache> sim_cache,
+ const MissRatioStats& miss_ratio_stats,
+ uint64_t expected_usage, uint64_t expected_num_accesses,
+ uint64_t expected_num_misses,
+ std::vector<std::string> blocks,
+ std::vector<std::string> keys) {
+ EXPECT_EQ(expected_usage, sim_cache->GetUsage());
+ EXPECT_EQ(expected_num_accesses, miss_ratio_stats.total_accesses());
+ EXPECT_EQ(expected_num_misses, miss_ratio_stats.total_misses());
+ for (auto const& block : blocks) {
+ auto handle = sim_cache->Lookup(block);
+ EXPECT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ }
+ for (auto const& key : keys) {
+ std::string row_key = kRefKeyPrefix + key + kRefKeySequenceNumber;
+ auto handle =
+ sim_cache->Lookup("0_" + ExtractUserKey(row_key).ToString());
+ EXPECT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ }
+ }
+
+ Env* env_;
+};
+
+TEST_F(CacheSimulatorTest, GhostCache) {
+ const std::string key1 = "test1";
+ const std::string key2 = "test2";
+ std::unique_ptr<GhostCache> ghost_cache(new GhostCache(
+ NewLRUCache(/*capacity=*/kGhostCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ EXPECT_FALSE(ghost_cache->Admit(key1));
+ EXPECT_TRUE(ghost_cache->Admit(key1));
+ EXPECT_TRUE(ghost_cache->Admit(key1));
+ EXPECT_FALSE(ghost_cache->Admit(key2));
+ EXPECT_TRUE(ghost_cache->Admit(key2));
+}
+
+TEST_F(CacheSimulatorTest, CacheSimulator) {
+ const BlockCacheTraceRecord& access = GenerateGetRecord(kGetId);
+ const BlockCacheTraceRecord& compaction_access = GenerateCompactionRecord();
+ std::shared_ptr<Cache> sim_cache =
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0);
+ std::unique_ptr<CacheSimulator> cache_simulator(
+ new CacheSimulator(nullptr, sim_cache));
+ cache_simulator->Access(access);
+ cache_simulator->Access(access);
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(50, cache_simulator->miss_ratio_stats().miss_ratio());
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(50, cache_simulator->miss_ratio_stats().user_miss_ratio());
+
+ cache_simulator->Access(compaction_access);
+ cache_simulator->Access(compaction_access);
+ ASSERT_EQ(4, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(75, cache_simulator->miss_ratio_stats().miss_ratio());
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(50, cache_simulator->miss_ratio_stats().user_miss_ratio());
+
+ cache_simulator->reset_counter();
+ ASSERT_EQ(0, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(-1, cache_simulator->miss_ratio_stats().miss_ratio());
+ auto handle = sim_cache->Lookup(access.block_key);
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ handle = sim_cache->Lookup(compaction_access.block_key);
+ ASSERT_EQ(nullptr, handle);
+}
+
+TEST_F(CacheSimulatorTest, GhostCacheSimulator) {
+ const BlockCacheTraceRecord& access = GenerateGetRecord(kGetId);
+ std::unique_ptr<GhostCache> ghost_cache(new GhostCache(
+ NewLRUCache(/*capacity=*/kGhostCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ std::unique_ptr<CacheSimulator> cache_simulator(new CacheSimulator(
+ std::move(ghost_cache),
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ cache_simulator->Access(access);
+ cache_simulator->Access(access);
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses());
+ // Both of them will be miss since we have a ghost cache.
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio());
+}
+
+TEST_F(CacheSimulatorTest, PrioritizedCacheSimulator) {
+ const BlockCacheTraceRecord& access = GenerateGetRecord(kGetId);
+ std::shared_ptr<Cache> sim_cache =
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0);
+ std::unique_ptr<PrioritizedCacheSimulator> cache_simulator(
+ new PrioritizedCacheSimulator(nullptr, sim_cache));
+ cache_simulator->Access(access);
+ cache_simulator->Access(access);
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(50, cache_simulator->miss_ratio_stats().miss_ratio());
+
+ auto handle = sim_cache->Lookup(access.block_key);
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+}
+
+TEST_F(CacheSimulatorTest, GhostPrioritizedCacheSimulator) {
+ const BlockCacheTraceRecord& access = GenerateGetRecord(kGetId);
+ std::unique_ptr<GhostCache> ghost_cache(new GhostCache(
+ NewLRUCache(/*capacity=*/kGhostCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ std::unique_ptr<PrioritizedCacheSimulator> cache_simulator(
+ new PrioritizedCacheSimulator(
+ std::move(ghost_cache),
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ cache_simulator->Access(access);
+ cache_simulator->Access(access);
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses());
+ // Both of them will be miss since we have a ghost cache.
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio());
+}
+
+TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulator) {
+ uint64_t block_id = 100;
+ BlockCacheTraceRecord first_get = GenerateGetRecord(kGetId);
+ first_get.get_from_user_specified_snapshot = Boolean::kTrue;
+ BlockCacheTraceRecord second_get = GenerateGetRecord(kGetId + 1);
+ second_get.referenced_data_size = 0;
+ second_get.referenced_key_exist_in_block = Boolean::kFalse;
+ second_get.get_from_user_specified_snapshot = Boolean::kTrue;
+ BlockCacheTraceRecord third_get = GenerateGetRecord(kGetId + 2);
+ third_get.referenced_data_size = 0;
+ third_get.referenced_key_exist_in_block = Boolean::kFalse;
+ third_get.referenced_key = kRefKeyPrefix + "third_get";
+ // We didn't find the referenced key in the third get.
+ third_get.referenced_key_exist_in_block = Boolean::kFalse;
+ third_get.referenced_data_size = 0;
+ std::shared_ptr<Cache> sim_cache =
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0);
+ std::unique_ptr<HybridRowBlockCacheSimulator> cache_simulator(
+ new HybridRowBlockCacheSimulator(
+ nullptr, sim_cache, /*insert_blocks_row_kvpair_misses=*/true));
+ // The first get request accesses 10 blocks. We should only report 10 accesses
+ // and 100% miss.
+ for (uint32_t i = 0; i < 10; i++) {
+ first_get.block_key = kBlockKeyPrefix + std::to_string(block_id);
+ cache_simulator->Access(first_get);
+ block_id++;
+ }
+
+ ASSERT_EQ(10, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio());
+ ASSERT_EQ(10, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().user_miss_ratio());
+ auto handle =
+ sim_cache->Lookup(std::to_string(first_get.sst_fd_number) + "_" +
+ ExtractUserKey(first_get.referenced_key).ToString());
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ for (uint32_t i = 100; i < block_id; i++) {
+ handle = sim_cache->Lookup(kBlockKeyPrefix + std::to_string(i));
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ }
+
+ // The second get request accesses the same key. We should report 15
+ // access and 66% miss, 10 misses with 15 accesses.
+ // We do not consider these 5 block lookups as misses since the row hits the
+ // cache.
+ for (uint32_t i = 0; i < 5; i++) {
+ second_get.block_key = kBlockKeyPrefix + std::to_string(block_id);
+ cache_simulator->Access(second_get);
+ block_id++;
+ }
+ ASSERT_EQ(15, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(66, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().miss_ratio()));
+ ASSERT_EQ(15, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(66, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().user_miss_ratio()));
+ handle =
+ sim_cache->Lookup(std::to_string(second_get.sst_fd_number) + "_" +
+ ExtractUserKey(second_get.referenced_key).ToString());
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ for (uint32_t i = 100; i < block_id; i++) {
+ handle = sim_cache->Lookup(kBlockKeyPrefix + std::to_string(i));
+ if (i < 110) {
+ ASSERT_NE(nullptr, handle) << i;
+ sim_cache->Release(handle);
+ } else {
+ ASSERT_EQ(nullptr, handle) << i;
+ }
+ }
+
+ // The third get on a different key and does not have a size.
+ // This key should not be inserted into the cache.
+ for (uint32_t i = 0; i < 5; i++) {
+ third_get.block_key = kBlockKeyPrefix + std::to_string(block_id);
+ cache_simulator->Access(third_get);
+ block_id++;
+ }
+ ASSERT_EQ(20, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(75, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().miss_ratio()));
+ ASSERT_EQ(20, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(75, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().user_miss_ratio()));
+ // Assert that the third key is not inserted into the cache.
+ handle = sim_cache->Lookup(std::to_string(third_get.sst_fd_number) + "_" +
+ third_get.referenced_key);
+ ASSERT_EQ(nullptr, handle);
+ for (uint32_t i = 100; i < block_id; i++) {
+ if (i < 110 || i >= 115) {
+ handle = sim_cache->Lookup(kBlockKeyPrefix + std::to_string(i));
+ ASSERT_NE(nullptr, handle) << i;
+ sim_cache->Release(handle);
+ } else {
+ handle = sim_cache->Lookup(kBlockKeyPrefix + std::to_string(i));
+ ASSERT_EQ(nullptr, handle) << i;
+ }
+ }
+}
+
+TEST_F(CacheSimulatorTest, HybridRowBlockCacheSimulatorGetTest) {
+ BlockCacheTraceRecord get = GenerateGetRecord(kGetId);
+ get.block_size = 1;
+ get.referenced_data_size = 0;
+ get.access_timestamp = 0;
+ get.block_key = "1";
+ get.get_id = 1;
+ get.get_from_user_specified_snapshot = Boolean::kFalse;
+ get.referenced_key =
+ kRefKeyPrefix + std::to_string(1) + kRefKeySequenceNumber;
+ get.no_insert = Boolean::kFalse;
+ get.sst_fd_number = 0;
+ get.get_from_user_specified_snapshot = Boolean::kFalse;
+
+ LRUCacheOptions co;
+ co.capacity = 16;
+ co.num_shard_bits = 1;
+ co.strict_capacity_limit = false;
+ co.high_pri_pool_ratio = 0;
+ co.metadata_charge_policy = kDontChargeCacheMetadata;
+ std::shared_ptr<Cache> sim_cache = NewLRUCache(co);
+ std::unique_ptr<HybridRowBlockCacheSimulator> cache_simulator(
+ new HybridRowBlockCacheSimulator(
+ nullptr, sim_cache, /*insert_blocks_row_kvpair_misses=*/true));
+ // Expect a miss and does not insert the row key-value pair since it does not
+ // have size.
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 1, 1, 1, {"1"},
+ {});
+ get.access_timestamp += 1;
+ get.referenced_data_size = 1;
+ get.block_key = "2";
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 3, 2, 2,
+ {"1", "2"}, {"1"});
+ get.access_timestamp += 1;
+ get.block_key = "3";
+ // K1 should not inserted again.
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 4, 3, 3,
+ {"1", "2", "3"}, {"1"});
+
+ // A second get request referencing the same key.
+ get.access_timestamp += 1;
+ get.get_id = 2;
+ get.block_key = "4";
+ get.referenced_data_size = 0;
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 4, 4, 3,
+ {"1", "2", "3"}, {"1"});
+
+ // A third get request searches three files, three different keys.
+ // And the second key observes a hit.
+ get.access_timestamp += 1;
+ get.referenced_data_size = 1;
+ get.get_id = 3;
+ get.block_key = "3";
+ get.referenced_key = kRefKeyPrefix + "2" + kRefKeySequenceNumber;
+ // K2 should observe a miss. Block 3 observes a hit.
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 5, 5, 3,
+ {"1", "2", "3"}, {"1", "2"});
+
+ get.access_timestamp += 1;
+ get.referenced_data_size = 1;
+ get.get_id = 3;
+ get.block_key = "4";
+ get.referenced_data_size = 1;
+ get.referenced_key = kRefKeyPrefix + "1" + kRefKeySequenceNumber;
+ // K1 should observe a hit.
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 5, 6, 3,
+ {"1", "2", "3"}, {"1", "2"});
+
+ get.access_timestamp += 1;
+ get.referenced_data_size = 1;
+ get.get_id = 3;
+ get.block_key = "4";
+ get.referenced_data_size = 1;
+ get.referenced_key = kRefKeyPrefix + "3" + kRefKeySequenceNumber;
+ // K3 should observe a miss.
+ // However, as the get already complete, we should not access k3 any more.
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 5, 7, 3,
+ {"1", "2", "3"}, {"1", "2"});
+
+ // A fourth get request searches one file and two blocks. One row key.
+ get.access_timestamp += 1;
+ get.get_id = 4;
+ get.block_key = "5";
+ get.referenced_key = kRefKeyPrefix + "4" + kRefKeySequenceNumber;
+ get.referenced_data_size = 1;
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 7, 8, 4,
+ {"1", "2", "3", "5"}, {"1", "2", "4"});
+ for (auto const& key : {"1", "2", "4"}) {
+ auto handle = sim_cache->Lookup("0_" + kRefKeyPrefix + key);
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ }
+
+ // A bunch of insertions which evict cached row keys.
+ for (uint32_t i = 6; i < 100; i++) {
+ get.access_timestamp += 1;
+ get.get_id = 0;
+ get.block_key = std::to_string(i);
+ cache_simulator->Access(get);
+ }
+
+ get.get_id = 4;
+ // A different block.
+ get.block_key = "100";
+ // Same row key and should not be inserted again.
+ get.referenced_key = kRefKeyPrefix + "4" + kRefKeySequenceNumber;
+ get.referenced_data_size = 1;
+ cache_simulator->Access(get);
+ AssertCache(sim_cache, cache_simulator->miss_ratio_stats(), 16, 103, 99, {},
+ {});
+ for (auto const& key : {"1", "2", "4"}) {
+ auto handle = sim_cache->Lookup("0_" + kRefKeyPrefix + key);
+ ASSERT_EQ(nullptr, handle);
+ }
+}
+
+TEST_F(CacheSimulatorTest, HybridRowBlockNoInsertCacheSimulator) {
+ uint64_t block_id = 100;
+ BlockCacheTraceRecord first_get = GenerateGetRecord(kGetId);
+ std::shared_ptr<Cache> sim_cache =
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0);
+ std::unique_ptr<HybridRowBlockCacheSimulator> cache_simulator(
+ new HybridRowBlockCacheSimulator(
+ nullptr, sim_cache, /*insert_blocks_row_kvpair_misses=*/false));
+ for (uint32_t i = 0; i < 9; i++) {
+ first_get.block_key = kBlockKeyPrefix + std::to_string(block_id);
+ cache_simulator->Access(first_get);
+ block_id++;
+ }
+ auto handle =
+ sim_cache->Lookup(std::to_string(first_get.sst_fd_number) + "_" +
+ ExtractUserKey(first_get.referenced_key).ToString());
+ ASSERT_NE(nullptr, handle);
+ sim_cache->Release(handle);
+ // All blocks are missing from the cache since insert_blocks_row_kvpair_misses
+ // is set to false.
+ for (uint32_t i = 100; i < block_id; i++) {
+ handle = sim_cache->Lookup(kBlockKeyPrefix + std::to_string(i));
+ ASSERT_EQ(nullptr, handle);
+ }
+}
+
+TEST_F(CacheSimulatorTest, GhostHybridRowBlockCacheSimulator) {
+ std::unique_ptr<GhostCache> ghost_cache(new GhostCache(
+ NewLRUCache(/*capacity=*/kGhostCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0)));
+ const BlockCacheTraceRecord& first_get = GenerateGetRecord(kGetId);
+ const BlockCacheTraceRecord& second_get = GenerateGetRecord(kGetId + 1);
+ const BlockCacheTraceRecord& third_get = GenerateGetRecord(kGetId + 2);
+ std::unique_ptr<HybridRowBlockCacheSimulator> cache_simulator(
+ new HybridRowBlockCacheSimulator(
+ std::move(ghost_cache),
+ NewLRUCache(/*capacity=*/kCacheSize, /*num_shard_bits=*/1,
+ /*strict_capacity_limit=*/false,
+ /*high_pri_pool_ratio=*/0),
+ /*insert_blocks_row_kvpair_misses=*/false));
+ // Two get requests access the same key.
+ cache_simulator->Access(first_get);
+ cache_simulator->Access(second_get);
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().miss_ratio());
+ ASSERT_EQ(2, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(100, cache_simulator->miss_ratio_stats().user_miss_ratio());
+ // We insert the key-value pair upon the second get request. A third get
+ // request should observe a hit.
+ for (uint32_t i = 0; i < 10; i++) {
+ cache_simulator->Access(third_get);
+ }
+ ASSERT_EQ(12, cache_simulator->miss_ratio_stats().total_accesses());
+ ASSERT_EQ(16, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().miss_ratio()));
+ ASSERT_EQ(12, cache_simulator->miss_ratio_stats().user_accesses());
+ ASSERT_EQ(16, static_cast<uint64_t>(
+ cache_simulator->miss_ratio_stats().user_miss_ratio()));
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/utilities/simulator_cache/sim_cache.cc b/src/rocksdb/utilities/simulator_cache/sim_cache.cc
new file mode 100644
index 000000000..ec411cf9a
--- /dev/null
+++ b/src/rocksdb/utilities/simulator_cache/sim_cache.cc
@@ -0,0 +1,354 @@
+// 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/utilities/sim_cache.h"
+#include <atomic>
+#include "env/composite_env_wrapper.h"
+#include "file/writable_file_writer.h"
+#include "monitoring/statistics.h"
+#include "port/port.h"
+#include "rocksdb/env.h"
+#include "util/mutexlock.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+class CacheActivityLogger {
+ public:
+ CacheActivityLogger()
+ : activity_logging_enabled_(false), max_logging_size_(0) {}
+
+ ~CacheActivityLogger() {
+ MutexLock l(&mutex_);
+
+ StopLoggingInternal();
+ }
+
+ Status StartLogging(const std::string& activity_log_file, Env* env,
+ uint64_t max_logging_size = 0) {
+ assert(activity_log_file != "");
+ assert(env != nullptr);
+
+ Status status;
+ EnvOptions env_opts;
+ std::unique_ptr<WritableFile> log_file;
+
+ MutexLock l(&mutex_);
+
+ // Stop existing logging if any
+ StopLoggingInternal();
+
+ // Open log file
+ status = env->NewWritableFile(activity_log_file, &log_file, env_opts);
+ if (!status.ok()) {
+ return status;
+ }
+ file_writer_.reset(new WritableFileWriter(
+ NewLegacyWritableFileWrapper(std::move(log_file)), activity_log_file,
+ env_opts));
+
+ max_logging_size_ = max_logging_size;
+ activity_logging_enabled_.store(true);
+
+ return status;
+ }
+
+ void StopLogging() {
+ MutexLock l(&mutex_);
+
+ StopLoggingInternal();
+ }
+
+ void ReportLookup(const Slice& key) {
+ if (activity_logging_enabled_.load() == false) {
+ return;
+ }
+
+ std::string log_line = "LOOKUP - " + key.ToString(true) + "\n";
+
+ // line format: "LOOKUP - <KEY>"
+ MutexLock l(&mutex_);
+ Status s = file_writer_->Append(log_line);
+ if (!s.ok() && bg_status_.ok()) {
+ bg_status_ = s;
+ }
+ if (MaxLoggingSizeReached() || !bg_status_.ok()) {
+ // Stop logging if we have reached the max file size or
+ // encountered an error
+ StopLoggingInternal();
+ }
+ }
+
+ void ReportAdd(const Slice& key, size_t size) {
+ if (activity_logging_enabled_.load() == false) {
+ return;
+ }
+
+ std::string log_line = "ADD - ";
+ log_line += key.ToString(true);
+ log_line += " - ";
+ AppendNumberTo(&log_line, size);
+ // @lint-ignore TXT2 T25377293 Grandfathered in
+ log_line += "\n";
+
+ // line format: "ADD - <KEY> - <KEY-SIZE>"
+ MutexLock l(&mutex_);
+ Status s = file_writer_->Append(log_line);
+ if (!s.ok() && bg_status_.ok()) {
+ bg_status_ = s;
+ }
+
+ if (MaxLoggingSizeReached() || !bg_status_.ok()) {
+ // Stop logging if we have reached the max file size or
+ // encountered an error
+ StopLoggingInternal();
+ }
+ }
+
+ Status& bg_status() {
+ MutexLock l(&mutex_);
+ return bg_status_;
+ }
+
+ private:
+ bool MaxLoggingSizeReached() {
+ mutex_.AssertHeld();
+
+ return (max_logging_size_ > 0 &&
+ file_writer_->GetFileSize() >= max_logging_size_);
+ }
+
+ void StopLoggingInternal() {
+ mutex_.AssertHeld();
+
+ if (!activity_logging_enabled_) {
+ return;
+ }
+
+ activity_logging_enabled_.store(false);
+ Status s = file_writer_->Close();
+ if (!s.ok() && bg_status_.ok()) {
+ bg_status_ = s;
+ }
+ }
+
+ // Mutex to sync writes to file_writer, and all following
+ // class data members
+ port::Mutex mutex_;
+ // Indicates if logging is currently enabled
+ // atomic to allow reads without mutex
+ std::atomic<bool> activity_logging_enabled_;
+ // When reached, we will stop logging and close the file
+ // Value of 0 means unlimited
+ uint64_t max_logging_size_;
+ std::unique_ptr<WritableFileWriter> file_writer_;
+ Status bg_status_;
+};
+
+// SimCacheImpl definition
+class SimCacheImpl : public SimCache {
+ public:
+ // capacity for real cache (ShardedLRUCache)
+ // test_capacity for key only cache
+ SimCacheImpl(std::shared_ptr<Cache> sim_cache, std::shared_ptr<Cache> cache)
+ : cache_(cache),
+ key_only_cache_(sim_cache),
+ miss_times_(0),
+ hit_times_(0),
+ stats_(nullptr) {}
+
+ ~SimCacheImpl() override {}
+ void SetCapacity(size_t capacity) override { cache_->SetCapacity(capacity); }
+
+ void SetStrictCapacityLimit(bool strict_capacity_limit) override {
+ cache_->SetStrictCapacityLimit(strict_capacity_limit);
+ }
+
+ Status Insert(const Slice& key, void* value, size_t charge,
+ void (*deleter)(const Slice& key, void* value), Handle** handle,
+ Priority priority) override {
+ // The handle and value passed in are for real cache, so we pass nullptr
+ // to key_only_cache_ for both instead. Also, the deleter function pointer
+ // will be called by user to perform some external operation which should
+ // be applied only once. Thus key_only_cache accepts an empty function.
+ // *Lambda function without capture can be assgined to a function pointer
+ Handle* h = key_only_cache_->Lookup(key);
+ if (h == nullptr) {
+ key_only_cache_->Insert(key, nullptr, charge,
+ [](const Slice& /*k*/, void* /*v*/) {}, nullptr,
+ priority);
+ } else {
+ key_only_cache_->Release(h);
+ }
+
+ cache_activity_logger_.ReportAdd(key, charge);
+ if (!cache_) {
+ return Status::OK();
+ }
+ return cache_->Insert(key, value, charge, deleter, handle, priority);
+ }
+
+ Handle* Lookup(const Slice& key, Statistics* stats) override {
+ Handle* h = key_only_cache_->Lookup(key);
+ if (h != nullptr) {
+ key_only_cache_->Release(h);
+ inc_hit_counter();
+ RecordTick(stats, SIM_BLOCK_CACHE_HIT);
+ } else {
+ inc_miss_counter();
+ RecordTick(stats, SIM_BLOCK_CACHE_MISS);
+ }
+
+ cache_activity_logger_.ReportLookup(key);
+ if (!cache_) {
+ return nullptr;
+ }
+ return cache_->Lookup(key, stats);
+ }
+
+ bool Ref(Handle* handle) override { return cache_->Ref(handle); }
+
+ bool Release(Handle* handle, bool force_erase = false) override {
+ return cache_->Release(handle, force_erase);
+ }
+
+ void Erase(const Slice& key) override {
+ cache_->Erase(key);
+ key_only_cache_->Erase(key);
+ }
+
+ void* Value(Handle* handle) override { return cache_->Value(handle); }
+
+ uint64_t NewId() override { return cache_->NewId(); }
+
+ size_t GetCapacity() const override { return cache_->GetCapacity(); }
+
+ bool HasStrictCapacityLimit() const override {
+ return cache_->HasStrictCapacityLimit();
+ }
+
+ size_t GetUsage() const override { return cache_->GetUsage(); }
+
+ size_t GetUsage(Handle* handle) const override {
+ return cache_->GetUsage(handle);
+ }
+
+ size_t GetCharge(Handle* handle) const override {
+ return cache_->GetCharge(handle);
+ }
+
+ size_t GetPinnedUsage() const override { return cache_->GetPinnedUsage(); }
+
+ void DisownData() override {
+ cache_->DisownData();
+ key_only_cache_->DisownData();
+ }
+
+ void ApplyToAllCacheEntries(void (*callback)(void*, size_t),
+ bool thread_safe) override {
+ // only apply to _cache since key_only_cache doesn't hold value
+ cache_->ApplyToAllCacheEntries(callback, thread_safe);
+ }
+
+ void EraseUnRefEntries() override {
+ cache_->EraseUnRefEntries();
+ key_only_cache_->EraseUnRefEntries();
+ }
+
+ size_t GetSimCapacity() const override {
+ return key_only_cache_->GetCapacity();
+ }
+ size_t GetSimUsage() const override { return key_only_cache_->GetUsage(); }
+ void SetSimCapacity(size_t capacity) override {
+ key_only_cache_->SetCapacity(capacity);
+ }
+
+ uint64_t get_miss_counter() const override {
+ return miss_times_.load(std::memory_order_relaxed);
+ }
+
+ uint64_t get_hit_counter() const override {
+ return hit_times_.load(std::memory_order_relaxed);
+ }
+
+ void reset_counter() override {
+ miss_times_.store(0, std::memory_order_relaxed);
+ hit_times_.store(0, std::memory_order_relaxed);
+ SetTickerCount(stats_, SIM_BLOCK_CACHE_HIT, 0);
+ SetTickerCount(stats_, SIM_BLOCK_CACHE_MISS, 0);
+ }
+
+ std::string ToString() const override {
+ std::string res;
+ res.append("SimCache MISSes: " + std::to_string(get_miss_counter()) + "\n");
+ res.append("SimCache HITs: " + std::to_string(get_hit_counter()) + "\n");
+ char buff[350];
+ auto lookups = get_miss_counter() + get_hit_counter();
+ snprintf(buff, sizeof(buff), "SimCache HITRATE: %.2f%%\n",
+ (lookups == 0 ? 0 : get_hit_counter() * 100.0f / lookups));
+ res.append(buff);
+ return res;
+ }
+
+ std::string GetPrintableOptions() const override {
+ std::string ret;
+ ret.reserve(20000);
+ ret.append(" cache_options:\n");
+ ret.append(cache_->GetPrintableOptions());
+ ret.append(" sim_cache_options:\n");
+ ret.append(key_only_cache_->GetPrintableOptions());
+ return ret;
+ }
+
+ Status StartActivityLogging(const std::string& activity_log_file, Env* env,
+ uint64_t max_logging_size = 0) override {
+ return cache_activity_logger_.StartLogging(activity_log_file, env,
+ max_logging_size);
+ }
+
+ void StopActivityLogging() override { cache_activity_logger_.StopLogging(); }
+
+ Status GetActivityLoggingStatus() override {
+ return cache_activity_logger_.bg_status();
+ }
+
+ private:
+ std::shared_ptr<Cache> cache_;
+ std::shared_ptr<Cache> key_only_cache_;
+ std::atomic<uint64_t> miss_times_;
+ std::atomic<uint64_t> hit_times_;
+ Statistics* stats_;
+ CacheActivityLogger cache_activity_logger_;
+
+ void inc_miss_counter() {
+ miss_times_.fetch_add(1, std::memory_order_relaxed);
+ }
+ void inc_hit_counter() { hit_times_.fetch_add(1, std::memory_order_relaxed); }
+};
+
+} // end anonymous namespace
+
+// For instrumentation purpose, use NewSimCache instead
+std::shared_ptr<SimCache> NewSimCache(std::shared_ptr<Cache> cache,
+ size_t sim_capacity, int num_shard_bits) {
+ LRUCacheOptions co;
+ co.capacity = sim_capacity;
+ co.num_shard_bits = num_shard_bits;
+ co.metadata_charge_policy = kDontChargeCacheMetadata;
+ return NewSimCache(NewLRUCache(co), cache, num_shard_bits);
+}
+
+std::shared_ptr<SimCache> NewSimCache(std::shared_ptr<Cache> sim_cache,
+ std::shared_ptr<Cache> cache,
+ int num_shard_bits) {
+ if (num_shard_bits >= 20) {
+ return nullptr; // the cache cannot be sharded into too many fine pieces
+ }
+ return std::make_shared<SimCacheImpl>(sim_cache, cache);
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/utilities/simulator_cache/sim_cache_test.cc b/src/rocksdb/utilities/simulator_cache/sim_cache_test.cc
new file mode 100644
index 000000000..6cb495813
--- /dev/null
+++ b/src/rocksdb/utilities/simulator_cache/sim_cache_test.cc
@@ -0,0 +1,225 @@
+// 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/utilities/sim_cache.h"
+#include <cstdlib>
+#include "db/db_test_util.h"
+#include "port/stack_trace.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class SimCacheTest : public DBTestBase {
+ private:
+ size_t miss_count_ = 0;
+ size_t hit_count_ = 0;
+ size_t insert_count_ = 0;
+ size_t failure_count_ = 0;
+
+ public:
+ const size_t kNumBlocks = 5;
+ const size_t kValueSize = 1000;
+
+ SimCacheTest() : DBTestBase("/sim_cache_test") {}
+
+ BlockBasedTableOptions GetTableOptions() {
+ BlockBasedTableOptions table_options;
+ // Set a small enough block size so that each key-value get its own block.
+ table_options.block_size = 1;
+ return table_options;
+ }
+
+ Options GetOptions(const BlockBasedTableOptions& table_options) {
+ Options options = CurrentOptions();
+ options.create_if_missing = true;
+ // options.compression = kNoCompression;
+ options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ return options;
+ }
+
+ void InitTable(const Options& /*options*/) {
+ std::string value(kValueSize, 'a');
+ for (size_t i = 0; i < kNumBlocks * 2; i++) {
+ ASSERT_OK(Put(ToString(i), value.c_str()));
+ }
+ }
+
+ void RecordCacheCounters(const Options& options) {
+ miss_count_ = TestGetTickerCount(options, BLOCK_CACHE_MISS);
+ hit_count_ = TestGetTickerCount(options, BLOCK_CACHE_HIT);
+ insert_count_ = TestGetTickerCount(options, BLOCK_CACHE_ADD);
+ failure_count_ = TestGetTickerCount(options, BLOCK_CACHE_ADD_FAILURES);
+ }
+
+ void CheckCacheCounters(const Options& options, size_t expected_misses,
+ size_t expected_hits, size_t expected_inserts,
+ size_t expected_failures) {
+ size_t new_miss_count = TestGetTickerCount(options, BLOCK_CACHE_MISS);
+ size_t new_hit_count = TestGetTickerCount(options, BLOCK_CACHE_HIT);
+ size_t new_insert_count = TestGetTickerCount(options, BLOCK_CACHE_ADD);
+ size_t new_failure_count =
+ TestGetTickerCount(options, BLOCK_CACHE_ADD_FAILURES);
+ ASSERT_EQ(miss_count_ + expected_misses, new_miss_count);
+ ASSERT_EQ(hit_count_ + expected_hits, new_hit_count);
+ ASSERT_EQ(insert_count_ + expected_inserts, new_insert_count);
+ ASSERT_EQ(failure_count_ + expected_failures, new_failure_count);
+ miss_count_ = new_miss_count;
+ hit_count_ = new_hit_count;
+ insert_count_ = new_insert_count;
+ failure_count_ = new_failure_count;
+ }
+};
+
+TEST_F(SimCacheTest, SimCache) {
+ ReadOptions read_options;
+ auto table_options = GetTableOptions();
+ auto options = GetOptions(table_options);
+ InitTable(options);
+ LRUCacheOptions co;
+ co.capacity = 0;
+ co.num_shard_bits = 0;
+ co.strict_capacity_limit = false;
+ co.metadata_charge_policy = kDontChargeCacheMetadata;
+ std::shared_ptr<SimCache> simCache = NewSimCache(NewLRUCache(co), 20000, 0);
+ table_options.block_cache = simCache;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ Reopen(options);
+ RecordCacheCounters(options);
+
+ std::vector<std::unique_ptr<Iterator>> iterators(kNumBlocks);
+ Iterator* iter = nullptr;
+
+ // Load blocks into cache.
+ for (size_t i = 0; i < kNumBlocks; i++) {
+ iter = db_->NewIterator(read_options);
+ iter->Seek(ToString(i));
+ ASSERT_OK(iter->status());
+ CheckCacheCounters(options, 1, 0, 1, 0);
+ iterators[i].reset(iter);
+ }
+ ASSERT_EQ(kNumBlocks,
+ simCache->get_hit_counter() + simCache->get_miss_counter());
+ ASSERT_EQ(0, simCache->get_hit_counter());
+ size_t usage = simCache->GetUsage();
+ ASSERT_LT(0, usage);
+ ASSERT_EQ(usage, simCache->GetSimUsage());
+ simCache->SetCapacity(usage);
+ ASSERT_EQ(usage, simCache->GetPinnedUsage());
+
+ // Test with strict capacity limit.
+ simCache->SetStrictCapacityLimit(true);
+ iter = db_->NewIterator(read_options);
+ iter->Seek(ToString(kNumBlocks * 2 - 1));
+ ASSERT_TRUE(iter->status().IsIncomplete());
+ CheckCacheCounters(options, 1, 0, 0, 1);
+ delete iter;
+ iter = nullptr;
+
+ // Release iterators and access cache again.
+ for (size_t i = 0; i < kNumBlocks; i++) {
+ iterators[i].reset();
+ CheckCacheCounters(options, 0, 0, 0, 0);
+ }
+ // Add kNumBlocks again
+ for (size_t i = 0; i < kNumBlocks; i++) {
+ std::unique_ptr<Iterator> it(db_->NewIterator(read_options));
+ it->Seek(ToString(i));
+ ASSERT_OK(it->status());
+ CheckCacheCounters(options, 0, 1, 0, 0);
+ }
+ ASSERT_EQ(5, simCache->get_hit_counter());
+ for (size_t i = kNumBlocks; i < kNumBlocks * 2; i++) {
+ std::unique_ptr<Iterator> it(db_->NewIterator(read_options));
+ it->Seek(ToString(i));
+ ASSERT_OK(it->status());
+ CheckCacheCounters(options, 1, 0, 1, 0);
+ }
+ ASSERT_EQ(0, simCache->GetPinnedUsage());
+ ASSERT_EQ(3 * kNumBlocks + 1,
+ simCache->get_hit_counter() + simCache->get_miss_counter());
+ ASSERT_EQ(6, simCache->get_hit_counter());
+}
+
+TEST_F(SimCacheTest, SimCacheLogging) {
+ auto table_options = GetTableOptions();
+ auto options = GetOptions(table_options);
+ options.disable_auto_compactions = true;
+ LRUCacheOptions co;
+ co.capacity = 1024 * 1024;
+ co.metadata_charge_policy = kDontChargeCacheMetadata;
+ std::shared_ptr<SimCache> sim_cache = NewSimCache(NewLRUCache(co), 20000, 0);
+ table_options.block_cache = sim_cache;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ Reopen(options);
+
+ int num_block_entries = 20;
+ for (int i = 0; i < num_block_entries; i++) {
+ Put(Key(i), "val");
+ Flush();
+ }
+
+ std::string log_file = test::PerThreadDBPath(env_, "cache_log.txt");
+ ASSERT_OK(sim_cache->StartActivityLogging(log_file, env_));
+ for (int i = 0; i < num_block_entries; i++) {
+ ASSERT_EQ(Get(Key(i)), "val");
+ }
+ for (int i = 0; i < num_block_entries; i++) {
+ ASSERT_EQ(Get(Key(i)), "val");
+ }
+ sim_cache->StopActivityLogging();
+ ASSERT_OK(sim_cache->GetActivityLoggingStatus());
+
+ std::string file_contents = "";
+ ReadFileToString(env_, log_file, &file_contents);
+
+ int lookup_num = 0;
+ int add_num = 0;
+ std::string::size_type pos;
+
+ // count number of lookups
+ pos = 0;
+ while ((pos = file_contents.find("LOOKUP -", pos)) != std::string::npos) {
+ ++lookup_num;
+ pos += 1;
+ }
+
+ // count number of additions
+ pos = 0;
+ while ((pos = file_contents.find("ADD -", pos)) != std::string::npos) {
+ ++add_num;
+ pos += 1;
+ }
+
+ // We asked for every block twice
+ ASSERT_EQ(lookup_num, num_block_entries * 2);
+
+ // We added every block only once, since the cache can hold all blocks
+ ASSERT_EQ(add_num, num_block_entries);
+
+ // Log things again but stop logging automatically after reaching 512 bytes
+ // @lint-ignore TXT2 T25377293 Grandfathered in
+ int max_size = 512;
+ ASSERT_OK(sim_cache->StartActivityLogging(log_file, env_, max_size));
+ for (int it = 0; it < 10; it++) {
+ for (int i = 0; i < num_block_entries; i++) {
+ ASSERT_EQ(Get(Key(i)), "val");
+ }
+ }
+ ASSERT_OK(sim_cache->GetActivityLoggingStatus());
+
+ uint64_t fsize = 0;
+ ASSERT_OK(env_->GetFileSize(log_file, &fsize));
+ // error margin of 100 bytes
+ ASSERT_LT(fsize, max_size + 100);
+ ASSERT_GT(fsize, max_size - 100);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}