Use deleters to label cache entries and collect stats (#8297)

Summary:
This change gathers and publishes statistics about the
kinds of items in block cache. This is especially important for
profiling relative usage of cache by index vs. filter vs. data blocks.
It works by iterating over the cache during periodic stats dump
(InternalStats, stats_dump_period_sec) or on demand when
DB::Get(Map)Property(kBlockCacheEntryStats), except that for
efficiency and sharing among column families, saved data from
the last scan is used when the data is not considered too old.

The new information can be seen in info LOG, for example:

    Block cache LRUCache@0x7fca62229330 capacity: 95.37 MB collections: 8 last_copies: 0 last_secs: 0.00178 secs_since: 0
    Block cache entry stats(count,size,portion): DataBlock(7092,28.24 MB,29.6136%) FilterBlock(215,867.90 KB,0.888728%) FilterMetaBlock(2,5.31 KB,0.00544%) IndexBlock(217,180.11 KB,0.184432%) WriteBuffer(1,256.00 KB,0.262144%) Misc(1,0.00 KB,0%)

And also through DB::GetProperty and GetMapProperty (here using
ldb just for demonstration):

    $ ./ldb --db=/dev/shm/dbbench/ get_property rocksdb.block-cache-entry-stats
    rocksdb.block-cache-entry-stats.bytes.data-block: 0
    rocksdb.block-cache-entry-stats.bytes.deprecated-filter-block: 0
    rocksdb.block-cache-entry-stats.bytes.filter-block: 0
    rocksdb.block-cache-entry-stats.bytes.filter-meta-block: 0
    rocksdb.block-cache-entry-stats.bytes.index-block: 178992
    rocksdb.block-cache-entry-stats.bytes.misc: 0
    rocksdb.block-cache-entry-stats.bytes.other-block: 0
    rocksdb.block-cache-entry-stats.bytes.write-buffer: 0
    rocksdb.block-cache-entry-stats.capacity: 8388608
    rocksdb.block-cache-entry-stats.count.data-block: 0
    rocksdb.block-cache-entry-stats.count.deprecated-filter-block: 0
    rocksdb.block-cache-entry-stats.count.filter-block: 0
    rocksdb.block-cache-entry-stats.count.filter-meta-block: 0
    rocksdb.block-cache-entry-stats.count.index-block: 215
    rocksdb.block-cache-entry-stats.count.misc: 1
    rocksdb.block-cache-entry-stats.count.other-block: 0
    rocksdb.block-cache-entry-stats.count.write-buffer: 0
    rocksdb.block-cache-entry-stats.id: LRUCache@0x7f3636661290
    rocksdb.block-cache-entry-stats.percent.data-block: 0.000000
    rocksdb.block-cache-entry-stats.percent.deprecated-filter-block: 0.000000
    rocksdb.block-cache-entry-stats.percent.filter-block: 0.000000
    rocksdb.block-cache-entry-stats.percent.filter-meta-block: 0.000000
    rocksdb.block-cache-entry-stats.percent.index-block: 2.133751
    rocksdb.block-cache-entry-stats.percent.misc: 0.000000
    rocksdb.block-cache-entry-stats.percent.other-block: 0.000000
    rocksdb.block-cache-entry-stats.percent.write-buffer: 0.000000
    rocksdb.block-cache-entry-stats.secs_for_last_collection: 0.000052
    rocksdb.block-cache-entry-stats.secs_since_last_collection: 0

Solution detail - We need some way to flag what kind of blocks each
entry belongs to, preferably without changing the Cache API.
One of the complications is that Cache is a general interface that could
have other users that don't adhere to whichever convention we decide
on for keys and values. Or we would pay for an extra field in the Handle
that would only be used for this purpose.

This change uses a back-door approach, the deleter, to indicate the
"role" of a Cache entry (in addition to the value type, implicitly).
This has the added benefit of ensuring proper code origin whenever we
recognize a particular role for a cache entry; if the entry came from
some other part of the code, it will use an unrecognized deleter, which
we simply attribute to the "Misc" role.

An internal API makes for simple instantiation and automatic
registration of Cache deleters for a given value type and "role".

Another internal API, CacheEntryStatsCollector, solves the problem of
caching the results of a scan and sharing them, to ensure scans are
neither excessive nor redundant so as not to harm Cache performance.

Because code is added to BlocklikeTraits, it is pulled out of
block_based_table_reader.cc into its own file.

This is a reformulation of https://github.com/facebook/rocksdb/issues/8276, without the type checking option
(could still be added), and with actual stat gathering.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/8297

Test Plan: manual testing with db_bench, and a couple of basic unit tests

Reviewed By: ltamasi

Differential Revision: D28488721

Pulled By: pdillinger

fbshipit-source-id: 472f524a9691b5afb107934be2d41d84f2b129fb
main
Peter Dillinger 3 years ago committed by Facebook GitHub Bot
parent 748e3acc11
commit 311a544c2a
  1. 1
      CMakeLists.txt
  2. 1
      HISTORY.md
  3. 2
      TARGETS
  4. 66
      cache/cache_entry_roles.cc
  5. 122
      cache/cache_entry_roles.h
  6. 152
      cache/cache_entry_stats.h
  7. 11
      cache/cache_helpers.h
  8. 3
      cache/cache_test.cc
  9. 14
      cache/clock_cache.cc
  10. 18
      cache/lru_cache.cc
  11. 11
      cache/lru_cache.h
  12. 4
      cache/sharded_cache.cc
  13. 5
      cache/sharded_cache.h
  14. 130
      db/db_block_cache_test.cc
  15. 4
      db/db_test_util.h
  16. 160
      db/internal_stats.cc
  17. 65
      db/internal_stats.h
  18. 6
      include/rocksdb/cache.h
  19. 4
      include/rocksdb/db.h
  20. 7
      memtable/write_buffer_manager.cc
  21. 25
      port/lang.h
  22. 1
      src.mk
  23. 87
      table/block_based/block_based_table_reader.cc
  24. 128
      table/block_based/block_like_traits.h
  25. 2
      tools/db_bench_tool.cc
  26. 4
      utilities/simulator_cache/sim_cache.cc

@ -599,6 +599,7 @@ find_package(Threads REQUIRED)
set(SOURCES
cache/cache.cc
cache/cache_entry_roles.cc
cache/clock_cache.cc
cache/lru_cache.cc
cache/sharded_cache.cc

@ -16,6 +16,7 @@
* Add new option allow_stall passed during instance creation of WriteBufferManager. When allow_stall is set, WriteBufferManager will stall all writers shared across multiple DBs and columns if memory usage goes beyond specified WriteBufferManager::buffer_size (soft limit). Stall will be cleared when memory is freed after flush and memory usage goes down below buffer_size.
* Allow `CompactionFilter`s to apply in more table file creation scenarios such as flush and recovery. For compatibility, `CompactionFilter`s by default apply during compaction. Users can customize this behavior by overriding `CompactionFilterFactory::ShouldFilterTableFileCreation()`.
* Added more fields to FilterBuildingContext with LSM details, for custom filter policies that vary behavior based on where they are in the LSM-tree.
* Added DB::Properties::kBlockCacheEntryStats for querying statistics on what percentage of block cache is used by various kinds of blocks, etc. using DB::GetProperty and DB::GetMapProperty. The same information is now dumped to info LOG periodically according to `stats_dump_period_sec`.
### Performance Improvements
* BlockPrefetcher is used by iterators to prefetch data if they anticipate more data to be used in future. It is enabled implicitly by rocksdb. Added change to take in account read pattern if reads are sequential. This would disable prefetching for random reads in MultiGet and iterators as readahead_size is increased exponential doing large prefetches.

@ -132,6 +132,7 @@ cpp_library(
name = "rocksdb_lib",
srcs = [
"cache/cache.cc",
"cache/cache_entry_roles.cc",
"cache/clock_cache.cc",
"cache/lru_cache.cc",
"cache/sharded_cache.cc",
@ -442,6 +443,7 @@ cpp_library(
name = "rocksdb_whole_archive_lib",
srcs = [
"cache/cache.cc",
"cache/cache_entry_roles.cc",
"cache/clock_cache.cc",
"cache/lru_cache.cc",
"cache/sharded_cache.cc",

@ -0,0 +1,66 @@
// 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<const char*, kNumCacheEntryRoles> kCacheEntryRoleToCamelString{{
"DataBlock",
"FilterBlock",
"FilterMetaBlock",
"DeprecatedFilterBlock",
"IndexBlock",
"OtherBlock",
"WriteBuffer",
"Misc",
}};
std::array<const char*, kNumCacheEntryRoles> kCacheEntryRoleToHyphenString{{
"data-block",
"filter-block",
"filter-meta-block",
"deprecated-filter-block",
"index-block",
"other-block",
"write-buffer",
"misc",
}};
namespace {
struct Registry {
std::mutex mutex;
std::unordered_map<Cache::DeleterFn, CacheEntryRole> role_map;
void Register(Cache::DeleterFn fn, CacheEntryRole role) {
std::lock_guard<std::mutex> lock(mutex);
role_map[fn] = role;
}
std::unordered_map<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);
}
std::unordered_map<Cache::DeleterFn, CacheEntryRole> CopyCacheDeleterRoleMap() {
return GetRegistry().Copy();
}
} // namespace ROCKSDB_NAMESPACE

@ -0,0 +1,122 @@
// 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 <unordered_map>
#include "rocksdb/cache.h"
namespace ROCKSDB_NAMESPACE {
// Classifications of block cache entries, for reporting statistics
enum class CacheEntryRole {
// Block-based table data block
kDataBlock,
// Block-based table filter block (full or partitioned)
kFilterBlock,
// Block-based table metadata block for partitioned filter
kFilterMetaBlock,
// Block-based table deprecated filter block (old "block-based" filter)
kDeprecatedFilterBlock,
// Block-based table index block
kIndexBlock,
// Other kinds of block-based table block
kOtherBlock,
// WriteBufferManager resevations to account for memtable usage
kWriteBuffer,
// Default bucket, for miscellaneous cache entries. Do not use for
// entries that could potentially add up to large usage.
kMisc,
};
constexpr uint32_t kNumCacheEntryRoles =
static_cast<uint32_t>(CacheEntryRole::kMisc) + 1;
extern std::array<const char*, kNumCacheEntryRoles>
kCacheEntryRoleToCamelString;
extern std::array<const char*, 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).
std::unordered_map<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) {
delete static_cast<T*>(value);
}
};
template <CacheEntryRole R>
struct RegisteredNoopDeleter {
RegisteredNoopDeleter() { RegisterCacheDeleterRole(Delete, R); }
static void Delete(const Slice& /* key */, void* value) {
(void)value;
assert(value == nullptr);
}
};
} // 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

