提交 8b0097b4 编写于 作者: M Maysam Yabandeh 提交者: Facebook Github Bot

Readers for partition filter

Summary:
This is the last split of this pull request: https://github.com/facebook/rocksdb/pull/1891 which includes the reader part as well as the tests.
Closes https://github.com/facebook/rocksdb/pull/1961

Differential Revision: D4672216

Pulled By: maysamyabandeh

fbshipit-source-id: 6a2b829
上级 9ef3627f
......@@ -354,6 +354,7 @@ TESTS = \
file_reader_writer_test \
block_based_filter_block_test \
full_filter_block_test \
partitioned_filter_block_test \
hash_table_test \
histogram_test \
log_test \
......@@ -1158,6 +1159,9 @@ block_based_filter_block_test: table/block_based_filter_block_test.o $(LIBOBJECT
full_filter_block_test: table/full_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)
partitioned_filter_block_test: table/partitioned_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)
log_test: db/log_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)
......
此差异已折叠。
......@@ -1221,7 +1221,7 @@ class PinL0IndexAndFilterBlocksTest : public DBTestBase,
PinL0IndexAndFilterBlocksTest() : DBTestBase("/db_pin_l0_index_bloom_test") {}
virtual void SetUp() override { infinite_max_files_ = GetParam(); }
void CreateTwoLevels(Options* options) {
void CreateTwoLevels(Options* options, bool close_afterwards) {
if (infinite_max_files_) {
options->max_open_files = -1;
}
......@@ -1249,6 +1249,9 @@ class PinL0IndexAndFilterBlocksTest : public DBTestBase,
Put(1, "z2", "end2");
ASSERT_OK(Flush(1));
if (close_afterwards) {
Close(); // This ensures that there is no ref to block cache entries
}
table_options.block_cache->EraseUnRefEntries();
}
......@@ -1303,7 +1306,7 @@ TEST_P(PinL0IndexAndFilterBlocksTest,
TEST_P(PinL0IndexAndFilterBlocksTest,
MultiLevelIndexAndFilterBlocksCachedWithPinning) {
Options options = CurrentOptions();
PinL0IndexAndFilterBlocksTest::CreateTwoLevels(&options);
PinL0IndexAndFilterBlocksTest::CreateTwoLevels(&options, false);
// get base cache values
uint64_t fm = TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
uint64_t fh = TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT);
......@@ -1332,7 +1335,10 @@ TEST_P(PinL0IndexAndFilterBlocksTest,
TEST_P(PinL0IndexAndFilterBlocksTest, DisablePrefetchingNonL0IndexAndFilter) {
Options options = CurrentOptions();
PinL0IndexAndFilterBlocksTest::CreateTwoLevels(&options);
// This ensures that db does not ref anything in the block cache, so
// EraseUnRefEntries could clear them up.
bool close_afterwards = true;
PinL0IndexAndFilterBlocksTest::CreateTwoLevels(&options, close_afterwards);
// Get base cache values
uint64_t fm = TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
......
......@@ -230,6 +230,8 @@ bool DBTestBase::ChangeFilterOptions() {
option_config_ = kFilter;
} else if (option_config_ == kFilter) {
option_config_ = kFullFilterWithNewTableReaderForCompactions;
} else if (option_config_ == kFullFilterWithNewTableReaderForCompactions) {
option_config_ = kPartitionedFilterWithNewTableReaderForCompactions;
} else {
return false;
}
......@@ -325,6 +327,14 @@ Options DBTestBase::CurrentOptions(
options.new_table_reader_for_compaction_inputs = true;
options.compaction_readahead_size = 10 * 1024 * 1024;
break;
case kPartitionedFilterWithNewTableReaderForCompactions:
table_options.filter_policy.reset(NewBloomFilterPolicy(10, false));
table_options.partition_filters = true;
table_options.index_type =
BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
options.new_table_reader_for_compaction_inputs = true;
options.compaction_readahead_size = 10 * 1024 * 1024;
break;
case kUncompressed:
options.compression = kNoCompression;
break;
......@@ -426,6 +436,8 @@ Options DBTestBase::CurrentOptions(
if (options_override.filter_policy) {
table_options.filter_policy = options_override.filter_policy;
table_options.partition_filters = options_override.partition_filters;
table_options.index_per_partition = options_override.index_per_partition;
}
if (set_block_based_table_factory) {
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
......
......@@ -109,6 +109,11 @@ class AtomicCounter {
struct OptionsOverride {
std::shared_ptr<const FilterPolicy> filter_policy = nullptr;
// These will be used only if filter_policy is set
bool partition_filters = false;
uint64_t index_per_partition = 1024;
BlockBasedTableOptions::IndexType index_type =
BlockBasedTableOptions::IndexType::kBinarySearch;
// Used as a bit mask of individual enums in which to skip an XF test point
int skip_policy = 0;
......@@ -617,6 +622,7 @@ class DBTestBase : public testing::Test {
kUniversalSubcompactions = 32,
kBlockBasedTableWithIndexRestartInterval = 33,
kBlockBasedTableWithPartitionedIndex = 34,
kPartitionedFilterWithNewTableReaderForCompactions = 35,
};
int option_config_;
......
......@@ -184,8 +184,9 @@ BlockBasedFilterBlockReader::BlockBasedFilterBlockReader(
num_ = (n - 5 - last_word) / 4;
}
bool BlockBasedFilterBlockReader::KeyMayMatch(const Slice& key,
uint64_t block_offset) {
bool BlockBasedFilterBlockReader::KeyMayMatch(
const Slice& key, uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr) {
assert(block_offset != kNotValid);
if (!whole_key_filtering_) {
return true;
......@@ -193,8 +194,9 @@ bool BlockBasedFilterBlockReader::KeyMayMatch(const Slice& key,
return MayMatch(key, block_offset);
}
bool BlockBasedFilterBlockReader::PrefixMayMatch(const Slice& prefix,
uint64_t block_offset) {
bool BlockBasedFilterBlockReader::PrefixMayMatch(
const Slice& prefix, uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr) {
assert(block_offset != kNotValid);
if (!prefix_extractor_) {
return true;
......
......@@ -81,10 +81,14 @@ class BlockBasedFilterBlockReader : public FilterBlockReader {
bool whole_key_filtering,
BlockContents&& contents, Statistics* statistics);
virtual bool IsBlockBased() override { return true; }
virtual bool KeyMayMatch(const Slice& key,
uint64_t block_offset = kNotValid) override;
virtual bool PrefixMayMatch(const Slice& prefix,
uint64_t block_offset = kNotValid) override;
virtual bool KeyMayMatch(
const Slice& key, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual bool PrefixMayMatch(
const Slice& prefix, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual size_t ApproximateMemoryUsage() const override;
// convert this object to a human readable form
......
此差异已折叠。
......@@ -11,6 +11,7 @@
#include <stdint.h>
#include <memory>
#include <set>
#include <string>
#include <utility>
#include <vector>
......@@ -193,17 +194,29 @@ class BlockBasedTable : public TableReader {
class BlockEntryIteratorState;
private:
friend class PartitionIndexReader;
protected:
template <class TValue>
struct CachableEntry;
struct Rep;
Rep* rep_;
explicit BlockBasedTable(Rep* rep)
: rep_(rep), compaction_optimized_(false) {}
private:
bool compaction_optimized_;
// input_iter: if it is not null, update this one and return it as Iterator
static InternalIterator* NewDataBlockIterator(
Rep* rep, const ReadOptions& ro, const Slice& index_value,
BlockIter* input_iter = nullptr);
static InternalIterator* NewDataBlockIterator(Rep* rep, const ReadOptions& ro,
const Slice& index_value,
BlockIter* input_iter = nullptr,
bool is_index = false);
static InternalIterator* NewDataBlockIterator(Rep* rep, const ReadOptions& ro,
const BlockHandle& block_hanlde,
BlockIter* input_iter = nullptr,
bool is_index = false,
Status s = Status());
// If block cache enabled (compressed or uncompressed), looks for the block
// identified by handle in (1) uncompressed cache, (2) compressed cache, and
// then (3) file. If found, inserts into the cache(s) that were searched
......@@ -213,14 +226,19 @@ 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.
static Status MaybeLoadDataBlockToCache(
Rep* rep, const ReadOptions& ro, const BlockHandle& handle,
Slice compression_dict, CachableEntry<Block>* block_entry);
static Status MaybeLoadDataBlockToCache(Rep* rep, const ReadOptions& ro,
const BlockHandle& handle,
Slice compression_dict,
CachableEntry<Block>* block_entry,
bool is_index = false);
// 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(bool no_io = false) const;
virtual CachableEntry<FilterBlockReader> GetFilter(
const BlockHandle& filter_blk_handle, const bool is_a_filter_partition,
bool no_io) const;
// Get the iterator from the index reader.
// If input_iter is not set, return new Iterator
......@@ -247,7 +265,8 @@ class BlockBasedTable : public TableReader {
Cache* block_cache, Cache* block_cache_compressed,
const ImmutableCFOptions& ioptions, const ReadOptions& read_options,
BlockBasedTable::CachableEntry<Block>* block, uint32_t format_version,
const Slice& compression_dict, size_t read_amp_bytes_per_bit);
const Slice& compression_dict, size_t read_amp_bytes_per_bit,
bool is_index = false);
// Put a raw block (maybe compressed) to the corresponding block caches.
// This method will perform decompression against raw_block if needed and then
......@@ -264,7 +283,8 @@ class BlockBasedTable : public TableReader {
Cache* block_cache, Cache* block_cache_compressed,
const ReadOptions& read_options, const ImmutableCFOptions& ioptions,
CachableEntry<Block>* block, Block* raw_block, uint32_t format_version,
const Slice& compression_dict, size_t read_amp_bytes_per_bit);
const Slice& compression_dict, size_t read_amp_bytes_per_bit,
bool is_index = false, Cache::Priority pri = Cache::Priority::LOW);
// Calls (*handle_result)(arg, ...) repeatedly, starting with the entry found
// after a call to Seek(key), until handle_result returns false.
......@@ -280,24 +300,23 @@ class BlockBasedTable : public TableReader {
// helps avoid re-reading meta index block if caller already created one.
Status CreateIndexReader(
IndexReader** index_reader,
InternalIterator* preloaded_meta_index_iter = nullptr);
InternalIterator* preloaded_meta_index_iter = nullptr,
const int level = -1);
bool FullFilterKeyMayMatch(const ReadOptions& read_options,
FilterBlockReader* filter,
const Slice& user_key) const;
FilterBlockReader* filter, const Slice& user_key,
const bool no_io) const;
// Read the meta block from sst.
static Status ReadMetaBlock(Rep* rep, std::unique_ptr<Block>* meta_block,
std::unique_ptr<InternalIterator>* iter);
// Create the filter from the filter block.
FilterBlockReader* ReadFilter(Rep* rep) const;
FilterBlockReader* ReadFilter(const BlockHandle& filter_handle,
const bool is_a_filter_partition) const;
static void SetupCacheKeyPrefix(Rep* rep, uint64_t file_size);
explicit BlockBasedTable(Rep* rep)
: rep_(rep), compaction_optimized_(false) {}
// Generate a cache key prefix from the file
static void GenerateCachePrefix(Cache* cc,
RandomAccessFile* file, char* buffer, size_t* size);
......@@ -313,13 +332,18 @@ class BlockBasedTable : public TableReader {
// No copying allowed
explicit BlockBasedTable(const TableReader&) = delete;
void operator=(const TableReader&) = delete;
friend class PartitionedFilterBlockReader;
friend class PartitionedFilterBlockTest;
};
// Maitaning state of a two-level iteration on a partitioned index structure
class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
public:
BlockEntryIteratorState(BlockBasedTable* table,
const ReadOptions& read_options, bool skip_filters);
const ReadOptions& read_options, bool skip_filters,
bool is_index = false,
Cleanable* block_cache_cleaner = nullptr);
InternalIterator* NewSecondaryIterator(const Slice& index_value) override;
bool PrefixMayMatch(const Slice& internal_key) override;
......@@ -328,6 +352,11 @@ class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
BlockBasedTable* table_;
const ReadOptions read_options_;
bool skip_filters_;
// true if the 2nd level iterator is on indexes instead of on user data.
bool is_index_;
Cleanable* block_cache_cleaner_;
std::set<uint64_t> cleaner_set;
port::RWMutex cleaner_mu;
};
// CachableEntry represents the entries that *may* be fetched from block cache.
......
......@@ -82,16 +82,35 @@ class FilterBlockReader {
virtual ~FilterBlockReader() {}
virtual bool IsBlockBased() = 0; // If is blockbased filter
virtual bool KeyMayMatch(const Slice& key,
uint64_t block_offset = kNotValid) = 0;
/**
* 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
* avoid reading partitions that are not in block cache already
*
* Normally filters are built on only the user keys and the InternalKey is not
* needed for a query. The index in PartitionedFilterBlockReader however is
* built upon InternalKey and must be provided via const_ikey_ptr when running
* queries.
*/
virtual bool KeyMayMatch(const Slice& key, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) = 0;
/**
* no_io and const_ikey_ptr here means the same as in KeyMayMatch
*/
virtual bool PrefixMayMatch(const Slice& prefix,
uint64_t block_offset = kNotValid) = 0;
uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) = 0;
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_; }
int GetLevel() const { return level_; }
void SetLevel(int level) { level_ = level; }
// convert this object to a human readable form
virtual std::string ToString() const {
std::string error_msg("Unsupported filter \n");
......@@ -107,6 +126,7 @@ class FilterBlockReader {
void operator=(const FilterBlockReader&);
size_t size_;
Statistics* statistics_;
int level_ = -1;
};
} // namespace rocksdb
......@@ -73,8 +73,9 @@ FullFilterBlockReader::FullFilterBlockReader(
block_contents_ = std::move(contents);
}
bool FullFilterBlockReader::KeyMayMatch(const Slice& key,
uint64_t block_offset) {
bool FullFilterBlockReader::KeyMayMatch(const Slice& key, uint64_t block_offset,
const bool no_io,
const Slice* const const_ikey_ptr) {
assert(block_offset == kNotValid);
if (!whole_key_filtering_) {
return true;
......@@ -83,7 +84,9 @@ bool FullFilterBlockReader::KeyMayMatch(const Slice& key,
}
bool FullFilterBlockReader::PrefixMayMatch(const Slice& prefix,
uint64_t block_offset) {
uint64_t block_offset,
const bool no_io,
const Slice* const const_ikey_ptr) {
assert(block_offset == kNotValid);
if (!prefix_extractor_) {
return true;
......
......@@ -91,10 +91,14 @@ class FullFilterBlockReader : public FilterBlockReader {
~FullFilterBlockReader() {}
virtual bool IsBlockBased() override { return false; }
virtual bool KeyMayMatch(const Slice& key,
uint64_t block_offset = kNotValid) override;
virtual bool PrefixMayMatch(const Slice& prefix,
uint64_t block_offset = kNotValid) override;
virtual bool KeyMayMatch(
const Slice& key, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual bool PrefixMayMatch(
const Slice& prefix, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual size_t ApproximateMemoryUsage() const override;
private:
......
......@@ -5,7 +5,12 @@
#include "table/partitioned_filter_block.h"
#include <utility>
#include "port/port.h"
#include "rocksdb/filter_policy.h"
#include "table/block.h"
#include "table/block_based_table_reader.h"
#include "util/coding.h"
namespace rocksdb {
......@@ -67,4 +72,149 @@ Slice PartitionedFilterBlockBuilder::Finish(
}
}
PartitionedFilterBlockReader::PartitionedFilterBlockReader(
const SliceTransform* prefix_extractor, bool _whole_key_filtering,
BlockContents&& contents, FilterBitsReader* filter_bits_reader,
Statistics* stats, const Comparator& comparator,
const BlockBasedTable* table)
: FilterBlockReader(contents.data.size(), stats, _whole_key_filtering),
prefix_extractor_(prefix_extractor),
comparator_(comparator),
table_(table) {
idx_on_fltr_blk_.reset(new Block(std::move(contents),
kDisableGlobalSequenceNumber,
0 /* read_amp_bytes_per_bit */, stats));
}
PartitionedFilterBlockReader::~PartitionedFilterBlockReader() {
ReadLock rl(&mu_);
for (auto it = handle_list_.begin(); it != handle_list_.end(); ++it) {
table_->rep_->table_options.block_cache.get()->Release(*it);
}
}
bool PartitionedFilterBlockReader::KeyMayMatch(
const Slice& key, uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr) {
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;
}
// This is the user key vs. the full key in the partition index. We assume
// that user key <= full key
auto filter_handle = GetFilterPartitionHandle(*const_ikey_ptr);
if (UNLIKELY(filter_handle.size() == 0)) { // key is out of range
return false;
}
bool cached = false;
auto filter_partition = GetFilterPartition(&filter_handle, no_io, &cached);
if (UNLIKELY(!filter_partition.value)) {
return true;
}
auto res = filter_partition.value->KeyMayMatch(key, block_offset, no_io);
if (cached) {
return res;
}
if (LIKELY(filter_partition.IsSet())) {
filter_partition.Release(table_->rep_->table_options.block_cache.get());
} else {
delete filter_partition.value;
}
return res;
}
bool PartitionedFilterBlockReader::PrefixMayMatch(
const Slice& prefix, uint64_t block_offset, const bool no_io,
const Slice* const const_ikey_ptr) {
assert(const_ikey_ptr != nullptr);
assert(block_offset == kNotValid);
if (!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;
}
bool cached = false;
auto filter_partition = GetFilterPartition(&filter_handle, no_io, &cached);
if (UNLIKELY(!filter_partition.value)) {
return true;
}
auto res = filter_partition.value->PrefixMayMatch(prefix, kNotValid, no_io);
if (cached) {
return res;
}
if (LIKELY(filter_partition.IsSet())) {
filter_partition.Release(table_->rep_->table_options.block_cache.get());
} else {
delete filter_partition.value;
}
return res;
}
Slice PartitionedFilterBlockReader::GetFilterPartitionHandle(
const Slice& entry) {
BlockIter iter;
idx_on_fltr_blk_->NewIterator(&comparator_, &iter, true);
iter.Seek(entry);
if (UNLIKELY(!iter.Valid())) {
return Slice();
}
assert(iter.Valid());
Slice handle_value = iter.value();
return handle_value;
}
BlockBasedTable::CachableEntry<FilterBlockReader>
PartitionedFilterBlockReader::GetFilterPartition(Slice* handle_value,
const bool no_io,
bool* cached) {
BlockHandle fltr_blk_handle;
auto s = fltr_blk_handle.DecodeFrom(handle_value);
assert(s.ok());
const bool is_a_filter_partition = true;
auto block_cache = table_->rep_->table_options.block_cache.get();
if (LIKELY(block_cache != nullptr)) {
bool pin_cached_filters =
GetLevel() == 0 &&
table_->rep_->table_options.pin_l0_filter_and_index_blocks_in_cache;
if (pin_cached_filters) {
ReadLock rl(&mu_);
auto iter = filter_cache_.find(fltr_blk_handle.offset());
if (iter != filter_cache_.end()) {
RecordTick(statistics(), BLOCK_CACHE_FILTER_HIT);
*cached = true;
return {iter->second, nullptr};
}
}
auto filter =
table_->GetFilter(fltr_blk_handle, is_a_filter_partition, no_io);
if (pin_cached_filters && filter.IsSet()) {
WriteLock wl(&mu_);
std::pair<uint64_t, FilterBlockReader*> pair(fltr_blk_handle.offset(),
filter.value);
auto succ = filter_cache_.insert(pair).second;
if (succ) {
handle_list_.push_back(filter.cache_handle);
} // Otherwise it is already inserted by a concurrent thread
*cached = true;
}
return filter;
} else {
auto filter = table_->ReadFilter(fltr_blk_handle, is_a_filter_partition);
return {filter, nullptr};
}
}
size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
return idx_on_fltr_blk_->size();
}
} // namespace rocksdb
......@@ -7,14 +7,17 @@
#include <list>
#include <string>
#include <vector>
#include <unordered_map>
#include "db/dbformat.h"
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "table/block.h"
#include "table/block_based_table_reader.h"
#include "table/full_filter_block.h"
#include "table/index_builder.h"
#include "util/autovector.h"
namespace rocksdb {
......@@ -49,4 +52,40 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
PartitionedIndexBuilder* const p_index_builder_;
};
class PartitionedFilterBlockReader : public FilterBlockReader {
public:
explicit PartitionedFilterBlockReader(const SliceTransform* prefix_extractor,
bool whole_key_filtering,
BlockContents&& contents,
FilterBitsReader* filter_bits_reader,
Statistics* stats,
const Comparator& comparator,
const BlockBasedTable* table);
virtual ~PartitionedFilterBlockReader();
virtual bool IsBlockBased() override { return false; }
virtual bool KeyMayMatch(
const Slice& key, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual bool PrefixMayMatch(
const Slice& prefix, uint64_t block_offset = kNotValid,
const bool no_io = false,
const Slice* const const_ikey_ptr = nullptr) override;
virtual size_t ApproximateMemoryUsage() const override;
private:
Slice GetFilterPartitionHandle(const Slice& entry);
BlockBasedTable::CachableEntry<FilterBlockReader> GetFilterPartition(
Slice* handle, const bool no_io, bool* cached);
const SliceTransform* prefix_extractor_;
std::unique_ptr<Block> idx_on_fltr_blk_;
const Comparator& comparator_;
const BlockBasedTable* table_;
std::unordered_map<uint64_t, FilterBlockReader*> filter_cache_;
autovector<Cache::Handle*> handle_list_;
port::RWMutex mu_;
};
} // namespace rocksdb
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include <map>
#include "rocksdb/filter_policy.h"
#include "table/partitioned_filter_block.h"
#include "util/coding.h"
#include "util/hash.h"
#include "util/logging.h"
#include "util/testharness.h"
#include "util/testutil.h"
namespace rocksdb {
std::map<uint64_t, Slice> slices;
class MockedBlockBasedTable : public BlockBasedTable {
public:
explicit MockedBlockBasedTable(Rep* rep) : BlockBasedTable(rep) {}
virtual CachableEntry<FilterBlockReader> GetFilter(
const BlockHandle& filter_blk_handle, const bool is_a_filter_partition,
bool no_io) const override {
Slice slice = slices[filter_blk_handle.offset()];
auto obj = new FullFilterBlockReader(
nullptr, true, BlockContents(slice, false, kNoCompression),
rep_->table_options.filter_policy->GetFilterBitsReader(slice), nullptr);
return {obj, nullptr};
}
};
class PartitionedFilterBlockTest : public testing::Test {
public:
BlockBasedTableOptions table_options_;
InternalKeyComparator icomp = InternalKeyComparator(BytewiseComparator());
PartitionedFilterBlockTest() {
table_options_.filter_policy.reset(NewBloomFilterPolicy(10, false));
table_options_.no_block_cache = true; // Otherwise BlockBasedTable::Close
// will access variable that are not
// initialized in our mocked version
}
std::shared_ptr<Cache> cache_;
~PartitionedFilterBlockTest() {}
const std::string keys[4] = {"afoo", "bar", "box", "hello"};
const std::string missing_keys[2] = {"missing", "other"};
int last_offset = 10;
BlockHandle Write(const Slice& slice) {
BlockHandle bh(last_offset + 1, slice.size());
slices[bh.offset()] = slice;
last_offset += bh.size();
return bh;
}
PartitionedIndexBuilder* NewIndexBuilder() {
return PartitionedIndexBuilder::CreateIndexBuilder(&icomp, table_options_);
}
PartitionedFilterBlockBuilder* NewBuilder(
PartitionedIndexBuilder* const p_index_builder) {
return new PartitionedFilterBlockBuilder(
nullptr, table_options_.whole_key_filtering,
table_options_.filter_policy->GetFilterBitsBuilder(),
table_options_.index_block_restart_interval, p_index_builder);
}
std::unique_ptr<MockedBlockBasedTable> table;
PartitionedFilterBlockReader* NewReader(
PartitionedFilterBlockBuilder* builder) {
BlockHandle bh;
Status status;
Slice slice;
do {
slice = builder->Finish(bh, &status);
bh = Write(slice);
} while (status.IsIncomplete());
const Options options;
const ImmutableCFOptions ioptions(options);
const EnvOptions env_options;
table.reset(new MockedBlockBasedTable(new BlockBasedTable::Rep(
ioptions, env_options, table_options_, icomp, false)));
auto reader = new PartitionedFilterBlockReader(
nullptr, true, BlockContents(slice, false, kNoCompression), nullptr,
nullptr, *icomp.user_comparator(), table.get());
return reader;
}
void VerifyReader(PartitionedFilterBlockBuilder* builder,
bool empty = false) {
std::unique_ptr<PartitionedFilterBlockReader> reader(NewReader(builder));
// 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, kNotValid, !no_io, &ikey_slice));
}
{
// 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], kNotValid, !no_io, &ikey_slice));
}
// 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, kNotValid, !no_io, &ikey_slice));
} else {
// assuming a good hash function
ASSERT_FALSE(reader->KeyMayMatch(key, kNotValid, !no_io, &ikey_slice));
}
}
}
void TestBlockPerKey() {
table_options_.index_per_partition = 1;
std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
std::unique_ptr<PartitionedFilterBlockBuilder> builder(
NewBuilder(pib.get()));
int i = 0;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i], keys[i + 1]);
i++;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i], keys[i + 1]);
i++;
builder->Add(keys[i]);
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i], keys[i + 1]);
i++;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i]);
VerifyReader(builder.get());
}
void TestBlockPerTwoKeys() {
std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
std::unique_ptr<PartitionedFilterBlockBuilder> builder(
NewBuilder(pib.get()));
int i = 0;
builder->Add(keys[i]);
i++;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i], keys[i + 1]);
i++;
builder->Add(keys[i]);
builder->Add(keys[i]);
i++;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i]);
VerifyReader(builder.get());
}
void TestBlockPerAllKeys() {
std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
std::unique_ptr<PartitionedFilterBlockBuilder> builder(
NewBuilder(pib.get()));
int i = 0;
builder->Add(keys[i]);
i++;
builder->Add(keys[i]);
i++;
builder->Add(keys[i]);
builder->Add(keys[i]);
i++;
builder->Add(keys[i]);
CutABlock(pib.get(), keys[i]);
VerifyReader(builder.get());
}
void CutABlock(PartitionedIndexBuilder* builder,
const std::string& user_key) {
// Assuming a block is cut, add an entry to the index
std::string key =
std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep());
BlockHandle dont_care_block_handle(1, 1);
builder->AddIndexEntry(&key, nullptr, dont_care_block_handle);
}
void CutABlock(PartitionedIndexBuilder* builder, const std::string& user_key,
const std::string& next_user_key) {
// Assuming a block is cut, add an entry to the index
std::string key =
std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep());
std::string next_key = std::string(
*InternalKey(next_user_key, 0, ValueType::kTypeValue).rep());
BlockHandle dont_care_block_handle(1, 1);
Slice slice = Slice(next_key.data(), next_key.size());
builder->AddIndexEntry(&key, &slice, dont_care_block_handle);
}
};
TEST_F(PartitionedFilterBlockTest, EmptyBuilder) {
std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
std::unique_ptr<PartitionedFilterBlockBuilder> builder(NewBuilder(pib.get()));
const bool empty = true;
VerifyReader(builder.get(), empty);
}
TEST_F(PartitionedFilterBlockTest, OneBlock) {
int num_keys = sizeof(keys) / sizeof(*keys);
for (int i = 1; i < num_keys + 1; i++) {
table_options_.index_per_partition = i;
TestBlockPerAllKeys();
}
}
TEST_F(PartitionedFilterBlockTest, TwoBlocksPerKey) {
int num_keys = sizeof(keys) / sizeof(*keys);
for (int i = 1; i < num_keys + 1; i++) {
table_options_.index_per_partition = i;
TestBlockPerTwoKeys();
}
}
TEST_F(PartitionedFilterBlockTest, OneBlockPerKey) {
int num_keys = sizeof(keys) / sizeof(*keys);
for (int i = 1; i < num_keys + 1; i++) {
table_options_.index_per_partition = i;
TestBlockPerKey();
}
}
} // namespace rocksdb
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册