Move the filter readers out of the block cache (#5504)

Summary:
Currently, when the block cache is used for the filter block, it is not
really the block itself that is stored in the cache but a FilterBlockReader
object. Since this object is not pure data (it has, for instance, pointers that
might dangle, including in one case a back pointer to the TableReader), it's not
really sharable. To avoid the issues around this, the current code erases the
cache entries when the TableReader is closed (which, BTW, is not sufficient
since a concurrent TableReader might have picked up the object in the meantime).
Instead of doing this, the patch moves the FilterBlockReader out of the cache
altogether, and decouples the filter reader object from the filter block.
In particular, instead of the TableReader owning, or caching/pinning the
FilterBlockReader (based on the customer's settings), with the change the
TableReader unconditionally owns the FilterBlockReader, which in turn
owns/caches/pins the filter block. This change also enables us to reuse the code
paths historically used for data blocks for filters as well.

Note:
Eviction statistics for filter blocks are temporarily broken. We plan to fix this in a
separate phase.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5504

Test Plan: make asan_check

Differential Revision: D16036974

Pulled By: ltamasi

fbshipit-source-id: 770f543c5fb4ed126fd1e04bfd3809cf4ff9c091
main
Levi Tamasi 5 years ago committed by Facebook Github Bot
parent cd2520361d
commit 3bde41b5a3
  1. 1
      CMakeLists.txt
  2. 3
      HISTORY.md
  3. 1
      TARGETS
  4. 14
      db/db_block_cache_test.cc
  5. 1
      src.mk
  6. 178
      table/block_based/block_based_filter_block.cc
  7. 45
      table/block_based/block_based_filter_block.h
  8. 322
      table/block_based/block_based_filter_block_test.cc
  9. 626
      table/block_based/block_based_table_reader.cc
  10. 69
      table/block_based/block_based_table_reader.h
  11. 1
      table/block_based/cachable_entry.h
  12. 66
      table/block_based/filter_block.h
  13. 90
      table/block_based/filter_block_reader_common.cc
  14. 54
      table/block_based/filter_block_reader_common.h
  15. 156
      table/block_based/full_filter_block.cc
  16. 62
      table/block_based/full_filter_block.h
  17. 204
      table/block_based/full_filter_block_test.cc
  18. 303
      table/block_based/partitioned_filter_block.cc
  19. 68
      table/block_based/partitioned_filter_block.h
  20. 118
      table/block_based/partitioned_filter_block_test.cc
  21. 3
      table/table_reader.h
  22. 6
      table/table_test.cc
  23. 3
      tools/sst_dump_tool.cc

@ -595,6 +595,7 @@ set(SOURCES
table/block_based/block_prefix_index.cc
table/block_based/data_block_hash_index.cc
table/block_based/data_block_footer.cc
table/block_based/filter_block_reader_common.cc
table/block_based/flush_block_policy.cc
table/block_based/full_filter_block.cc
table/block_based/index_builder.cc

@ -6,8 +6,9 @@
### Public API Change
* Now DB::Close() will return Aborted() error when there is unreleased snapshot. Users can retry after all snapshots are released.
* Index and filter blocks are now handled similarly to data blocks with regards to the block cache: instead of storing reader objects in the cache, only the blocks themselves are cached. In addition, index and filter blocks (as well as filter partitions) no longer get evicted from the cache when a table is closed. Moreover, index blocks can now use the compressed block cache (if any).
* Partitions of partitioned indexes no longer affect the read amplification statistics.
* Due to a refactoring, block cache eviction statistics for indexes are temporarily broken. We plan to reintroduce them in a later phase.
* Due to the above refactoring, block cache eviction statistics for indexes and filters are temporarily broken. We plan to reintroduce them in a later phase.
* options.keep_log_file_num will be enforced strictly all the time. File names of all log files will be tracked, which may take significantly amount of memory if options.keep_log_file_num is large and either of options.max_log_file_size or options.log_file_time_to_roll is set.
* Add initial support for Get/Put with user timestamps. Users can specify timestamps via ReadOptions and WriteOptions when calling DB::Get and DB::Put.
* Accessing a partition of a partitioned filter or index through a pinned reference is no longer considered a cache hit.

@ -192,6 +192,7 @@ cpp_library(
"table/block_based/block_prefix_index.cc",
"table/block_based/data_block_footer.cc",
"table/block_based/data_block_hash_index.cc",
"table/block_based/filter_block_reader_common.cc",
"table/block_based/flush_block_policy.cc",
"table/block_based/full_filter_block.cc",
"table/block_based/index_builder.cc",

@ -365,11 +365,11 @@ TEST_F(DBBlockCacheTest, IndexAndFilterBlocksStats) {
ASSERT_EQ(cache->GetUsage(), index_bytes_insert + filter_bytes_insert);
// set the cache capacity to the current usage
cache->SetCapacity(index_bytes_insert + filter_bytes_insert);
// The index eviction statistics were broken by the refactoring that moved
// the index readers out of the block cache. Disabling these until we can
// The index and filter eviction statistics were broken by the refactoring
// that moved the readers out of the block cache. Disabling these until we can
// bring the stats back.
// ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_INDEX_BYTES_EVICT), 0);
ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_FILTER_BYTES_EVICT), 0);
// ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_FILTER_BYTES_EVICT), 0);
// Note that the second key needs to be no longer than the first one.
// Otherwise the second index block may not fit in cache.
ASSERT_OK(Put(1, "key", "val"));
@ -380,13 +380,13 @@ TEST_F(DBBlockCacheTest, IndexAndFilterBlocksStats) {
index_bytes_insert);
ASSERT_GT(TestGetTickerCount(options, BLOCK_CACHE_FILTER_BYTES_INSERT),
filter_bytes_insert);
// The index eviction statistics were broken by the refactoring that moved
// the index readers out of the block cache. Disabling these until we can
// The index and filter eviction statistics were broken by the refactoring
// that moved the readers out of the block cache. Disabling these until we can
// bring the stats back.
// ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_INDEX_BYTES_EVICT),
// index_bytes_insert);
ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_FILTER_BYTES_EVICT),
filter_bytes_insert);
// ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_FILTER_BYTES_EVICT),
// filter_bytes_insert);
}
namespace {

@ -115,6 +115,7 @@ LIB_SOURCES = \
table/block_based/block_prefix_index.cc \
table/block_based/data_block_hash_index.cc \
table/block_based/data_block_footer.cc \
table/block_based/filter_block_reader_common.cc \
table/block_based/flush_block_policy.cc \
table/block_based/full_filter_block.cc \
table/block_based/index_builder.cc \

@ -13,6 +13,7 @@
#include "db/dbformat.h"
#include "monitoring/perf_context_imp.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "util/coding.h"
#include "util/string_util.h"
@ -162,58 +163,120 @@ void BlockBasedFilterBlockBuilder::GenerateFilter() {
}
BlockBasedFilterBlockReader::BlockBasedFilterBlockReader(
const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt, bool _whole_key_filtering,
BlockContents&& contents, Statistics* stats)
: FilterBlockReader(contents.data.size(), stats, _whole_key_filtering),
policy_(table_opt.filter_policy.get()),
prefix_extractor_(prefix_extractor),
data_(nullptr),
offset_(nullptr),
num_(0),
base_lg_(0),
contents_(std::move(contents)) {
assert(policy_);
size_t n = contents_.data.size();
if (n < 5) return; // 1 byte for base_lg_ and 4 for start of offset array
base_lg_ = contents_.data[n - 1];
uint32_t last_word = DecodeFixed32(contents_.data.data() + n - 5);
if (last_word > n - 5) return;
data_ = contents_.data.data();
offset_ = data_ + last_word;
num_ = (n - 5 - last_word) / 4;
const BlockBasedTable* t, CachableEntry<BlockContents>&& filter_block)
: FilterBlockReaderCommon(t, std::move(filter_block)) {
assert(table());
assert(table()->get_rep());
assert(table()->get_rep()->filter_policy);
}
std::unique_ptr<FilterBlockReader> BlockBasedFilterBlockReader::Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context) {
assert(table);
assert(table->get_rep());
assert(!pin || prefetch);
CachableEntry<BlockContents> filter_block;
if (prefetch || !use_cache) {
const Status s = ReadFilterBlock(table, prefetch_buffer, ReadOptions(),
nullptr /* get_context */, lookup_context,
&filter_block);
if (!s.ok()) {
return std::unique_ptr<FilterBlockReader>();
}
if (use_cache && !pin) {
filter_block.Reset();
}
}
return std::unique_ptr<FilterBlockReader>(
new BlockBasedFilterBlockReader(table, std::move(filter_block)));
}
bool BlockBasedFilterBlockReader::KeyMayMatch(
const Slice& key, const SliceTransform* /* prefix_extractor */,
uint64_t block_offset, const bool /*no_io*/,
const Slice* const /*const_ikey_ptr*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
assert(block_offset != kNotValid);
if (!whole_key_filtering_) {
if (!whole_key_filtering()) {
return true;
}
return MayMatch(key, block_offset);
return MayMatch(key, block_offset, no_io, get_context, lookup_context);
}
bool BlockBasedFilterBlockReader::PrefixMayMatch(
const Slice& prefix, const SliceTransform* /* prefix_extractor */,
uint64_t block_offset, const bool /*no_io*/,
const Slice* const /*const_ikey_ptr*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
assert(block_offset != kNotValid);
return MayMatch(prefix, block_offset);
return MayMatch(prefix, block_offset, no_io, get_context, lookup_context);
}
bool BlockBasedFilterBlockReader::ParseFieldsFromBlock(
const BlockContents& contents, const char** data, const char** offset,
size_t* num, size_t* base_lg) {
assert(data);
assert(offset);
assert(num);
assert(base_lg);
const size_t n = contents.data.size();
if (n < 5) { // 1 byte for base_lg and 4 for start of offset array
return false;
}
const uint32_t last_word = DecodeFixed32(contents.data.data() + n - 5);
if (last_word > n - 5) {
return false;
}
*data = contents.data.data();
*offset = (*data) + last_word;
*num = (n - 5 - last_word) / 4;
*base_lg = contents.data[n - 1];
return true;
}
bool BlockBasedFilterBlockReader::MayMatch(const Slice& entry,
uint64_t block_offset) {
uint64_t index = block_offset >> base_lg_;
if (index < num_) {
uint32_t start = DecodeFixed32(offset_ + index * 4);
uint32_t limit = DecodeFixed32(offset_ + index * 4 + 4);
if (start <= limit && limit <= (uint32_t)(offset_ - data_)) {
Slice filter = Slice(data_ + start, limit - start);
bool const may_match = policy_->KeyMayMatch(entry, filter);
bool BlockBasedFilterBlockReader::MayMatch(
const Slice& entry, uint64_t block_offset, bool no_io,
GetContext* get_context, BlockCacheLookupContext* lookup_context) const {
CachableEntry<BlockContents> filter_block;
const Status s =
GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block);
if (!s.ok()) {
return true;
}
assert(filter_block.GetValue());
const char* data = nullptr;
const char* offset = nullptr;
size_t num = 0;
size_t base_lg = 0;
if (!ParseFieldsFromBlock(*filter_block.GetValue(), &data, &offset, &num,
&base_lg)) {
return true; // Errors are treated as potential matches
}
const uint64_t index = block_offset >> base_lg;
if (index < num) {
const uint32_t start = DecodeFixed32(offset + index * 4);
const uint32_t limit = DecodeFixed32(offset + index * 4 + 4);
if (start <= limit && limit <= (uint32_t)(offset - data)) {
const Slice filter = Slice(data + start, limit - start);
assert(table());
assert(table()->get_rep());
const FilterPolicy* const policy = table()->get_rep()->filter_policy;
const bool may_match = policy->KeyMayMatch(entry, filter);
if (may_match) {
PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
return true;
@ -230,27 +293,54 @@ bool BlockBasedFilterBlockReader::MayMatch(const Slice& entry,
}
size_t BlockBasedFilterBlockReader::ApproximateMemoryUsage() const {
return num_ * 4 + 5 + (offset_ - data_);
size_t usage = ApproximateFilterBlockMemoryUsage();
#ifdef ROCKSDB_MALLOC_USABLE_SIZE
usage += malloc_usable_size(const_cast<BlockBasedFilterBlockReader*>(this));
#else
usage += sizeof(*this);
#endif // ROCKSDB_MALLOC_USABLE_SIZE
return usage;
}
std::string BlockBasedFilterBlockReader::ToString() const {
CachableEntry<BlockContents> filter_block;
const Status s =
GetOrReadFilterBlock(false /* no_io */, nullptr /* get_context */,
nullptr /* lookup_context */, &filter_block);
if (!s.ok()) {
return std::string("Unable to retrieve filter block");
}
assert(filter_block.GetValue());
const char* data = nullptr;
const char* offset = nullptr;
size_t num = 0;
size_t base_lg = 0;
if (!ParseFieldsFromBlock(*filter_block.GetValue(), &data, &offset, &num,
&base_lg)) {
return std::string("Error parsing filter block");
}
std::string result;
result.reserve(1024);
std::string s_bo("Block offset"), s_hd("Hex dump"), s_fb("# filter blocks");
AppendItem(&result, s_fb, rocksdb::ToString(num_));
AppendItem(&result, s_fb, rocksdb::ToString(num));
AppendItem(&result, s_bo, s_hd);
for (size_t index = 0; index < num_; index++) {
uint32_t start = DecodeFixed32(offset_ + index * 4);
uint32_t limit = DecodeFixed32(offset_ + index * 4 + 4);
for (size_t index = 0; index < num; index++) {
uint32_t start = DecodeFixed32(offset + index * 4);
uint32_t limit = DecodeFixed32(offset + index * 4 + 4);
if (start != limit) {
result.append(" filter block # " + rocksdb::ToString(index + 1) + "\n");
Slice filter = Slice(data_ + start, limit - start);
Slice filter = Slice(data + start, limit - start);
AppendItem(&result, start, filter.ToString(true));
}
}
return result;
}
} // namespace rocksdb

