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: 472f524a9691b5afb107934be2d41d84f2b129fbmain
parent
748e3acc11
commit
311a544c2a
@ -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
|
@ -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
|
Loading…
Reference in new issue