@ -0,0 +1,152 @@
// 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 "port/lang.h"
#include "rocksdb/cache.h"
#include "rocksdb/status.h"
#include "rocksdb/system_clock.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:
// Gathers stats and saves results into `stats`
void GetStats(Stats *stats, int maximum_age_in_seconds = 180) {
// Waits for any pending reader or writer (collector)
std::lock_guard<std::mutex> lock(mutex_);
// Maximum allowed age is nominally given by the parameter
uint64_t max_age_micros =
static_cast<uint64_t>(std::min(maximum_age_in_seconds, 0)) * 1000000U;
// But we will re-scan more frequently if it means scanning < 1%
// of the time and no more than once per second.
max_age_micros = std::min(
max_age_micros,
std::max(uint64_t{1000000},
100U * (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;
saved_stats_.BeginCollection(cache_, clock_, start_time_micros);
cache_->ApplyToAllEntries(saved_stats_.GetEntryCallback(), {});
uint64_t end_time_micros = clock_->NowMicros();
last_end_time_micros_ = end_time_micros;
saved_stats_.EndCollection(cache_, clock_, end_time_micros);
} else {
saved_stats_.SkippedCollection();
}
// Copy to caller
*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) {
std::array<uint64_t, 3> cache_key_data{
{// First 16 bytes == md5 of class name
0x7eba5a8fb5437c90U, 0x8ca68c9b11655855U,
// Last 8 bytes based on a function pointer to make unique for each
// template instantiation
reinterpret_cast<uint64_t>(&CacheEntryStatsCollector::GetShared)}};
Slice cache_key = GetSlice(&cache_key_data);
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);
if (!s.ok()) {
assert(h == nullptr);
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_(),
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);
}
std::mutex mutex_;
Stats saved_stats_;
uint64_t last_start_time_micros_;
uint64_t last_end_time_micros_;
Cache *const cache_;
SystemClock *const clock_;
};
} // namespace ROCKSDB_NAMESPACE