@ -22,7 +22,8 @@
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "table/block_based/filter_block.h"
#include "table/block_based/filter_block_reader_common.h"
#include "table/format.h"
#include "util/hash.h"
namespace rocksdb {
@ -75,42 +76,42 @@ class BlockBasedFilterBlockBuilder : public FilterBlockBuilder {
// A FilterBlockReader is used to parse filter from SST table.
// KeyMayMatch and PrefixMayMatch would trigger filter checking
class BlockBasedFilterBlockReader : public FilterBlockReader {
class BlockBasedFilterBlockReader
: public FilterBlockReaderCommon<BlockContents> {
public:
// REQUIRES: "contents" and *policy must stay live while *this is live.
BlockBasedFilterBlockReader(const SliceTransform* prefix_extractor,
const BlockBasedTableOptions& table_opt,
bool whole_key_filtering,
BlockContents&& contents, Statistics* statistics);
BlockBasedFilterBlockReader(const BlockBasedTable* t,
CachableEntry<BlockContents>&& filter_block);
static std::unique_ptr<FilterBlockReader> Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context);
bool IsBlockBased() override { return true; }
bool KeyMayMatch(const Slice& key, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
const Slice* const const_ikey_ptr, GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
bool PrefixMayMatch(const Slice& prefix,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
size_t ApproximateMemoryUsage() const override;
// convert this object to a human readable form
std::string ToString() const override;
private:
const FilterPolicy* policy_;
const SliceTransform* prefix_extractor_;
const char* data_; // Pointer to filter data (at block-start)
const char* offset_; // Pointer to beginning of offset array (at block-end)
size_t num_; // Number of entries in offset array
size_t base_lg_; // Encoding parameter (see kFilterBaseLg in .cc file)
BlockContents contents_;
static bool ParseFieldsFromBlock(const BlockContents& contents,
const char** data, const char** offset,
size_t* num, size_t* base_lg);
bool MayMatch(const Slice& entry, uint64_t block_offset);
// No copying allowed
BlockBasedFilterBlockReader(const BlockBasedFilterBlockReader&);
void operator=(const BlockBasedFilterBlockReader&);
bool MayMatch(const Slice& entry, uint64_t block_offset, bool no_io,
GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
};
} // namespace rocksdb

@ -10,6 +10,7 @@
#include "table/block_based/block_based_filter_block.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "test_util/testharness.h"
#include "test_util/testutil.h"
#include "util/coding.h"
@ -41,28 +42,58 @@ class TestHashFilter : public FilterPolicy {
}
};
class MockBlockBasedTable : public BlockBasedTable {
public:
explicit MockBlockBasedTable(Rep* rep)
: BlockBasedTable(rep, nullptr /* block_cache_tracer */) {
// Initialize what Open normally does as much as necessary for the test
rep->cache_key_prefix_size = 10;
}
};
class FilterBlockTest : public testing::Test {
public:
TestHashFilter policy_;
Options options_;
ImmutableCFOptions ioptions_;
EnvOptions env_options_;
BlockBasedTableOptions table_options_;
FilterBlockTest() {
table_options_.filter_policy.reset(new TestHashFilter());
InternalKeyComparator icomp_;
std::unique_ptr<BlockBasedTable> table_;
FilterBlockTest()
: ioptions_(options_),
env_options_(options_),
icomp_(options_.comparator) {
table_options_.no_block_cache = true;
table_options_.filter_policy.reset(new TestHashFilter);
constexpr bool skip_filters = false;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table)));
}
};
TEST_F(FilterBlockTest, EmptyBuilder) {
BlockBasedFilterBlockBuilder builder(nullptr, table_options_);
BlockContents block(builder.Finish());
ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block.data));
BlockBasedFilterBlockReader reader(nullptr, table_options_, true,
std::move(block), nullptr);
Slice slice(builder.Finish());
ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(slice));
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
BlockBasedFilterBlockReader reader(table_.get(), std::move(block));
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/100000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
TEST_F(FilterBlockTest, SingleChunk) {
@ -77,30 +108,46 @@ TEST_F(FilterBlockTest, SingleChunk) {
builder.StartBlock(300);
builder.Add("hello");
ASSERT_EQ(5, builder.NumAdded());
BlockContents block(builder.Finish());
BlockBasedFilterBlockReader reader(nullptr, table_options_, true,
std::move(block), nullptr);
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
Slice slice(builder.Finish());
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
BlockBasedFilterBlockReader reader(table_.get(), std::move(block));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("bar", /*prefix_extractor=*/nullptr,
/*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("box", /*prefix_extractor=*/nullptr,
/*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("hello", /*prefix_extractor=*/nullptr,
/*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"missing", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"other", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
TEST_F(FilterBlockTest, MultiChunk) {
@ -123,93 +170,139 @@ TEST_F(FilterBlockTest, MultiChunk) {
builder.Add("box");
builder.Add("hello");
BlockContents block(builder.Finish());
BlockBasedFilterBlockReader reader(nullptr, table_options_, true,
std::move(block), nullptr);
Slice slice(builder.Finish());
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
BlockBasedFilterBlockReader reader(table_.get(), std::move(block));
// Check first filter
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/2000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("bar", /*prefix_extractor=*/nullptr,
/*block_offset=*/2000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check second filter
ASSERT_TRUE(reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("box", /*prefix_extractor=*/nullptr,
/*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check third filter (empty)
ASSERT_TRUE(!reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check last filter
ASSERT_TRUE(reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("box", /*prefix_extractor=*/nullptr,
/*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("hello", /*prefix_extractor=*/nullptr,
/*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
// Test for block based filter block
// use new interface in FilterPolicy to create filter builder/reader
class BlockBasedFilterBlockTest : public testing::Test {
public:
Options options_;
ImmutableCFOptions ioptions_;
EnvOptions env_options_;
BlockBasedTableOptions table_options_;
BlockBasedFilterBlockTest() {
InternalKeyComparator icomp_;
std::unique_ptr<BlockBasedTable> table_;
BlockBasedFilterBlockTest()
: ioptions_(options_),
env_options_(options_),
icomp_(options_.comparator) {
table_options_.no_block_cache = true;
table_options_.filter_policy.reset(NewBloomFilterPolicy(10));
}
~BlockBasedFilterBlockTest() override {}
constexpr bool skip_filters = false;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table)));
}
};
TEST_F(BlockBasedFilterBlockTest, BlockBasedEmptyBuilder) {
FilterBlockBuilder* builder =
new BlockBasedFilterBlockBuilder(nullptr, table_options_);
BlockContents block(builder->Finish());
ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block.data));
FilterBlockReader* reader = new BlockBasedFilterBlockReader(
nullptr, table_options_, true, std::move(block), nullptr);
Slice slice(builder->Finish());
ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(slice));
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FilterBlockReader* reader =
new BlockBasedFilterBlockReader(table_.get(), std::move(block));
ASSERT_TRUE(reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/10000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
delete builder;
delete reader;
@ -226,30 +319,42 @@ TEST_F(BlockBasedFilterBlockTest, BlockBasedSingleChunk) {
builder->Add("box");
builder->StartBlock(300);
builder->Add("hello");
BlockContents block(builder->Finish());
FilterBlockReader* reader = new BlockBasedFilterBlockReader(
nullptr, table_options_, true, std::move(block), nullptr);
Slice slice(builder->Finish());
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FilterBlockReader* reader =
new BlockBasedFilterBlockReader(table_.get(), std::move(block));
ASSERT_TRUE(reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"missing", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"other", /*prefix_extractor=*/nullptr, /*block_offset=*/100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
delete builder;
delete reader;
@ -276,65 +381,86 @@ TEST_F(BlockBasedFilterBlockTest, BlockBasedMultiChunk) {
builder->Add("box");
builder->Add("hello");
BlockContents block(builder->Finish());
FilterBlockReader* reader = new BlockBasedFilterBlockReader(
nullptr, table_options_, true, std::move(block), nullptr);
Slice slice(builder->Finish());
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FilterBlockReader* reader =
new BlockBasedFilterBlockReader(table_.get(), std::move(block));
// Check first filter
ASSERT_TRUE(reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/2000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/uint64_t{0},
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check second filter
ASSERT_TRUE(reader->KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/3100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check third filter (empty)
ASSERT_TRUE(!reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/4100,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
// Check last filter
ASSERT_TRUE(reader->KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader->KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/9000,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
delete builder;
delete reader;

File diff suppressed because it is too large Load Diff

@ -172,8 +172,7 @@ class BlockBasedTable : public TableReader {
size_t ApproximateMemoryUsage() const override;
// convert SST file to a human readable form
Status DumpTable(WritableFile* out_file,
const SliceTransform* prefix_extractor = nullptr) override;
Status DumpTable(WritableFile* out_file) override;
Status VerifyChecksum(TableReaderCaller caller) override;
@ -181,7 +180,7 @@ class BlockBasedTable : public TableReader {
~BlockBasedTable();
bool TEST_filter_block_preloaded() const;
bool TEST_FilterBlockInCache() const;
bool TEST_IndexBlockInCache() const;
// IndexReader is the interface that provides the functionality for index
@ -241,6 +240,8 @@ class BlockBasedTable : public TableReader {
class PartitionedIndexIteratorState;
template <typename TBlocklike>
friend class FilterBlockReaderCommon;
friend class PartitionIndexReader;
protected:
@ -278,21 +279,23 @@ class BlockBasedTable : public TableReader {
// @param block_entry value is set to the uncompressed block if found. If
// in uncompressed block cache, also sets cache_handle to reference that
// block.
template <typename TBlocklike>
Status MaybeReadBlockAndLoadToCache(
FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
const BlockHandle& handle, const UncompressionDict& uncompression_dict,
CachableEntry<Block>* block_entry, BlockType block_type,
CachableEntry<TBlocklike>* block_entry, BlockType block_type,
GetContext* get_context, BlockCacheLookupContext* lookup_context,
BlockContents* contents) const;
// Similar to the above, with one crucial difference: it will retrieve the
// block from the file even if there are no caches configured (assuming the
// read options allow I/O).
template <typename TBlocklike>
Status RetrieveBlock(FilePrefetchBuffer* prefetch_buffer,
const ReadOptions& ro, const BlockHandle& handle,
const UncompressionDict& uncompression_dict,
CachableEntry<Block>* block_entry, BlockType block_type,
GetContext* get_context,
CachableEntry<TBlocklike>* block_entry,
BlockType block_type, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
bool for_compaction = false) const;
@ -310,19 +313,6 @@ class BlockBasedTable : public TableReader {
CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>* results,
char* scratch, const UncompressionDict& uncompression_dict) const;
// For the following two functions:
// if `no_io == true`, we will not try to read filter/index from sst file
// were they not present in cache yet.
CachableEntry<FilterBlockReader> GetFilter(
const SliceTransform* prefix_extractor,
FilePrefetchBuffer* prefetch_buffer, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
virtual CachableEntry<FilterBlockReader> GetFilter(
FilePrefetchBuffer* prefetch_buffer, const BlockHandle& filter_blk_handle,
const bool is_a_filter_partition, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
const SliceTransform* prefix_extractor) const;
CachableEntry<UncompressionDict> GetUncompressionDict(
FilePrefetchBuffer* prefetch_buffer, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
@ -348,12 +338,13 @@ class BlockBasedTable : public TableReader {
// pointer to the block as well as its block handle.
// @param uncompression_dict Data for presetting the compression library's
// dictionary.
template <typename TBlocklike>
Status GetDataBlockFromCache(
const Slice& block_cache_key, const Slice& compressed_block_cache_key,
Cache* block_cache, Cache* block_cache_compressed,
const ReadOptions& read_options, CachableEntry<Block>* block,
const ReadOptions& read_options, CachableEntry<TBlocklike>* block,
const UncompressionDict& uncompression_dict, BlockType block_type,
GetContext* get_context = nullptr) const;
GetContext* get_context) const;
// Put a raw block (maybe compressed) to the corresponding block caches.
// This method will perform decompression against raw_block if needed and then
@ -365,11 +356,12 @@ class BlockBasedTable : public TableReader {
// PutDataBlockToCache(). After the call, the object will be invalid.
// @param uncompression_dict Data for presetting the compression library's
// dictionary.
template <typename TBlocklike>
Status PutDataBlockToCache(
const Slice& block_cache_key, const Slice& compressed_block_cache_key,
Cache* block_cache, Cache* block_cache_compressed,
CachableEntry<Block>* cached_block, BlockContents* raw_block_contents,
CompressionType raw_block_comp_type,
CachableEntry<TBlocklike>* cached_block,
BlockContents* raw_block_contents, CompressionType raw_block_comp_type,
const UncompressionDict& uncompression_dict, SequenceNumber seq_no,
MemoryAllocator* memory_allocator, BlockType block_type,
GetContext* get_context) const;
@ -387,13 +379,14 @@ class BlockBasedTable : public TableReader {
Status CreateIndexReader(FilePrefetchBuffer* prefetch_buffer,
InternalIterator* preloaded_meta_index_iter,
bool use_cache, bool prefetch, bool pin,
IndexReader** index_reader,
BlockCacheLookupContext* lookup_context);
BlockCacheLookupContext* lookup_context,
std::unique_ptr<IndexReader>* index_reader);
bool FullFilterKeyMayMatch(const ReadOptions& read_options,
FilterBlockReader* filter, const Slice& user_key,
const bool no_io,
const SliceTransform* prefix_extractor,
GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
void FullFilterKeysMayMatch(const ReadOptions& read_options,
@ -435,10 +428,9 @@ class BlockBasedTable : public TableReader {
Status VerifyChecksumInBlocks(InternalIteratorBase<IndexValue>* index_iter);
// Create the filter from the filter block.
virtual FilterBlockReader* ReadFilter(
FilePrefetchBuffer* prefetch_buffer, const BlockHandle& filter_handle,
const bool is_a_filter_partition,
const SliceTransform* prefix_extractor = nullptr) const;
std::unique_ptr<FilterBlockReader> CreateFilterBlockReader(
FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
bool pin, BlockCacheLookupContext* lookup_context);
static void SetupCacheKeyPrefix(Rep* rep);
@ -516,17 +508,7 @@ struct BlockBasedTable::Rep {
// Footer contains the fixed table information
Footer footer;
// `filter` and `uncompression_dict` will be populated (i.e., non-nullptr)
// and used only when options.block_cache is nullptr or when
// `cache_index_and_filter_blocks == false`. Otherwise, we will get the
// filter and compression dictionary blocks via the block cache. In that case,
// `filter_handle`, and `compression_dict_handle` are used to lookup these
// meta-blocks in block cache.
//
// Note: the IndexReader object is always stored in this member variable;
// the index block itself, however, may or may not be in the block cache
// based on the settings above. We plan to change the handling of the
// filter and compression dictionary similarly.
std::unique_ptr<IndexReader> index_reader;
std::unique_ptr<FilterBlockReader> filter;
std::unique_ptr<UncompressionDict> uncompression_dict;
@ -553,13 +535,6 @@ struct BlockBasedTable::Rep {
std::unique_ptr<SliceTransform> internal_prefix_transform;
std::shared_ptr<const SliceTransform> table_prefix_extractor;
// only used in level 0 files when pin_l0_filter_and_index_blocks_in_cache is
// true or in all levels when pin_top_level_index_and_filter is set in
// combination with partitioned filters: then we do use the LRU cache,
// but we always keep the filter block's handle checked out here (=we
// don't call Release()), plus the parsed out objects the LRU cache will never
// push flush them out, hence they're pinned
CachableEntry<FilterBlockReader> filter_entry;
std::shared_ptr<const FragmentedRangeTombstoneList> fragmented_range_dels;
// If global_seqno is used, all Keys in this file will have the same

@ -10,6 +10,7 @@
#pragma once
#include <cassert>
#include "port/likely.h"
#include "rocksdb/cache.h"
#include "rocksdb/cleanable.h"

@ -38,6 +38,7 @@ namespace rocksdb {
const uint64_t kNotValid = ULLONG_MAX;
class FilterPolicy;
class GetContext;
using MultiGetRange = MultiGetContext::Range;
// A FilterBlockBuilder is used to construct all of the filters for a
@ -78,16 +79,14 @@ class FilterBlockBuilder {
// BlockBased/Full FilterBlock would be called in the same way.
class FilterBlockReader {
public:
explicit FilterBlockReader()
: whole_key_filtering_(true), size_(0), statistics_(nullptr) {}
explicit FilterBlockReader(size_t s, Statistics* stats,
bool _whole_key_filtering)
: whole_key_filtering_(_whole_key_filtering),
size_(s),
statistics_(stats) {}
virtual ~FilterBlockReader() {}
FilterBlockReader() = default;
virtual ~FilterBlockReader() = default;
FilterBlockReader(const FilterBlockReader&) = delete;
FilterBlockReader& operator=(const FilterBlockReader&) = delete;
virtual bool IsBlockBased() = 0; // If is blockbased filter
/**
* If no_io is set, then it returns true if it cannot answer the query without
* reading data from disk. This is used in PartitionedFilterBlockReader to
@ -102,17 +101,19 @@ class FilterBlockReader {
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) = 0;
GetContext* get_context,
BlockCacheLookupContext* lookup_context) = 0;
virtual void KeysMayMatch(MultiGetRange* range,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* context) {
BlockCacheLookupContext* lookup_context) {
for (auto iter = range->begin(); iter != range->end(); ++iter) {
const Slice ukey = iter->ukey;
const Slice ikey = iter->ikey;
GetContext* const get_context = iter->get_context;
if (!KeyMayMatch(ukey, prefix_extractor, block_offset, no_io, &ikey,
context)) {
get_context, lookup_context)) {
range->SkipKey(iter);
}
}
@ -125,27 +126,26 @@ class FilterBlockReader {
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) = 0;
GetContext* get_context,
BlockCacheLookupContext* lookup_context) = 0;
virtual void PrefixesMayMatch(MultiGetRange* range,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* context) {
BlockCacheLookupContext* lookup_context) {
for (auto iter = range->begin(); iter != range->end(); ++iter) {
const Slice ukey = iter->ukey;
const Slice ikey = iter->ikey;
GetContext* const get_context = iter->get_context;
if (!KeyMayMatch(prefix_extractor->Transform(ukey), prefix_extractor,
block_offset, no_io, &ikey, context)) {
block_offset, no_io, &ikey, get_context,
lookup_context)) {
range->SkipKey(iter);
}
}
}
virtual size_t ApproximateMemoryUsage() const = 0;
virtual size_t size() const { return size_; }
virtual Statistics* statistics() const { return statistics_; }
bool whole_key_filtering() const { return whole_key_filtering_; }
// convert this object to a human readable form
virtual std::string ToString() const {
@ -153,30 +153,22 @@ class FilterBlockReader {
return error_msg;
}
virtual void CacheDependencies(bool /*pin*/,
const SliceTransform* /*prefix_extractor*/) {}
virtual void CacheDependencies(bool /*pin*/) {}
virtual bool RangeMayExist(
const Slice* /*iterate_upper_bound*/, const Slice& user_key,
const SliceTransform* prefix_extractor, const Comparator* /*comparator*/,
const Slice* const const_ikey_ptr, bool* filter_checked,
bool /*need_upper_bound_check*/, BlockCacheLookupContext* context) {
virtual bool RangeMayExist(const Slice* /*iterate_upper_bound*/,
const Slice& user_key,
const SliceTransform* prefix_extractor,
const Comparator* /*comparator*/,
const Slice* const const_ikey_ptr,
bool* filter_checked,
bool /*need_upper_bound_check*/,
BlockCacheLookupContext* lookup_context) {
*filter_checked = true;
Slice prefix = prefix_extractor->Transform(user_key);
return PrefixMayMatch(prefix, prefix_extractor, kNotValid, false,
const_ikey_ptr, context);
const_ikey_ptr, /* get_context */ nullptr,
lookup_context);
}
protected:
bool whole_key_filtering_;
private:
// No copying allowed
FilterBlockReader(const FilterBlockReader&);
void operator=(const FilterBlockReader&);
size_t size_;
Statistics* statistics_;
int level_ = -1;
};
} // namespace rocksdb

@ -0,0 +1,90 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
#include "table/block_based/filter_block_reader_common.h"
#include "monitoring/perf_context_imp.h"
#include "table/block_based/block_based_table_reader.h"
namespace rocksdb {
template <typename TBlocklike>
Status FilterBlockReaderCommon<TBlocklike>::ReadFilterBlock(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
const ReadOptions& read_options, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<TBlocklike>* filter_block) {
PERF_TIMER_GUARD(read_filter_block_nanos);
assert(table);
assert(filter_block);
assert(filter_block->IsEmpty());
const BlockBasedTable::Rep* const rep = table->get_rep();
assert(rep);
const Status s =
table->RetrieveBlock(prefetch_buffer, read_options, rep->filter_handle,
UncompressionDict::GetEmptyDict(), filter_block,
BlockType::kFilter, get_context, lookup_context);
return s;
}
template <typename TBlocklike>
const SliceTransform*
FilterBlockReaderCommon<TBlocklike>::table_prefix_extractor() const {
assert(table_);
const BlockBasedTable::Rep* const rep = table_->get_rep();
assert(rep);
return rep->prefix_filtering ? rep->table_prefix_extractor.get() : nullptr;
}
template <typename TBlocklike>
bool FilterBlockReaderCommon<TBlocklike>::whole_key_filtering() const {
assert(table_);
assert(table_->get_rep());
return table_->get_rep()->whole_key_filtering;
}
template <typename TBlocklike>
Status FilterBlockReaderCommon<TBlocklike>::GetOrReadFilterBlock(
bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<TBlocklike>* filter_block) const {
assert(filter_block);
if (!filter_block_.IsEmpty()) {
filter_block->SetUnownedValue(filter_block_.GetValue());
return Status::OK();
}
ReadOptions read_options;
if (no_io) {
read_options.read_tier = kBlockCacheTier;
}
return ReadFilterBlock(table_, nullptr /* prefetch_buffer */, read_options,
get_context, lookup_context, filter_block);
}
template <typename TBlocklike>
size_t FilterBlockReaderCommon<TBlocklike>::ApproximateFilterBlockMemoryUsage()
const {
assert(!filter_block_.GetOwnValue() || filter_block_.GetValue() != nullptr);
return filter_block_.GetOwnValue()
? filter_block_.GetValue()->ApproximateMemoryUsage()
: 0;
}
// Explicitly instantiate templates for both "blocklike" types we use.
// This makes it possible to keep the template definitions in the .cc file.
template class FilterBlockReaderCommon<BlockContents>;
template class FilterBlockReaderCommon<Block>;
} // namespace rocksdb

@ -0,0 +1,54 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
#pragma once
#include <cassert>
#include "table/block_based/cachable_entry.h"
#include "table/block_based/filter_block.h"
namespace rocksdb {
class BlockBasedTable;
class FilePrefetchBuffer;
// Encapsulates common functionality for the various filter block reader
// implementations. Provides access to the filter block regardless of whether
// it is owned by the reader or stored in the cache, or whether it is pinned
// in the cache or not.
template <typename TBlocklike>
class FilterBlockReaderCommon : public FilterBlockReader {
public:
FilterBlockReaderCommon(const BlockBasedTable* t,
CachableEntry<TBlocklike>&& filter_block)
: table_(t), filter_block_(std::move(filter_block)) {
assert(table_);
}
protected:
static Status ReadFilterBlock(const BlockBasedTable* table,
FilePrefetchBuffer* prefetch_buffer,
const ReadOptions& read_options,
GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<TBlocklike>* filter_block);
const BlockBasedTable* table() const { return table_; }
const SliceTransform* table_prefix_extractor() const;
bool whole_key_filtering() const;
Status GetOrReadFilterBlock(bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<TBlocklike>* filter_block) const;
size_t ApproximateFilterBlockMemoryUsage() const;
private:
const BlockBasedTable* table_;
CachableEntry<TBlocklike> filter_block_;
};
} // namespace rocksdb

@ -16,6 +16,7 @@
#include "monitoring/perf_context_imp.h"
#include "port/port.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "util/coding.h"
namespace rocksdb {
@ -98,59 +99,91 @@ Slice FullFilterBlockBuilder::Finish(const BlockHandle& /*tmp*/,
}
FullFilterBlockReader::FullFilterBlockReader(
const SliceTransform* prefix_extractor, bool _whole_key_filtering,
const Slice& contents, FilterBitsReader* filter_bits_reader,
Statistics* stats)
: FilterBlockReader(contents.size(), stats, _whole_key_filtering),
prefix_extractor_(prefix_extractor),
contents_(contents) {
assert(filter_bits_reader != nullptr);
filter_bits_reader_.reset(filter_bits_reader);
if (prefix_extractor_ != nullptr) {
const BlockBasedTable* t, CachableEntry<BlockContents>&& filter_block)
: FilterBlockReaderCommon(t, std::move(filter_block)) {
const SliceTransform* const prefix_extractor = table_prefix_extractor();
if (prefix_extractor) {
full_length_enabled_ =
prefix_extractor_->FullLengthEnabled(&prefix_extractor_full_length_);
prefix_extractor->FullLengthEnabled(&prefix_extractor_full_length_);
}
}
FullFilterBlockReader::FullFilterBlockReader(
const SliceTransform* prefix_extractor, bool _whole_key_filtering,
BlockContents&& contents, FilterBitsReader* filter_bits_reader,
Statistics* stats)
: FullFilterBlockReader(prefix_extractor, _whole_key_filtering,
contents.data, filter_bits_reader, stats) {
block_contents_ = std::move(contents);
}
bool FullFilterBlockReader::KeyMayMatch(
const Slice& key, const SliceTransform* /*prefix_extractor*/,
uint64_t block_offset, const bool /*no_io*/,
const Slice* const /*const_ikey_ptr*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
if (!whole_key_filtering_) {
if (!whole_key_filtering()) {
return true;
}
return MayMatch(key);
return MayMatch(key, no_io, get_context, lookup_context);
}
std::unique_ptr<FilterBlockReader> FullFilterBlockReader::Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context) {
assert(table);
assert(table->get_rep());
assert(!pin || prefetch);
CachableEntry<BlockContents> filter_block;
if (prefetch || !use_cache) {
const Status s = ReadFilterBlock(table, prefetch_buffer, ReadOptions(),
nullptr /* get_context */, lookup_context,
&filter_block);
if (!s.ok()) {
return std::unique_ptr<FilterBlockReader>();
}
if (use_cache && !pin) {
filter_block.Reset();
}
}
return std::unique_ptr<FilterBlockReader>(
new FullFilterBlockReader(table, std::move(filter_block)));
}
bool FullFilterBlockReader::PrefixMayMatch(
const Slice& prefix, const SliceTransform* /* prefix_extractor */,
uint64_t block_offset, const bool /*no_io*/,
const Slice* const /*const_ikey_ptr*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(block_offset == kNotValid);
return MayMatch(prefix);
return MayMatch(prefix, no_io, get_context, lookup_context);
}
bool FullFilterBlockReader::MayMatch(const Slice& entry) {
if (contents_.size() != 0) {
if (filter_bits_reader_->MayMatch(entry)) {
bool FullFilterBlockReader::MayMatch(
const Slice& entry, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const {
CachableEntry<BlockContents> filter_block;
const Status s =
GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block);
if (!s.ok()) {
return true;
}
assert(filter_block.GetValue());
if (filter_block.GetValue()->data.size() != 0) {
assert(table());
assert(table()->get_rep());
std::unique_ptr<FilterBitsReader> filter_bits_reader(
table()->get_rep()->filter_policy->GetFilterBitsReader(
filter_block.GetValue()->data));
assert(filter_bits_reader != nullptr);
if (filter_bits_reader->MayMatch(entry)) {
PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
return true;
} else {
@ -163,38 +196,58 @@ bool FullFilterBlockReader::MayMatch(const Slice& entry) {
void FullFilterBlockReader::KeysMayMatch(
MultiGetRange* range, const SliceTransform* /*prefix_extractor*/,
uint64_t block_offset, const bool /*no_io*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)range;
(void)block_offset;
#endif
assert(block_offset == kNotValid);
if (!whole_key_filtering_) {
if (!whole_key_filtering()) {
// Simply return. Don't skip any key - consider all keys as likely to be
// present
return;
}
MayMatch(range);
MayMatch(range, no_io, lookup_context);
}
void FullFilterBlockReader::PrefixesMayMatch(
MultiGetRange* range, const SliceTransform* /* prefix_extractor */,
uint64_t block_offset, const bool /*no_io*/,
BlockCacheLookupContext* /*context*/) {
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)range;
(void)block_offset;
#endif
assert(block_offset == kNotValid);
MayMatch(range);
MayMatch(range, no_io, lookup_context);
}
void FullFilterBlockReader::MayMatch(MultiGetRange* range) {
if (contents_.size() == 0) {
void FullFilterBlockReader::MayMatch(
MultiGetRange* range, bool no_io,
BlockCacheLookupContext* lookup_context) const {
CachableEntry<BlockContents> filter_block;
const Status s = GetOrReadFilterBlock(no_io, range->begin()->get_context,
lookup_context, &filter_block);
if (!s.ok()) {
return;
}
assert(filter_block.GetValue());
if (filter_block.GetValue()->data.size() == 0) {
return;
}
assert(table());
assert(table()->get_rep());
std::unique_ptr<FilterBitsReader> filter_bits_reader(
table()->get_rep()->filter_policy->GetFilterBitsReader(
filter_block.GetValue()->data));
assert(filter_bits_reader != nullptr);
// We need to use an array instead of autovector for may_match since
// &may_match[0] doesn't work for autovector<bool> (compiler error). So
// declare both keys and may_match as arrays, which is also slightly less
@ -205,7 +258,7 @@ void FullFilterBlockReader::MayMatch(MultiGetRange* range) {
for (auto iter = range->begin(); iter != range->end(); ++iter) {
keys[num_keys++] = &iter->ukey;
}
filter_bits_reader_->MayMatch(num_keys, &keys[0], &may_match[0]);
filter_bits_reader->MayMatch(num_keys, &keys[0], &may_match[0]);
int i = 0;
for (auto iter = range->begin(); iter != range->end(); ++iter) {
@ -217,13 +270,11 @@ void FullFilterBlockReader::MayMatch(MultiGetRange* range) {
}
size_t FullFilterBlockReader::ApproximateMemoryUsage() const {
size_t usage = block_contents_.usable_size();
size_t usage = ApproximateFilterBlockMemoryUsage();
#ifdef ROCKSDB_MALLOC_USABLE_SIZE
usage += malloc_usable_size((void*)this);
usage += malloc_usable_size(filter_bits_reader_.get());
usage += malloc_usable_size(const_cast<FullFilterBlockReader*>(this));
#else
usage += sizeof(*this);
usage += sizeof(*filter_bits_reader_.get());
#endif // ROCKSDB_MALLOC_USABLE_SIZE
return usage;
}
@ -232,7 +283,7 @@ bool FullFilterBlockReader::RangeMayExist(
const Slice* iterate_upper_bound, const Slice& user_key,
const SliceTransform* prefix_extractor, const Comparator* comparator,
const Slice* const const_ikey_ptr, bool* filter_checked,
bool need_upper_bound_check, BlockCacheLookupContext* context) {
bool need_upper_bound_check, BlockCacheLookupContext* lookup_context) {
if (!prefix_extractor || !prefix_extractor->InDomain(user_key)) {
*filter_checked = false;
return true;
@ -245,22 +296,23 @@ bool FullFilterBlockReader::RangeMayExist(
} else {
*filter_checked = true;
return PrefixMayMatch(prefix, prefix_extractor, kNotValid, false,
const_ikey_ptr, context);
const_ikey_ptr, /* get_context */ nullptr,
lookup_context);
}
}
bool FullFilterBlockReader::IsFilterCompatible(
const Slice* iterate_upper_bound, const Slice& prefix,
const Comparator* comparator) {
const Comparator* comparator) const {
// Try to reuse the bloom filter in the SST table if prefix_extractor in
// mutable_cf_options has changed. If range [user_key, upper_bound) all
// share the same prefix then we may still be able to use the bloom filter.
if (iterate_upper_bound != nullptr && prefix_extractor_) {
if (!prefix_extractor_->InDomain(*iterate_upper_bound)) {
const SliceTransform* const prefix_extractor = table_prefix_extractor();
if (iterate_upper_bound != nullptr && prefix_extractor) {
if (!prefix_extractor->InDomain(*iterate_upper_bound)) {
return false;
}
Slice upper_bound_xform =
prefix_extractor_->Transform(*iterate_upper_bound);
Slice upper_bound_xform = prefix_extractor->Transform(*iterate_upper_bound);
// first check if user_key and upper_bound all share the same prefix
if (!comparator->Equal(prefix, upper_bound_xform)) {
// second check if user_key's prefix is the immediate predecessor of

@ -15,7 +15,8 @@
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "table/block_based/filter_block.h"
#include "table/block_based/filter_block_reader_common.h"
#include "table/format.h"
#include "util/hash.h"
namespace rocksdb {
@ -78,71 +79,58 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
// A FilterBlockReader is used to parse filter from SST table.
// KeyMayMatch and PrefixMayMatch would trigger filter checking
class FullFilterBlockReader : public FilterBlockReader {
class FullFilterBlockReader : public FilterBlockReaderCommon<BlockContents> {
public:
// REQUIRES: "contents" and filter_bits_reader must stay live
// while *this is live.
explicit FullFilterBlockReader(const SliceTransform* prefix_extractor,
bool whole_key_filtering,
const Slice& contents,
FilterBitsReader* filter_bits_reader,
Statistics* statistics);
explicit FullFilterBlockReader(const SliceTransform* prefix_extractor,
bool whole_key_filtering,
BlockContents&& contents,
FilterBitsReader* filter_bits_reader,
Statistics* statistics);
// bits_reader is created in filter_policy, it should be passed in here
// directly. and be deleted here
~FullFilterBlockReader() override {}
FullFilterBlockReader(const BlockBasedTable* t,
CachableEntry<BlockContents>&& filter_block);
static std::unique_ptr<FilterBlockReader> Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context);
bool IsBlockBased() override { return false; }
bool KeyMayMatch(const Slice& key, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
const Slice* const const_ikey_ptr, GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
bool PrefixMayMatch(const Slice& prefix,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
void KeysMayMatch(MultiGetRange* range,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* context) override;
BlockCacheLookupContext* lookup_context) override;
void PrefixesMayMatch(MultiGetRange* range,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
BlockCacheLookupContext* context) override;
BlockCacheLookupContext* lookup_context) override;
size_t ApproximateMemoryUsage() const override;
bool RangeMayExist(const Slice* iterate_upper_bound, const Slice& user_key,
const SliceTransform* prefix_extractor,
const Comparator* comparator,
const Slice* const const_ikey_ptr, bool* filter_checked,
bool need_upper_bound_check,
BlockCacheLookupContext* context) override;
BlockCacheLookupContext* lookup_context) override;
private:
bool MayMatch(const Slice& entry, bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
void MayMatch(MultiGetRange* range, bool no_io,
BlockCacheLookupContext* lookup_context) const;
bool IsFilterCompatible(const Slice* iterate_upper_bound, const Slice& prefix,
const Comparator* comparator) const;
private:
const SliceTransform* prefix_extractor_;
Slice contents_;
std::unique_ptr<FilterBitsReader> filter_bits_reader_;
BlockContents block_contents_;
bool full_length_enabled_;
size_t prefix_extractor_full_length_;
// No copying allowed
FullFilterBlockReader(const FullFilterBlockReader&);
bool MayMatch(const Slice& entry);
void MayMatch(MultiGetRange* range);
void operator=(const FullFilterBlockReader&);
bool IsFilterCompatible(const Slice* iterate_upper_bound,
const Slice& prefix, const Comparator* comparator);
};
} // namespace rocksdb

@ -6,6 +6,7 @@
#include "table/block_based/full_filter_block.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "table/full_filter_bits_builder.h"
#include "test_util/testharness.h"
#include "test_util/testutil.h"
@ -40,6 +41,15 @@ class TestFilterBitsBuilder : public FilterBitsBuilder {
std::vector<uint32_t> hash_entries_;
};
class MockBlockBasedTable : public BlockBasedTable {
public:
explicit MockBlockBasedTable(Rep* rep)
: BlockBasedTable(rep, nullptr /* block_cache_tracer */) {
// Initialize what Open normally does as much as necessary for the test
rep->cache_key_prefix_size = 10;
}
};
class TestFilterBitsReader : public FilterBitsReader {
public:
explicit TestFilterBitsReader(const Slice& contents)
@ -95,26 +105,46 @@ class TestHashFilter : public FilterPolicy {
class PluginFullFilterBlockTest : public testing::Test {
public:
Options options_;
ImmutableCFOptions ioptions_;
EnvOptions env_options_;
BlockBasedTableOptions table_options_;
InternalKeyComparator icomp_;
std::unique_ptr<BlockBasedTable> table_;
PluginFullFilterBlockTest() {
table_options_.filter_policy.reset(new TestHashFilter());
PluginFullFilterBlockTest()
: ioptions_(options_),
env_options_(options_),
icomp_(options_.comparator) {
table_options_.no_block_cache = true;
table_options_.filter_policy.reset(new TestHashFilter);
constexpr bool skip_filters = false;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table)));
}
};
TEST_F(PluginFullFilterBlockTest, PluginEmptyBuilder) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
Slice block = builder.Finish();
ASSERT_EQ("", EscapeString(block));
Slice slice = builder.Finish();
ASSERT_EQ("", EscapeString(slice));
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FullFilterBlockReader reader(
nullptr, true, block,
table_options_.filter_policy->GetFilterBitsReader(block), nullptr);
FullFilterBlockReader reader(table_.get(), std::move(block));
// Remain same symantic with blockbased filter
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
TEST_F(PluginFullFilterBlockTest, PluginSingleChunk) {
@ -125,57 +155,90 @@ TEST_F(PluginFullFilterBlockTest, PluginSingleChunk) {
builder.Add("box");
builder.Add("box");
builder.Add("hello");
Slice block = builder.Finish();
FullFilterBlockReader reader(
nullptr, true, block,
table_options_.filter_policy->GetFilterBitsReader(block), nullptr);
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
Slice slice = builder.Finish();
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FullFilterBlockReader reader(table_.get(), std::move(block));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("bar", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("box", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("hello", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"missing", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"other", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
class FullFilterBlockTest : public testing::Test {
public:
Options options_;
ImmutableCFOptions ioptions_;
EnvOptions env_options_;
BlockBasedTableOptions table_options_;
InternalKeyComparator icomp_;
std::unique_ptr<BlockBasedTable> table_;
FullFilterBlockTest() {
FullFilterBlockTest()
: ioptions_(options_),
env_options_(options_),
icomp_(options_.comparator) {
table_options_.no_block_cache = true;
table_options_.filter_policy.reset(NewBloomFilterPolicy(10, false));
}
~FullFilterBlockTest() override {}
constexpr bool skip_filters = false;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table)));
}
};
TEST_F(FullFilterBlockTest, EmptyBuilder) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
Slice block = builder.Finish();
ASSERT_EQ("", EscapeString(block));
Slice slice = builder.Finish();
ASSERT_EQ("", EscapeString(slice));
FullFilterBlockReader reader(
nullptr, true, block,
table_options_.filter_policy->GetFilterBitsReader(block), nullptr);
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FullFilterBlockReader reader(table_.get(), std::move(block));
// Remain same symantic with blockbased filter
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
TEST_F(FullFilterBlockTest, DuplicateEntries) {
@ -221,31 +284,46 @@ TEST_F(FullFilterBlockTest, SingleChunk) {
builder.Add("box");
builder.Add("hello");
ASSERT_EQ(5, builder.NumAdded());
Slice block = builder.Finish();
FullFilterBlockReader reader(
nullptr, true, block,
table_options_.filter_policy->GetFilterBitsReader(block), nullptr);
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"bar", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"box", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"hello", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch(
"foo", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
Slice slice = builder.Finish();
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */);
FullFilterBlockReader reader(table_.get(), std::move(block));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("bar", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("box", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("hello", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(reader.KeyMayMatch("foo", /*prefix_extractor=*/nullptr,
/*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr,
/*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"missing", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
ASSERT_TRUE(!reader.KeyMayMatch(
"other", /*prefix_extractor=*/nullptr, /*block_offset=*/kNotValid,
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*context=*/nullptr));
/*no_io=*/false, /*const_ikey_ptr=*/nullptr, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
} // namespace rocksdb

@ -119,113 +119,77 @@ Slice PartitionedFilterBlockBuilder::Finish(
}
PartitionedFilterBlockReader::PartitionedFilterBlockReader(
const SliceTransform* prefix_extractor, bool _whole_key_filtering,
BlockContents&& contents, FilterBitsReader* /*filter_bits_reader*/,
Statistics* stats, const InternalKeyComparator comparator,
const BlockBasedTable* table, const bool index_key_includes_seq,
const bool index_value_is_full)
: FilterBlockReader(contents.data.size(), stats, _whole_key_filtering),
prefix_extractor_(prefix_extractor),
comparator_(comparator),
table_(table),
index_key_includes_seq_(index_key_includes_seq),
index_value_is_full_(index_value_is_full) {
idx_on_fltr_blk_.reset(new Block(std::move(contents),
kDisableGlobalSequenceNumber,
0 /* read_amp_bytes_per_bit */, stats));
}
const BlockBasedTable* t, CachableEntry<Block>&& filter_block)
: FilterBlockReaderCommon(t, std::move(filter_block)) {}
PartitionedFilterBlockReader::~PartitionedFilterBlockReader() {
// TODO(myabandeh): if instead of filter object we store only the blocks in
// block cache, then we don't have to manually earse them from block cache
// here.
auto block_cache = table_->rep_->table_options.block_cache.get();
if (UNLIKELY(block_cache == nullptr)) {
return;
}
char cache_key[BlockBasedTable::kMaxCacheKeyPrefixSize + kMaxVarint64Length];
IndexBlockIter biter;
BlockHandle handle;
Statistics* kNullStats = nullptr;
idx_on_fltr_blk_->NewIndexIterator(
&comparator_, comparator_.user_comparator(), &biter, kNullStats, true,
/* have_first_key */ false, index_key_includes_seq_,
index_value_is_full_);
biter.SeekToFirst();
for (; biter.Valid(); biter.Next()) {
handle = biter.value().handle;
auto key = BlockBasedTable::GetCacheKey(table_->rep_->cache_key_prefix,
table_->rep_->cache_key_prefix_size,
handle, cache_key);
block_cache->Erase(key);
std::unique_ptr<FilterBlockReader> PartitionedFilterBlockReader::Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context) {
assert(table);
assert(table->get_rep());
assert(!pin || prefetch);
CachableEntry<Block> filter_block;
if (prefetch || !use_cache) {
const Status s = ReadFilterBlock(table, prefetch_buffer, ReadOptions(),
nullptr /* get_context */, lookup_context,
&filter_block);
if (!s.ok()) {
return std::unique_ptr<FilterBlockReader>();
}
if (use_cache && !pin) {
filter_block.Reset();
}
}
return std::unique_ptr<FilterBlockReader>(
new PartitionedFilterBlockReader(table, std::move(filter_block)));
}
bool PartitionedFilterBlockReader::KeyMayMatch(
const Slice& key, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io, const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) {
GetContext* get_context, BlockCacheLookupContext* lookup_context) {
assert(const_ikey_ptr != nullptr);
assert(block_offset == kNotValid);
if (!whole_key_filtering_) {
return true;
}
if (UNLIKELY(idx_on_fltr_blk_->size() == 0)) {
return true;
}
auto filter_handle = GetFilterPartitionHandle(*const_ikey_ptr);
if (UNLIKELY(filter_handle.size() == 0)) { // key is out of range
return false;
}
auto filter_partition =
GetFilterPartition(/*prefetch_buffer=*/nullptr, filter_handle, no_io,
prefix_extractor, context);
if (UNLIKELY(!filter_partition.GetValue())) {
if (!whole_key_filtering()) {
return true;
}
return filter_partition.GetValue()->KeyMayMatch(
key, prefix_extractor, block_offset, no_io, /*const_ikey_ptr=*/nullptr,
context);
return MayMatch(key, prefix_extractor, block_offset, no_io, const_ikey_ptr,
get_context, lookup_context,
&FullFilterBlockReader::KeyMayMatch);
}
bool PartitionedFilterBlockReader::PrefixMayMatch(
const Slice& prefix, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io, const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) {
GetContext* get_context, BlockCacheLookupContext* lookup_context) {
#ifdef NDEBUG
(void)block_offset;
#endif
assert(const_ikey_ptr != nullptr);
assert(block_offset == kNotValid);
if (!prefix_extractor_ && !prefix_extractor) {
if (!table_prefix_extractor() && !prefix_extractor) {
return true;
}
if (UNLIKELY(idx_on_fltr_blk_->size() == 0)) {
return true;
}
auto filter_handle = GetFilterPartitionHandle(*const_ikey_ptr);
if (UNLIKELY(filter_handle.size() == 0)) { // prefix is out of range
return false;
}
auto filter_partition =
GetFilterPartition(/*prefetch_buffer=*/nullptr, filter_handle, no_io,
prefix_extractor, context);
if (UNLIKELY(!filter_partition.GetValue())) {
return true;
}
return filter_partition.GetValue()->PrefixMayMatch(
prefix, prefix_extractor, kNotValid, no_io, /*const_ikey_ptr=*/nullptr,
context);
return MayMatch(prefix, prefix_extractor, block_offset, no_io, const_ikey_ptr,
get_context, lookup_context,
&FullFilterBlockReader::PrefixMayMatch);
}
BlockHandle PartitionedFilterBlockReader::GetFilterPartitionHandle(
const Slice& entry) {
const CachableEntry<Block>& filter_block, const Slice& entry) const {
IndexBlockIter iter;
const InternalKeyComparator* const comparator = internal_comparator();
Statistics* kNullStats = nullptr;
idx_on_fltr_blk_->NewIndexIterator(
&comparator_, comparator_.user_comparator(), &iter, kNullStats, true,
/* have_first_key */ false, index_key_includes_seq_,
index_value_is_full_);
filter_block.GetValue()->NewIndexIterator(
comparator, comparator->user_comparator(), &iter, kNullStats,
true /* total_order_seek */, false /* have_first_key */,
index_key_includes_seq(), index_value_is_full());
iter.Seek(entry);
if (UNLIKELY(!iter.Valid())) {
return BlockHandle(0, 0);
@ -235,39 +199,78 @@ BlockHandle PartitionedFilterBlockReader::GetFilterPartitionHandle(
return fltr_blk_handle;
}
CachableEntry<FilterBlockReader>
PartitionedFilterBlockReader::GetFilterPartition(
FilePrefetchBuffer* prefetch_buffer, BlockHandle& fltr_blk_handle,
const bool no_io, const SliceTransform* prefix_extractor,
BlockCacheLookupContext* context) {
const bool is_a_filter_partition = true;
auto block_cache = table_->rep_->table_options.block_cache.get();
if (LIKELY(block_cache != nullptr)) {
if (filter_map_.size() != 0) {
auto iter = filter_map_.find(fltr_blk_handle.offset());
// This is a possible scenario since block cache might not have had space
// for the partition
if (iter != filter_map_.end()) {
return {iter->second.GetValue(), nullptr /* cache */,
nullptr /* cache_handle */, false /* own_value */};
}
Status PartitionedFilterBlockReader::GetFilterPartitionBlock(
FilePrefetchBuffer* prefetch_buffer, const BlockHandle& fltr_blk_handle,
bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<BlockContents>* filter_block) const {
assert(table());
assert(filter_block);
assert(filter_block->IsEmpty());
if (!filter_map_.empty()) {
auto iter = filter_map_.find(fltr_blk_handle.offset());
// This is a possible scenario since block cache might not have had space
// for the partition
if (iter != filter_map_.end()) {
filter_block->SetUnownedValue(iter->second.GetValue());
return Status::OK();
}
return table_->GetFilter(/*prefetch_buffer=*/nullptr, fltr_blk_handle,
is_a_filter_partition, no_io,
/*get_context=*/nullptr, context,
prefix_extractor);
} else {
auto filter = table_->ReadFilter(prefetch_buffer, fltr_blk_handle,
is_a_filter_partition, prefix_extractor);
return {filter, nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */};
}
ReadOptions read_options;
if (no_io) {
read_options.read_tier = kBlockCacheTier;
}
const Status s =
table()->RetrieveBlock(prefetch_buffer, read_options, fltr_blk_handle,
UncompressionDict::GetEmptyDict(), filter_block,
BlockType::kFilter, get_context, lookup_context);
return s;
}
bool PartitionedFilterBlockReader::MayMatch(
const Slice& slice, const SliceTransform* prefix_extractor,
uint64_t block_offset, bool no_io, const Slice* const_ikey_ptr,
GetContext* get_context, BlockCacheLookupContext* lookup_context,
FilterFunction filter_function) const {
CachableEntry<Block> filter_block;
Status s =
GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block);
if (UNLIKELY(!s.ok())) {
return true;
}
if (UNLIKELY(filter_block.GetValue()->size() == 0)) {
return true;
}
auto filter_handle = GetFilterPartitionHandle(filter_block, *const_ikey_ptr);
if (UNLIKELY(filter_handle.size() == 0)) { // key is out of range
return false;
}
CachableEntry<BlockContents> filter_partition_block;
s = GetFilterPartitionBlock(nullptr /* prefetch_buffer */, filter_handle,
no_io, get_context, lookup_context,
&filter_partition_block);
if (UNLIKELY(!s.ok())) {
return true;
}
FullFilterBlockReader filter_partition(table(),
std::move(filter_partition_block));
return (filter_partition.*filter_function)(
slice, prefix_extractor, block_offset, no_io, const_ikey_ptr, get_context,
lookup_context);
}
size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
size_t usage = idx_on_fltr_blk_->usable_size();
size_t usage = ApproximateFilterBlockMemoryUsage();
#ifdef ROCKSDB_MALLOC_USABLE_SIZE
usage += malloc_usable_size((void*)this);
usage += malloc_usable_size(const_cast<PartitionedFilterBlockReader*>(this));
#else
usage += sizeof(*this);
#endif // ROCKSDB_MALLOC_USABLE_SIZE
@ -276,16 +279,36 @@ size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
}
// TODO(myabandeh): merge this with the same function in IndexReader
void PartitionedFilterBlockReader::CacheDependencies(
bool pin, const SliceTransform* prefix_extractor) {
// Before read partitions, prefetch them to avoid lots of IOs
void PartitionedFilterBlockReader::CacheDependencies(bool pin) {
assert(table());
const BlockBasedTable::Rep* const rep = table()->get_rep();
assert(rep);
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
CachableEntry<Block> filter_block;
Status s = GetOrReadFilterBlock(false /* no_io */, nullptr /* get_context */,
&lookup_context, &filter_block);
if (!s.ok()) {
ROCKS_LOG_WARN(rep->ioptions.info_log,
"Error retrieving top-level filter block while trying to "
"cache filter partitions: %s",
s.ToString().c_str());
return;
}
// Before read partitions, prefetch them to avoid lots of IOs
assert(filter_block.GetValue());
IndexBlockIter biter;
const InternalKeyComparator* const comparator = internal_comparator();
Statistics* kNullStats = nullptr;
idx_on_fltr_blk_->NewIndexIterator(
&comparator_, comparator_.user_comparator(), &biter, kNullStats, true,
/* have_first_key */ false, index_key_includes_seq_,
index_value_is_full_);
filter_block.GetValue()->NewIndexIterator(
comparator, comparator->user_comparator(), &biter, kNullStats,
true /* total_order_seek */, false /* have_first_key */,
index_key_includes_seq(), index_value_is_full());
// Index partitions are assumed to be consecuitive. Prefetch them all.
// Read the first block offset
biter.SeekToFirst();
@ -298,27 +321,55 @@ void PartitionedFilterBlockReader::CacheDependencies(
uint64_t last_off = handle.offset() + handle.size() + kBlockTrailerSize;
uint64_t prefetch_len = last_off - prefetch_off;
std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
auto& file = table_->rep_->file;
prefetch_buffer.reset(new FilePrefetchBuffer());
Status s;
s = prefetch_buffer->Prefetch(file.get(), prefetch_off,
s = prefetch_buffer->Prefetch(rep->file.get(), prefetch_off,
static_cast<size_t>(prefetch_len));
// After prefetch, read the partitions one by one
biter.SeekToFirst();
for (; biter.Valid(); biter.Next()) {
ReadOptions read_options;
for (biter.SeekToFirst(); biter.Valid(); biter.Next()) {
handle = biter.value().handle;
const bool no_io = true;
const bool is_a_filter_partition = true;
auto filter = table_->GetFilter(
prefetch_buffer.get(), handle, is_a_filter_partition, !no_io,
/*get_context=*/nullptr, &lookup_context, prefix_extractor);
if (LIKELY(filter.IsCached())) {
if (pin) {
filter_map_[handle.offset()] = std::move(filter);
CachableEntry<BlockContents> block;
// TODO: Support counter batch update for partitioned index and
// filter blocks
s = table()->MaybeReadBlockAndLoadToCache(
prefetch_buffer.get(), read_options, handle,
UncompressionDict::GetEmptyDict(), &block, BlockType::kFilter,
nullptr /* get_context */, &lookup_context, nullptr /* contents */);
assert(s.ok() || block.GetValue() == nullptr);
if (s.ok() && block.GetValue() != nullptr) {
if (block.IsCached()) {
if (pin) {
filter_map_[handle.offset()] = std::move(block);
}
}
}
}
}
const InternalKeyComparator* PartitionedFilterBlockReader::internal_comparator()
const {
assert(table());
assert(table()->get_rep());
return &table()->get_rep()->internal_comparator;
}
bool PartitionedFilterBlockReader::index_key_includes_seq() const {
assert(table());
assert(table()->get_rep());
return table()->get_rep()->index_key_includes_seq;
}
bool PartitionedFilterBlockReader::index_value_is_full() const {
assert(table());
assert(table()->get_rep());
return table()->get_rep()->index_value_is_full;
}
} // namespace rocksdb

@ -14,8 +14,7 @@
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "table/block_based/block.h"
#include "table/block_based/block_based_table_reader.h"
#include "table/block_based/cachable_entry.h"
#include "table/block_based/filter_block_reader_common.h"
#include "table/block_based/full_filter_block.h"
#include "util/autovector.h"
@ -69,44 +68,57 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
BlockHandle last_encoded_handle_;
};
class PartitionedFilterBlockReader : public FilterBlockReader {
class PartitionedFilterBlockReader : public FilterBlockReaderCommon<Block> {
public:
explicit PartitionedFilterBlockReader(
const SliceTransform* prefix_extractor, bool whole_key_filtering,
BlockContents&& contents, FilterBitsReader* filter_bits_reader,
Statistics* stats, const InternalKeyComparator comparator,
const BlockBasedTable* table, const bool index_key_includes_seq,
const bool index_value_is_full);
~PartitionedFilterBlockReader() override;
PartitionedFilterBlockReader(const BlockBasedTable* t,
CachableEntry<Block>&& filter_block);
static std::unique_ptr<FilterBlockReader> Create(
const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context);
bool IsBlockBased() override { return false; }
bool KeyMayMatch(const Slice& key, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
const Slice* const const_ikey_ptr, GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
bool PrefixMayMatch(const Slice& prefix,
const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr,
BlockCacheLookupContext* context) override;
GetContext* get_context,
BlockCacheLookupContext* lookup_context) override;
size_t ApproximateMemoryUsage() const override;
private:
BlockHandle GetFilterPartitionHandle(const Slice& entry);
CachableEntry<FilterBlockReader> GetFilterPartition(
FilePrefetchBuffer* prefetch_buffer, BlockHandle& handle,
const bool no_io, const SliceTransform* prefix_extractor,
BlockCacheLookupContext* context);
void CacheDependencies(bool bin,
const SliceTransform* prefix_extractor) override;
const SliceTransform* prefix_extractor_;
std::unique_ptr<Block> idx_on_fltr_blk_;
const InternalKeyComparator comparator_;
const BlockBasedTable* table_;
const bool index_key_includes_seq_;
const bool index_value_is_full_;
std::unordered_map<uint64_t, CachableEntry<FilterBlockReader>> filter_map_;
BlockHandle GetFilterPartitionHandle(const CachableEntry<Block>& filter_block,
const Slice& entry) const;
Status GetFilterPartitionBlock(
FilePrefetchBuffer* prefetch_buffer, const BlockHandle& handle,
bool no_io, GetContext* get_context,
BlockCacheLookupContext* lookup_context,
CachableEntry<BlockContents>* filter_block) const;
using FilterFunction = bool (FullFilterBlockReader::*)(
const Slice& slice, const SliceTransform* prefix_extractor,
uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr, GetContext* get_context,
BlockCacheLookupContext* lookup_context);
bool MayMatch(const Slice& slice, const SliceTransform* prefix_extractor,
uint64_t block_offset, bool no_io, const Slice* const_ikey_ptr,
GetContext* get_context,
BlockCacheLookupContext* lookup_context,
FilterFunction filter_function) const;
void CacheDependencies(bool pin) override;
const InternalKeyComparator* internal_comparator() const;
bool index_key_includes_seq() const;
bool index_value_is_full() const;
protected:
std::unordered_map<uint64_t, CachableEntry<BlockContents>> filter_map_;
};
} // namespace rocksdb

@ -7,6 +7,7 @@
#include "rocksdb/filter_policy.h"
#include "table/block_based/block_based_table_reader.h"
#include "table/block_based/partitioned_filter_block.h"
#include "table/full_filter_bits_builder.h"
@ -23,34 +24,29 @@ std::map<uint64_t, Slice> slices;
class MockedBlockBasedTable : public BlockBasedTable {
public:
explicit MockedBlockBasedTable(Rep* rep)
MockedBlockBasedTable(Rep* rep, PartitionedIndexBuilder* pib)
: BlockBasedTable(rep, /*block_cache_tracer=*/nullptr) {
// Initialize what Open normally does as much as necessary for the test
rep->cache_key_prefix_size = 10;
rep->index_key_includes_seq = pib->seperator_is_key_plus_seq();
rep->index_value_is_full = !pib->get_use_value_delta_encoding();
}
};
CachableEntry<FilterBlockReader> GetFilter(
FilePrefetchBuffer*, const BlockHandle& filter_blk_handle,
const bool /* unused */, bool /* unused */, GetContext* /* unused */,
BlockCacheLookupContext* /*context*/,
const SliceTransform* prefix_extractor) const override {
Slice slice = slices[filter_blk_handle.offset()];
auto obj = new FullFilterBlockReader(
prefix_extractor, true, BlockContents(slice),
rep_->table_options.filter_policy->GetFilterBitsReader(slice), nullptr);
return {obj, nullptr /* cache */, nullptr /* cache_handle */,
true /* own_value */};
}
FilterBlockReader* ReadFilter(
FilePrefetchBuffer*, const BlockHandle& filter_blk_handle,
const bool /* unused */,
const SliceTransform* prefix_extractor) const override {
Slice slice = slices[filter_blk_handle.offset()];
auto obj = new FullFilterBlockReader(
prefix_extractor, true, BlockContents(slice),
rep_->table_options.filter_policy->GetFilterBitsReader(slice), nullptr);
return obj;
class MyPartitionedFilterBlockReader : public PartitionedFilterBlockReader {
public:
MyPartitionedFilterBlockReader(BlockBasedTable* t,
CachableEntry<Block>&& filter_block)
: PartitionedFilterBlockReader(t, std::move(filter_block)) {
for (const auto& pair : slices) {
const uint64_t offset = pair.first;
const Slice& slice = pair.second;
CachableEntry<BlockContents> block(
new BlockContents(slice), nullptr /* cache */,
nullptr /* cache_handle */, true /* own_value */);
filter_map_[offset] = std::move(block);
}
}
};
@ -58,10 +54,18 @@ class PartitionedFilterBlockTest
: public testing::Test,
virtual public ::testing::WithParamInterface<uint32_t> {
public:
Options options_;
ImmutableCFOptions ioptions_;
EnvOptions env_options_;
BlockBasedTableOptions table_options_;
InternalKeyComparator icomp = InternalKeyComparator(BytewiseComparator());
InternalKeyComparator icomp_;
std::unique_ptr<BlockBasedTable> table_;
std::shared_ptr<Cache> cache_;
PartitionedFilterBlockTest() {
PartitionedFilterBlockTest()
: ioptions_(options_),
env_options_(options_),
icomp_(options_.comparator) {
table_options_.filter_policy.reset(NewBloomFilterPolicy(10, false));
table_options_.no_block_cache = true; // Otherwise BlockBasedTable::Close
// will access variable that are not
@ -70,7 +74,6 @@ class PartitionedFilterBlockTest
table_options_.index_block_restart_interval = 3;
}
std::shared_ptr<Cache> cache_;
~PartitionedFilterBlockTest() override {}
const std::string keys[4] = {"afoo", "bar", "box", "hello"};
@ -110,7 +113,7 @@ class PartitionedFilterBlockTest
PartitionedIndexBuilder* NewIndexBuilder() {
const bool kValueDeltaEncoded = true;
return PartitionedIndexBuilder::CreateIndexBuilder(
&icomp, !kValueDeltaEncoded, table_options_);
&icomp_, !kValueDeltaEncoded, table_options_);
}
PartitionedFilterBlockBuilder* NewBuilder(
@ -131,11 +134,8 @@ class PartitionedFilterBlockTest
p_index_builder, partition_size);
}
std::unique_ptr<MockedBlockBasedTable> table;
PartitionedFilterBlockReader* NewReader(
PartitionedFilterBlockBuilder* builder, PartitionedIndexBuilder* pib,
const SliceTransform* prefix_extractor) {
PartitionedFilterBlockBuilder* builder, PartitionedIndexBuilder* pib) {
BlockHandle bh;
Status status;
Slice slice;
@ -143,19 +143,21 @@ class PartitionedFilterBlockTest
slice = builder->Finish(bh, &status);
bh = Write(slice);
} while (status.IsIncomplete());
const Options options;
const ImmutableCFOptions ioptions(options);
const MutableCFOptions moptions(options);
const EnvOptions env_options;
const bool kSkipFilters = true;
const bool kImmortal = true;
table.reset(new MockedBlockBasedTable(
new BlockBasedTable::Rep(ioptions, env_options, table_options_, icomp,
!kSkipFilters, 0, !kImmortal)));
auto reader = new PartitionedFilterBlockReader(
prefix_extractor, true, BlockContents(slice), nullptr, nullptr, icomp,
table.get(), pib->seperator_is_key_plus_seq(),
!pib->get_use_value_delta_encoding());
constexpr bool skip_filters = false;
constexpr int level = 0;
constexpr bool immortal_table = false;
table_.reset(new MockedBlockBasedTable(
new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
icomp_, skip_filters, level, immortal_table),
pib));
BlockContents contents(slice);
CachableEntry<Block> block(
new Block(std::move(contents), kDisableGlobalSequenceNumber,
0 /* read_amp_bytes_per_bit */, nullptr),
nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
auto reader =
new MyPartitionedFilterBlockReader(table_.get(), std::move(block));
return reader;
}
@ -163,36 +165,37 @@ class PartitionedFilterBlockTest
PartitionedIndexBuilder* pib, bool empty = false,
const SliceTransform* prefix_extractor = nullptr) {
std::unique_ptr<PartitionedFilterBlockReader> reader(
NewReader(builder, pib, prefix_extractor));
NewReader(builder, pib));
// Querying added keys
const bool no_io = true;
for (auto key : keys) {
auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
const Slice ikey_slice = Slice(*ikey.rep());
ASSERT_TRUE(reader->KeyMayMatch(key, prefix_extractor, kNotValid, !no_io,
&ikey_slice, /*context=*/nullptr));
&ikey_slice, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
{
// querying a key twice
auto ikey = InternalKey(keys[0], 0, ValueType::kTypeValue);
const Slice ikey_slice = Slice(*ikey.rep());
ASSERT_TRUE(reader->KeyMayMatch(keys[0], prefix_extractor, kNotValid,
!no_io, &ikey_slice,
/*context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
keys[0], prefix_extractor, kNotValid, !no_io, &ikey_slice,
/*get_context=*/nullptr, /*lookup_context=*/nullptr));
}
// querying missing keys
for (auto key : missing_keys) {
auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
const Slice ikey_slice = Slice(*ikey.rep());
if (empty) {
ASSERT_TRUE(reader->KeyMayMatch(key, prefix_extractor, kNotValid,
!no_io, &ikey_slice,
/*context=*/nullptr));
ASSERT_TRUE(reader->KeyMayMatch(
key, prefix_extractor, kNotValid, !no_io, &ikey_slice,
/*get_context=*/nullptr, /*lookup_context=*/nullptr));
} else {
// assuming a good hash function
ASSERT_FALSE(reader->KeyMayMatch(key, prefix_extractor, kNotValid,
!no_io, &ikey_slice,
/*context=*/nullptr));
ASSERT_FALSE(reader->KeyMayMatch(
key, prefix_extractor, kNotValid, !no_io, &ikey_slice,
/*get_context=*/nullptr, /*lookup_context=*/nullptr));
}
}
}
@ -336,13 +339,14 @@ TEST_P(PartitionedFilterBlockTest, SamePrefixInMultipleBlocks) {
builder->Add(pkeys[2]);
CutABlock(pib.get(), pkeys[2]);
std::unique_ptr<PartitionedFilterBlockReader> reader(
NewReader(builder.get(), pib.get(), prefix_extractor.get()));
NewReader(builder.get(), pib.get()));
for (auto key : pkeys) {
auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
const Slice ikey_slice = Slice(*ikey.rep());
ASSERT_TRUE(reader->PrefixMayMatch(
prefix_extractor->Transform(key), prefix_extractor.get(), kNotValid,
/*no_io=*/false, &ikey_slice, /*context=*/nullptr));
/*no_io=*/false, &ikey_slice, /*get_context=*/nullptr,
/*lookup_context=*/nullptr));
}
}

@ -116,8 +116,7 @@ class TableReader {
}
// convert db file to a human readable form
virtual Status DumpTable(WritableFile* /*out_file*/,
const SliceTransform* /*prefix_extractor*/) {
virtual Status DumpTable(WritableFile* /*out_file*/) {
return Status::NotSupported("DumpTable() not supported");
}

@ -2296,7 +2296,7 @@ TEST_P(BlockBasedTableTest, BlockCacheDisabledTest) {
// preloading filter/index blocks is enabled.
auto reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
ASSERT_TRUE(reader->TEST_filter_block_preloaded());
ASSERT_FALSE(reader->TEST_FilterBlockInCache());
ASSERT_FALSE(reader->TEST_IndexBlockInCache());
{
@ -2343,7 +2343,7 @@ TEST_P(BlockBasedTableTest, FilterBlockInBlockCache) {
GetPlainInternalComparator(options.comparator), &keys, &kvmap);
// preloading filter/index blocks is prohibited.
auto* reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
ASSERT_TRUE(!reader->TEST_filter_block_preloaded());
ASSERT_FALSE(reader->TEST_FilterBlockInCache());
ASSERT_TRUE(reader->TEST_IndexBlockInCache());
// -- PART 1: Open with regular block cache.
@ -2476,7 +2476,7 @@ TEST_P(BlockBasedTableTest, FilterBlockInBlockCache) {
MutableCFOptions moptions4(options);
ASSERT_OK(c3.Reopen(ioptions4, moptions4));
reader = dynamic_cast<BlockBasedTable*>(c3.GetTableReader());
ASSERT_TRUE(!reader->TEST_filter_block_preloaded());
ASSERT_FALSE(reader->TEST_FilterBlockInCache());
PinnableSlice value;
GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
GetContext::kNotFound, user_key, &value, nullptr,

@ -150,8 +150,7 @@ Status SstFileDumper::DumpTable(const std::string& out_filename) {
std::unique_ptr<WritableFile> out_file;
Env* env = Env::Default();
env->NewWritableFile(out_filename, &out_file, soptions_);
Status s = table_reader_->DumpTable(out_file.get(),
moptions_.prefix_extractor.get());
Status s = table_reader_->DumpTable(out_file.get());
out_file->Close();
return s;
}

Loading…
Cancel
Save