diff --git a/cache/lru_cache_test.cc b/cache/lru_cache_test.cc index 5a8f2e1d0..31e730f07 100644 --- a/cache/lru_cache_test.cc +++ b/cache/lru_cache_test.cc @@ -8,8 +8,13 @@ #include #include +#include "db/db_test_util.h" +#include "file/sst_file_manager_impl.h" #include "port/port.h" +#include "port/stack_trace.h" #include "rocksdb/cache.h" +#include "rocksdb/io_status.h" +#include "rocksdb/sst_file_manager.h" #include "test_util/testharness.h" #include "util/coding.h" #include "util/random.h" @@ -199,7 +204,7 @@ TEST_F(LRUCacheTest, EntriesWithPriority) { class TestSecondaryCache : public SecondaryCache { public: explicit TestSecondaryCache(size_t capacity) - : num_inserts_(0), num_lookups_(0) { + : num_inserts_(0), num_lookups_(0), inject_failure_(false) { cache_ = NewLRUCache(capacity, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); } @@ -207,8 +212,15 @@ class TestSecondaryCache : public SecondaryCache { std::string Name() override { return "TestSecondaryCache"; } + void InjectFailure() { inject_failure_ = true; } + + void ResetInjectFailure() { inject_failure_ = false; } + Status Insert(const Slice& key, void* value, const Cache::CacheItemHelper* helper) override { + if (inject_failure_) { + return Status::Corruption("Insertion Data Corrupted"); + } size_t size; char* buf; Status s; @@ -287,6 +299,13 @@ class TestSecondaryCache : public SecondaryCache { std::shared_ptr cache_; uint32_t num_inserts_; uint32_t num_lookups_; + bool inject_failure_; +}; + +class DBSecondaryCacheTest : public DBTestBase { + public: + DBSecondaryCacheTest() + : DBTestBase("/db_secondary_cache_test", /*env_do_fsync=*/true) {} }; class LRUSecondaryCacheTest : public LRUCacheTest { @@ -314,13 +333,13 @@ class LRUSecondaryCacheTest : public LRUCacheTest { return reinterpret_cast(obj)->Size(); } - static Status SaveToCallback(void* obj, size_t offset, size_t size, - void* out) { - TestItem* item = reinterpret_cast(obj); + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + TestItem* item = reinterpret_cast(from_obj); char* buf = item->Buf(); - EXPECT_EQ(size, item->Size()); - EXPECT_EQ(offset, 0); - memcpy(out, buf, size); + EXPECT_EQ(length, item->Size()); + EXPECT_EQ(from_offset, 0); + memcpy(out, buf, length); return Status::OK(); } @@ -547,6 +566,372 @@ TEST_F(LRUSecondaryCacheTest, FullCapacityTest) { cache.reset(); secondary_cache.reset(); } + +// In this test, the block cache size is set to 4096, after insert 6 KV-pairs +// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta +// blocks. block_1 size is 4096 and block_2 size is 2056. The total size +// of the meta blocks are about 900 to 1000. Therefore, in any situation, +// if we try to insert block_1 to the block cache, it will always fails. Only +// block_2 will be successfully inserted into the block cache. +TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness1) { + LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, + kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + std::shared_ptr secondary_cache( + new TestSecondaryCache(2048 * 1024)); + opts.secondary_cache = secondary_cache; + std::shared_ptr cache = NewLRUCache(opts); + BlockBasedTableOptions table_options; + table_options.block_cache = cache; + table_options.block_size = 4 * 1024; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + + // Set the file paranoid check, so after flush, the file will be read + // all the blocks will be accessed. + options.paranoid_file_checks = true; + DestroyAndReopen(options); + Random rnd(301); + const int N = 6; + for (int i = 0; i < N; i++) { + std::string p_v = rnd.RandomString(1007); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + // After Flush is successful, RocksDB do the paranoid check for the new + // SST file. Meta blocks are always cached in the block cache and they + // will not be evicted. When block_2 is cache miss and read out, it is + // inserted to the block cache. Note that, block_1 is never successfully + // inserted to the block cache. Here are 2 lookups in the secondary cache + // for block_1 and block_2 + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + Compact("a", "z"); + // Compaction will create the iterator to scan the whole file. So all the + // blocks are needed. Meta blocks are always cached. When block_1 is read + // out, block_2 is evicted from block cache and inserted to secondary + // cache. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 3u); + + std::string v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // The first data block is not in the cache, similarly, trigger the block + // cache Lookup and secondary cache lookup for block_1. But block_1 will not + // be inserted successfully due to the size. Currently, cache only has + // the meta blocks. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 4u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // The second data block is not in the cache, similarly, trigger the block + // cache Lookup and secondary cache lookup for block_2 and block_2 is found + // in the secondary cache. Now block cache has block_2 + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // block_2 is in the block cache. There is a block cache hit. No need to + // lookup or insert the secondary cache. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // Lookup the first data block, not in the block cache, so lookup the + // secondary cache. Also not in the secondary cache. After Get, still + // block_1 is will not be cached. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 6u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // Lookup the first data block, not in the block cache, so lookup the + // secondary cache. Also not in the secondary cache. After Get, still + // block_1 is will not be cached. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 7u); + + Destroy(options); +} + +// In this test, the block cache size is set to 5100, after insert 6 KV-pairs +// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta +// blocks. block_1 size is 4096 and block_2 size is 2056. The total size +// of the meta blocks are about 900 to 1000. Therefore, we can successfully +// insert and cache block_1 in the block cache (this is the different place +// from TestSecondaryCacheCorrectness1) +TEST_F(DBSecondaryCacheTest, TestSecondaryCacheCorrectness2) { + LRUCacheOptions opts(5100, 0, false, 0.5, nullptr, kDefaultToAdaptiveMutex, + kDontChargeCacheMetadata); + std::shared_ptr secondary_cache( + new TestSecondaryCache(2048 * 1024)); + opts.secondary_cache = secondary_cache; + std::shared_ptr cache = NewLRUCache(opts); + BlockBasedTableOptions table_options; + table_options.block_cache = cache; + table_options.block_size = 4 * 1024; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + options.paranoid_file_checks = true; + DestroyAndReopen(options); + Random rnd(301); + const int N = 6; + for (int i = 0; i < N; i++) { + std::string p_v = rnd.RandomString(1007); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + // After Flush is successful, RocksDB do the paranoid check for the new + // SST file. Meta blocks are always cached in the block cache and they + // will not be evicted. When block_2 is cache miss and read out, it is + // inserted to the block cache. Thefore, block_1 is evicted from block + // cache and successfully inserted to the secondary cache. Here are 2 + // lookups in the secondary cache for block_1 and block_2. + ASSERT_EQ(secondary_cache->num_inserts(), 1u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + Compact("a", "z"); + // Compaction will create the iterator to scan the whole file. So all the + // blocks are needed. After Flush, only block_2 is cached in block cache + // and block_1 is in the secondary cache. So when read block_1, it is + // read out from secondary cache and inserted to block cache. At the same + // time, block_2 is inserted to secondary cache. Now, secondary cache has + // both block_1 and block_2. After compaction, block_1 is in the cache. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 3u); + + std::string v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // This Get needs to access block_1, since block_1 is cached in block cache + // there is no secondary cache lookup. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 3u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // This Get needs to access block_2 which is not in the block cache. So + // it will lookup the secondary cache for block_2 and cache it in the + // block_cache. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 4u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // This Get needs to access block_2 which is already in the block cache. + // No need to lookup secondary cache. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 4u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // This Get needs to access block_1, since block_1 is not in block cache + // there is one econdary cache lookup. Then, block_1 is cached in the + // block cache. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // This Get needs to access block_1, since block_1 is cached in block cache + // there is no secondary cache lookup. + ASSERT_EQ(secondary_cache->num_inserts(), 2u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + Destroy(options); +} + +// The block cache size is set to 1024*1024, after insert 6 KV-pairs +// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta +// blocks. block_1 size is 4096 and block_2 size is 2056. The total size +// of the meta blocks are about 900 to 1000. Therefore, we can successfully +// cache all the blocks in the block cache and there is not secondary cache +// insertion. 2 lookup is needed for the blocks. +TEST_F(DBSecondaryCacheTest, NoSecondaryCacheInsertion) { + LRUCacheOptions opts(1024 * 1024, 0, false, 0.5, nullptr, + kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + std::shared_ptr secondary_cache( + new TestSecondaryCache(2048 * 1024)); + opts.secondary_cache = secondary_cache; + std::shared_ptr cache = NewLRUCache(opts); + BlockBasedTableOptions table_options; + table_options.block_cache = cache; + table_options.block_size = 4 * 1024; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.paranoid_file_checks = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + DestroyAndReopen(options); + Random rnd(301); + const int N = 6; + for (int i = 0; i < N; i++) { + std::string p_v = rnd.RandomString(1000); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + // After Flush is successful, RocksDB do the paranoid check for the new + // SST file. Meta blocks are always cached in the block cache and they + // will not be evicted. Now, block cache is large enough, it cache + // both block_1 and block_2. When first time read block_1 and block_2 + // there are cache misses. So 2 secondary cache lookups are needed for + // the 2 blocks + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + Compact("a", "z"); + // Compaction will iterate the whole SST file. Since all the data blocks + // are in the block cache. No need to lookup the secondary cache. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + std::string v = Get(Key(0)); + ASSERT_EQ(1000, v.size()); + // Since the block cache is large enough, all the blocks are cached. we + // do not need to lookup the seondary cache. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + Destroy(options); +} + +TEST_F(DBSecondaryCacheTest, SecondaryCacheIntensiveTesting) { + LRUCacheOptions opts(8 * 1024, 0, false, 0.5, nullptr, + kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + std::shared_ptr secondary_cache( + new TestSecondaryCache(2048 * 1024)); + opts.secondary_cache = secondary_cache; + std::shared_ptr cache = NewLRUCache(opts); + BlockBasedTableOptions table_options; + table_options.block_cache = cache; + table_options.block_size = 4 * 1024; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + DestroyAndReopen(options); + Random rnd(301); + const int N = 256; + for (int i = 0; i < N; i++) { + std::string p_v = rnd.RandomString(1000); + ASSERT_OK(Put(Key(i), p_v)); + } + ASSERT_OK(Flush()); + Compact("a", "z"); + + Random r_index(47); + std::string v; + for (int i = 0; i < 1000; i++) { + uint32_t key_i = r_index.Next() % N; + v = Get(Key(key_i)); + } + + // We have over 200 data blocks there will be multiple insertion + // and lookups. + ASSERT_GE(secondary_cache->num_inserts(), 1u); + ASSERT_GE(secondary_cache->num_lookups(), 1u); + + Destroy(options); +} + +// In this test, the block cache size is set to 4096, after insert 6 KV-pairs +// and flush, there are 5 blocks in this SST file, 2 data blocks and 3 meta +// blocks. block_1 size is 4096 and block_2 size is 2056. The total size +// of the meta blocks are about 900 to 1000. Therefore, in any situation, +// if we try to insert block_1 to the block cache, it will always fails. Only +// block_2 will be successfully inserted into the block cache. +TEST_F(DBSecondaryCacheTest, SecondaryCacheFailureTest) { + LRUCacheOptions opts(4 * 1024, 0, false, 0.5, nullptr, + kDefaultToAdaptiveMutex, kDontChargeCacheMetadata); + std::shared_ptr secondary_cache( + new TestSecondaryCache(2048 * 1024)); + opts.secondary_cache = secondary_cache; + std::shared_ptr cache = NewLRUCache(opts); + BlockBasedTableOptions table_options; + table_options.block_cache = cache; + table_options.block_size = 4 * 1024; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.paranoid_file_checks = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + DestroyAndReopen(options); + Random rnd(301); + const int N = 6; + for (int i = 0; i < N; i++) { + std::string p_v = rnd.RandomString(1007); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + // After Flush is successful, RocksDB do the paranoid check for the new + // SST file. Meta blocks are always cached in the block cache and they + // will not be evicted. When block_2 is cache miss and read out, it is + // inserted to the block cache. Note that, block_1 is never successfully + // inserted to the block cache. Here are 2 lookups in the secondary cache + // for block_1 and block_2 + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 2u); + + // Fail the insertion, in LRU cache, the secondary insertion returned status + // is not checked, therefore, the DB will not be influenced. + secondary_cache->InjectFailure(); + Compact("a", "z"); + // Compaction will create the iterator to scan the whole file. So all the + // blocks are needed. Meta blocks are always cached. When block_1 is read + // out, block_2 is evicted from block cache and inserted to secondary + // cache. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 3u); + + std::string v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // The first data block is not in the cache, similarly, trigger the block + // cache Lookup and secondary cache lookup for block_1. But block_1 will not + // be inserted successfully due to the size. Currently, cache only has + // the meta blocks. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 4u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // The second data block is not in the cache, similarly, trigger the block + // cache Lookup and secondary cache lookup for block_2 and block_2 is found + // in the secondary cache. Now block cache has block_2 + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + // block_2 is in the block cache. There is a block cache hit. No need to + // lookup or insert the secondary cache. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 5u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // Lookup the first data block, not in the block cache, so lookup the + // secondary cache. Also not in the secondary cache. After Get, still + // block_1 is will not be cached. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 6u); + + v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + // Lookup the first data block, not in the block cache, so lookup the + // secondary cache. Also not in the secondary cache. After Get, still + // block_1 is will not be cached. + ASSERT_EQ(secondary_cache->num_inserts(), 0u); + ASSERT_EQ(secondary_cache->num_lookups(), 7u); + secondary_cache->ResetInjectFailure(); + + Destroy(options); +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/db_block_cache_test.cc b/db/db_block_cache_test.cc index c55b8b83d..54e52aa50 100644 --- a/db/db_block_cache_test.cc +++ b/db/db_block_cache_test.cc @@ -460,15 +460,17 @@ class MockCache : public LRUCache { } using ShardedCache::Insert; - Status Insert(const Slice& key, void* value, size_t charge, - void (*deleter)(const Slice& key, void* value), Handle** handle, - Priority priority) override { + + Status Insert(const Slice& key, void* value, + const Cache::CacheItemHelper* helper_cb, size_t charge, + Handle** handle, Priority priority) override { + DeleterFn delete_cb = helper_cb->del_cb; if (priority == Priority::LOW) { low_pri_insert_count++; } else { high_pri_insert_count++; } - return LRUCache::Insert(key, value, charge, deleter, handle, priority); + return LRUCache::Insert(key, value, charge, delete_cb, handle, priority); } }; diff --git a/include/rocksdb/cache.h b/include/rocksdb/cache.h index 43b725630..d9d99e983 100644 --- a/include/rocksdb/cache.h +++ b/include/rocksdb/cache.h @@ -176,8 +176,8 @@ class Cache { // data into a buffer. The secondary cache may decide to not store it in a // contiguous buffer, in which case this callback will be called multiple // times with increasing offset - using SaveToCallback = Status (*)(void* obj, size_t offset, size_t size, - void* out); + using SaveToCallback = Status (*)(void* from_obj, size_t from_offset, + size_t length, void* out); // A function pointer type for custom destruction of an entry's // value. The Cache is responsible for copying and reclaiming space diff --git a/table/block_based/block_based_table_reader.cc b/table/block_based/block_based_table_reader.cc index 4eba2223a..c92b9d3de 100644 --- a/table/block_based/block_based_table_reader.cc +++ b/table/block_based/block_based_table_reader.cc @@ -354,8 +354,11 @@ void BlockBasedTable::UpdateCacheInsertionMetrics(BlockType block_type, Cache::Handle* BlockBasedTable::GetEntryFromCache( Cache* block_cache, const Slice& key, BlockType block_type, - GetContext* get_context) const { - auto cache_handle = block_cache->Lookup(key, rep_->ioptions.stats); + GetContext* get_context, const Cache::CacheItemHelper* cache_helper, + const Cache::CreateCallback& create_cb, Cache::Priority priority) const { + auto cache_handle = + block_cache->Lookup(key, cache_helper, create_cb, priority, true, + rep_->ioptions.statistics.get()); if (cache_handle != nullptr) { UpdateCacheHitMetrics(block_type, get_context, @@ -1107,15 +1110,30 @@ Status BlockBasedTable::GetDataBlockFromCache( : 0; assert(block); assert(block->IsEmpty()); + const Cache::Priority priority = + rep_->table_options.cache_index_and_filter_blocks_with_high_priority && + (block_type == BlockType::kFilter || + block_type == BlockType::kCompressionDictionary || + block_type == BlockType::kIndex) + ? Cache::Priority::HIGH + : Cache::Priority::LOW; Status s; BlockContents* compressed_block = nullptr; Cache::Handle* block_cache_compressed_handle = nullptr; + Statistics* statistics = rep_->ioptions.statistics.get(); + bool using_zstd = rep_->blocks_definitely_zstd_compressed; + const FilterPolicy* filter_policy = rep_->filter_policy; + Cache::CreateCallback create_cb = + GetCreateCallback(read_amp_bytes_per_bit, statistics, using_zstd, + filter_policy, *block->GetValue()); // Lookup uncompressed cache first if (block_cache != nullptr) { - auto cache_handle = GetEntryFromCache(block_cache, block_cache_key, - block_type, get_context); + auto cache_handle = GetEntryFromCache( + block_cache, block_cache_key, block_type, get_context, + BlocklikeTraits::GetCacheItemHelper(block_type), create_cb, + priority); if (cache_handle != nullptr) { block->SetCachedValue( reinterpret_cast(block_cache->Value(cache_handle)), @@ -1132,10 +1150,13 @@ Status BlockBasedTable::GetDataBlockFromCache( } assert(!compressed_block_cache_key.empty()); - block_cache_compressed_handle = - block_cache_compressed->Lookup(compressed_block_cache_key); - - Statistics* statistics = rep_->ioptions.stats; + BlockContents contents; + Cache::CreateCallback create_cb_special = GetCreateCallback( + read_amp_bytes_per_bit, statistics, using_zstd, filter_policy, contents); + block_cache_compressed_handle = block_cache_compressed->Lookup( + compressed_block_cache_key, + BlocklikeTraits::GetCacheItemHelper(block_type), + create_cb_special, priority, true); // if we found in the compressed cache, then uncompress and insert into // uncompressed cache @@ -1152,7 +1173,6 @@ Status BlockBasedTable::GetDataBlockFromCache( assert(compression_type != kNoCompression); // Retrieve the uncompressed contents into a new buffer - BlockContents contents; UncompressionContext context(compression_type); UncompressionInfo info(context, uncompression_dict, compression_type); s = UncompressBlockContents( @@ -1160,7 +1180,8 @@ Status BlockBasedTable::GetDataBlockFromCache( &contents, rep_->table_options.format_version, rep_->ioptions, GetMemoryAllocator(rep_->table_options)); - // Insert uncompressed block into block cache + // Insert uncompressed block into block cache, the priority is based on the + // data block type. if (s.ok()) { std::unique_ptr block_holder( BlocklikeTraits::Create( @@ -1172,9 +1193,10 @@ Status BlockBasedTable::GetDataBlockFromCache( read_options.fill_cache) { size_t charge = block_holder->ApproximateMemoryUsage(); Cache::Handle* cache_handle = nullptr; - auto deleter = BlocklikeTraits::GetDeleter(block_type); - s = block_cache->Insert(block_cache_key, block_holder.get(), charge, - deleter, &cache_handle); + s = block_cache->Insert( + block_cache_key, block_holder.get(), + BlocklikeTraits::GetCacheItemHelper(block_type), charge, + &cache_handle, priority); if (s.ok()) { assert(cache_handle != nullptr); block->SetCachedValue(block_holder.release(), block_cache, @@ -1261,10 +1283,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::GetDeleter(block_type); s = block_cache_compressed->Insert( compressed_block_cache_key, block_cont_for_comp_cache, - block_cont_for_comp_cache->ApproximateMemoryUsage(), deleter); + BlocklikeTraits::GetCacheItemHelper(block_type), + block_cont_for_comp_cache->ApproximateMemoryUsage()); if (s.ok()) { // Avoid the following code to delete this cached block. RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD); @@ -1278,9 +1300,10 @@ 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::GetDeleter(block_type); - s = block_cache->Insert(block_cache_key, block_holder.get(), charge, - deleter, &cache_handle, priority); + s = block_cache->Insert( + block_cache_key, block_holder.get(), + BlocklikeTraits::GetCacheItemHelper(block_type), charge, + &cache_handle, priority); if (s.ok()) { assert(cache_handle != nullptr); cached_block->SetCachedValue(block_holder.release(), block_cache, diff --git a/table/block_based/block_based_table_reader.h b/table/block_based/block_based_table_reader.h index 81ffd1cf0..2b51f817a 100644 --- a/table/block_based/block_based_table_reader.h +++ b/table/block_based/block_based_table_reader.h @@ -270,7 +270,10 @@ class BlockBasedTable : public TableReader { bool redundant) const; Cache::Handle* GetEntryFromCache(Cache* block_cache, const Slice& key, BlockType block_type, - GetContext* get_context) const; + GetContext* get_context, + const Cache::CacheItemHelper* cache_helper, + const Cache::CreateCallback& create_cb, + Cache::Priority priority) const; // Either Block::NewDataIterator() or Block::NewIndexIterator(). template diff --git a/table/block_based/block_like_traits.h b/table/block_based/block_like_traits.h index 33241a769..55562bdad 100644 --- a/table/block_based/block_like_traits.h +++ b/table/block_based/block_like_traits.h @@ -17,6 +17,29 @@ namespace ROCKSDB_NAMESPACE { template class BlocklikeTraits; +template +Cache::CacheItemHelper* GetCacheItemHelperForRole(); + +template +Cache::CreateCallback GetCreateCallback(size_t read_amp_bytes_per_bit, + Statistics* statistics, bool using_zstd, + const FilterPolicy* filter_policy, + const TBlocklike& /*block*/) { + return [read_amp_bytes_per_bit, statistics, using_zstd, filter_policy]( + void* buf, size_t size, void** out_obj, size_t* charge) -> Status { + assert(buf != nullptr); + std::unique_ptr buf_data(new char[size]()); + memcpy(buf_data.get(), buf, size); + BlockContents bc = BlockContents(std::move(buf_data), size); + TBlocklike* ucd_ptr = BlocklikeTraits::Create( + std::move(bc), read_amp_bytes_per_bit, statistics, using_zstd, + filter_policy); + *out_obj = reinterpret_cast(ucd_ptr); + *charge = size; + return Status::OK(); + }; +} + template <> class BlocklikeTraits { public: @@ -32,14 +55,31 @@ class BlocklikeTraits { return 0; } - static Cache::DeleterFn GetDeleter(BlockType block_type) { + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + BlockContents* ptr = static_cast(obj); + return ptr->data.size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + BlockContents* ptr = static_cast(from_obj); + const char* buf = ptr->data.data(); + assert(length == ptr->data.size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { if (block_type == BlockType::kFilter) { - return GetCacheEntryDeleterForRole< + return GetCacheItemHelperForRole< BlockContents, CacheEntryRole::kDeprecatedFilterBlock>(); } else { // E.g. compressed cache - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); } } }; @@ -59,11 +99,28 @@ class BlocklikeTraits { return 0; } - static Cache::DeleterFn GetDeleter(BlockType block_type) { + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + ParsedFullFilterBlock* ptr = static_cast(obj); + return ptr->GetBlockContentsData().size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + ParsedFullFilterBlock* ptr = static_cast(from_obj); + const char* buf = ptr->GetBlockContentsData().data(); + assert(length == ptr->GetBlockContentsData().size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { (void)block_type; assert(block_type == BlockType::kFilter); - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); } }; @@ -80,23 +137,38 @@ class BlocklikeTraits { return block.NumRestarts(); } - static Cache::DeleterFn GetDeleter(BlockType block_type) { + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + Block* ptr = static_cast(obj); + return ptr->size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + Block* ptr = static_cast(from_obj); + const char* buf = ptr->data(); + assert(length == ptr->size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { switch (block_type) { case BlockType::kData: - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); case BlockType::kIndex: - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); case BlockType::kFilter: - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); default: // Not a recognized combination assert(false); FALLTHROUGH_INTENDED; case BlockType::kRangeDeletion: - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); } } }; @@ -117,12 +189,38 @@ class BlocklikeTraits { return 0; } - static Cache::DeleterFn GetDeleter(BlockType block_type) { + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + UncompressionDict* ptr = static_cast(obj); + return ptr->slice_.size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + UncompressionDict* ptr = static_cast(from_obj); + const char* buf = ptr->slice_.data(); + assert(length == ptr->slice_.size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { (void)block_type; assert(block_type == BlockType::kCompressionDictionary); - return GetCacheEntryDeleterForRole(); + return GetCacheItemHelperForRole(); } }; +// Get an CacheItemHelper pointer for value type T and role R. +template +Cache::CacheItemHelper* GetCacheItemHelperForRole() { + static Cache::CacheItemHelper cache_helper( + BlocklikeTraits::SizeCallback, BlocklikeTraits::SaveToCallback, + GetCacheEntryDeleterForRole()); + return &cache_helper; +} + } // namespace ROCKSDB_NAMESPACE diff --git a/table/block_based/parsed_full_filter_block.h b/table/block_based/parsed_full_filter_block.h index 36c619921..95d7b5208 100644 --- a/table/block_based/parsed_full_filter_block.h +++ b/table/block_based/parsed_full_filter_block.h @@ -32,6 +32,8 @@ class ParsedFullFilterBlock { bool own_bytes() const { return block_contents_.own_bytes(); } + const Slice GetBlockContentsData() const { return block_contents_.data; } + private: BlockContents block_contents_; std::unique_ptr filter_bits_reader_;