@ -111,4 +111,15 @@ class CacheHandleGuard {
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

@ -822,11 +822,12 @@ TEST_P(CacheTest, DefaultShardBits) {
ASSERT_EQ(6, sc->GetNumShardBits());
}
TEST_P(CacheTest, GetCharge) {
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);
}

@ -178,7 +178,7 @@ struct CacheHandle {
Slice key;
void* value;
size_t charge;
void (*deleter)(const Slice&, void* value);
Cache::DeleterFn deleter;
uint32_t hash;
// Addition to "charge" to get "total charge" under metadata policy.
@ -803,16 +803,14 @@ class ClockCache final : public ShardedCache {
return reinterpret_cast<const CacheHandle*>(handle)->hash;
}
DeleterFn GetDeleter(Handle* handle) const override {
return reinterpret_cast<const CacheHandle*>(handle)->deleter;
}
void DisownData() override {
#if defined(__clang__)
#if !defined(__has_feature) || !__has_feature(address_sanitizer)
#ifndef MUST_FREE_HEAP_ALLOCATIONS
shards_ = nullptr;
#endif
#else // __clang__
#ifndef __SANITIZE_ADDRESS__
shards_ = nullptr;
#endif // !__SANITIZE_ADDRESS__
#endif // __clang__
}
void WaitAll(std::vector<Handle*>& /*handles*/) override {}

18
cache/lru_cache.cc vendored

@ -625,23 +625,25 @@ size_t LRUCache::GetCharge(Handle* handle) const {
return reinterpret_cast<const LRUHandle*>(handle)->charge;
}
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;
}
}
uint32_t LRUCache::GetHash(Handle* handle) const {
return reinterpret_cast<const LRUHandle*>(handle)->hash;
}
void LRUCache::DisownData() {
// Do not drop data if compile with ASAN to suppress leak warning.
#if defined(__clang__)
#if !defined(__has_feature) || !__has_feature(address_sanitizer)
#ifndef MUST_FREE_HEAP_ALLOCATIONS
shards_ = nullptr;
num_shards_ = 0;
#endif
#else // __clang__
#ifndef __SANITIZE_ADDRESS__
shards_ = nullptr;
num_shards_ = 0;
#endif // !__SANITIZE_ADDRESS__
#endif // __clang__
}
size_t LRUCache::TEST_GetLRUSize() {

11
cache/lru_cache.h vendored

@ -53,7 +53,7 @@ struct LRUHandle {
union Info {
Info() {}
~Info() {}
void (*deleter)(const Slice&, void* value);
Cache::DeleterFn deleter;
const ShardedCache::CacheItemHelper* helper;
} info_;
LRUHandle* next_hash;
@ -267,8 +267,7 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard {
// Like Cache methods, but with an extra "hash" parameter.
virtual Status Insert(const Slice& key, uint32_t hash, void* value,
size_t charge,
void (*deleter)(const Slice& key, void* value),
size_t charge, Cache::DeleterFn deleter,
Cache::Handle** handle,
Cache::Priority priority) override {
return Insert(key, hash, value, charge, deleter, nullptr, handle, priority);
@ -329,9 +328,8 @@ class ALIGN_AS(CACHE_LINE_SIZE) LRUCacheShard final : public CacheShard {
private:
Status InsertItem(LRUHandle* item, Cache::Handle** handle);
Status Insert(const Slice& key, uint32_t hash, void* value, size_t charge,
void (*deleter)(const Slice& key, void* value),
const Cache::CacheItemHelper* helper, Cache::Handle** handle,
Cache::Priority priority);
DeleterFn deleter, const Cache::CacheItemHelper* helper,
Cache::Handle** handle, Cache::Priority priority);
void LRU_Remove(LRUHandle* e);
void LRU_Insert(LRUHandle* e);
@ -416,6 +414,7 @@ class LRUCache
virtual void* Value(Handle* handle) override;
virtual size_t GetCharge(Handle* handle) const override;
virtual uint32_t GetHash(Handle* handle) const override;
virtual DeleterFn GetDeleter(Handle* handle) const override;
virtual void DisownData() override;
virtual void WaitAll(std::vector<Handle*>& /*handles*/) override {}

@ -56,8 +56,8 @@ void ShardedCache::SetStrictCapacityLimit(bool strict_capacity_limit) {
}
Status ShardedCache::Insert(const Slice& key, void* value, size_t charge,
void (*deleter)(const Slice& key, void* value),
Handle** handle, Priority priority) {
DeleterFn deleter, Handle** handle,
Priority priority) {
uint32_t hash = HashSlice(key);
return GetShard(Shard(hash))
->Insert(key, hash, value, charge, deleter, handle, priority);

@ -73,15 +73,10 @@ class ShardedCache : public Cache {
ShardedCache(size_t capacity, int num_shard_bits, bool strict_capacity_limit,
std::shared_ptr<MemoryAllocator> memory_allocator = nullptr);
virtual ~ShardedCache() = default;
virtual const char* Name() const override = 0;
virtual CacheShard* GetShard(uint32_t shard) = 0;
virtual const CacheShard* GetShard(uint32_t shard) const = 0;
virtual void* Value(Handle* handle) override = 0;
virtual size_t GetCharge(Handle* handle) const override = 0;
virtual void WaitAll(std::vector<Handle*>& handles) override = 0;
virtual uint32_t GetHash(Handle* handle) const = 0;
virtual void DisownData() override = 0;
virtual void SetCapacity(size_t capacity) override;
virtual void SetStrictCapacityLimit(bool strict_capacity_limit) override;

@ -7,10 +7,13 @@
// 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 <cstdlib>
#include <memory>
#include "cache/cache_entry_roles.h"
#include "cache/lru_cache.h"
#include "db/db_test_util.h"
#include "port/stack_trace.h"
#include "rocksdb/table.h"
#include "util/compression.h"
#include "util/random.h"
@ -147,6 +150,16 @@ class DBBlockCacheTest : public DBTestBase {
compressed_insert_count_ = new_insert_count;
compressed_failure_count_ = new_failure_count;
}
#ifndef ROCKSDB_LITE
const std::array<size_t, kNumCacheEntryRoles>& GetCacheEntryRoleCounts() {
// Verify in cache entry role stats
ColumnFamilyHandleImpl* cfh =
static_cast<ColumnFamilyHandleImpl*>(dbfull()->DefaultColumnFamily());
InternalStats* internal_stats_ptr = cfh->cfd()->internal_stats();
return internal_stats_ptr->TEST_GetCacheEntryRoleStats().entry_counts;
}
#endif // ROCKSDB_LITE
};
TEST_F(DBBlockCacheTest, IteratorBlockCacheUsage) {
@ -889,6 +902,123 @@ TEST_F(DBBlockCacheTest, CacheCompressionDict) {
}
}
static void ClearCache(Cache* cache) {
std::deque<std::string> keys;
Cache::ApplyToAllEntriesOptions opts;
auto callback = [&](const Slice& key, void* /*value*/, size_t /*charge*/,
Cache::DeleterFn /*deleter*/) {
keys.push_back(key.ToString());
};
cache->ApplyToAllEntries(callback, opts);
for (auto& k : keys) {
cache->Erase(k);
}
}
TEST_F(DBBlockCacheTest, CacheEntryRoleStats) {
const size_t capacity = size_t{1} << 25;
int iterations_tested = 0;
for (bool partition : {false, true}) {
for (std::shared_ptr<Cache> cache :
{NewLRUCache(capacity), NewClockCache(capacity)}) {
if (!cache) {
// Skip clock cache when not supported
continue;
}
++iterations_tested;
Options options = CurrentOptions();
options.create_if_missing = true;
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
options.stats_dump_period_sec = 0;
options.max_open_files = 13;
options.table_cache_numshardbits = 0;
BlockBasedTableOptions table_options;
table_options.block_cache = cache;
table_options.cache_index_and_filter_blocks = true;
table_options.filter_policy.reset(NewBloomFilterPolicy(50));
if (partition) {
table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
table_options.partition_filters = true;
}
table_options.metadata_cache_options.top_level_index_pinning =
PinningTier::kNone;
table_options.metadata_cache_options.partition_pinning =
PinningTier::kNone;
table_options.metadata_cache_options.unpartitioned_pinning =
PinningTier::kNone;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
DestroyAndReopen(options);
// Create a new table.
ASSERT_OK(Put("foo", "value"));
ASSERT_OK(Put("bar", "value"));
ASSERT_OK(Flush());
ASSERT_OK(Put("zfoo", "value"));
ASSERT_OK(Put("zbar", "value"));
ASSERT_OK(Flush());
ASSERT_EQ(2, NumTableFilesAtLevel(0));
// Fresh cache
ClearCache(cache.get());
std::array<size_t, kNumCacheEntryRoles> expected{};
// For CacheEntryStatsCollector
expected[static_cast<size_t>(CacheEntryRole::kMisc)] = 1;
EXPECT_EQ(expected, GetCacheEntryRoleCounts());
// First access only filters
ASSERT_EQ("NOT_FOUND", Get("different from any key added"));
expected[static_cast<size_t>(CacheEntryRole::kFilterBlock)] += 2;
if (partition) {
expected[static_cast<size_t>(CacheEntryRole::kFilterMetaBlock)] += 2;
}
EXPECT_EQ(expected, GetCacheEntryRoleCounts());
// Now access index and data block
ASSERT_EQ("value", Get("foo"));
expected[static_cast<size_t>(CacheEntryRole::kIndexBlock)]++;
if (partition) {
// top-level
expected[static_cast<size_t>(CacheEntryRole::kIndexBlock)]++;
}
expected[static_cast<size_t>(CacheEntryRole::kDataBlock)]++;
EXPECT_EQ(expected, GetCacheEntryRoleCounts());
// The same for other file
ASSERT_EQ("value", Get("zfoo"));
expected[static_cast<size_t>(CacheEntryRole::kIndexBlock)]++;
if (partition) {
// top-level
expected[static_cast<size_t>(CacheEntryRole::kIndexBlock)]++;
}
expected[static_cast<size_t>(CacheEntryRole::kDataBlock)]++;
EXPECT_EQ(expected, GetCacheEntryRoleCounts());
// Also check the GetProperty interface
std::map<std::string, std::string> values;
ASSERT_TRUE(
db_->GetMapProperty(DB::Properties::kBlockCacheEntryStats, &values));
EXPECT_EQ(
ToString(expected[static_cast<size_t>(CacheEntryRole::kIndexBlock)]),
values["count.index-block"]);
EXPECT_EQ(
ToString(expected[static_cast<size_t>(CacheEntryRole::kDataBlock)]),
values["count.data-block"]);
EXPECT_EQ(
ToString(expected[static_cast<size_t>(CacheEntryRole::kFilterBlock)]),
values["count.filter-block"]);
EXPECT_EQ(ToString(expected[static_cast<size_t>(CacheEntryRole::kMisc)]),
values["count.misc"]);
}
EXPECT_GE(iterations_tested, 1);
}
}
#endif // ROCKSDB_LITE
class DBBlockCachePinningTest

@ -875,6 +875,10 @@ class CacheWrapper : public Cache {
return target_->GetCharge(handle);
}
DeleterFn GetDeleter(Handle* handle) const override {
return target_->GetDeleter(handle);
}
void ApplyToAllCacheEntries(void (*callback)(void*, size_t),
bool thread_safe) override {
target_->ApplyToAllCacheEntries(callback, thread_safe);

@ -12,15 +12,20 @@
#include <algorithm>
#include <cinttypes>
#include <cstddef>
#include <limits>
#include <sstream>
#include <string>
#include <utility>
#include <vector>
#include "cache/cache_entry_roles.h"
#include "cache/cache_entry_stats.h"
#include "db/column_family.h"
#include "db/db_impl/db_impl.h"
#include "rocksdb/system_clock.h"
#include "rocksdb/table.h"
#include "table/block_based/cachable_entry.h"
#include "util/string_util.h"
namespace ROCKSDB_NAMESPACE {
@ -220,6 +225,7 @@ static const std::string cfstats_no_file_histogram =
static const std::string cf_file_histogram = "cf-file-histogram";
static const std::string dbstats = "dbstats";
static const std::string levelstats = "levelstats";
static const std::string block_cache_entry_stats = "block-cache-entry-stats";
static const std::string num_immutable_mem_table = "num-immutable-mem-table";
static const std::string num_immutable_mem_table_flushed =
"num-immutable-mem-table-flushed";
@ -286,6 +292,8 @@ const std::string DB::Properties::kCFFileHistogram =
rocksdb_prefix + cf_file_histogram;
const std::string DB::Properties::kDBStats = rocksdb_prefix + dbstats;
const std::string DB::Properties::kLevelStats = rocksdb_prefix + levelstats;
const std::string DB::Properties::kBlockCacheEntryStats =
rocksdb_prefix + block_cache_entry_stats;
const std::string DB::Properties::kNumImmutableMemTable =
rocksdb_prefix + num_immutable_mem_table;
const std::string DB::Properties::kNumImmutableMemTableFlushed =
@ -385,6 +393,9 @@ const std::unordered_map<std::string, DBPropertyInfo>
nullptr}},
{DB::Properties::kDBStats,
{false, &InternalStats::HandleDBStats, nullptr, nullptr, nullptr}},
{DB::Properties::kBlockCacheEntryStats,
{false, &InternalStats::HandleBlockCacheEntryStats, nullptr,
&InternalStats::HandleBlockCacheEntryStatsMap, nullptr}},
{DB::Properties::kSSTables,
{false, &InternalStats::HandleSsTables, nullptr, nullptr, nullptr}},
{DB::Properties::kAggregatedTableProperties,
@ -503,6 +514,146 @@ const std::unordered_map<std::string, DBPropertyInfo>
&DBImpl::GetPropertyHandleOptionsStatistics}},
};
InternalStats::InternalStats(int num_levels, SystemClock* clock,
ColumnFamilyData* cfd)
: db_stats_{},
cf_stats_value_{},
cf_stats_count_{},
comp_stats_(num_levels),
comp_stats_by_pri_(Env::Priority::TOTAL),
file_read_latency_(num_levels),
bg_error_count_(0),
number_levels_(num_levels),
clock_(clock),
cfd_(cfd),
started_at_(clock->NowMicros()) {}
Status InternalStats::CollectCacheEntryStats() {
using Collector = CacheEntryStatsCollector<CacheEntryRoleStats>;
Cache* block_cache;
bool ok = HandleBlockCacheStat(&block_cache);
if (ok) {
// Extract or create stats collector.
std::shared_ptr<Collector> collector;
Status s = Collector::GetShared(block_cache, clock_, &collector);
if (s.ok()) {
collector->GetStats(&cache_entry_stats);
} else {
// Block cache likely under pressure. Scanning could make it worse,
// so skip.
}
return s;
} else {
return Status::NotFound("block cache not configured");
}
}
std::function<void(const Slice&, void*, size_t, Cache::DeleterFn)>
InternalStats::CacheEntryRoleStats::GetEntryCallback() {
return [&](const Slice& /*key*/, void* /*value*/, size_t charge,
Cache::DeleterFn deleter) {
auto e = role_map_.find(deleter);
size_t role_idx;
if (e == role_map_.end()) {
role_idx = static_cast<size_t>(CacheEntryRole::kMisc);
} else {
role_idx = static_cast<size_t>(e->second);
}
entry_counts[role_idx]++;
total_charges[role_idx] += charge;
};
}
void InternalStats::CacheEntryRoleStats::BeginCollection(
Cache* cache, SystemClock*, uint64_t start_time_micros) {
Clear();
last_start_time_micros_ = start_time_micros;
++collection_count;
role_map_ = CopyCacheDeleterRoleMap();
std::ostringstream str;
str << cache->Name() << "@" << static_cast<void*>(cache);
cache_id = str.str();
cache_capacity = cache->GetCapacity();
}
void InternalStats::CacheEntryRoleStats::EndCollection(
Cache*, SystemClock*, uint64_t end_time_micros) {
last_end_time_micros_ = end_time_micros;
}
void InternalStats::CacheEntryRoleStats::SkippedCollection() {
++copies_of_last_collection;
}
uint64_t InternalStats::CacheEntryRoleStats::GetLastDurationMicros() const {
if (last_end_time_micros_ > last_start_time_micros_) {
return last_end_time_micros_ - last_start_time_micros_;
} else {
return 0U;
}
}
std::string InternalStats::CacheEntryRoleStats::ToString(
SystemClock* clock) const {
std::ostringstream str;
str << "Block cache " << cache_id
<< " capacity: " << BytesToHumanString(cache_capacity)
<< " collections: " << collection_count
<< " last_copies: " << copies_of_last_collection
<< " last_secs: " << (GetLastDurationMicros() / 1000000.0)
<< " secs_since: "
<< ((clock->NowMicros() - last_end_time_micros_) / 1000000U) << "\n";
str << "Block cache entry stats(count,size,portion):";
for (size_t i = 0; i < kNumCacheEntryRoles; ++i) {
if (entry_counts[i] > 0) {
str << " " << kCacheEntryRoleToCamelString[i] << "(" << entry_counts[i]
<< "," << BytesToHumanString(total_charges[i]) << ","
<< (100.0 * total_charges[i] / cache_capacity) << "%)";
}
}
str << "\n";
return str.str();
}
void InternalStats::CacheEntryRoleStats::ToMap(
std::map<std::string, std::string>* values, SystemClock* clock) const {
values->clear();
auto& v = *values;
v["id"] = cache_id;
v["capacity"] = ROCKSDB_NAMESPACE::ToString(cache_capacity);
v["secs_for_last_collection"] =
ROCKSDB_NAMESPACE::ToString(GetLastDurationMicros() / 1000000.0);
v["secs_since_last_collection"] = ROCKSDB_NAMESPACE::ToString(
(clock->NowMicros() - last_end_time_micros_) / 1000000U);
for (size_t i = 0; i < kNumCacheEntryRoles; ++i) {
std::string role = kCacheEntryRoleToHyphenString[i];
v["count." + role] = ROCKSDB_NAMESPACE::ToString(entry_counts[i]);
v["bytes." + role] = ROCKSDB_NAMESPACE::ToString(total_charges[i]);
v["percent." + role] =
ROCKSDB_NAMESPACE::ToString(100.0 * total_charges[i] / cache_capacity);
}
}
bool InternalStats::HandleBlockCacheEntryStats(std::string* value,
Slice /*suffix*/) {
Status s = CollectCacheEntryStats();
if (!s.ok()) {
return false;
}
*value = cache_entry_stats.ToString(clock_);
return true;
}
bool InternalStats::HandleBlockCacheEntryStatsMap(
std::map<std::string, std::string>* values, Slice /*suffix*/) {
Status s = CollectCacheEntryStats();
if (!s.ok()) {
return false;
}
cache_entry_stats.ToMap(values, clock_);
return true;
}
const DBPropertyInfo* GetPropertyInfo(const Slice& property) {
std::string ppt_name = GetPropertyNameAndArg(property).first.ToString();
auto ppt_info_iter = InternalStats::ppt_name_to_info.find(ppt_name);
@ -1454,6 +1605,15 @@ void InternalStats::DumpCFStatsNoFileHistogram(std::string* value) {
cf_stats_snapshot_.ingest_keys_addfile = ingest_keys_addfile;
cf_stats_snapshot_.comp_stats = compaction_stats_sum;
cf_stats_snapshot_.stall_count = total_stall_count;
Status s = CollectCacheEntryStats();
if (s.ok()) {
value->append(cache_entry_stats.ToString(clock_));
} else {
value->append("Block cache: ");
value->append(s.ToString());
value->append("\n");
}
}
void InternalStats::DumpCFFileHistogram(std::string* value) {

@ -9,10 +9,13 @@
//
#pragma once
#include <map>
#include <memory>
#include <string>
#include <vector>
#include "cache/cache_entry_roles.h"
#include "db/version_set.h"
#include "rocksdb/system_clock.h"
@ -125,18 +128,7 @@ class InternalStats {
kIntStatsNumMax,
};
InternalStats(int num_levels, SystemClock* clock, ColumnFamilyData* cfd)
: db_stats_{},
cf_stats_value_{},
cf_stats_count_{},
comp_stats_(num_levels),
comp_stats_by_pri_(Env::Priority::TOTAL),
file_read_latency_(num_levels),
bg_error_count_(0),
number_levels_(num_levels),
clock_(clock),
cfd_(cfd),
started_at_(clock->NowMicros()) {}
InternalStats(int num_levels, SystemClock* clock, ColumnFamilyData* cfd);
// Per level compaction stats. comp_stats_[level] stores the stats for
// compactions that produced data for the specified "level".
@ -357,6 +349,39 @@ class InternalStats {
}
};
// For use with CacheEntryStatsCollector
struct CacheEntryRoleStats {
uint64_t cache_capacity = 0;
std::string cache_id;
std::array<uint64_t, kNumCacheEntryRoles> total_charges;
std::array<size_t, kNumCacheEntryRoles> entry_counts;
uint32_t collection_count = 0;
uint32_t copies_of_last_collection = 0;
uint64_t last_start_time_micros_ = 0;
uint64_t last_end_time_micros_ = 0;
void Clear() {
// Wipe everything except collection_count
uint32_t saved_collection_count = collection_count;
*this = CacheEntryRoleStats();
collection_count = saved_collection_count;
}
void BeginCollection(Cache*, SystemClock*, uint64_t start_time_micros);
std::function<void(const Slice&, void*, size_t, Cache::DeleterFn)>
GetEntryCallback();
void EndCollection(Cache*, SystemClock*, uint64_t end_time_micros);
void SkippedCollection();
std::string ToString(SystemClock* clock) const;
void ToMap(std::map<std::string, std::string>* values,
SystemClock* clock) const;
private:
std::unordered_map<Cache::DeleterFn, CacheEntryRole> role_map_;
uint64_t GetLastDurationMicros() const;
};
void Clear() {
for (int i = 0; i < kIntStatsNumMax; i++) {
db_stats_[i].store(0);
@ -365,6 +390,7 @@ class InternalStats {
cf_stats_count_[i] = 0;
cf_stats_value_[i] = 0;
}
cache_entry_stats.Clear();
for (auto& comp_stat : comp_stats_) {
comp_stat.Clear();
}
@ -437,6 +463,15 @@ class InternalStats {
return comp_stats_;
}
const CacheEntryRoleStats& TEST_GetCacheEntryRoleStats() {
Status s = CollectCacheEntryStats();
if (!s.ok()) {
assert(false);
cache_entry_stats.Clear();
}
return cache_entry_stats;
}
// Store a mapping from the user-facing DB::Properties string to our
// DBPropertyInfo struct used internally for retrieving properties.
static const std::unordered_map<std::string, DBPropertyInfo> ppt_name_to_info;
@ -457,11 +492,14 @@ class InternalStats {
bool HandleBlockCacheStat(Cache** block_cache);
Status CollectCacheEntryStats();
// Per-DB stats
std::atomic<uint64_t> db_stats_[kIntStatsNumMax];
// Per-ColumnFamily stats
uint64_t cf_stats_value_[INTERNAL_CF_STATS_ENUM_MAX];
uint64_t cf_stats_count_[INTERNAL_CF_STATS_ENUM_MAX];
CacheEntryRoleStats cache_entry_stats;
// Per-ColumnFamily/level compaction stats
std::vector<CompactionStats> comp_stats_;
std::vector<CompactionStats> comp_stats_by_pri_;
@ -629,6 +667,9 @@ class InternalStats {
bool HandleBlockCacheUsage(uint64_t* value, DBImpl* db, Version* version);
bool HandleBlockCachePinnedUsage(uint64_t* value, DBImpl* db,
Version* version);
bool HandleBlockCacheEntryStats(std::string* value, Slice suffix);
bool HandleBlockCacheEntryStatsMap(std::map<std::string, std::string>* values,
Slice suffix);
// Total number of background errors encountered. Every time a flush task
// or compaction task fails, this counter is incremented. The failure can
// be caused by any possible reason, including file system errors, out of

@ -336,6 +336,12 @@ class Cache {
// returns the charge for the specific entry in the cache.
virtual size_t GetCharge(Handle* handle) const = 0;
// Returns the deleter for the specified entry. This might seem useless
// as the Cache itself is responsible for calling the deleter, but
// the deleter can essentially verify that a cache entry is of an
// expected type from an expected code source.
virtual DeleterFn GetDeleter(Handle* handle) const = 0;
// Call this on shutdown if you want to speed it up. Cache will disown
// any underlying data and will not free it on delete. This call will leak
// memory - call this only if you're shutting down the process.

@ -763,6 +763,10 @@ class DB {
// of files per level and total size of each level (MB).
static const std::string kLevelStats;
// "rocksdb.block-cache-entry-stats" - returns a multi-line string or
// map with statistics on block cache usage.
static const std::string kBlockCacheEntryStats;
// "rocksdb.num-immutable-mem-table" - returns number of immutable
// memtables that have not yet been flushed.
static const std::string kNumImmutableMemTable;

@ -9,6 +9,7 @@
#include "rocksdb/write_buffer_manager.h"
#include "cache/cache_entry_roles.h"
#include "db/db_impl/db_impl.h"
#include "util/coding.h"
@ -107,9 +108,9 @@ void WriteBufferManager::ReserveMemWithCache(size_t mem) {
// Expand size by at least 256KB.
// Add a dummy record to the cache
Cache::Handle* handle = nullptr;
Status s =
cache_rep_->cache_->Insert(cache_rep_->GetNextCacheKey(), nullptr,
kSizeDummyEntry, nullptr, &handle);
Status s = cache_rep_->cache_->Insert(
cache_rep_->GetNextCacheKey(), nullptr, kSizeDummyEntry,
GetNoopDeleterForRole<CacheEntryRole::kWriteBuffer>(), &handle);
s.PermitUncheckedError(); // TODO: What to do on error?
// We keep the handle even if insertion fails and a null handle is
// returned, so that when memory shrinks, we don't release extra

@ -14,3 +14,28 @@
#define FALLTHROUGH_INTENDED do {} while (0)
#endif
#endif
#if defined(__clang__)
#if defined(__has_feature)
#if __has_feature(address_sanitizer)
#define MUST_FREE_HEAP_ALLOCATIONS 1
#endif // __has_feature(address_sanitizer)
#endif // defined(__has_feature)
#else // __clang__
#ifdef __SANITIZE_ADDRESS__
#define MUST_FREE_HEAP_ALLOCATIONS 1
#endif // __SANITIZE_ADDRESS__
#endif // __clang__
// Coding guidelines say to avoid static objects with non-trivial destructors,
// because it's easy to cause trouble (UB) in static destruction. This
// macro makes it easier to define static objects that are normally never
// destructed, except are destructed when running under ASAN. This should
// avoid unexpected, unnecessary destruction behavior in production.
// Note that constructor arguments can be provided as in
// STATIC_AVOID_DESTRUCTION(Foo, foo)(arg1, arg2);
#ifdef MUST_FREE_HEAP_ALLOCATIONS
#define STATIC_AVOID_DESTRUCTION(Type, name) static Type name
#else
#define STATIC_AVOID_DESTRUCTION(Type, name) static Type& name = *new Type
#endif

@ -1,6 +1,7 @@
# These are the sources from which librocksdb.a is built:
LIB_SOURCES = \
cache/cache.cc \
cache/cache_entry_roles.cc \
cache/clock_cache.cc \
cache/lru_cache.cc \
cache/sharded_cache.cc \

@ -15,6 +15,7 @@
#include <utility>
#include <vector>
#include "cache/cache_entry_roles.h"
#include "cache/sharded_cache.h"
#include "db/dbformat.h"
#include "db/pinned_iterators_manager.h"
@ -40,7 +41,9 @@
#include "table/block_based/block_based_filter_block.h"
#include "table/block_based/block_based_table_factory.h"
#include "table/block_based/block_based_table_iterator.h"
#include "table/block_based/block_like_traits.h"
#include "table/block_based/block_prefix_index.h"
#include "table/block_based/block_type.h"
#include "table/block_based/filter_block.h"
#include "table/block_based/full_filter_block.h"
#include "table/block_based/hash_index_reader.h"
@ -73,72 +76,6 @@ BlockBasedTable::~BlockBasedTable() {
std::atomic<uint64_t> BlockBasedTable::next_cache_key_id_(0);
template <typename TBlocklike>
class BlocklikeTraits;
template <>
class BlocklikeTraits<BlockContents> {
public:
static BlockContents* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool /* using_zstd */,
const FilterPolicy* /* filter_policy */) {
return new BlockContents(std::move(contents));
}
static uint32_t GetNumRestarts(const BlockContents& /* contents */) {
return 0;
}
};
template <>
class BlocklikeTraits<ParsedFullFilterBlock> {
public:
static ParsedFullFilterBlock* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool /* using_zstd */,
const FilterPolicy* filter_policy) {
return new ParsedFullFilterBlock(filter_policy, std::move(contents));
}
static uint32_t GetNumRestarts(const ParsedFullFilterBlock& /* block */) {
return 0;
}
};
template <>
class BlocklikeTraits<Block> {
public:
static Block* Create(BlockContents&& contents, size_t read_amp_bytes_per_bit,
Statistics* statistics, bool /* using_zstd */,
const FilterPolicy* /* filter_policy */) {
return new Block(std::move(contents), read_amp_bytes_per_bit, statistics);
}
static uint32_t GetNumRestarts(const Block& block) {
return block.NumRestarts();
}
};
template <>
class BlocklikeTraits<UncompressionDict> {
public:
static UncompressionDict* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool using_zstd,
const FilterPolicy* /* filter_policy */) {
return new UncompressionDict(contents.data, std::move(contents.allocation),
using_zstd);
}
static uint32_t GetNumRestarts(const UncompressionDict& /* dict */) {
return 0;
}
};
namespace {
// Read the block identified by "handle" from "file".
// The only relevant option is options.verify_checksums for now.
@ -173,13 +110,6 @@ Status ReadBlockFromFile(
return s;
}
// Delete the entry resided in the cache.
template <class Entry>
void DeleteCachedEntry(const Slice& /*key*/, void* value) {
auto entry = reinterpret_cast<Entry*>(value);
delete entry;
}
// Release the cached entry and decrement its ref count.
// Do not force erase
void ReleaseCachedEntry(void* arg, void* h) {
@ -1242,8 +1172,9 @@ Status BlockBasedTable::GetDataBlockFromCache(
read_options.fill_cache) {
size_t charge = block_holder->ApproximateMemoryUsage();
Cache::Handle* cache_handle = nullptr;
auto deleter = BlocklikeTraits<TBlocklike>::GetDeleter(block_type);
s = block_cache->Insert(block_cache_key, block_holder.get(), charge,
&DeleteCachedEntry<TBlocklike>, &cache_handle);
deleter, &cache_handle);
if (s.ok()) {
assert(cache_handle != nullptr);
block->SetCachedValue(block_holder.release(), block_cache,
@ -1330,10 +1261,10 @@ Status BlockBasedTable::PutDataBlockToCache(
// an object in the stack.
BlockContents* block_cont_for_comp_cache =
new BlockContents(std::move(*raw_block_contents));
auto deleter = BlocklikeTraits<BlockContents>::GetDeleter(block_type);
s = block_cache_compressed->Insert(
compressed_block_cache_key, block_cont_for_comp_cache,
block_cont_for_comp_cache->ApproximateMemoryUsage(),
&DeleteCachedEntry<BlockContents>);
block_cont_for_comp_cache->ApproximateMemoryUsage(), deleter);
if (s.ok()) {
// Avoid the following code to delete this cached block.
RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD);
@ -1347,9 +1278,9 @@ Status BlockBasedTable::PutDataBlockToCache(
if (block_cache != nullptr && block_holder->own_bytes()) {
size_t charge = block_holder->ApproximateMemoryUsage();
Cache::Handle* cache_handle = nullptr;
auto deleter = BlocklikeTraits<TBlocklike>::GetDeleter(block_type);
s = block_cache->Insert(block_cache_key, block_holder.get(), charge,
&DeleteCachedEntry<TBlocklike>, &cache_handle,
priority);
deleter, &cache_handle, priority);
if (s.ok()) {
assert(cache_handle != nullptr);
cached_block->SetCachedValue(block_holder.release(), block_cache,

@ -0,0 +1,128 @@
// 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 "cache/cache_entry_roles.h"
#include "port/lang.h"
#include "table/block_based/block.h"
#include "table/block_based/block_type.h"
#include "table/block_based/parsed_full_filter_block.h"
#include "table/format.h"
namespace ROCKSDB_NAMESPACE {
template <typename TBlocklike>
class BlocklikeTraits;
template <>
class BlocklikeTraits<BlockContents> {
public:
static BlockContents* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool /* using_zstd */,
const FilterPolicy* /* filter_policy */) {
return new BlockContents(std::move(contents));
}
static uint32_t GetNumRestarts(const BlockContents& /* contents */) {
return 0;
}
static Cache::DeleterFn GetDeleter(BlockType block_type) {
if (block_type == BlockType::kFilter) {
return GetCacheEntryDeleterForRole<
BlockContents, CacheEntryRole::kDeprecatedFilterBlock>();
} else {
// E.g. compressed cache
return GetCacheEntryDeleterForRole<BlockContents,
CacheEntryRole::kOtherBlock>();
}
}
};
template <>
class BlocklikeTraits<ParsedFullFilterBlock> {
public:
static ParsedFullFilterBlock* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool /* using_zstd */,
const FilterPolicy* filter_policy) {
return new ParsedFullFilterBlock(filter_policy, std::move(contents));
}
static uint32_t GetNumRestarts(const ParsedFullFilterBlock& /* block */) {
return 0;
}
static Cache::DeleterFn GetDeleter(BlockType block_type) {
(void)block_type;
assert(block_type == BlockType::kFilter);
return GetCacheEntryDeleterForRole<ParsedFullFilterBlock,
CacheEntryRole::kFilterBlock>();
}
};
template <>
class BlocklikeTraits<Block> {
public:
static Block* Create(BlockContents&& contents, size_t read_amp_bytes_per_bit,
Statistics* statistics, bool /* using_zstd */,
const FilterPolicy* /* filter_policy */) {
return new Block(std::move(contents), read_amp_bytes_per_bit, statistics);
}
static uint32_t GetNumRestarts(const Block& block) {
return block.NumRestarts();
}
static Cache::DeleterFn GetDeleter(BlockType block_type) {
switch (block_type) {
case BlockType::kData:
return GetCacheEntryDeleterForRole<Block, CacheEntryRole::kDataBlock>();
case BlockType::kIndex:
return GetCacheEntryDeleterForRole<Block,
CacheEntryRole::kIndexBlock>();
case BlockType::kFilter:
return GetCacheEntryDeleterForRole<Block,
CacheEntryRole::kFilterMetaBlock>();
default:
// Not a recognized combination
assert(false);
FALLTHROUGH_INTENDED;
case BlockType::kRangeDeletion:
return GetCacheEntryDeleterForRole<Block,
CacheEntryRole::kOtherBlock>();
}
}
};
template <>
class BlocklikeTraits<UncompressionDict> {
public:
static UncompressionDict* Create(BlockContents&& contents,
size_t /* read_amp_bytes_per_bit */,
Statistics* /* statistics */,
bool using_zstd,
const FilterPolicy* /* filter_policy */) {
return new UncompressionDict(contents.data, std::move(contents.allocation),
using_zstd);
}
static uint32_t GetNumRestarts(const UncompressionDict& /* dict */) {
return 0;
}
static Cache::DeleterFn GetDeleter(BlockType block_type) {
(void)block_type;
assert(block_type == BlockType::kCompressionDictionary);
return GetCacheEntryDeleterForRole<UncompressionDict,
CacheEntryRole::kOtherBlock>();
}
};
} // namespace ROCKSDB_NAMESPACE

@ -2925,6 +2925,8 @@ class Benchmark {
}
delete prefix_extractor_;
if (cache_.get() != nullptr) {
// Clear cache reference first
open_options_.write_buffer_manager.reset();
// this will leak, but we're shutting down so nobody cares
cache_->DisownData();
}

@ -245,6 +245,10 @@ class SimCacheImpl : public SimCache {
return cache_->GetCharge(handle);
}
DeleterFn GetDeleter(Handle* handle) const override {
return cache_->GetDeleter(handle);
}
size_t GetPinnedUsage() const override { return cache_->GetPinnedUsage(); }
void DisownData() override {

Loading…
Cancel
Save