diff --git a/Makefile b/Makefile index b9c00158a..c05d82af7 100644 --- a/Makefile +++ b/Makefile @@ -90,7 +90,8 @@ TESTS = \ blob_store_test \ filelock_test \ filename_test \ - filter_block_test \ + block_based_filter_block_test \ + full_filter_block_test \ histogram_test \ log_test \ manual_compaction_test \ @@ -393,8 +394,11 @@ rate_limiter_test: util/rate_limiter_test.o $(LIBOBJECTS) $(TESTHARNESS) filename_test: db/filename_test.o $(LIBOBJECTS) $(TESTHARNESS) $(CXX) db/filename_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) -filter_block_test: table/filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) - $(CXX) table/filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) +block_based_filter_block_test: table/block_based_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) + $(CXX) table/block_based_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) + +full_filter_block_test: table/full_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) + $(CXX) table/full_filter_block_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) log_test: db/log_test.o $(LIBOBJECTS) $(TESTHARNESS) $(CXX) db/log_test.o $(LIBOBJECTS) $(TESTHARNESS) $(EXEC_LDFLAGS) -o $@ $(LDFLAGS) $(COVERAGEFLAGS) diff --git a/db/c.cc b/db/c.cc index 9ea549646..d9dee46fb 100644 --- a/db/c.cc +++ b/db/c.cc @@ -118,7 +118,7 @@ struct rocksdb_compactionfilter_t : public CompactionFilter { const Slice& existing_value, std::string* new_value, bool* value_changed) const { - char* c_new_value = NULL; + char* c_new_value = nullptr; size_t new_value_length = 0; unsigned char c_value_changed = 0; unsigned char result = (*filter_)( diff --git a/db/db_bench.cc b/db/db_bench.cc index ced93f227..eada95b6b 100644 --- a/db/db_bench.cc +++ b/db/db_bench.cc @@ -39,8 +39,8 @@ int main() { #include "rocksdb/memtablerep.h" #include "rocksdb/write_batch.h" #include "rocksdb/slice.h" +#include "rocksdb/filter_policy.h" #include "rocksdb/slice_transform.h" -#include "rocksdb/statistics.h" #include "rocksdb/perf_context.h" #include "port/port.h" #include "port/stack_trace.h" @@ -553,7 +553,9 @@ DEFINE_double(cuckoo_hash_ratio, 0.9, "Hash ratio for Cuckoo SST table."); DEFINE_bool(use_hash_search, false, "if use kHashSearch " "instead of kBinarySearch. " "This is valid if only we use BlockTable"); - +DEFINE_bool(use_block_based_filter, false, "if use kBlockBasedFilter " + "instead of kFullFilter for filter block. " + "This is valid if only we use BlockTable"); DEFINE_string(merge_operator, "", "The merge operator to use with the database." "If a new merge operator is specified, be sure to use fresh" " database The possible merge operators are defined in" @@ -1076,9 +1078,9 @@ class Benchmark { (FLAGS_cache_numshardbits >= 1 ? NewLRUCache(FLAGS_compressed_cache_size, FLAGS_cache_numshardbits) : NewLRUCache(FLAGS_compressed_cache_size)) : nullptr), - filter_policy_(FLAGS_bloom_bits >= 0 - ? NewBloomFilterPolicy(FLAGS_bloom_bits) - : nullptr), + filter_policy_(FLAGS_bloom_bits >= 0 ? + NewBloomFilterPolicy(FLAGS_bloom_bits, FLAGS_use_block_based_filter) + : nullptr), prefix_extractor_(NewFixedPrefixTransform(FLAGS_prefix_size)), num_(FLAGS_num), value_size_(FLAGS_value_size), diff --git a/db/db_test.cc b/db/db_test.cc index 5b913f43c..b30bfd70d 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -324,21 +324,22 @@ class DBTest { kHashCuckoo = 7, kMergePut = 8, kFilter = 9, - kUncompressed = 10, - kNumLevel_3 = 11, - kDBLogDir = 12, - kWalDir = 13, - kManifestFileSize = 14, - kCompactOnFlush = 15, - kPerfOptions = 16, - kDeletesFilterFirst = 17, - kHashSkipList = 18, - kUniversalCompaction = 19, - kCompressedBlockCache = 20, - kInfiniteMaxOpenFiles = 21, - kxxHashChecksum = 22, - kFIFOCompaction = 23, - kEnd = 24 + kFullFilter = 10, + kUncompressed = 11, + kNumLevel_3 = 12, + kDBLogDir = 13, + kWalDir = 14, + kManifestFileSize = 15, + kCompactOnFlush = 16, + kPerfOptions = 17, + kDeletesFilterFirst = 18, + kHashSkipList = 19, + kUniversalCompaction = 20, + kCompressedBlockCache = 21, + kInfiniteMaxOpenFiles = 22, + kxxHashChecksum = 23, + kFIFOCompaction = 24, + kEnd = 25 }; int option_config_; @@ -448,6 +449,30 @@ class DBTest { } } + // Switch between different filter policy + // Jump from kDefault to kFilter to kFullFilter + bool ChangeFilterOptions(Options* prev_options = nullptr) { + if (option_config_ == kDefault) { + option_config_ = kFilter; + if (prev_options == nullptr) { + prev_options = &last_options_; + } + Destroy(prev_options); + TryReopen(); + return true; + } else if (option_config_ == kFilter) { + option_config_ = kFullFilter; + if (prev_options == nullptr) { + prev_options = &last_options_; + } + Destroy(prev_options); + TryReopen(); + return true; + } else { + return false; + } + } + // Return the current option configuration. Options CurrentOptions( const anon::OptionsOverride& options_override = anon::OptionsOverride()) { @@ -486,7 +511,10 @@ class DBTest { options.merge_operator = MergeOperators::CreatePutOperator(); break; case kFilter: - table_options.filter_policy.reset(NewBloomFilterPolicy(10)); + table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); + break; + case kFullFilter: + table_options.filter_policy.reset(NewBloomFilterPolicy(10, false)); break; case kUncompressed: options.compression = kNoCompression; @@ -5744,6 +5772,92 @@ TEST(DBTest, BloomFilter) { } while (ChangeCompactOptions()); } +TEST(DBTest, BloomFilterRate) { + while (ChangeFilterOptions()) { + Options options = CurrentOptions(); + options.statistics = rocksdb::CreateDBStatistics(); + CreateAndReopenWithCF({"pikachu"}, &options); + + const int maxKey = 10000; + for (int i = 0; i < maxKey; i++) { + ASSERT_OK(Put(1, Key(i), Key(i))); + } + // Add a large key to make the file contain wide range + ASSERT_OK(Put(1, Key(maxKey + 55555), Key(maxKey + 55555))); + Flush(1); + + // Check if they can be found + for (int i = 0; i < maxKey; i++) { + ASSERT_EQ(Key(i), Get(1, Key(i))); + } + ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 0); + + // Check if filter is useful + for (int i = 0; i < maxKey; i++) { + ASSERT_EQ("NOT_FOUND", Get(1, Key(i+33333))); + } + ASSERT_GE(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), maxKey*0.98); + } +} + +TEST(DBTest, BloomFilterCompatibility) { + Options options; + options.statistics = rocksdb::CreateDBStatistics(); + BlockBasedTableOptions table_options; + table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + + // Create with block based filter + CreateAndReopenWithCF({"pikachu"}, &options); + + const int maxKey = 10000; + for (int i = 0; i < maxKey; i++) { + ASSERT_OK(Put(1, Key(i), Key(i))); + } + ASSERT_OK(Put(1, Key(maxKey + 55555), Key(maxKey + 55555))); + Flush(1); + + // Check db with full filter + table_options.filter_policy.reset(NewBloomFilterPolicy(10, false)); + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + ReopenWithColumnFamilies({"default", "pikachu"}, &options); + + // Check if they can be found + for (int i = 0; i < maxKey; i++) { + ASSERT_EQ(Key(i), Get(1, Key(i))); + } + ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 0); +} + +TEST(DBTest, BloomFilterReverseCompatibility) { + Options options; + options.statistics = rocksdb::CreateDBStatistics(); + BlockBasedTableOptions table_options; + table_options.filter_policy.reset(NewBloomFilterPolicy(10, false)); + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + + // Create with full filter + CreateAndReopenWithCF({"pikachu"}, &options); + + const int maxKey = 10000; + for (int i = 0; i < maxKey; i++) { + ASSERT_OK(Put(1, Key(i), Key(i))); + } + ASSERT_OK(Put(1, Key(maxKey + 55555), Key(maxKey + 55555))); + Flush(1); + + // Check db with block_based filter + table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + ReopenWithColumnFamilies({"default", "pikachu"}, &options); + + // Check if they can be found + for (int i = 0; i < maxKey; i++) { + ASSERT_EQ(Key(i), Get(1, Key(i))); + } + ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 0); +} + TEST(DBTest, SnapshotFiles) { do { Options options = CurrentOptions(); @@ -7194,47 +7308,49 @@ void PrefixScanInit(DBTest *dbtest) { } // namespace TEST(DBTest, PrefixScan) { - int count; - Slice prefix; - Slice key; - char buf[100]; - Iterator* iter; - snprintf(buf, sizeof(buf), "03______:"); - prefix = Slice(buf, 8); - key = Slice(buf, 9); - // db configs - env_->count_random_reads_ = true; - Options options = CurrentOptions(); - options.env = env_; - options.prefix_extractor.reset(NewFixedPrefixTransform(8)); - options.disable_auto_compactions = true; - options.max_background_compactions = 2; - options.create_if_missing = true; - options.memtable_factory.reset(NewHashSkipListRepFactory(16)); + while (ChangeFilterOptions()) { + int count; + Slice prefix; + Slice key; + char buf[100]; + Iterator* iter; + snprintf(buf, sizeof(buf), "03______:"); + prefix = Slice(buf, 8); + key = Slice(buf, 9); + // db configs + env_->count_random_reads_ = true; + Options options = CurrentOptions(); + options.env = env_; + options.prefix_extractor.reset(NewFixedPrefixTransform(8)); + options.disable_auto_compactions = true; + options.max_background_compactions = 2; + options.create_if_missing = true; + options.memtable_factory.reset(NewHashSkipListRepFactory(16)); - BlockBasedTableOptions table_options; - table_options.no_block_cache = true; - table_options.filter_policy.reset(NewBloomFilterPolicy(10)); - table_options.whole_key_filtering = false; - options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + BlockBasedTableOptions table_options; + table_options.no_block_cache = true; + table_options.filter_policy.reset(NewBloomFilterPolicy(10)); + table_options.whole_key_filtering = false; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); - // 11 RAND I/Os - DestroyAndReopen(&options); - PrefixScanInit(this); - count = 0; - env_->random_read_counter_.Reset(); - iter = db_->NewIterator(ReadOptions()); - for (iter->Seek(prefix); iter->Valid(); iter->Next()) { - if (! iter->key().starts_with(prefix)) { - break; + // 11 RAND I/Os + DestroyAndReopen(&options); + PrefixScanInit(this); + count = 0; + env_->random_read_counter_.Reset(); + iter = db_->NewIterator(ReadOptions()); + for (iter->Seek(prefix); iter->Valid(); iter->Next()) { + if (! iter->key().starts_with(prefix)) { + break; + } + count++; } - count++; - } - ASSERT_OK(iter->status()); - delete iter; - ASSERT_EQ(count, 2); - ASSERT_EQ(env_->random_read_counter_.Read(), 2); - Close(); + ASSERT_OK(iter->status()); + delete iter; + ASSERT_EQ(count, 2); + ASSERT_EQ(env_->random_read_counter_.Read(), 2); + Close(); + } // end of while } TEST(DBTest, TailingIteratorSingle) { diff --git a/include/rocksdb/filter_policy.h b/include/rocksdb/filter_policy.h index fa44db45f..90aefb388 100644 --- a/include/rocksdb/filter_policy.h +++ b/include/rocksdb/filter_policy.h @@ -21,11 +21,52 @@ #define STORAGE_ROCKSDB_INCLUDE_FILTER_POLICY_H_ #include +#include namespace rocksdb { class Slice; +// A class that takes a bunch of keys, then generates filter +class FilterBitsBuilder { + public: + virtual ~FilterBitsBuilder() {} + + // Add Key to filter, you could use any way to store the key. + // Such as: storing hashes or original keys + // Keys are in sorted order and duplicated keys are possible. + virtual void AddKey(const Slice& key) = 0; + + // Generate the filter using the keys that are added + // The return value of this function would be the filter bits, + // The ownership of actual data is set to buf + virtual Slice Finish(std::unique_ptr* buf) = 0; +}; + +// A class that checks if a key can be in filter +// It should be initialized by Slice generated by BitsBuilder +class FilterBitsReader { + public: + virtual ~FilterBitsReader() {} + + // Check if the entry match the bits in filter + virtual bool MayMatch(const Slice& entry) = 0; +}; + +// We add a new format of filter block called full filter block +// This new interface gives you more space of customization +// +// For the full filter block, you can plug in your version by implement +// the FilterBitsBuilder and FilterBitsReader +// +// There are two sets of interface in FilterPolicy +// Set 1: CreateFilter, KeyMayMatch: used for blockbased filter +// Set 2: GetFilterBitsBuilder, GetFilterBitsReader, they are used for +// full filter. +// Set 1 MUST be implemented correctly, Set 2 is optional +// RocksDB would first try using functions in Set 2. if they return nullptr, +// it would use Set 1 instead. +// You can choose filter type in NewBloomFilterPolicy class FilterPolicy { public: virtual ~FilterPolicy(); @@ -51,11 +92,28 @@ class FilterPolicy { // This method may return true or false if the key was not on the // list, but it should aim to return false with a high probability. virtual bool KeyMayMatch(const Slice& key, const Slice& filter) const = 0; + + // Get the FilterBitsBuilder, which is ONLY used for full filter block + // It contains interface to take individual key, then generate filter + virtual FilterBitsBuilder* GetFilterBitsBuilder() const { + return nullptr; + } + + // Get the FilterBitsReader, which is ONLY used for full filter block + // It contains interface to tell if key can be in filter + // The input slice should NOT be deleted by FilterPolicy + virtual FilterBitsReader* GetFilterBitsReader(const Slice& contents) const { + return nullptr; + } }; // Return a new filter policy that uses a bloom filter with approximately -// the specified number of bits per key. A good value for bits_per_key +// the specified number of bits per key. +// +// bits_per_key: bits per key in bloom filter. A good value for bits_per_key // is 10, which yields a filter with ~ 1% false positive rate. +// use_block_based_builder: use block based filter rather than full fiter. +// If you want to builder full filter, it needs to be set to false. // // Callers must delete the result after any database that is using the // result has been closed. @@ -67,8 +125,8 @@ class FilterPolicy { // ignores trailing spaces, it would be incorrect to use a // FilterPolicy (like NewBloomFilterPolicy) that does not ignore // trailing spaces in keys. -extern const FilterPolicy* NewBloomFilterPolicy(int bits_per_key); - +extern const FilterPolicy* NewBloomFilterPolicy(int bits_per_key, + bool use_block_based_builder = true); } #endif // STORAGE_ROCKSDB_INCLUDE_FILTER_POLICY_H_ diff --git a/include/rocksdb/statistics.h b/include/rocksdb/statistics.h index 6785833b4..a7f2c1408 100644 --- a/include/rocksdb/statistics.h +++ b/include/rocksdb/statistics.h @@ -115,7 +115,7 @@ enum Tickers : uint32_t { // head of the writers queue. WRITE_DONE_BY_SELF, WRITE_DONE_BY_OTHER, - WRITE_TIMEDOUT, // Number of writes ending up with timed-out. + WRITE_TIMEDOUT, // Number of writes ending up with timed-out. WRITE_WITH_WAL, // Number of Write calls that request WAL COMPACT_READ_BYTES, // Bytes read during compaction COMPACT_WRITE_BYTES, // Bytes written during compaction diff --git a/table/filter_block.cc b/table/block_based_filter_block.cc similarity index 60% rename from table/filter_block.cc rename to table/block_based_filter_block.cc index 30284017b..c2c34c628 100644 --- a/table/filter_block.cc +++ b/table/block_based_filter_block.cc @@ -7,7 +7,7 @@ // Use of this source code is governed by a BSD-style license that can be // found in the LICENSE file. See the AUTHORS file for names of contributors. -#include "table/filter_block.h" +#include "table/block_based_filter_block.h" #include "db/dbformat.h" #include "rocksdb/filter_policy.h" @@ -15,21 +15,39 @@ namespace rocksdb { +namespace { +bool SamePrefix(const SliceTransform* prefix_extractor, + const Slice& key1, const Slice& key2) { + if (!prefix_extractor->InDomain(key1) && + !prefix_extractor->InDomain(key2)) { + return true; + } else if (!prefix_extractor->InDomain(key1) || + !prefix_extractor->InDomain(key2)) { + return false; + } else { + return (prefix_extractor->Transform(key1) == + prefix_extractor->Transform(key2)); + } +} +} // namespace + + // See doc/table_format.txt for an explanation of the filter block format. // Generate new filter every 2KB of data static const size_t kFilterBaseLg = 11; static const size_t kFilterBase = 1 << kFilterBaseLg; -FilterBlockBuilder::FilterBlockBuilder(const SliceTransform* prefix_extractor, - const BlockBasedTableOptions& table_opt, - const Comparator* internal_comparator) +BlockBasedFilterBlockBuilder::BlockBasedFilterBlockBuilder( + const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt) : policy_(table_opt.filter_policy.get()), prefix_extractor_(prefix_extractor), - whole_key_filtering_(table_opt.whole_key_filtering), - comparator_(internal_comparator) {} + whole_key_filtering_(table_opt.whole_key_filtering) { + assert(policy_); +} -void FilterBlockBuilder::StartBlock(uint64_t block_offset) { +void BlockBasedFilterBlockBuilder::StartBlock(uint64_t block_offset) { uint64_t filter_index = (block_offset / kFilterBase); assert(filter_index >= filter_offsets_.size()); while (filter_index > filter_offsets_.size()) { @@ -37,53 +55,45 @@ void FilterBlockBuilder::StartBlock(uint64_t block_offset) { } } -bool FilterBlockBuilder::SamePrefix(const Slice &key1, - const Slice &key2) const { - if (!prefix_extractor_->InDomain(key1) && - !prefix_extractor_->InDomain(key2)) { - return true; - } else if (!prefix_extractor_->InDomain(key1) || - !prefix_extractor_->InDomain(key2)) { - return false; - } else { - return (prefix_extractor_->Transform(key1) == - prefix_extractor_->Transform(key2)); +void BlockBasedFilterBlockBuilder::Add(const Slice& key) { + added_to_start_ = 0; + if (whole_key_filtering_) { + AddKey(key); + added_to_start_ = 1; + } + if (prefix_extractor_ && prefix_extractor_->InDomain(key)) { + AddPrefix(key); } } -void FilterBlockBuilder::AddKey(const Slice& key) { +// Add key to filter if needed +inline void BlockBasedFilterBlockBuilder::AddKey(const Slice& key) { + start_.push_back(entries_.size()); + entries_.append(key.data(), key.size()); +} + +// Add prefix to filter if needed +inline void BlockBasedFilterBlockBuilder::AddPrefix(const Slice& key) { // get slice for most recently added entry Slice prev; - size_t added_to_start = 0; - - // add key to filter if needed - if (whole_key_filtering_) { - start_.push_back(entries_.size()); - ++added_to_start; - entries_.append(key.data(), key.size()); - } - - if (start_.size() > added_to_start) { - size_t prev_start = start_[start_.size() - 1 - added_to_start]; + if (start_.size() > added_to_start_) { + size_t prev_start = start_[start_.size() - 1 - added_to_start_]; const char* base = entries_.data() + prev_start; size_t length = entries_.size() - prev_start; prev = Slice(base, length); } - // add prefix to filter if needed - if (prefix_extractor_ && prefix_extractor_->InDomain(key)) { - // this assumes prefix(prefix(key)) == prefix(key), as the last - // entry in entries_ may be either a key or prefix, and we use - // prefix(last entry) to get the prefix of the last key. - if (prev.size() == 0 || !SamePrefix(key, prev)) { - Slice prefix = prefix_extractor_->Transform(key); - start_.push_back(entries_.size()); - entries_.append(prefix.data(), prefix.size()); - } + // this assumes prefix(prefix(key)) == prefix(key), as the last + // entry in entries_ may be either a key or prefix, and we use + // prefix(last entry) to get the prefix of the last key. + if (prev.size() == 0 || !SamePrefix(prefix_extractor_, key, prev)) { + Slice prefix = prefix_extractor_->Transform(key); + start_.push_back(entries_.size()); + entries_.append(prefix.data(), prefix.size()); } } -Slice FilterBlockBuilder::Finish() { +Slice BlockBasedFilterBlockBuilder::Finish() { if (!start_.empty()) { GenerateFilter(); } @@ -99,7 +109,7 @@ Slice FilterBlockBuilder::Finish() { return Slice(result_); } -void FilterBlockBuilder::GenerateFilter() { +void BlockBasedFilterBlockBuilder::GenerateFilter() { const size_t num_entries = start_.size(); if (num_entries == 0) { // Fast path if there are no keys for this filter @@ -112,7 +122,7 @@ void FilterBlockBuilder::GenerateFilter() { tmp_entries_.resize(num_entries); for (size_t i = 0; i < num_entries; i++) { const char* base = entries_.data() + start_[i]; - size_t length = start_[i+1] - start_[i]; + size_t length = start_[i + 1] - start_[i]; tmp_entries_[i] = Slice(base, length); } @@ -125,7 +135,7 @@ void FilterBlockBuilder::GenerateFilter() { start_.clear(); } -FilterBlockReader::FilterBlockReader( +BlockBasedFilterBlockReader::BlockBasedFilterBlockReader( const SliceTransform* prefix_extractor, const BlockBasedTableOptions& table_opt, const Slice& contents, bool delete_contents_after_use) @@ -136,9 +146,10 @@ FilterBlockReader::FilterBlockReader( offset_(nullptr), num_(0), base_lg_(0) { + assert(policy_); size_t n = contents.size(); if (n < 5) return; // 1 byte for base_lg_ and 4 for start of offset array - base_lg_ = contents[n-1]; + base_lg_ = contents[n - 1]; uint32_t last_word = DecodeFixed32(contents.data() + n - 5); if (last_word > n - 5) return; data_ = contents.data(); @@ -149,27 +160,30 @@ FilterBlockReader::FilterBlockReader( } } -bool FilterBlockReader::KeyMayMatch(uint64_t block_offset, - const Slice& key) { +bool BlockBasedFilterBlockReader::KeyMayMatch(const Slice& key, + uint64_t block_offset) { + assert(block_offset != kNotValid); if (!whole_key_filtering_) { return true; } - return MayMatch(block_offset, key); + return MayMatch(key, block_offset); } -bool FilterBlockReader::PrefixMayMatch(uint64_t block_offset, - const Slice& prefix) { +bool BlockBasedFilterBlockReader::PrefixMayMatch(const Slice& prefix, + uint64_t block_offset) { + assert(block_offset != kNotValid); if (!prefix_extractor_) { return true; } - return MayMatch(block_offset, prefix); + return MayMatch(prefix, block_offset); } -bool FilterBlockReader::MayMatch(uint64_t block_offset, const Slice& entry) { +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); + 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); return policy_->KeyMayMatch(entry, filter); @@ -181,7 +195,7 @@ bool FilterBlockReader::MayMatch(uint64_t block_offset, const Slice& entry) { return true; // Errors are treated as potential matches } -size_t FilterBlockReader::ApproximateMemoryUsage() const { +size_t BlockBasedFilterBlockReader::ApproximateMemoryUsage() const { return num_ * 4 + 5 + (offset_ - data_); } } diff --git a/table/block_based_filter_block.h b/table/block_based_filter_block.h new file mode 100644 index 000000000..9bbc93531 --- /dev/null +++ b/table/block_based_filter_block.h @@ -0,0 +1,102 @@ +// Copyright (c) 2013, 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. +// +// Copyright (c) 2012 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +// +// A filter block is stored near the end of a Table file. It contains +// filters (e.g., bloom filters) for all data blocks in the table combined +// into a single filter block. + +#pragma once + +#include +#include +#include +#include +#include +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "table/filter_block.h" +#include "util/hash.h" + +namespace rocksdb { + + +// A BlockBasedFilterBlockBuilder is used to construct all of the filters for a +// particular Table. It generates a single string which is stored as +// a special block in the Table. +// +// The sequence of calls to BlockBasedFilterBlockBuilder must match the regexp: +// (StartBlock Add*)* Finish +class BlockBasedFilterBlockBuilder : public FilterBlockBuilder { + public: + BlockBasedFilterBlockBuilder(const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt); + + virtual bool IsBlockBased() override { return true; } + virtual void StartBlock(uint64_t block_offset) override; + virtual void Add(const Slice& key) override; + virtual Slice Finish() override; + + private: + void AddKey(const Slice& key); + void AddPrefix(const Slice& key); + void GenerateFilter(); + + // important: all of these might point to invalid addresses + // at the time of destruction of this filter block. destructor + // should NOT dereference them. + const FilterPolicy* policy_; + const SliceTransform* prefix_extractor_; + bool whole_key_filtering_; + + std::string entries_; // Flattened entry contents + std::vector start_; // Starting index in entries_ of each entry + uint32_t added_to_start_; // To indicate if key is added + std::string result_; // Filter data computed so far + std::vector tmp_entries_; // policy_->CreateFilter() argument + std::vector filter_offsets_; + + // No copying allowed + BlockBasedFilterBlockBuilder(const BlockBasedFilterBlockBuilder&); + void operator=(const BlockBasedFilterBlockBuilder&); +}; + +// A FilterBlockReader is used to parse filter from SST table. +// KeyMayMatch and PrefixMayMatch would trigger filter checking +class BlockBasedFilterBlockReader : public FilterBlockReader { + public: + // REQUIRES: "contents" and *policy must stay live while *this is live. + BlockBasedFilterBlockReader(const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, + const Slice& contents, + bool delete_contents_after_use = false); + 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 size_t ApproximateMemoryUsage() const override; + + private: + const FilterPolicy* policy_; + const SliceTransform* prefix_extractor_; + bool whole_key_filtering_; + 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) + std::unique_ptr filter_data; + + bool MayMatch(const Slice& entry, uint64_t block_offset); + + // No copying allowed + BlockBasedFilterBlockReader(const BlockBasedFilterBlockReader&); + void operator=(const BlockBasedFilterBlockReader&); +}; +} // namespace rocksdb diff --git a/table/block_based_filter_block_test.cc b/table/block_based_filter_block_test.cc new file mode 100644 index 000000000..4fd8c1cf5 --- /dev/null +++ b/table/block_based_filter_block_test.cc @@ -0,0 +1,242 @@ +// Copyright (c) 2013, 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. +// +// Copyright (c) 2012 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include "table/block_based_filter_block.h" + +#include "rocksdb/filter_policy.h" +#include "util/coding.h" +#include "util/hash.h" +#include "util/logging.h" +#include "util/testharness.h" +#include "util/testutil.h" + +namespace rocksdb { + +// For testing: emit an array with one hash value per key +class TestHashFilter : public FilterPolicy { + public: + virtual const char* Name() const { + return "TestHashFilter"; + } + + virtual void CreateFilter(const Slice* keys, int n, std::string* dst) const { + for (int i = 0; i < n; i++) { + uint32_t h = Hash(keys[i].data(), keys[i].size(), 1); + PutFixed32(dst, h); + } + } + + virtual bool KeyMayMatch(const Slice& key, const Slice& filter) const { + uint32_t h = Hash(key.data(), key.size(), 1); + for (unsigned int i = 0; i + 4 <= filter.size(); i += 4) { + if (h == DecodeFixed32(filter.data() + i)) { + return true; + } + } + return false; + } +}; + +class FilterBlockTest { + public: + TestHashFilter policy_; + BlockBasedTableOptions table_options_; + + FilterBlockTest() { + table_options_.filter_policy.reset(new TestHashFilter()); + } +}; + +TEST(FilterBlockTest, EmptyBuilder) { + BlockBasedFilterBlockBuilder builder(nullptr, table_options_); + Slice block = builder.Finish(); + ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block)); + BlockBasedFilterBlockReader reader(nullptr, table_options_, block); + ASSERT_TRUE(reader.KeyMayMatch("foo", 0)); + ASSERT_TRUE(reader.KeyMayMatch("foo", 100000)); +} + +TEST(FilterBlockTest, SingleChunk) { + BlockBasedFilterBlockBuilder builder(nullptr, table_options_); + builder.StartBlock(100); + builder.Add("foo"); + builder.Add("bar"); + builder.Add("box"); + builder.StartBlock(200); + builder.Add("box"); + builder.StartBlock(300); + builder.Add("hello"); + Slice block = builder.Finish(); + BlockBasedFilterBlockReader reader(nullptr, table_options_, block); + ASSERT_TRUE(reader.KeyMayMatch("foo", 100)); + ASSERT_TRUE(reader.KeyMayMatch("bar", 100)); + ASSERT_TRUE(reader.KeyMayMatch("box", 100)); + ASSERT_TRUE(reader.KeyMayMatch("hello", 100)); + ASSERT_TRUE(reader.KeyMayMatch("foo", 100)); + ASSERT_TRUE(!reader.KeyMayMatch("missing", 100)); + ASSERT_TRUE(!reader.KeyMayMatch("other", 100)); +} + +TEST(FilterBlockTest, MultiChunk) { + BlockBasedFilterBlockBuilder builder(nullptr, table_options_); + + // First filter + builder.StartBlock(0); + builder.Add("foo"); + builder.StartBlock(2000); + builder.Add("bar"); + + // Second filter + builder.StartBlock(3100); + builder.Add("box"); + + // Third filter is empty + + // Last filter + builder.StartBlock(9000); + builder.Add("box"); + builder.Add("hello"); + + Slice block = builder.Finish(); + BlockBasedFilterBlockReader reader(nullptr, table_options_, block); + + // Check first filter + ASSERT_TRUE(reader.KeyMayMatch("foo", 0)); + ASSERT_TRUE(reader.KeyMayMatch("bar", 2000)); + ASSERT_TRUE(!reader.KeyMayMatch("box", 0)); + ASSERT_TRUE(!reader.KeyMayMatch("hello", 0)); + + // Check second filter + ASSERT_TRUE(reader.KeyMayMatch("box", 3100)); + ASSERT_TRUE(!reader.KeyMayMatch("foo", 3100)); + ASSERT_TRUE(!reader.KeyMayMatch("bar", 3100)); + ASSERT_TRUE(!reader.KeyMayMatch("hello", 3100)); + + // Check third filter (empty) + ASSERT_TRUE(!reader.KeyMayMatch("foo", 4100)); + ASSERT_TRUE(!reader.KeyMayMatch("bar", 4100)); + ASSERT_TRUE(!reader.KeyMayMatch("box", 4100)); + ASSERT_TRUE(!reader.KeyMayMatch("hello", 4100)); + + // Check last filter + ASSERT_TRUE(reader.KeyMayMatch("box", 9000)); + ASSERT_TRUE(reader.KeyMayMatch("hello", 9000)); + ASSERT_TRUE(!reader.KeyMayMatch("foo", 9000)); + ASSERT_TRUE(!reader.KeyMayMatch("bar", 9000)); +} + +// Test for block based filter block +// use new interface in FilterPolicy to create filter builder/reader +class BlockBasedFilterBlockTest { + public: + BlockBasedTableOptions table_options_; + + BlockBasedFilterBlockTest() { + table_options_.filter_policy.reset(NewBloomFilterPolicy(10)); + } + + ~BlockBasedFilterBlockTest() {} +}; + +TEST(BlockBasedFilterBlockTest, BlockBasedEmptyBuilder) { + FilterBlockBuilder* builder = new BlockBasedFilterBlockBuilder( + nullptr, table_options_); + Slice block = builder->Finish(); + ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block)); + FilterBlockReader* reader = new BlockBasedFilterBlockReader( + nullptr, table_options_, block); + ASSERT_TRUE(reader->KeyMayMatch("foo", 0)); + ASSERT_TRUE(reader->KeyMayMatch("foo", 100000)); + + delete builder; + delete reader; +} + +TEST(BlockBasedFilterBlockTest, BlockBasedSingleChunk) { + FilterBlockBuilder* builder = new BlockBasedFilterBlockBuilder( + nullptr, table_options_); + builder->StartBlock(100); + builder->Add("foo"); + builder->Add("bar"); + builder->Add("box"); + builder->StartBlock(200); + builder->Add("box"); + builder->StartBlock(300); + builder->Add("hello"); + Slice block = builder->Finish(); + FilterBlockReader* reader = new BlockBasedFilterBlockReader( + nullptr, table_options_, block); + ASSERT_TRUE(reader->KeyMayMatch("foo", 100)); + ASSERT_TRUE(reader->KeyMayMatch("bar", 100)); + ASSERT_TRUE(reader->KeyMayMatch("box", 100)); + ASSERT_TRUE(reader->KeyMayMatch("hello", 100)); + ASSERT_TRUE(reader->KeyMayMatch("foo", 100)); + ASSERT_TRUE(!reader->KeyMayMatch("missing", 100)); + ASSERT_TRUE(!reader->KeyMayMatch("other", 100)); + + delete builder; + delete reader; +} + +TEST(BlockBasedFilterBlockTest, BlockBasedMultiChunk) { + FilterBlockBuilder* builder = new BlockBasedFilterBlockBuilder( + nullptr, table_options_); + + // First filter + builder->StartBlock(0); + builder->Add("foo"); + builder->StartBlock(2000); + builder->Add("bar"); + + // Second filter + builder->StartBlock(3100); + builder->Add("box"); + + // Third filter is empty + + // Last filter + builder->StartBlock(9000); + builder->Add("box"); + builder->Add("hello"); + + Slice block = builder->Finish(); + FilterBlockReader* reader = new BlockBasedFilterBlockReader( + nullptr, table_options_, block); + + // Check first filter + ASSERT_TRUE(reader->KeyMayMatch("foo", 0)); + ASSERT_TRUE(reader->KeyMayMatch("bar", 2000)); + ASSERT_TRUE(!reader->KeyMayMatch("box", 0)); + ASSERT_TRUE(!reader->KeyMayMatch("hello", 0)); + + // Check second filter + ASSERT_TRUE(reader->KeyMayMatch("box", 3100)); + ASSERT_TRUE(!reader->KeyMayMatch("foo", 3100)); + ASSERT_TRUE(!reader->KeyMayMatch("bar", 3100)); + ASSERT_TRUE(!reader->KeyMayMatch("hello", 3100)); + + // Check third filter (empty) + ASSERT_TRUE(!reader->KeyMayMatch("foo", 4100)); + ASSERT_TRUE(!reader->KeyMayMatch("bar", 4100)); + ASSERT_TRUE(!reader->KeyMayMatch("box", 4100)); + ASSERT_TRUE(!reader->KeyMayMatch("hello", 4100)); + + // Check last filter + ASSERT_TRUE(reader->KeyMayMatch("box", 9000)); + ASSERT_TRUE(reader->KeyMayMatch("hello", 9000)); + ASSERT_TRUE(!reader->KeyMayMatch("foo", 9000)); + ASSERT_TRUE(!reader->KeyMayMatch("bar", 9000)); + + delete builder; + delete reader; +} + +} // namespace rocksdb + +int main(int argc, char** argv) { return rocksdb::test::RunAllTests(); } diff --git a/table/block_based_table_builder.cc b/table/block_based_table_builder.cc index fde363760..7fb662d88 100644 --- a/table/block_based_table_builder.cc +++ b/table/block_based_table_builder.cc @@ -31,6 +31,8 @@ #include "table/block_based_table_reader.h" #include "table/block_builder.h" #include "table/filter_block.h" +#include "table/block_based_filter_block.h" +#include "table/full_filter_block.h" #include "table/format.h" #include "table/meta_blocks.h" #include "table/table_builder.h" @@ -274,6 +276,21 @@ IndexBuilder* CreateIndexBuilder(IndexType type, const Comparator* comparator, return nullptr; } +// Create a index builder based on its type. +FilterBlockBuilder* CreateFilterBlockBuilder(const ImmutableCFOptions& opt, + const BlockBasedTableOptions& table_opt) { + if (table_opt.filter_policy == nullptr) return nullptr; + + FilterBitsBuilder* filter_bits_builder = + table_opt.filter_policy->GetFilterBitsBuilder(); + if (filter_bits_builder == nullptr) { + return new BlockBasedFilterBlockBuilder(opt.prefix_extractor, table_opt); + } else { + return new FullFilterBlockBuilder(opt.prefix_extractor, table_opt, + filter_bits_builder); + } +} + bool GoodCompressionRatio(size_t compressed_size, size_t raw_size) { // Check to see if compressed less than 12.5% return compressed_size < raw_size - (raw_size / 8u); @@ -365,7 +382,6 @@ class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector std::string val; PutFixed32(&val, static_cast(index_type_)); properties->insert({BlockBasedTablePropertyNames::kIndexType, val}); - return Status::OK(); } @@ -428,11 +444,7 @@ struct BlockBasedTableBuilder::Rep { table_options.index_type, &internal_comparator, &this->internal_prefix_transform)), compression_type(compression_type), - compression_opts(compression_opts), - filter_block(table_options.filter_policy == nullptr ? - nullptr : - new FilterBlockBuilder(ioptions.prefix_extractor, - table_options, &internal_comparator)), + filter_block(CreateFilterBlockBuilder(ioptions, table_options)), flush_block_policy( table_options.flush_block_policy_factory->NewFlushBlockPolicy( table_options, data_block)) { @@ -497,7 +509,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) { } if (r->filter_block != nullptr) { - r->filter_block->AddKey(ExtractUserKey(key)); + r->filter_block->Add(ExtractUserKey(key)); } r->last_key.assign(key.data(), key.size()); @@ -661,10 +673,7 @@ Status BlockBasedTableBuilder::Finish() { assert(!r->closed); r->closed = true; - BlockHandle filter_block_handle, - metaindex_block_handle, - index_block_handle; - + BlockHandle filter_block_handle, metaindex_block_handle, index_block_handle; // Write filter block if (ok() && r->filter_block != nullptr) { auto filter_contents = r->filter_block->Finish(); @@ -703,7 +712,12 @@ Status BlockBasedTableBuilder::Finish() { if (r->filter_block != nullptr) { // Add mapping from ".Name" to location // of filter data. - std::string key = BlockBasedTable::kFilterBlockPrefix; + std::string key; + if (r->filter_block->IsBlockBased()) { + key = BlockBasedTable::kFilterBlockPrefix; + } else { + key = BlockBasedTable::kFullFilterBlockPrefix; + } key.append(r->table_options.filter_policy->Name()); meta_index_builder.Add(key, filter_block_handle); } @@ -807,5 +821,6 @@ uint64_t BlockBasedTableBuilder::FileSize() const { } const std::string BlockBasedTable::kFilterBlockPrefix = "filter."; +const std::string BlockBasedTable::kFullFilterBlockPrefix = "fullfilter."; } // namespace rocksdb diff --git a/table/block_based_table_reader.cc b/table/block_based_table_reader.cc index cf915e105..b38f88588 100644 --- a/table/block_based_table_reader.cc +++ b/table/block_based_table_reader.cc @@ -26,6 +26,8 @@ #include "table/block.h" #include "table/filter_block.h" +#include "table/block_based_filter_block.h" +#include "table/full_filter_block.h" #include "table/block_hash_index.h" #include "table/block_prefix_index.h" #include "table/format.h" @@ -46,7 +48,6 @@ using std::unique_ptr; typedef BlockBasedTable::IndexReader IndexReader; namespace { - // The longest the prefix of the cache key used to identify blocks can be. // We are using the fact that we know for Posix files the unique ID is three // varints. @@ -527,11 +528,18 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions, // Set filter block if (rep->filter_policy) { - std::string key = kFilterBlockPrefix; - key.append(rep->filter_policy->Name()); - BlockHandle handle; - if (FindMetaBlock(meta_iter.get(), key, &handle).ok()) { - rep->filter.reset(ReadFilter(handle, rep)); + // First try reading full_filter, then reading block_based_filter + for (auto filter_block_prefix : { kFullFilterBlockPrefix, + kFilterBlockPrefix }) { + std::string key = filter_block_prefix; + key.append(rep->filter_policy->Name()); + + BlockHandle handle; + if (FindMetaBlock(meta_iter.get(), key, &handle).ok()) { + rep->filter.reset(ReadFilter(handle, rep, + filter_block_prefix, nullptr)); + break; + } } } } else { @@ -741,9 +749,9 @@ Status BlockBasedTable::PutDataBlockToCache( return s; } -FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle, - BlockBasedTable::Rep* rep, - size_t* filter_size) { +FilterBlockReader* BlockBasedTable::ReadFilter( + const BlockHandle& filter_handle, BlockBasedTable::Rep* rep, + const std::string& filter_block_prefix, size_t* filter_size) { // TODO: We might want to unify with ReadBlockFromFile() if we start // requiring checksum verification in Table::Open. ReadOptions opt; @@ -757,13 +765,25 @@ FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle, *filter_size = block.data.size(); } - return new FilterBlockReader( - rep->ioptions.prefix_extractor, rep->table_options, - block.data, block.heap_allocated); + assert(rep->filter_policy); + if (kFilterBlockPrefix == filter_block_prefix) { + return new BlockBasedFilterBlockReader(rep->ioptions.prefix_extractor, + rep->table_options, block.data, block.heap_allocated); + } else if (kFullFilterBlockPrefix == filter_block_prefix) { + auto filter_bits_reader = rep->filter_policy-> + GetFilterBitsReader(block.data); + + if (filter_bits_reader != nullptr) { + return new FullFilterBlockReader(rep->ioptions.prefix_extractor, + rep->table_options, block.data, filter_bits_reader, + block.heap_allocated); + } + } + return nullptr; } BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( - bool no_io) const { + bool no_io) const { // filter pre-populated if (rep_->filter != nullptr) { return {rep_->filter.get(), nullptr /* cache handle */}; @@ -777,11 +797,9 @@ BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( // Fetching from the cache char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; - auto key = GetCacheKey( - rep_->cache_key_prefix, - rep_->cache_key_prefix_size, - rep_->footer.metaindex_handle(), - cache_key + auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, + rep_->footer.metaindex_handle(), + cache_key ); Statistics* statistics = rep_->ioptions.statistics; @@ -791,8 +809,8 @@ BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( FilterBlockReader* filter = nullptr; if (cache_handle != nullptr) { - filter = reinterpret_cast( - block_cache->Value(cache_handle)); + filter = reinterpret_cast( + block_cache->Value(cache_handle)); } else if (no_io) { // Do not invoke any io. return CachableEntry(); @@ -803,17 +821,22 @@ BlockBasedTable::CachableEntry BlockBasedTable::GetFilter( auto s = ReadMetaBlock(rep_, &meta, &iter); if (s.ok()) { - std::string filter_block_key = kFilterBlockPrefix; - filter_block_key.append(rep_->filter_policy->Name()); - BlockHandle handle; - if (FindMetaBlock(iter.get(), filter_block_key, &handle).ok()) { - filter = ReadFilter(handle, rep_, &filter_size); - assert(filter); - assert(filter_size > 0); - - cache_handle = block_cache->Insert( - key, filter, filter_size, &DeleteCachedEntry); - RecordTick(statistics, BLOCK_CACHE_ADD); + // First try reading full_filter, then reading block_based_filter + for (auto filter_block_prefix : {kFullFilterBlockPrefix, + kFilterBlockPrefix}) { + std::string filter_block_key = filter_block_prefix; + filter_block_key.append(rep_->filter_policy->Name()); + BlockHandle handle; + if (FindMetaBlock(iter.get(), filter_block_key, &handle).ok()) { + filter = ReadFilter(handle, rep_, filter_block_prefix, &filter_size); + + if (filter == nullptr) break; // err happen in ReadFilter + assert(filter_size > 0); + cache_handle = block_cache->Insert( + key, filter, filter_size, &DeleteCachedEntry); + RecordTick(statistics, BLOCK_CACHE_ADD); + break; + } } } } @@ -918,8 +941,8 @@ Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep, // create key for block cache if (block_cache != nullptr) { - key = GetCacheKey(rep->cache_key_prefix, - rep->cache_key_prefix_size, handle, cache_key); + key = GetCacheKey(rep->cache_key_prefix, rep->cache_key_prefix_size, + handle, cache_key); } if (block_cache_compressed != nullptr) { @@ -1039,42 +1062,50 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) { // loaded to memory. ReadOptions no_io_read_options; no_io_read_options.read_tier = kBlockCacheTier; - unique_ptr iiter(NewIndexIterator(no_io_read_options)); - iiter->Seek(internal_prefix); - - if (!iiter->Valid()) { - // we're past end of file - // if it's incomplete, it means that we avoided I/O - // and we're not really sure that we're past the end - // of the file - may_match = iiter->status().IsIncomplete(); - } else if (ExtractUserKey(iiter->key()).starts_with( - ExtractUserKey(internal_prefix))) { - // we need to check for this subtle case because our only - // guarantee is that "the key is a string >= last key in that data - // block" according to the doc/table_format.txt spec. - // - // Suppose iiter->key() starts with the desired prefix; it is not - // necessarily the case that the corresponding data block will - // contain the prefix, since iiter->key() need not be in the - // block. However, the next data block may contain the prefix, so - // we return true to play it safe. - may_match = true; - } else { - // iiter->key() does NOT start with the desired prefix. Because - // Seek() finds the first key that is >= the seek target, this - // means that iiter->key() > prefix. Thus, any data blocks coming - // after the data block corresponding to iiter->key() cannot - // possibly contain the key. Thus, the corresponding data block - // is the only one which could potentially contain the prefix. - Slice handle_value = iiter->value(); - BlockHandle handle; - s = handle.DecodeFrom(&handle_value); - assert(s.ok()); - auto filter_entry = GetFilter(true /* no io */); - may_match = filter_entry.value == nullptr || - filter_entry.value->PrefixMayMatch(handle.offset(), prefix); - filter_entry.Release(rep_->table_options.block_cache.get()); + + // First, try check with full filter + auto filter_entry = GetFilter(true /* no io */); + FilterBlockReader* filter = filter_entry.value; + if (filter != nullptr && !filter->IsBlockBased()) { + may_match = filter->PrefixMayMatch(prefix); + } + + // Then, try find it within each block + if (may_match) { + unique_ptr iiter(NewIndexIterator(no_io_read_options)); + iiter->Seek(internal_prefix); + + if (!iiter->Valid()) { + // we're past end of file + // if it's incomplete, it means that we avoided I/O + // and we're not really sure that we're past the end + // of the file + may_match = iiter->status().IsIncomplete(); + } else if (ExtractUserKey(iiter->key()).starts_with( + ExtractUserKey(internal_prefix))) { + // we need to check for this subtle case because our only + // guarantee is that "the key is a string >= last key in that data + // block" according to the doc/table_format.txt spec. + // + // Suppose iiter->key() starts with the desired prefix; it is not + // necessarily the case that the corresponding data block will + // contain the prefix, since iiter->key() need not be in the + // block. However, the next data block may contain the prefix, so + // we return true to play it safe. + may_match = true; + } else if (filter != nullptr && filter->IsBlockBased()) { + // iiter->key() does NOT start with the desired prefix. Because + // Seek() finds the first key that is >= the seek target, this + // means that iiter->key() > prefix. Thus, any data blocks coming + // after the data block corresponding to iiter->key() cannot + // possibly contain the key. Thus, the corresponding data block + // is the only on could potentially contain the prefix. + Slice handle_value = iiter->value(); + BlockHandle handle; + s = handle.DecodeFrom(&handle_value); + assert(s.ok()); + may_match = filter->PrefixMayMatch(prefix, handle.offset()); + } } Statistics* statistics = rep_->ioptions.statistics; @@ -1083,6 +1114,7 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) { RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL); } + filter_entry.Release(rep_->table_options.block_cache.get()); return may_match; } @@ -1098,64 +1130,72 @@ Status BlockBasedTable::Get( const Slice& v), void (*mark_key_may_exist_handler)(void* handle_context)) { Status s; - BlockIter iiter; - NewIndexIterator(read_options, &iiter); - auto filter_entry = GetFilter(read_options.read_tier == kBlockCacheTier); FilterBlockReader* filter = filter_entry.value; - bool done = false; - for (iiter.Seek(key); iiter.Valid() && !done; iiter.Next()) { - Slice handle_value = iiter.value(); - BlockHandle handle; - bool may_not_exist_in_filter = - filter != nullptr && handle.DecodeFrom(&handle_value).ok() && - !filter->KeyMayMatch(handle.offset(), ExtractUserKey(key)); - - if (may_not_exist_in_filter) { - // Not found - // TODO: think about interaction with Merge. If a user key cannot - // cross one data block, we should be fine. - RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL); - break; - } else { - BlockIter biter; - NewDataBlockIterator(rep_, read_options, iiter.value(), &biter); - - if (read_options.read_tier && biter.status().IsIncomplete()) { - // couldn't get block from block_cache - // Update Saver.state to Found because we are only looking for whether - // we can guarantee the key is not there when "no_io" is set - (*mark_key_may_exist_handler)(handle_context); - break; - } - if (!biter.status().ok()) { - s = biter.status(); - break; - } + // First check the full filter + // If full filter not useful, Then go into each block + if (filter != nullptr && !filter->IsBlockBased() + && !filter->KeyMayMatch(ExtractUserKey(key))) { + RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL); + } else { + BlockIter iiter; + NewIndexIterator(read_options, &iiter); - // Call the *saver function on each entry/block until it returns false - for (biter.Seek(key); biter.Valid(); biter.Next()) { - ParsedInternalKey parsed_key; - if (!ParseInternalKey(biter.key(), &parsed_key)) { - s = Status::Corruption(Slice()); - } + bool done = false; + for (iiter.Seek(key); iiter.Valid() && !done; iiter.Next()) { + Slice handle_value = iiter.value(); - if (!(*result_handler)(handle_context, parsed_key, - biter.value())) { - done = true; + BlockHandle handle; + bool not_exist_in_filter = + filter != nullptr && filter->IsBlockBased() == true && + handle.DecodeFrom(&handle_value).ok() && + !filter->KeyMayMatch(ExtractUserKey(key), handle.offset()); + + if (not_exist_in_filter) { + // Not found + // TODO: think about interaction with Merge. If a user key cannot + // cross one data block, we should be fine. + RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL); + break; + } else { + BlockIter biter; + NewDataBlockIterator(rep_, read_options, iiter.value(), &biter); + + if (read_options.read_tier && biter.status().IsIncomplete()) { + // couldn't get block from block_cache + // Update Saver.state to Found because we are only looking for whether + // we can guarantee the key is not there when "no_io" is set + (*mark_key_may_exist_handler)(handle_context); break; } + if (!biter.status().ok()) { + s = biter.status(); + break; + } + + // Call the *saver function on each entry/block until it returns false + for (biter.Seek(key); biter.Valid(); biter.Next()) { + ParsedInternalKey parsed_key; + if (!ParseInternalKey(biter.key(), &parsed_key)) { + s = Status::Corruption(Slice()); + } + + if (!(*result_handler)(handle_context, parsed_key, + biter.value())) { + done = true; + break; + } + } + s = biter.status(); } - s = biter.status(); + } + if (s.ok()) { + s = iiter.status(); } } filter_entry.Release(rep_->table_options.block_cache.get()); - if (s.ok()) { - s = iiter.status(); - } - return s; } @@ -1175,8 +1215,8 @@ bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options, char cache_key_storage[kMaxCacheKeyPrefixSize + kMaxVarint64Length]; Slice cache_key = - GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, handle, - cache_key_storage); + GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size, + handle, cache_key_storage); Slice ckey; s = GetDataBlockFromCache(cache_key, ckey, block_cache, nullptr, nullptr, diff --git a/table/block_based_table_reader.h b/table/block_based_table_reader.h index b5686d265..503a91bb3 100644 --- a/table/block_based_table_reader.h +++ b/table/block_based_table_reader.h @@ -28,6 +28,8 @@ class BlockIter; class BlockHandle; class Cache; class FilterBlockReader; +class BlockBasedFilterBlockReader; +class FullFilterBlockReader; class Footer; class InternalKeyComparator; class Iterator; @@ -47,6 +49,7 @@ using std::unique_ptr; class BlockBasedTable : public TableReader { public: static const std::string kFilterBlockPrefix; + static const std::string kFullFilterBlockPrefix; // Attempt to open the table that is stored in bytes [0..file_size) // of "file", and read the metadata entries necessary to allow @@ -184,7 +187,9 @@ class BlockBasedTable : public TableReader { // Create the filter from the filter block. static FilterBlockReader* ReadFilter(const BlockHandle& filter_handle, - Rep* rep, size_t* filter_size = nullptr); + Rep* rep, + const std::string& filter_block_prefix, + size_t* filter_size = nullptr); static void SetupCacheKeyPrefix(Rep* rep); diff --git a/table/filter_block.h b/table/filter_block.h index efee5ac71..adbb7c496 100644 --- a/table/filter_block.h +++ b/table/filter_block.h @@ -10,86 +10,70 @@ // A filter block is stored near the end of a Table file. It contains // filters (e.g., bloom filters) for all data blocks in the table combined // into a single filter block. +// +// It is a base class for BlockBasedFilter and FullFilter. +// These two are both used in BlockBasedTable. The first one contain filter +// For a part of keys in sst file, the second contain filter for all keys +// in sst file. #pragma once -#include #include #include #include #include +#include "rocksdb/options.h" #include "rocksdb/slice.h" -#include "rocksdb/slice_transform.h" #include "rocksdb/table.h" -#include "util/hash.h" namespace rocksdb { -class FilterPolicy; +const uint64_t kNotValid = ULLONG_MAX; // A FilterBlockBuilder is used to construct all of the filters for a // particular Table. It generates a single string which is stored as // a special block in the Table. // // The sequence of calls to FilterBlockBuilder must match the regexp: -// (StartBlock AddKey*)* Finish +// (StartBlock Add*)* Finish +// +// BlockBased/Full FilterBlock would be called in the same way. class FilterBlockBuilder { public: - explicit FilterBlockBuilder(const SliceTransform* prefix_extractor, - const BlockBasedTableOptions& table_opt, - const Comparator* internal_comparator); + explicit FilterBlockBuilder() {} + virtual ~FilterBlockBuilder() {} - void StartBlock(uint64_t block_offset); - void AddKey(const Slice& key); - Slice Finish(); + virtual bool IsBlockBased() = 0; // If is blockbased filter + virtual void StartBlock(uint64_t block_offset) = 0; // Start new block filter + virtual void Add(const Slice& key) = 0; // Add a key to current filter + virtual Slice Finish() = 0; // Generate Filter private: - bool SamePrefix(const Slice &key1, const Slice &key2) const; - void GenerateFilter(); - - // important: all of these might point to invalid addresses - // at the time of destruction of this filter block. destructor - // should NOT dereference them. - const FilterPolicy* policy_; - const SliceTransform* prefix_extractor_; - bool whole_key_filtering_; - const Comparator* comparator_; - - std::string entries_; // Flattened entry contents - std::vector start_; // Starting index in entries_ of each entry - std::string result_; // Filter data computed so far - std::vector tmp_entries_; // policy_->CreateFilter() argument - std::vector filter_offsets_; - // No copying allowed FilterBlockBuilder(const FilterBlockBuilder&); void operator=(const FilterBlockBuilder&); }; +// A FilterBlockReader is used to parse filter from SST table. +// KeyMayMatch and PrefixMayMatch would trigger filter checking +// +// BlockBased/Full FilterBlock would be called in the same way. class FilterBlockReader { public: - // REQUIRES: "contents" and *policy must stay live while *this is live. - FilterBlockReader( - const SliceTransform* prefix_extractor, - const BlockBasedTableOptions& table_opt, - const Slice& contents, - bool delete_contents_after_use = false); - bool KeyMayMatch(uint64_t block_offset, const Slice& key); - bool PrefixMayMatch(uint64_t block_offset, const Slice& prefix); - size_t ApproximateMemoryUsage() const; - - private: - const FilterPolicy* policy_; - const SliceTransform* prefix_extractor_; - bool whole_key_filtering_; - 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) - std::unique_ptr filter_data; + explicit FilterBlockReader() {} + virtual ~FilterBlockReader() {} + virtual bool IsBlockBased() = 0; // If is blockbased filter + virtual bool KeyMayMatch(const Slice& key, + uint64_t block_offset = kNotValid) = 0; + virtual bool PrefixMayMatch(const Slice& prefix, + uint64_t block_offset = kNotValid) = 0; + virtual size_t ApproximateMemoryUsage() const = 0; - bool MayMatch(uint64_t block_offset, const Slice& entry); + private: + // No copying allowed + FilterBlockReader(const FilterBlockReader&); + void operator=(const FilterBlockReader&); }; -} +} // namespace rocksdb diff --git a/table/filter_block_test.cc b/table/filter_block_test.cc deleted file mode 100644 index 903247e80..000000000 --- a/table/filter_block_test.cc +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright (c) 2013, 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. -// -// Copyright (c) 2012 The LevelDB Authors. All rights reserved. -// Use of this source code is governed by a BSD-style license that can be -// found in the LICENSE file. See the AUTHORS file for names of contributors. - -#include "table/filter_block.h" - -#include "rocksdb/filter_policy.h" -#include "util/coding.h" -#include "util/hash.h" -#include "util/logging.h" -#include "util/testharness.h" -#include "util/testutil.h" - -namespace rocksdb { - -// For testing: emit an array with one hash value per key -class TestHashFilter : public FilterPolicy { - public: - virtual const char* Name() const { - return "TestHashFilter"; - } - - virtual void CreateFilter(const Slice* keys, int n, std::string* dst) const { - for (int i = 0; i < n; i++) { - uint32_t h = Hash(keys[i].data(), keys[i].size(), 1); - PutFixed32(dst, h); - } - } - - virtual bool KeyMayMatch(const Slice& key, const Slice& filter) const { - uint32_t h = Hash(key.data(), key.size(), 1); - for (unsigned int i = 0; i + 4 <= filter.size(); i += 4) { - if (h == DecodeFixed32(filter.data() + i)) { - return true; - } - } - return false; - } -}; - -class FilterBlockTest { - public: - const Comparator* comparator_; - BlockBasedTableOptions table_options_; - - FilterBlockTest() - : comparator_(BytewiseComparator()) { - table_options_.filter_policy.reset(new TestHashFilter()); - } -}; - -TEST(FilterBlockTest, EmptyBuilder) { - FilterBlockBuilder builder(nullptr, table_options_, comparator_); - Slice block = builder.Finish(); - ASSERT_EQ("\\x00\\x00\\x00\\x00\\x0b", EscapeString(block)); - FilterBlockReader reader(nullptr, table_options_, block); - ASSERT_TRUE(reader.KeyMayMatch(0, "foo")); - ASSERT_TRUE(reader.KeyMayMatch(100000, "foo")); -} - -TEST(FilterBlockTest, SingleChunk) { - FilterBlockBuilder builder(nullptr, table_options_, comparator_); - builder.StartBlock(100); - builder.AddKey("foo"); - builder.AddKey("bar"); - builder.AddKey("box"); - builder.StartBlock(200); - builder.AddKey("box"); - builder.StartBlock(300); - builder.AddKey("hello"); - Slice block = builder.Finish(); - FilterBlockReader reader(nullptr, table_options_, block); - ASSERT_TRUE(reader.KeyMayMatch(100, "foo")); - ASSERT_TRUE(reader.KeyMayMatch(100, "bar")); - ASSERT_TRUE(reader.KeyMayMatch(100, "box")); - ASSERT_TRUE(reader.KeyMayMatch(100, "hello")); - ASSERT_TRUE(reader.KeyMayMatch(100, "foo")); - ASSERT_TRUE(! reader.KeyMayMatch(100, "missing")); - ASSERT_TRUE(! reader.KeyMayMatch(100, "other")); -} - -TEST(FilterBlockTest, MultiChunk) { - FilterBlockBuilder builder(nullptr, table_options_, comparator_); - - // First filter - builder.StartBlock(0); - builder.AddKey("foo"); - builder.StartBlock(2000); - builder.AddKey("bar"); - - // Second filter - builder.StartBlock(3100); - builder.AddKey("box"); - - // Third filter is empty - - // Last filter - builder.StartBlock(9000); - builder.AddKey("box"); - builder.AddKey("hello"); - - Slice block = builder.Finish(); - FilterBlockReader reader(nullptr, table_options_, block); - - // Check first filter - ASSERT_TRUE(reader.KeyMayMatch(0, "foo")); - ASSERT_TRUE(reader.KeyMayMatch(2000, "bar")); - ASSERT_TRUE(! reader.KeyMayMatch(0, "box")); - ASSERT_TRUE(! reader.KeyMayMatch(0, "hello")); - - // Check second filter - ASSERT_TRUE(reader.KeyMayMatch(3100, "box")); - ASSERT_TRUE(! reader.KeyMayMatch(3100, "foo")); - ASSERT_TRUE(! reader.KeyMayMatch(3100, "bar")); - ASSERT_TRUE(! reader.KeyMayMatch(3100, "hello")); - - // Check third filter (empty) - ASSERT_TRUE(! reader.KeyMayMatch(4100, "foo")); - ASSERT_TRUE(! reader.KeyMayMatch(4100, "bar")); - ASSERT_TRUE(! reader.KeyMayMatch(4100, "box")); - ASSERT_TRUE(! reader.KeyMayMatch(4100, "hello")); - - // Check last filter - ASSERT_TRUE(reader.KeyMayMatch(9000, "box")); - ASSERT_TRUE(reader.KeyMayMatch(9000, "hello")); - ASSERT_TRUE(! reader.KeyMayMatch(9000, "foo")); - ASSERT_TRUE(! reader.KeyMayMatch(9000, "bar")); -} - -} // namespace rocksdb - -int main(int argc, char** argv) { - return rocksdb::test::RunAllTests(); -} diff --git a/table/full_filter_block.cc b/table/full_filter_block.cc new file mode 100644 index 000000000..8a481b7d0 --- /dev/null +++ b/table/full_filter_block.cc @@ -0,0 +1,99 @@ +// Copyright (c) 2014, 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 "table/full_filter_block.h" + +#include "rocksdb/filter_policy.h" +#include "port/port.h" +#include "util/coding.h" + +namespace rocksdb { + +FullFilterBlockBuilder::FullFilterBlockBuilder( + const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, + FilterBitsBuilder* filter_bits_builder) + : prefix_extractor_(prefix_extractor), + whole_key_filtering_(table_opt.whole_key_filtering), + num_added_(0) { + assert(filter_bits_builder != nullptr); + filter_bits_builder_.reset(filter_bits_builder); +} + +void FullFilterBlockBuilder::Add(const Slice& key) { + if (whole_key_filtering_) { + AddKey(key); + } + if (prefix_extractor_ && prefix_extractor_->InDomain(key)) { + AddPrefix(key); + } +} + +// Add key to filter if needed +inline void FullFilterBlockBuilder::AddKey(const Slice& key) { + filter_bits_builder_->AddKey(key); + num_added_++; +} + +// Add prefix to filter if needed +inline void FullFilterBlockBuilder::AddPrefix(const Slice& key) { + Slice prefix = prefix_extractor_->Transform(key); + filter_bits_builder_->AddKey(prefix); + num_added_++; +} + +Slice FullFilterBlockBuilder::Finish() { + if (num_added_ != 0) { + num_added_ = 0; + return filter_bits_builder_->Finish(&filter_data); + } + return Slice(); +} + +FullFilterBlockReader::FullFilterBlockReader( + const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, + const Slice& contents, + FilterBitsReader* filter_bits_reader, bool delete_contents_after_use) + : prefix_extractor_(prefix_extractor), + whole_key_filtering_(table_opt.whole_key_filtering), + contents_(contents) { + assert(filter_bits_reader != nullptr); + filter_bits_reader_.reset(filter_bits_reader); + + if (delete_contents_after_use) { + filter_data.reset(contents.data()); + } +} + +bool FullFilterBlockReader::KeyMayMatch(const Slice& key, + uint64_t block_offset) { + assert(block_offset == kNotValid); + if (!whole_key_filtering_) { + return true; + } + return MayMatch(key); +} + +bool FullFilterBlockReader::PrefixMayMatch(const Slice& prefix, + uint64_t block_offset) { + assert(block_offset == kNotValid); + if (!prefix_extractor_) { + return true; + } + return MayMatch(prefix); +} + +bool FullFilterBlockReader::MayMatch(const Slice& entry) { + if (contents_.size() != 0) { + return filter_bits_reader_->MayMatch(entry); + } + return true; // remain the same with block_based filter +} + +size_t FullFilterBlockReader::ApproximateMemoryUsage() const { + return contents_.size(); +} +} // namespace rocksdb diff --git a/table/full_filter_block.h b/table/full_filter_block.h new file mode 100644 index 000000000..24d20e032 --- /dev/null +++ b/table/full_filter_block.h @@ -0,0 +1,107 @@ +// Copyright (c) 2014, 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. + +#pragma once + +#include +#include +#include +#include +#include +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "db/dbformat.h" +#include "util/hash.h" +#include "table/filter_block.h" + +namespace rocksdb { + +class FilterPolicy; +class FilterBitsBuilder; +class FilterBitsReader; + +// A FullFilterBlockBuilder is used to construct a full filter for a +// particular Table. It generates a single string which is stored as +// a special block in the Table. +// The format of full filter block is: +// +----------------------------------------------------------------+ +// | full filter for all keys in sst file | +// +----------------------------------------------------------------+ +// The full filter can be very large. At the end of it, we put +// num_probes: how many hash functions are used in bloom filter +// +class FullFilterBlockBuilder : public FilterBlockBuilder { + public: + explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, + FilterBitsBuilder* filter_bits_builder); + // bits_builder is created in filter_policy, it should be passed in here + // directly. and be deleted here + ~FullFilterBlockBuilder() {} + + virtual bool IsBlockBased() override { return false; } + virtual void StartBlock(uint64_t block_offset) override {} + virtual void Add(const Slice& key) override; + virtual Slice Finish() override; + + private: + // important: all of these might point to invalid addresses + // at the time of destruction of this filter block. destructor + // should NOT dereference them. + const SliceTransform* prefix_extractor_; + bool whole_key_filtering_; + + uint32_t num_added_; + std::unique_ptr filter_bits_builder_; + std::unique_ptr filter_data; + + void AddKey(const Slice& key); + void AddPrefix(const Slice& key); + + // No copying allowed + FullFilterBlockBuilder(const FullFilterBlockBuilder&); + void operator=(const FullFilterBlockBuilder&); +}; + +// A FilterBlockReader is used to parse filter from SST table. +// KeyMayMatch and PrefixMayMatch would trigger filter checking +class FullFilterBlockReader : public FilterBlockReader { + public: + // REQUIRES: "contents" and filter_bits_reader must stay live + // while *this is live. + explicit FullFilterBlockReader(const SliceTransform* prefix_extractor, + const BlockBasedTableOptions& table_opt, + const Slice& contents, + FilterBitsReader* filter_bits_reader, + bool delete_contents_after_use = false); + + // bits_reader is created in filter_policy, it should be passed in here + // directly. and be deleted here + ~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 size_t ApproximateMemoryUsage() const override; + + private: + const SliceTransform* prefix_extractor_; + bool whole_key_filtering_; + + std::unique_ptr filter_bits_reader_; + Slice contents_; + std::unique_ptr filter_data; + + bool MayMatch(const Slice& entry); + + // No copying allowed + FullFilterBlockReader(const FullFilterBlockReader&); + void operator=(const FullFilterBlockReader&); +}; + +} // namespace rocksdb diff --git a/table/full_filter_block_test.cc b/table/full_filter_block_test.cc new file mode 100644 index 000000000..12e783b4a --- /dev/null +++ b/table/full_filter_block_test.cc @@ -0,0 +1,181 @@ +// Copyright (c) 2014, 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 "table/full_filter_block.h" + +#include "rocksdb/filter_policy.h" +#include "util/coding.h" +#include "util/hash.h" +#include "util/logging.h" +#include "util/testharness.h" +#include "util/testutil.h" + +namespace rocksdb { + +class TestFilterBitsBuilder : public FilterBitsBuilder { + public: + explicit TestFilterBitsBuilder() {} + + // Add Key to filter + virtual void AddKey(const Slice& key) override { + hash_entries_.push_back(Hash(key.data(), key.size(), 1)); + } + + // Generate the filter using the keys that are added + virtual Slice Finish(std::unique_ptr* buf) override { + uint32_t len = hash_entries_.size() * 4; + char* data = new char[len]; + for (size_t i = 0; i < hash_entries_.size(); i++) { + EncodeFixed32(data + i * 4, hash_entries_[i]); + } + buf->reset(data); + return Slice(data, len); + } + + private: + std::vector hash_entries_; +}; + +class TestFilterBitsReader : public FilterBitsReader { + public: + explicit TestFilterBitsReader(const Slice& contents) + : data_(contents.data()), len_(contents.size()) {} + + virtual bool MayMatch(const Slice& entry) override { + uint32_t h = Hash(entry.data(), entry.size(), 1); + for (size_t i = 0; i + 4 <= len_; i += 4) { + if (h == DecodeFixed32(data_ + i)) { + return true; + } + } + return false; + } + + private: + const char* data_; + uint32_t len_; +}; + + +class TestHashFilter : public FilterPolicy { + public: + virtual const char* Name() const { + return "TestHashFilter"; + } + + virtual void CreateFilter(const Slice* keys, int n, std::string* dst) const { + for (int i = 0; i < n; i++) { + uint32_t h = Hash(keys[i].data(), keys[i].size(), 1); + PutFixed32(dst, h); + } + } + + virtual bool KeyMayMatch(const Slice& key, const Slice& filter) const { + uint32_t h = Hash(key.data(), key.size(), 1); + for (unsigned int i = 0; i + 4 <= filter.size(); i += 4) { + if (h == DecodeFixed32(filter.data() + i)) { + return true; + } + } + return false; + } + + virtual FilterBitsBuilder* GetFilterBitsBuilder() const override { + return new TestFilterBitsBuilder(); + } + + virtual FilterBitsReader* GetFilterBitsReader(const Slice& contents) + const override { + return new TestFilterBitsReader(contents); + } +}; + +class PluginFullFilterBlockTest { + public: + BlockBasedTableOptions table_options_; + + PluginFullFilterBlockTest() { + table_options_.filter_policy.reset(new TestHashFilter()); + } +}; + +TEST(PluginFullFilterBlockTest, PluginEmptyBuilder) { + FullFilterBlockBuilder builder(nullptr, table_options_, + table_options_.filter_policy->GetFilterBitsBuilder()); + Slice block = builder.Finish(); + ASSERT_EQ("", EscapeString(block)); + + FullFilterBlockReader reader(nullptr, table_options_, block, + table_options_.filter_policy->GetFilterBitsReader(block)); + // Remain same symantic with blockbased filter + ASSERT_TRUE(reader.KeyMayMatch("foo")); +} + +TEST(PluginFullFilterBlockTest, PluginSingleChunk) { + FullFilterBlockBuilder builder(nullptr, table_options_, + table_options_.filter_policy->GetFilterBitsBuilder()); + builder.Add("foo"); + builder.Add("bar"); + builder.Add("box"); + builder.Add("box"); + builder.Add("hello"); + Slice block = builder.Finish(); + FullFilterBlockReader reader(nullptr, table_options_, block, + table_options_.filter_policy->GetFilterBitsReader(block)); + ASSERT_TRUE(reader.KeyMayMatch("foo")); + ASSERT_TRUE(reader.KeyMayMatch("bar")); + ASSERT_TRUE(reader.KeyMayMatch("box")); + ASSERT_TRUE(reader.KeyMayMatch("hello")); + ASSERT_TRUE(reader.KeyMayMatch("foo")); + ASSERT_TRUE(!reader.KeyMayMatch("missing")); + ASSERT_TRUE(!reader.KeyMayMatch("other")); +} + +class FullFilterBlockTest { + public: + BlockBasedTableOptions table_options_; + + FullFilterBlockTest() { + table_options_.filter_policy.reset(NewBloomFilterPolicy(10, false)); + } + + ~FullFilterBlockTest() {} +}; + +TEST(FullFilterBlockTest, EmptyBuilder) { + FullFilterBlockBuilder builder(nullptr, table_options_, + table_options_.filter_policy->GetFilterBitsBuilder()); + Slice block = builder.Finish(); + ASSERT_EQ("", EscapeString(block)); + + FullFilterBlockReader reader(nullptr, table_options_, block, + table_options_.filter_policy->GetFilterBitsReader(block)); + // Remain same symantic with blockbased filter + ASSERT_TRUE(reader.KeyMayMatch("foo")); +} + +TEST(FullFilterBlockTest, SingleChunk) { + FullFilterBlockBuilder builder(nullptr, table_options_, + table_options_.filter_policy->GetFilterBitsBuilder()); + builder.Add("foo"); + builder.Add("bar"); + builder.Add("box"); + builder.Add("box"); + builder.Add("hello"); + Slice block = builder.Finish(); + FullFilterBlockReader reader(nullptr, table_options_, block, + table_options_.filter_policy->GetFilterBitsReader(block)); + ASSERT_TRUE(reader.KeyMayMatch("foo")); + ASSERT_TRUE(reader.KeyMayMatch("bar")); + ASSERT_TRUE(reader.KeyMayMatch("box")); + ASSERT_TRUE(reader.KeyMayMatch("hello")); + ASSERT_TRUE(reader.KeyMayMatch("foo")); + ASSERT_TRUE(!reader.KeyMayMatch("missing")); + ASSERT_TRUE(!reader.KeyMayMatch("other")); +} + +} // namespace rocksdb + +int main(int argc, char** argv) { return rocksdb::test::RunAllTests(); } diff --git a/table/plain_table_builder.cc b/table/plain_table_builder.cc index 49489ed64..b5914554b 100644 --- a/table/plain_table_builder.cc +++ b/table/plain_table_builder.cc @@ -20,7 +20,6 @@ #include "table/block_builder.h" #include "table/bloom_block.h" #include "table/plain_table_index.h" -#include "table/filter_block.h" #include "table/format.h" #include "table/meta_blocks.h" #include "util/coding.h" diff --git a/table/plain_table_reader.cc b/table/plain_table_reader.cc index 3a6d48be8..23e53bcf7 100644 --- a/table/plain_table_reader.cc +++ b/table/plain_table_reader.cc @@ -20,7 +20,6 @@ #include "table/block.h" #include "table/bloom_block.h" -#include "table/filter_block.h" #include "table/format.h" #include "table/meta_blocks.h" #include "table/two_level_iterator.h" diff --git a/tools/db_sanity_test.cc b/tools/db_sanity_test.cc index 4ae120c21..7cf7c1cca 100644 --- a/tools/db_sanity_test.cc +++ b/tools/db_sanity_test.cc @@ -16,6 +16,7 @@ #include "include/rocksdb/comparator.h" #include "include/rocksdb/table.h" #include "include/rocksdb/slice_transform.h" +#include "include/rocksdb/filter_policy.h" namespace rocksdb { @@ -146,13 +147,30 @@ class SanityTestPlainTableFactory : public SanityTest { Options options_; }; +class SanityTestBloomFilter : public SanityTest { + public: + explicit SanityTestBloomFilter(const std::string& path) + : SanityTest(path) { + table_options_.filter_policy.reset(NewBloomFilterPolicy(10)); + options_.table_factory.reset(NewBlockBasedTableFactory(table_options_)); + } + ~SanityTestBloomFilter() {} + virtual Options GetOptions() const { return options_; } + virtual std::string Name() const { return "BloomFilter"; } + + private: + Options options_; + BlockBasedTableOptions table_options_; +}; + namespace { bool RunSanityTests(const std::string& command, const std::string& path) { std::vector sanity_tests = { new SanityTestBasic(path), new SanityTestSpecialComparator(path), new SanityTestZlibCompression(path), - new SanityTestPlainTableFactory(path)}; + new SanityTestPlainTableFactory(path), + new SanityTestBloomFilter(path)}; if (command == "create") { fprintf(stderr, "Creating...\n"); diff --git a/tools/db_stress.cc b/tools/db_stress.cc index e9955953d..b5c79bf3b 100644 --- a/tools/db_stress.cc +++ b/tools/db_stress.cc @@ -209,6 +209,9 @@ static const bool FLAGS_reopen_dummy __attribute__((unused)) = DEFINE_int32(bloom_bits, 10, "Bloom filter bits per key. " "Negative means use default settings."); +DEFINE_bool(use_block_based_filter, false, "use block based filter" + "instead of full filter for block based table"); + DEFINE_string(db, "", "Use the db with the following name."); DEFINE_bool(verify_checksum, false, @@ -757,8 +760,10 @@ class StressTest { ? NewLRUCache(FLAGS_compressed_cache_size) : nullptr), filter_policy_(FLAGS_bloom_bits >= 0 - ? NewBloomFilterPolicy(FLAGS_bloom_bits) - : nullptr), + ? FLAGS_use_block_based_filter + ? NewBloomFilterPolicy(FLAGS_bloom_bits, true) + : NewBloomFilterPolicy(FLAGS_bloom_bits, false) + : nullptr), db_(nullptr), new_column_family_name_(1), num_times_reopened_(0) { diff --git a/util/bloom.cc b/util/bloom.cc index 723adf843..f19e2a670 100644 --- a/util/bloom.cc +++ b/util/bloom.cc @@ -1,4 +1,4 @@ -// Copyright (c) 2013, Facebook, Inc. All rights reserved. +// Copyright (c) 2014, 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. @@ -10,42 +10,266 @@ #include "rocksdb/filter_policy.h" #include "rocksdb/slice.h" +#include "table/block_based_filter_block.h" +#include "table/full_filter_block.h" #include "util/hash.h" +#include "util/coding.h" namespace rocksdb { +class BlockBasedFilterBlockBuilder; +class FullFilterBlockBuilder; + namespace { +class FullFilterBitsBuilder : public FilterBitsBuilder { + public: + explicit FullFilterBitsBuilder(const size_t bits_per_key, + const size_t num_probes) + : bits_per_key_(bits_per_key), + num_probes_(num_probes) { + assert(bits_per_key_); + } + + ~FullFilterBitsBuilder() {} + + virtual void AddKey(const Slice& key) override { + uint32_t hash = BloomHash(key); + if (hash_entries_.size() == 0 || hash != hash_entries_.back()) { + hash_entries_.push_back(hash); + } + } + + // Create a filter that for hashes [0, n-1], the filter is allocated here + // When creating filter, it is ensured that + // total_bits = num_lines * CACHE_LINE_SIZE * 8 + // dst len is >= 5, 1 for num_probes, 4 for num_lines + // Then total_bits = (len - 5) * 8, and cache_line_size could be calulated + // +----------------------------------------------------------------+ + // | filter data with length total_bits/8 | + // +----------------------------------------------------------------+ + // | | + // | ... | + // | | + // +----------------------------------------------------------------+ + // | ... | num_probes : 1 byte | num_lines : 4 bytes | + // +----------------------------------------------------------------+ + virtual Slice Finish(std::unique_ptr* buf) override { + char* data = nullptr; + uint32_t total_bits, num_lines; + data = ReserveSpace(hash_entries_.size(), &total_bits, &num_lines); + assert(data); + + if (total_bits != 0 && num_lines != 0) { + for (auto h : hash_entries_) { + AddHash(h, data, num_lines, total_bits); + } + } + data[total_bits/8] = static_cast(num_probes_); + EncodeFixed32(data + total_bits/8 + 1, static_cast(num_lines)); + + buf->reset(data); + hash_entries_.clear(); + + return Slice(data, total_bits / 8 + 5); + } -class BloomFilterPolicy : public FilterPolicy { private: size_t bits_per_key_; - size_t k_; - uint32_t (*hash_func_)(const Slice& key); + size_t num_probes_; + std::vector hash_entries_; - void initialize() { - // We intentionally round down to reduce probing cost a little bit - k_ = static_cast(bits_per_key_ * 0.69); // 0.69 =~ ln(2) - if (k_ < 1) k_ = 1; - if (k_ > 30) k_ = 30; + // Get totalbits that optimized for cpu cache line + uint32_t GetTotalBitsForLocality(uint32_t total_bits); + + // Reserve space for new filter + char* ReserveSpace(const int num_entry, uint32_t* total_bits, + uint32_t* num_lines); + + // Assuming single threaded access to this function. + void AddHash(uint32_t h, char* data, uint32_t num_lines, + uint32_t total_bits); + + // No Copy allowed + FullFilterBitsBuilder(const FullFilterBitsBuilder&); + void operator=(const FullFilterBitsBuilder&); +}; + +uint32_t FullFilterBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) { + uint32_t num_lines = + (total_bits + CACHE_LINE_SIZE * 8 - 1) / (CACHE_LINE_SIZE * 8); + + // Make num_lines an odd number to make sure more bits are involved + // when determining which block. + if (num_lines % 2 == 0) { + num_lines++; + } + return num_lines * (CACHE_LINE_SIZE * 8); +} + +char* FullFilterBitsBuilder::ReserveSpace(const int num_entry, + uint32_t* total_bits, uint32_t* num_lines) { + assert(bits_per_key_); + char* data = nullptr; + if (num_entry != 0) { + uint32_t total_bits_tmp = num_entry * bits_per_key_; + + *total_bits = GetTotalBitsForLocality(total_bits_tmp); + *num_lines = *total_bits / (CACHE_LINE_SIZE * 8); + assert(*total_bits > 0 && *total_bits % 8 == 0); + } else { + // filter is empty, just leave space for metadata + *total_bits = 0; + *num_lines = 0; } + // Reserve space for Filter + uint32_t sz = *total_bits / 8; + sz += 5; // 4 bytes for num_lines, 1 byte for num_probes + + data = new char[sz]; + memset(data, 0, sz); + return data; +} + +inline void FullFilterBitsBuilder::AddHash(uint32_t h, char* data, + uint32_t num_lines, uint32_t total_bits) { + assert(num_lines > 0 && total_bits > 0); + + const uint32_t delta = (h >> 17) | (h << 15); // Rotate right 17 bits + uint32_t b = (h % num_lines) * (CACHE_LINE_SIZE * 8); + + for (uint32_t i = 0; i < num_probes_; ++i) { + // Since CACHE_LINE_SIZE is defined as 2^n, this line will be optimized + // to a simple operation by compiler. + const uint32_t bitpos = b + (h % (CACHE_LINE_SIZE * 8)); + data[bitpos / 8] |= (1 << (bitpos % 8)); + + h += delta; + } +} + +class FullFilterBitsReader : public FilterBitsReader { public: - explicit BloomFilterPolicy(int bits_per_key, - uint32_t (*hash_func)(const Slice& key)) - : bits_per_key_(bits_per_key), hash_func_(hash_func) { - initialize(); + explicit FullFilterBitsReader(const Slice& contents) + : data_(const_cast(contents.data())), + data_len_(contents.size()), + num_probes_(0), num_lines_(0) { + assert(data_); + GetFilterMeta(contents, &num_probes_, &num_lines_); + // Sanitize broken parameter + if (num_lines_ != 0 && (data_len_-5) % num_lines_ != 0) { + num_lines_ = 0; + num_probes_ = 0; + } + } + + ~FullFilterBitsReader() {} + + virtual bool MayMatch(const Slice& entry) override { + if (data_len_ <= 5) { // remain same with original filter + return false; + } + // Other Error params, including a broken filter, regarded as match + if (num_probes_ == 0 || num_lines_ == 0) return true; + uint32_t hash = BloomHash(entry); + return HashMayMatch(hash, Slice(data_, data_len_), + num_probes_, num_lines_); } - explicit BloomFilterPolicy(int bits_per_key) - : bits_per_key_(bits_per_key) { - hash_func_ = BloomHash; + + private: + // Filter meta data + char* data_; + uint32_t data_len_; + size_t num_probes_; + uint32_t num_lines_; + + // Get num_probes, and num_lines from filter + // If filter format broken, set both to 0. + void GetFilterMeta(const Slice& filter, size_t* num_probes, + uint32_t* num_lines); + + // "filter" contains the data appended by a preceding call to + // CreateFilterFromHash() on this class. This method must return true if + // the key was in the list of keys passed to CreateFilter(). + // This method may return true or false if the key was not on the + // list, but it should aim to return false with a high probability. + // + // hash: target to be checked + // filter: the whole filter, including meta data bytes + // num_probes: number of probes, read before hand + // num_lines: filter metadata, read before hand + // Before calling this function, need to ensure the input meta data + // is valid. + bool HashMayMatch(const uint32_t& hash, const Slice& filter, + const size_t& num_probes, const uint32_t& num_lines); + + // No Copy allowed + FullFilterBitsReader(const FullFilterBitsReader&); + void operator=(const FullFilterBitsReader&); +}; + +void FullFilterBitsReader::GetFilterMeta(const Slice& filter, + size_t* num_probes, uint32_t* num_lines) { + uint32_t len = filter.size(); + if (len <= 5) { + // filter is empty or broken + *num_probes = 0; + *num_lines = 0; + return; + } + + *num_probes = filter.data()[len - 5]; + *num_lines = DecodeFixed32(filter.data() + len - 4); +} + +bool FullFilterBitsReader::HashMayMatch(const uint32_t& hash, + const Slice& filter, const size_t& num_probes, + const uint32_t& num_lines) { + uint32_t len = filter.size(); + if (len <= 5) return false; // remain the same with original filter + + // It is ensured the params are valid before calling it + assert(num_probes != 0); + assert(num_lines != 0 && (len - 5) % num_lines == 0); + uint32_t cache_line_size = (len - 5) / num_lines; + const char* data = filter.data(); + + uint32_t h = hash; + const uint32_t delta = (h >> 17) | (h << 15); // Rotate right 17 bits + uint32_t b = (h % num_lines) * (cache_line_size * 8); + + for (uint32_t i = 0; i < num_probes; ++i) { + // Since CACHE_LINE_SIZE is defined as 2^n, this line will be optimized + // to a simple and operation by compiler. + const uint32_t bitpos = b + (h % (cache_line_size * 8)); + if (((data[bitpos / 8]) & (1 << (bitpos % 8))) == 0) { + return false; + } + + h += delta; + } + + return true; +} + +// An implementation of filter policy +class BloomFilterPolicy : public FilterPolicy { + public: + explicit BloomFilterPolicy(int bits_per_key, bool use_block_based_builder) + : bits_per_key_(bits_per_key), hash_func_(BloomHash), + use_block_based_builder_(use_block_based_builder) { initialize(); } - virtual const char* Name() const { + ~BloomFilterPolicy() { + } + + virtual const char* Name() const override { return "rocksdb.BuiltinBloomFilter"; } - virtual void CreateFilter(const Slice* keys, int n, std::string* dst) const { + virtual void CreateFilter(const Slice* keys, int n, + std::string* dst) const override { // Compute bloom filter size (in both bits and bytes) size_t bits = n * bits_per_key_; @@ -58,14 +282,14 @@ class BloomFilterPolicy : public FilterPolicy { const size_t init_size = dst->size(); dst->resize(init_size + bytes, 0); - dst->push_back(static_cast(k_)); // Remember # of probes in filter + dst->push_back(static_cast(num_probes_)); // Remember # of probes char* array = &(*dst)[init_size]; for (size_t i = 0; i < (size_t)n; i++) { // Use double-hashing to generate a sequence of hash values. // See analysis in [Kirsch,Mitzenmacher 2006]. uint32_t h = hash_func_(keys[i]); const uint32_t delta = (h >> 17) | (h << 15); // Rotate right 17 bits - for (size_t j = 0; j < k_; j++) { + for (size_t j = 0; j < num_probes_; j++) { const uint32_t bitpos = h % bits; array[bitpos/8] |= (1 << (bitpos % 8)); h += delta; @@ -73,7 +297,8 @@ class BloomFilterPolicy : public FilterPolicy { } } - virtual bool KeyMayMatch(const Slice& key, const Slice& bloom_filter) const { + virtual bool KeyMayMatch(const Slice& key, + const Slice& bloom_filter) const override { const size_t len = bloom_filter.size(); if (len < 2) return false; @@ -98,11 +323,43 @@ class BloomFilterPolicy : public FilterPolicy { } return true; } + + virtual FilterBitsBuilder* GetFilterBitsBuilder() const override { + if (use_block_based_builder_) { + return nullptr; + } + + return new FullFilterBitsBuilder(bits_per_key_, num_probes_); + } + + virtual FilterBitsReader* GetFilterBitsReader(const Slice& contents) + const override { + return new FullFilterBitsReader(contents); + } + + // If choose to use block based builder + bool UseBlockBasedBuilder() { return use_block_based_builder_; } + + private: + size_t bits_per_key_; + size_t num_probes_; + uint32_t (*hash_func_)(const Slice& key); + + const bool use_block_based_builder_; + + void initialize() { + // We intentionally round down to reduce probing cost a little bit + num_probes_ = static_cast(bits_per_key_ * 0.69); // 0.69 =~ ln(2) + if (num_probes_ < 1) num_probes_ = 1; + if (num_probes_ > 30) num_probes_ = 30; + } }; -} -const FilterPolicy* NewBloomFilterPolicy(int bits_per_key) { - return new BloomFilterPolicy(bits_per_key); +} // namespace + +const FilterPolicy* NewBloomFilterPolicy(int bits_per_key, + bool use_block_based_builder) { + return new BloomFilterPolicy(bits_per_key, use_block_based_builder); } } // namespace rocksdb diff --git a/util/bloom_test.cc b/util/bloom_test.cc index 881e3b0f5..275592b70 100644 --- a/util/bloom_test.cc +++ b/util/bloom_test.cc @@ -16,12 +16,13 @@ int main() { #else #include +#include #include "rocksdb/filter_policy.h" - #include "util/logging.h" #include "util/testharness.h" #include "util/testutil.h" +#include "util/arena.h" using GFLAGS::ParseCommandLineFlags; @@ -36,6 +37,19 @@ static Slice Key(int i, char* buffer) { return Slice(buffer, sizeof(i)); } +static int NextLength(int length) { + if (length < 10) { + length += 1; + } else if (length < 100) { + length += 10; + } else if (length < 1000) { + length += 100; + } else { + length += 1000; + } + return length; +} + class BloomTest { private: const FilterPolicy* policy_; @@ -43,7 +57,8 @@ class BloomTest { std::vector keys_; public: - BloomTest() : policy_(NewBloomFilterPolicy(FLAGS_bits_per_key)) { } + BloomTest() : policy_( + NewBloomFilterPolicy(FLAGS_bits_per_key)) {} ~BloomTest() { delete policy_; @@ -117,19 +132,6 @@ TEST(BloomTest, Small) { ASSERT_TRUE(! Matches("foo")); } -static int NextLength(int length) { - if (length < 10) { - length += 1; - } else if (length < 100) { - length += 10; - } else if (length < 1000) { - length += 100; - } else { - length += 1000; - } - return length; -} - TEST(BloomTest, VaryingLengths) { char buffer[sizeof(int)]; @@ -171,6 +173,121 @@ TEST(BloomTest, VaryingLengths) { // Different bits-per-byte +class FullBloomTest { + private: + const FilterPolicy* policy_; + std::unique_ptr bits_builder_; + std::unique_ptr bits_reader_; + std::unique_ptr buf_; + size_t filter_size_; + + public: + FullBloomTest() : + policy_(NewBloomFilterPolicy(FLAGS_bits_per_key, false)), + filter_size_(0) { + Reset(); + } + + ~FullBloomTest() { + delete policy_; + } + + void Reset() { + bits_builder_.reset(policy_->GetFilterBitsBuilder()); + bits_reader_.reset(nullptr); + buf_.reset(nullptr); + filter_size_ = 0; + } + + void Add(const Slice& s) { + bits_builder_->AddKey(s); + } + + void Build() { + Slice filter = bits_builder_->Finish(&buf_); + bits_reader_.reset(policy_->GetFilterBitsReader(filter)); + filter_size_ = filter.size(); + } + + size_t FilterSize() const { + return filter_size_; + } + + bool Matches(const Slice& s) { + if (bits_reader_ == nullptr) { + Build(); + } + return bits_reader_->MayMatch(s); + } + + double FalsePositiveRate() { + char buffer[sizeof(int)]; + int result = 0; + for (int i = 0; i < 10000; i++) { + if (Matches(Key(i + 1000000000, buffer))) { + result++; + } + } + return result / 10000.0; + } +}; + +TEST(FullBloomTest, FullEmptyFilter) { + // Empty filter is not match, at this level + ASSERT_TRUE(!Matches("hello")); + ASSERT_TRUE(!Matches("world")); +} + +TEST(FullBloomTest, FullSmall) { + Add("hello"); + Add("world"); + ASSERT_TRUE(Matches("hello")); + ASSERT_TRUE(Matches("world")); + ASSERT_TRUE(!Matches("x")); + ASSERT_TRUE(!Matches("foo")); +} + +TEST(FullBloomTest, FullVaryingLengths) { + char buffer[sizeof(int)]; + + // Count number of filters that significantly exceed the false positive rate + int mediocre_filters = 0; + int good_filters = 0; + + for (int length = 1; length <= 10000; length = NextLength(length)) { + Reset(); + for (int i = 0; i < length; i++) { + Add(Key(i, buffer)); + } + Build(); + + ASSERT_LE(FilterSize(), (size_t)((length * 10 / 8) + 128 + 5)) << length; + + // All added keys must match + for (int i = 0; i < length; i++) { + ASSERT_TRUE(Matches(Key(i, buffer))) + << "Length " << length << "; key " << i; + } + + // Check false positive rate + double rate = FalsePositiveRate(); + if (kVerbose >= 1) { + fprintf(stderr, "False positives: %5.2f%% @ length = %6d ; bytes = %6d\n", + rate*100.0, length, static_cast(FilterSize())); + } + ASSERT_LE(rate, 0.02); // Must not be over 2% + if (rate > 0.0125) + mediocre_filters++; // Allowed, but not too often + else + good_filters++; + } + if (kVerbose >= 1) { + fprintf(stderr, "Filters: %d good, %d mediocre\n", + good_filters, mediocre_filters); + } + ASSERT_LE(mediocre_filters, good_filters/5); +} + } // namespace rocksdb int main(int argc, char** argv) {