New Bloom filter implementation for full and partitioned filters (#6007)

Summary:
Adds an improved, replacement Bloom filter implementation (FastLocalBloom) for full and partitioned filters in the block-based table. This replacement is faster and more accurate, especially for high bits per key or millions of keys in a single filter.

Speed

The improved speed, at least on recent x86_64, comes from
* Using fastrange instead of modulo (%)
* Using our new hash function (XXH3 preview, added in a previous commit), which is much faster for large keys and only *slightly* slower on keys around 12 bytes if hashing the same size many thousands of times in a row.
* Optimizing the Bloom filter queries with AVX2 SIMD operations. (Added AVX2 to the USE_SSE=1 build.) Careful design was required to support (a) SIMD-optimized queries, (b) compatible non-SIMD code that's simple and efficient, (c) flexible choice of number of probes, and (d) essentially maximized accuracy for a cache-local Bloom filter. Probes are made eight at a time, so any number of probes up to 8 is the same speed, then up to 16, etc.
* Prefetching cache lines when building the filter. Although this optimization could be applied to the old structure as well, it seems to balance out the small added cost of accumulating 64 bit hashes for adding to the filter rather than 32 bit hashes.

Here's nominal speed data from filter_bench (200MB in filters, about 10k keys each, 10 bits filter data / key, 6 probes, avg key size 24 bytes, includes hashing time) on Skylake DE (relatively low clock speed):

$ ./filter_bench -quick -impl=2 -net_includes_hashing # New Bloom filter
Build avg ns/key: 47.7135
Mixed inside/outside queries...
  Single filter net ns/op: 26.2825
  Random filter net ns/op: 150.459
    Average FP rate %: 0.954651
$ ./filter_bench -quick -impl=0 -net_includes_hashing # Old Bloom filter
Build avg ns/key: 47.2245
Mixed inside/outside queries...
  Single filter net ns/op: 63.2978
  Random filter net ns/op: 188.038
    Average FP rate %: 1.13823

Similar build time but dramatically faster query times on hot data (63 ns to 26 ns), and somewhat faster on stale data (188 ns to 150 ns). Performance differences on batched and skewed query loads are between these extremes as expected.

The only other interesting thing about speed is "inside" (query key was added to filter) vs. "outside" (query key was not added to filter) query times. The non-SIMD implementations are substantially slower when most queries are "outside" vs. "inside". This goes against what one might expect or would have observed years ago, as "outside" queries only need about two probes on average, due to short-circuiting, while "inside" always have num_probes (say 6). The problem is probably the nastily unpredictable branch. The SIMD implementation has few branches (very predictable) and has pretty consistent running time regardless of query outcome.

Accuracy

The generally improved accuracy (re: Issue https://github.com/facebook/rocksdb/issues/5857) comes from a better design for probing indices
within a cache line (re: Issue https://github.com/facebook/rocksdb/issues/4120) and improved accuracy for millions of keys in a single filter from using a 64-bit hash function (XXH3p). Design details in code comments.

Accuracy data (generalizes, except old impl gets worse with millions of keys):
Memory bits per key: FP rate percent old impl -> FP rate percent new impl
6: 5.70953 -> 5.69888
8: 2.45766 -> 2.29709
10: 1.13977 -> 0.959254
12: 0.662498 -> 0.411593
16: 0.353023 -> 0.0873754
24: 0.261552 -> 0.0060971
50: 0.225453 -> ~0.00003 (less than 1 in a million queries are FP)

Fixes https://github.com/facebook/rocksdb/issues/5857
Fixes https://github.com/facebook/rocksdb/issues/4120

Unlike the old implementation, this implementation has a fixed cache line size (64 bytes). At 10 bits per key, the accuracy of this new implementation is very close to the old implementation with 128-byte cache line size. If there's sufficient demand, this implementation could be generalized.

Compatibility

Although old releases would see the new structure as corrupt filter data and read the table as if there's no filter, we've decided only to enable the new Bloom filter with new format_version=5. This provides a smooth path for automatic adoption over time, with an option for early opt-in.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6007

Test Plan: filter_bench has been used thoroughly to validate speed, accuracy, and correctness. Unit tests have been carefully updated to exercise new and old implementations, as well as the logic to select an implementation based on context (format_version).

Differential Revision: D18294749

Pulled By: pdillinger

fbshipit-source-id: d44c9db3696e4d0a17caaec47075b7755c262c5f
main
Peter Dillinger 5 years ago committed by Facebook Github Bot
parent f382f44e39
commit f059c7d9b9
  1. 2
      CMakeLists.txt
  2. 3
      HISTORY.md
  3. 2
      Makefile
  4. 1
      TARGETS
  5. 1
      buckifier/targets_cfg.py
  6. 17
      build_tools/build_detect_platform
  7. 69
      db/db_bloom_filter_test.cc
  8. 15
      include/rocksdb/filter_policy.h
  9. 3
      include/rocksdb/table.h
  10. 4
      table/block_based/block_based_table_builder.cc
  11. 430
      table/block_based/filter_policy.cc
  12. 81
      table/block_based/filter_policy_internal.h
  13. 12
      table/block_based/full_filter_block_test.cc
  14. 2
      table/block_based/parsed_full_filter_block.h
  15. 2
      table/block_based/partitioned_filter_block_test.cc
  16. 2
      table/format.h
  17. 2
      test_util/testutil.cc
  18. 214
      util/bloom_impl.h
  19. 278
      util/bloom_test.cc
  20. 34
      util/filter_bench.cc

@ -470,7 +470,7 @@ endif()
include_directories(${PROJECT_SOURCE_DIR})
include_directories(${PROJECT_SOURCE_DIR}/include)
include_directories(SYSTEM ${PROJECT_SOURCE_DIR}/third-party/gtest-1.8.1/fused-src)
if(WITH_FOLLY_DISTRIBUTED_MUTEX)
if(NOT ROCKSDB_LITE)
include_directories(${PROJECT_SOURCE_DIR}/third-party/folly)
endif()
find_package(Threads REQUIRED)

@ -5,11 +5,14 @@
* Changed the default value of periodic_compaction_seconds to `UINT64_MAX` which allows RocksDB to auto-tune periodic compaction scheduling. When using the default value, periodic compactions are now auto-enabled if a compaction filter is used. A value of `0` will turn off the feature completely.
* With FIFO compaction style, options.periodic_compaction_seconds will have the same meaning as options.ttl. Whichever stricter will be used. With the default options.periodic_compaction_seconds value with options.ttl's default of 0, RocksDB will give a default of 30 days.
* Added an API GetCreationTimeOfOldestFile(uint64_t* creation_time) to get the file_creation_time of the oldest SST file in the DB.
* An unlikely usage of FilterPolicy is no longer supported. Calling GetFilterBitsBuilder() on the FilterPolicy returned by NewBloomFilterPolicy will now cause an assertion violation in debug builds, because RocksDB has internally migrated to a more elaborate interface that is expected to evolve further. Custom implementations of FilterPolicy should work as before, except those wrapping the return of NewBloomFilterPolicy, which will require a new override of a protected function in FilterPolicy.
### New Features
* Universal compaction to support options.periodic_compaction_seconds. A full compaction will be triggered if any file is over the threshold.
* `GetLiveFilesMetaData` and `GetColumnFamilyMetaData` now expose the file number of SST files as well as the oldest blob file referenced by each SST.
* A batched MultiGet API (DB::MultiGet()) that supports retrieving keys from multiple column families.
* Full and partitioned filters in the block-based table use an improved Bloom filter implementation, enabled with format_version 5 (or above) because previous releases cannot read this filter. This replacement is faster and more accurate, especially for high bits per key or millions of keys in a single (full) filter. For example, the new Bloom filter has a lower false positive rate at 16 bits per key than the old one at 100 bits per key.
* Added AVX2 instructions to USE_SSE builds to accelerate the new Bloom filter and XXH3-based hash function on compatible x86_64 platforms (Haswell and later, ~2014).
### Performance Improvements
* For 64-bit hashing, RocksDB is standardizing on a slightly modified preview version of XXH3. This function is now used for many non-persisted hashes, along with fastrange64() in place of the modulus operator, and some benchmarks show a slight improvement.

@ -320,7 +320,7 @@ else
PLATFORM_CXXFLAGS += -isystem $(GTEST_DIR)
endif
ifeq ($(USE_FOLLY_DISTRIBUTED_MUTEX),1)
ifeq ($(filter -DROCKSDB_LITE,$(OPT)),)
FOLLY_DIR = ./third-party/folly
# AIX: pre-defined system headers are surrounded by an extern "C" block
ifeq ($(PLATFORM), OS_AIX)

@ -77,6 +77,7 @@ ROCKSDB_PREPROCESSOR_FLAGS = [
# Directories with files for #include
"-I" + REPO_PATH + "include/",
"-I" + REPO_PATH,
"-I" + REPO_PATH + "third-party/folly/",
]
ROCKSDB_ARCH_PREPROCESSOR_FLAGS = {

@ -83,6 +83,7 @@ ROCKSDB_PREPROCESSOR_FLAGS = [
# Directories with files for #include
"-I" + REPO_PATH + "include/",
"-I" + REPO_PATH,
"-I" + REPO_PATH + "third-party/folly/",
]
ROCKSDB_ARCH_PREPROCESSOR_FLAGS = {

@ -572,6 +572,8 @@ if test "$TRY_SSE_ETC"; then
TRY_SSE42="-msse4.2"
# PCLMUL available since westmere, ca. 2010-2011
TRY_PCLMUL="-mpclmul"
# AVX2 available since haswell, ca. 2013-2015
TRY_AVX2="-mavx2"
fi
$CXX $PLATFORM_CXXFLAGS $COMMON_FLAGS $TRY_SSE42 -x c++ - -o /dev/null 2>/dev/null <<EOF
@ -605,6 +607,21 @@ elif test "$USE_SSE"; then
echo "warning: USE_SSE specified but compiler could not use PCLMUL intrinsics, disabling" >&2
fi
$CXX $PLATFORM_CXXFLAGS $COMMON_FLAGS $TRY_AVX2 -x c++ - -o /dev/null 2>/dev/null <<EOF
#include <cstdint>
#include <immintrin.h>
int main() {
const auto a = _mm256_setr_epi32(0, 1, 2, 3, 4, 7, 6, 5);
const auto b = _mm256_permutevar8x32_epi32(a, a);
(void)b;
}
EOF
if [ "$?" = 0 ]; then
COMMON_FLAGS="$COMMON_FLAGS $TRY_AVX2 -DHAVE_AVX2"
elif test "$USE_SSE"; then
echo "warning: USE_SSE specified but compiler could not use AVX2 intrinsics, disabling" >&2
fi
$CXX $PLATFORM_CXXFLAGS $COMMON_FLAGS -x c++ - -o /dev/null 2>/dev/null <<EOF
#include <cstdint>
int main() {

@ -7,6 +7,9 @@
// 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.
#ifndef ROCKSDB_LITE
#include <folly/Optional.h>
#endif // ROCKSDB_LITE
#include "db/db_test_util.h"
#include "port/stack_trace.h"
#include "rocksdb/perf_context.h"
@ -27,10 +30,10 @@ class DBBloomFilterTest : public DBTestBase {
class DBBloomFilterTestWithParam : public DBTestBase,
public testing::WithParamInterface<
std::tuple<BFP::Impl, bool, uint32_t>> {
std::tuple<BFP::Mode, bool, uint32_t>> {
// public testing::WithParamInterface<bool> {
protected:
BFP::Impl bfp_impl_;
BFP::Mode bfp_impl_;
bool partition_filters_;
uint32_t format_version_;
@ -506,23 +509,26 @@ TEST_P(DBBloomFilterTestWithParam, BloomFilter) {
INSTANTIATE_TEST_CASE_P(
FormatDef, DBBloomFilterTestDefFormatVersion,
::testing::Values(
std::make_tuple(BFP::kBlock, false, test::kDefaultFormatVersion),
std::make_tuple(BFP::kFull, true, test::kDefaultFormatVersion),
std::make_tuple(BFP::kFull, false, test::kDefaultFormatVersion)));
std::make_tuple(BFP::kDeprecatedBlock, false,
test::kDefaultFormatVersion),
std::make_tuple(BFP::kAuto, true, test::kDefaultFormatVersion),
std::make_tuple(BFP::kAuto, false, test::kDefaultFormatVersion)));
INSTANTIATE_TEST_CASE_P(
FormatDef, DBBloomFilterTestWithParam,
::testing::Values(
std::make_tuple(BFP::kBlock, false, test::kDefaultFormatVersion),
std::make_tuple(BFP::kFull, true, test::kDefaultFormatVersion),
std::make_tuple(BFP::kFull, false, test::kDefaultFormatVersion)));
std::make_tuple(BFP::kDeprecatedBlock, false,
test::kDefaultFormatVersion),
std::make_tuple(BFP::kAuto, true, test::kDefaultFormatVersion),
std::make_tuple(BFP::kAuto, false, test::kDefaultFormatVersion)));
INSTANTIATE_TEST_CASE_P(
FormatLatest, DBBloomFilterTestWithParam,
::testing::Values(
std::make_tuple(BFP::kBlock, false, test::kLatestFormatVersion),
std::make_tuple(BFP::kFull, true, test::kLatestFormatVersion),
std::make_tuple(BFP::kFull, false, test::kLatestFormatVersion)));
std::make_tuple(BFP::kDeprecatedBlock, false,
test::kLatestFormatVersion),
std::make_tuple(BFP::kAuto, true, test::kLatestFormatVersion),
std::make_tuple(BFP::kAuto, false, test::kLatestFormatVersion)));
#endif // ROCKSDB_VALGRIND_RUN
TEST_F(DBBloomFilterTest, BloomFilterRate) {
@ -861,27 +867,27 @@ TEST_F(DBBloomFilterTest, MemtablePrefixBloomOutOfDomain) {
#ifndef ROCKSDB_LITE
class BloomStatsTestWithParam
: public DBBloomFilterTest,
public testing::WithParamInterface<std::tuple<bool, BFP::Impl, bool>> {
public testing::WithParamInterface<
std::tuple<folly::Optional<BFP::Mode>, bool>> {
public:
BloomStatsTestWithParam() {
use_block_table_ = std::get<0>(GetParam());
bfp_impl_ = std::get<1>(GetParam());
partition_filters_ = std::get<2>(GetParam());
bfp_impl_ = std::get<0>(GetParam());
partition_filters_ = std::get<1>(GetParam());
options_.create_if_missing = true;
options_.prefix_extractor.reset(rocksdb::NewFixedPrefixTransform(4));
options_.memtable_prefix_bloom_size_ratio =
8.0 * 1024.0 / static_cast<double>(options_.write_buffer_size);
if (use_block_table_) {
if (bfp_impl_) {
BlockBasedTableOptions table_options;
table_options.hash_index_allow_collision = false;
if (partition_filters_) {
assert(bfp_impl_ != BFP::kBlock);
assert(*bfp_impl_ != BFP::kDeprecatedBlock);
table_options.partition_filters = partition_filters_;
table_options.index_type =
BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch;
}
table_options.filter_policy.reset(new BFP(10, bfp_impl_));
table_options.filter_policy.reset(new BFP(10, *bfp_impl_));
options_.table_factory.reset(NewBlockBasedTableFactory(table_options));
} else {
assert(!partition_filters_); // not supported in plain table
@ -903,8 +909,7 @@ class BloomStatsTestWithParam
static void SetUpTestCase() {}
static void TearDownTestCase() {}
bool use_block_table_;
BFP::Impl bfp_impl_;
folly::Optional<BFP::Mode> bfp_impl_;
bool partition_filters_;
Options options_;
};
@ -1008,7 +1013,7 @@ TEST_P(BloomStatsTestWithParam, BloomStatsTestWithIter) {
ASSERT_EQ(value3, iter->value().ToString());
// The seek doesn't check block-based bloom filter because last index key
// starts with the same prefix we're seeking to.
uint64_t expected_hits = bfp_impl_ == BFP::kBlock ? 1 : 2;
uint64_t expected_hits = bfp_impl_ == BFP::kDeprecatedBlock ? 1 : 2;
ASSERT_EQ(expected_hits, get_perf_context()->bloom_sst_hit_count);
iter->Seek(key2);
@ -1020,10 +1025,12 @@ TEST_P(BloomStatsTestWithParam, BloomStatsTestWithIter) {
INSTANTIATE_TEST_CASE_P(
BloomStatsTestWithParam, BloomStatsTestWithParam,
::testing::Values(std::make_tuple(true, BFP::kBlock, false),
std::make_tuple(true, BFP::kFull, false),
std::make_tuple(true, BFP::kFull, true),
std::make_tuple(false, BFP::kFull, false)));
::testing::Values(std::make_tuple(BFP::kDeprecatedBlock, false),
std::make_tuple(BFP::kLegacyBloom, false),
std::make_tuple(BFP::kLegacyBloom, true),
std::make_tuple(BFP::kFastLocalBloom, false),
std::make_tuple(BFP::kFastLocalBloom, true),
std::make_tuple(folly::Optional<BFP::Mode>(), false)));
namespace {
void PrefixScanInit(DBBloomFilterTest* dbtest) {
@ -1330,8 +1337,8 @@ int CountIter(std::unique_ptr<Iterator>& iter, const Slice& key) {
// into the same string, or 2) the transformed seek key is of the same length
// as the upper bound and two keys are adjacent according to the comparator.
TEST_F(DBBloomFilterTest, DynamicBloomFilterUpperBound) {
for (auto bfp_impl : BFP::kAllImpls) {
int using_full_builder = bfp_impl != BFP::kBlock;
for (auto bfp_impl : BFP::kAllFixedImpls) {
int using_full_builder = bfp_impl != BFP::kDeprecatedBlock;
Options options;
options.create_if_missing = true;
options.prefix_extractor.reset(NewCappedPrefixTransform(4));
@ -1461,8 +1468,8 @@ TEST_F(DBBloomFilterTest, DynamicBloomFilterUpperBound) {
// Create multiple SST files each with a different prefix_extractor config,
// verify iterators can read all SST files using the latest config.
TEST_F(DBBloomFilterTest, DynamicBloomFilterMultipleSST) {
for (auto bfp_impl : BFP::kAllImpls) {
int using_full_builder = bfp_impl != BFP::kBlock;
for (auto bfp_impl : BFP::kAllFixedImpls) {
int using_full_builder = bfp_impl != BFP::kDeprecatedBlock;
Options options;
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(1));
@ -1596,7 +1603,7 @@ TEST_F(DBBloomFilterTest, DynamicBloomFilterMultipleSST) {
// as expected
TEST_F(DBBloomFilterTest, DynamicBloomFilterNewColumnFamily) {
int iteration = 0;
for (auto bfp_impl : BFP::kAllImpls) {
for (auto bfp_impl : BFP::kAllFixedImpls) {
Options options = CurrentOptions();
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(1));
@ -1654,7 +1661,7 @@ TEST_F(DBBloomFilterTest, DynamicBloomFilterNewColumnFamily) {
// Verify it's possible to change prefix_extractor at runtime and iterators
// behaves as expected
TEST_F(DBBloomFilterTest, DynamicBloomFilterOptions) {
for (auto bfp_impl : BFP::kAllImpls) {
for (auto bfp_impl : BFP::kAllFixedImpls) {
Options options;
options.create_if_missing = true;
options.prefix_extractor.reset(NewFixedPrefixTransform(1));

@ -80,6 +80,9 @@ class FilterBitsReader {
}
};
// Internal type required for FilterPolicy
struct FilterBuildingContext;
// We add a new format of filter block called full filter block
// This new interface gives you more space of customization
//
@ -131,6 +134,18 @@ class FilterPolicy {
const Slice& /*contents*/) const {
return nullptr;
}
protected:
// An internal-use-only variant of GetFilterBitsBuilder that allows
// a built-in FilterPolicy to customize the builder for contextual
// constraints and hints. (Name changed to avoid triggering
// -Werror=overloaded-virtual.)
virtual FilterBitsBuilder* GetFilterBitsBuilderInternal(
const FilterBuildingContext&) const {
return GetFilterBitsBuilder();
}
friend FilterBuildingContext;
};
// Return a new filter policy that uses a bloom filter with approximately

@ -269,6 +269,9 @@ struct BlockBasedTableOptions {
// probably use this as it would reduce the index size.
// This option only affects newly written tables. When reading existing
// tables, the information about version is read from the footer.
// 5 -- Can be read by RocksDB's versions since X.X.X (something after 6.4.6)
// Full and partitioned filters use a generally faster and more accurate
// Bloom filter implementation, with a different schema.
uint32_t format_version = 2;
// Store index blocks on disk in compressed format. Changing this option to

@ -25,7 +25,6 @@
#include "rocksdb/cache.h"
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/filter_policy.h"
#include "rocksdb/flush_block_policy.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/table.h"
@ -36,6 +35,7 @@
#include "table/block_based/block_based_table_reader.h"
#include "table/block_based/block_builder.h"
#include "table/block_based/filter_block.h"
#include "table/block_based/filter_policy_internal.h"
#include "table/block_based/full_filter_block.h"
#include "table/block_based/partitioned_filter_block.h"
#include "table/format.h"
@ -68,7 +68,7 @@ FilterBlockBuilder* CreateFilterBlockBuilder(
if (table_opt.filter_policy == nullptr) return nullptr;
FilterBitsBuilder* filter_bits_builder =
table_opt.filter_policy->GetFilterBitsBuilder();
FilterBuildingContext(table_opt).GetBuilder();
if (filter_bits_builder == nullptr) {
return new BlockBasedFilterBlockBuilder(mopt.prefix_extractor.get(),
table_opt);

@ -7,6 +7,8 @@
// 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 <array>
#include "rocksdb/filter_policy.h"
#include "rocksdb/slice.h"
@ -22,34 +24,179 @@ namespace rocksdb {
namespace {
typedef LegacyLocalityBloomImpl</*ExtraRotates*/ false> LegacyFullFilterImpl;
// See description in FastLocalBloomImpl
class FastLocalBloomBitsBuilder : public BuiltinFilterBitsBuilder {
public:
FastLocalBloomBitsBuilder(const int bits_per_key, const int num_probes)
: bits_per_key_(bits_per_key), num_probes_(num_probes) {
assert(bits_per_key_);
}
// No Copy allowed
FastLocalBloomBitsBuilder(const FastLocalBloomBitsBuilder&) = delete;
void operator=(const FastLocalBloomBitsBuilder&) = delete;
~FastLocalBloomBitsBuilder() override {}
virtual void AddKey(const Slice& key) override {
uint64_t hash = GetSliceHash64(key);
if (hash_entries_.size() == 0 || hash != hash_entries_.back()) {
hash_entries_.push_back(hash);
}
}
virtual Slice Finish(std::unique_ptr<const char[]>* buf) override {
uint32_t len_with_metadata =
CalculateSpace(static_cast<uint32_t>(hash_entries_.size()));
char* data = new char[len_with_metadata];
memset(data, 0, len_with_metadata);
assert(data);
assert(len_with_metadata >= 5);
uint32_t len = len_with_metadata - 5;
if (len > 0) {
AddAllEntries(data, len);
}
// See BloomFilterPolicy::GetBloomBitsReader re: metadata
// -1 = Marker for newer Bloom implementations
data[len] = static_cast<char>(-1);
// 0 = Marker for this sub-implementation
data[len + 1] = static_cast<char>(0);
// num_probes (and 0 in upper bits for 64-byte block size)
data[len + 2] = static_cast<char>(num_probes_);
// rest of metadata stays zero
const char* const_data = data;
buf->reset(const_data);
hash_entries_.clear();
return Slice(data, len_with_metadata);
}
int CalculateNumEntry(const uint32_t bytes) override {
uint32_t bytes_no_meta = bytes >= 5u ? bytes - 5u : 0;
return static_cast<int>(uint64_t{8} * bytes_no_meta / bits_per_key_);
}
uint32_t CalculateSpace(const int num_entry) override {
uint32_t num_cache_lines = 0;
if (bits_per_key_ > 0 && num_entry > 0) {
num_cache_lines = static_cast<uint32_t>(
(int64_t{num_entry} * bits_per_key_ + 511) / 512);
}
return num_cache_lines * 64 + /*metadata*/ 5;
}
private:
void AddAllEntries(char* data, uint32_t len) const {
// Simple version without prefetching:
//
// for (auto h : hash_entries_) {
// FastLocalBloomImpl::AddHash(Lower32of64(h), Upper32of64(h), len,
// num_probes_, data);
// }
const size_t num_entries = hash_entries_.size();
constexpr size_t kBufferMask = 7;
static_assert(((kBufferMask + 1) & kBufferMask) == 0,
"Must be power of 2 minus 1");
std::array<uint32_t, kBufferMask + 1> hashes;
std::array<uint32_t, kBufferMask + 1> byte_offsets;
// Prime the buffer
size_t i = 0;
for (; i <= kBufferMask && i < num_entries; ++i) {
uint64_t h = hash_entries_[i];
FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data,
/*out*/ &byte_offsets[i]);
hashes[i] = Upper32of64(h);
}
// Process and buffer
for (; i < num_entries; ++i) {
uint32_t& hash_ref = hashes[i & kBufferMask];
uint32_t& byte_offset_ref = byte_offsets[i & kBufferMask];
// Process (add)
FastLocalBloomImpl::AddHashPrepared(hash_ref, num_probes_,
data + byte_offset_ref);
// And buffer
uint64_t h = hash_entries_[i];
FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data,
/*out*/ &byte_offset_ref);
hash_ref = Upper32of64(h);
}
class FullFilterBitsBuilder : public BuiltinFilterBitsBuilder {
// Finish processing
for (i = 0; i <= kBufferMask && i < num_entries; ++i) {
FastLocalBloomImpl::AddHashPrepared(hashes[i], num_probes_,
data + byte_offsets[i]);
}
}
int bits_per_key_;
int num_probes_;
std::vector<uint64_t> hash_entries_;
};
// See description in FastLocalBloomImpl
class FastLocalBloomBitsReader : public FilterBitsReader {
public:
explicit FullFilterBitsBuilder(const int bits_per_key, const int num_probes);
FastLocalBloomBitsReader(const char* data, int num_probes, uint32_t len_bytes)
: data_(data), num_probes_(num_probes), len_bytes_(len_bytes) {}
// No Copy allowed
FullFilterBitsBuilder(const FullFilterBitsBuilder&) = delete;
void operator=(const FullFilterBitsBuilder&) = delete;
FastLocalBloomBitsReader(const FastLocalBloomBitsReader&) = delete;
void operator=(const FastLocalBloomBitsReader&) = delete;
~FastLocalBloomBitsReader() override {}
bool MayMatch(const Slice& key) override {
uint64_t h = GetSliceHash64(key);
uint32_t byte_offset;
FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_,
/*out*/ &byte_offset);
return FastLocalBloomImpl::HashMayMatchPrepared(Upper32of64(h), num_probes_,
data_ + byte_offset);
}
virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override {
std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes;
std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets;
for (int i = 0; i < num_keys; ++i) {
uint64_t h = GetSliceHash64(*keys[i]);
FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_,
/*out*/ &byte_offsets[i]);
hashes[i] = Upper32of64(h);
}
for (int i = 0; i < num_keys; ++i) {
may_match[i] = FastLocalBloomImpl::HashMayMatchPrepared(
hashes[i], num_probes_, data_ + byte_offsets[i]);
}
}
private:
const char* data_;
const int num_probes_;
const uint32_t len_bytes_;
};
~FullFilterBitsBuilder() override;
using LegacyBloomImpl = LegacyLocalityBloomImpl</*ExtraRotates*/ false>;
class LegacyBloomBitsBuilder : public BuiltinFilterBitsBuilder {
public:
explicit LegacyBloomBitsBuilder(const int bits_per_key, const int num_probes);
// No Copy allowed
LegacyBloomBitsBuilder(const LegacyBloomBitsBuilder&) = delete;
void operator=(const LegacyBloomBitsBuilder&) = delete;
~LegacyBloomBitsBuilder() override;
void AddKey(const Slice& key) override;
// 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 calculated
// +----------------------------------------------------------------+
// | filter data with length total_bits/8 |
// +----------------------------------------------------------------+
// | |
// | ... |
// | |
// +----------------------------------------------------------------+
// | ... | num_probes : 1 byte | num_lines : 4 bytes |
// +----------------------------------------------------------------+
Slice Finish(std::unique_ptr<const char[]>* buf) override;
int CalculateNumEntry(const uint32_t bytes) override;
@ -81,22 +228,22 @@ class FullFilterBitsBuilder : public BuiltinFilterBitsBuilder {
void AddHash(uint32_t h, char* data, uint32_t num_lines, uint32_t total_bits);
};
FullFilterBitsBuilder::FullFilterBitsBuilder(const int bits_per_key,
const int num_probes)
LegacyBloomBitsBuilder::LegacyBloomBitsBuilder(const int bits_per_key,
const int num_probes)
: bits_per_key_(bits_per_key), num_probes_(num_probes) {
assert(bits_per_key_);
}
FullFilterBitsBuilder::~FullFilterBitsBuilder() {}
LegacyBloomBitsBuilder::~LegacyBloomBitsBuilder() {}
void FullFilterBitsBuilder::AddKey(const Slice& key) {
void LegacyBloomBitsBuilder::AddKey(const Slice& key) {
uint32_t hash = BloomHash(key);
if (hash_entries_.size() == 0 || hash != hash_entries_.back()) {
hash_entries_.push_back(hash);
}
}
Slice FullFilterBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) {
Slice LegacyBloomBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) {
uint32_t total_bits, num_lines;
char* data = ReserveSpace(static_cast<int>(hash_entries_.size()), &total_bits,
&num_lines);
@ -107,6 +254,7 @@ Slice FullFilterBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) {
AddHash(h, data, num_lines, total_bits);
}
}
// See BloomFilterPolicy::GetFilterBitsReader for metadata
data[total_bits / 8] = static_cast<char>(num_probes_);
EncodeFixed32(data + total_bits / 8 + 1, static_cast<uint32_t>(num_lines));
@ -117,7 +265,7 @@ Slice FullFilterBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) {
return Slice(data, total_bits / 8 + 5);
}
uint32_t FullFilterBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) {
uint32_t LegacyBloomBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) {
uint32_t num_lines =
(total_bits + CACHE_LINE_SIZE * 8 - 1) / (CACHE_LINE_SIZE * 8);
@ -129,9 +277,9 @@ uint32_t FullFilterBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) {
return num_lines * (CACHE_LINE_SIZE * 8);
}
uint32_t FullFilterBitsBuilder::CalculateSpace(const int num_entry,
uint32_t* total_bits,
uint32_t* num_lines) {
uint32_t LegacyBloomBitsBuilder::CalculateSpace(const int num_entry,
uint32_t* total_bits,
uint32_t* num_lines) {
assert(bits_per_key_);
if (num_entry != 0) {
uint32_t total_bits_tmp = static_cast<uint32_t>(num_entry * bits_per_key_);
@ -151,16 +299,16 @@ uint32_t FullFilterBitsBuilder::CalculateSpace(const int num_entry,
return sz;
}
char* FullFilterBitsBuilder::ReserveSpace(const int num_entry,
uint32_t* total_bits,
uint32_t* num_lines) {
char* LegacyBloomBitsBuilder::ReserveSpace(const int num_entry,
uint32_t* total_bits,
uint32_t* num_lines) {
uint32_t sz = CalculateSpace(num_entry, total_bits, num_lines);
char* data = new char[sz];
memset(data, 0, sz);
return data;
}
int FullFilterBitsBuilder::CalculateNumEntry(const uint32_t bytes) {
int LegacyBloomBitsBuilder::CalculateNumEntry(const uint32_t bytes) {
assert(bits_per_key_);
assert(bytes > 0);
int high = static_cast<int>(bytes * 8 / bits_per_key_ + 1);
@ -175,43 +323,32 @@ int FullFilterBitsBuilder::CalculateNumEntry(const uint32_t bytes) {
return n;
}
inline void FullFilterBitsBuilder::AddHash(uint32_t h, char* data,
uint32_t num_lines, uint32_t total_bits) {
inline void LegacyBloomBitsBuilder::AddHash(uint32_t h, char* data,
uint32_t num_lines,
uint32_t total_bits) {
#ifdef NDEBUG
static_cast<void>(total_bits);
#endif
assert(num_lines > 0 && total_bits > 0);
LegacyFullFilterImpl::AddHash(h, num_lines, num_probes_, data,
folly::constexpr_log2(CACHE_LINE_SIZE));
LegacyBloomImpl::AddHash(h, num_lines, num_probes_, data,
folly::constexpr_log2(CACHE_LINE_SIZE));
}
class AlwaysTrueFilter : public FilterBitsReader {
public:
bool MayMatch(const Slice&) override { return true; }
using FilterBitsReader::MayMatch; // inherit overload
};
class AlwaysFalseFilter : public FilterBitsReader {
public:
bool MayMatch(const Slice&) override { return false; }
using FilterBitsReader::MayMatch; // inherit overload
};
class FullFilterBitsReader : public FilterBitsReader {
class LegacyBloomBitsReader : public FilterBitsReader {
public:
FullFilterBitsReader(const char* data, int num_probes, uint32_t num_lines,
uint32_t log2_cache_line_size)
LegacyBloomBitsReader(const char* data, int num_probes, uint32_t num_lines,
uint32_t log2_cache_line_size)
: data_(data),
num_probes_(num_probes),
num_lines_(num_lines),
log2_cache_line_size_(log2_cache_line_size) {}
// No Copy allowed
FullFilterBitsReader(const FullFilterBitsReader&) = delete;
void operator=(const FullFilterBitsReader&) = delete;
LegacyBloomBitsReader(const LegacyBloomBitsReader&) = delete;
void operator=(const LegacyBloomBitsReader&) = delete;
~FullFilterBitsReader() override {}
~LegacyBloomBitsReader() override {}
// "contents" contains the data built by a preceding call to
// FilterBitsBuilder::Finish. MayMatch must return true if the key was
@ -221,23 +358,23 @@ class FullFilterBitsReader : public FilterBitsReader {
bool MayMatch(const Slice& key) override {
uint32_t hash = BloomHash(key);
uint32_t byte_offset;
LegacyFullFilterImpl::PrepareHashMayMatch(
LegacyBloomImpl::PrepareHashMayMatch(
hash, num_lines_, data_, /*out*/ &byte_offset, log2_cache_line_size_);
return LegacyFullFilterImpl::HashMayMatchPrepared(
return LegacyBloomImpl::HashMayMatchPrepared(
hash, num_probes_, data_ + byte_offset, log2_cache_line_size_);
}
virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override {
uint32_t hashes[MultiGetContext::MAX_BATCH_SIZE];
uint32_t byte_offsets[MultiGetContext::MAX_BATCH_SIZE];
std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes;
std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets;
for (int i = 0; i < num_keys; ++i) {
hashes[i] = BloomHash(*keys[i]);
LegacyFullFilterImpl::PrepareHashMayMatch(hashes[i], num_lines_, data_,
/*out*/ &byte_offsets[i],
log2_cache_line_size_);
LegacyBloomImpl::PrepareHashMayMatch(hashes[i], num_lines_, data_,
/*out*/ &byte_offsets[i],
log2_cache_line_size_);
}
for (int i = 0; i < num_keys; ++i) {
may_match[i] = LegacyFullFilterImpl::HashMayMatchPrepared(
may_match[i] = LegacyBloomImpl::HashMayMatchPrepared(
hashes[i], num_probes_, data_ + byte_offsets[i],
log2_cache_line_size_);
}
@ -250,15 +387,33 @@ class FullFilterBitsReader : public FilterBitsReader {
const uint32_t log2_cache_line_size_;
};
class AlwaysTrueFilter : public FilterBitsReader {
public:
bool MayMatch(const Slice&) override { return true; }
using FilterBitsReader::MayMatch; // inherit overload
};
class AlwaysFalseFilter : public FilterBitsReader {
public:
bool MayMatch(const Slice&) override { return false; }
using FilterBitsReader::MayMatch; // inherit overload
};
} // namespace
const std::vector<BloomFilterPolicy::Impl> BloomFilterPolicy::kAllImpls = {
kFull,
kBlock,
const std::vector<BloomFilterPolicy::Mode> BloomFilterPolicy::kAllFixedImpls = {
kLegacyBloom,
kDeprecatedBlock,
kFastLocalBloom,
};
BloomFilterPolicy::BloomFilterPolicy(int bits_per_key, Impl impl)
: bits_per_key_(bits_per_key), impl_(impl) {
const std::vector<BloomFilterPolicy::Mode> BloomFilterPolicy::kAllUserModes = {
kDeprecatedBlock,
kAuto,
};
BloomFilterPolicy::BloomFilterPolicy(int bits_per_key, Mode mode)
: bits_per_key_(bits_per_key), mode_(mode) {
// We intentionally round down to reduce probing cost a little bit
num_probes_ = static_cast<int>(bits_per_key_ * 0.69); // 0.69 =~ ln(2)
if (num_probes_ < 1) num_probes_ = 1;
@ -275,7 +430,7 @@ void BloomFilterPolicy::CreateFilter(const Slice* keys, int n,
std::string* dst) const {
// We should ideally only be using this deprecated interface for
// appropriately constructed BloomFilterPolicy
assert(impl_ == kBlock);
assert(mode_ == kDeprecatedBlock);
// Compute bloom filter size (in both bits and bytes)
uint32_t bits = static_cast<uint32_t>(n * bits_per_key_);
@ -321,11 +476,41 @@ bool BloomFilterPolicy::KeyMayMatch(const Slice& key,
}
FilterBitsBuilder* BloomFilterPolicy::GetFilterBitsBuilder() const {
if (impl_ == kBlock) {
return nullptr;
} else {
return new FullFilterBitsBuilder(bits_per_key_, num_probes_);
// This code path should no longer be used, for the built-in
// BloomFilterPolicy. Internal to RocksDB and outside BloomFilterPolicy,
// only get a FilterBitsBuilder with FilterBuildingContext::GetBuilder(),
// which will call BloomFilterPolicy::GetFilterBitsBuilderInternal.
// RocksDB users have been warned (HISTORY.md) that they can no longer
// call this on the built-in BloomFilterPolicy (unlikely).
assert(false);
return GetFilterBitsBuilderInternal(
FilterBuildingContext(BlockBasedTableOptions()));
}
FilterBitsBuilder* BloomFilterPolicy::GetFilterBitsBuilderInternal(
const FilterBuildingContext& context) const {
Mode cur = mode_;
// Unusual code construction so that we can have just
// one exhaustive switch without (risky) recursion
for (int i = 0; i < 2; ++i) {
switch (cur) {
case kAuto:
if (context.table_options_.format_version < 5) {
cur = kLegacyBloom;
} else {
cur = kFastLocalBloom;
}
break;
case kDeprecatedBlock:
return nullptr;
case kFastLocalBloom:
return new FastLocalBloomBitsBuilder(bits_per_key_, num_probes_);
case kLegacyBloom:
return new LegacyBloomBitsBuilder(bits_per_key_, num_probes_);
}
}
assert(false);
return nullptr; // something legal
}
// Read metadata to determine what kind of FilterBitsReader is needed
@ -338,19 +523,37 @@ FilterBitsReader* BloomFilterPolicy::GetFilterBitsReader(
return new AlwaysFalseFilter();
}
// Legacy Bloom filter data:
// 0 +-----------------------------------+
// | Raw Bloom filter data |
// | ... |
// len +-----------------------------------+
// | byte for num_probes or |
// | marker for new implementations |
// len+1 +-----------------------------------+
// | four bytes for number of cache |
// | lines |
// len_with_meta +-----------------------------------+
int8_t raw_num_probes =
static_cast<int8_t>(contents.data()[len_with_meta - 5]);
// NB: *num_probes > 30 and < 128 probably have not been used, because of
// BloomFilterPolicy::initialize, unless directly calling
// FullFilterBitsBuilder as an API, but we are leaving those cases in
// limbo with FullFilterBitsReader for now.
// LegacyBloomBitsBuilder as an API, but we are leaving those cases in
// limbo with LegacyBloomBitsReader for now.
if (raw_num_probes < 1) {
// Note: < 0 (or unsigned > 127) indicate special new implementations
// (or reserved for future use)
if (raw_num_probes == -1) {
// Marker for newer Bloom implementations
return GetBloomBitsReader(contents);
}
// otherwise
// Treat as zero probes (always FP) for now.
// NB: < 0 (or unsigned > 127) effectively reserved for future use.
return new AlwaysTrueFilter();
}
// else attempt decode for FullFilterBitsReader
// else attempt decode for LegacyBloomBitsReader
int num_probes = raw_num_probes;
assert(num_probes >= 1);
@ -382,17 +585,82 @@ FilterBitsReader* BloomFilterPolicy::GetFilterBitsReader(
}
}
// if not early return
return new FullFilterBitsReader(contents.data(), num_probes, num_lines,
log2_cache_line_size);
return new LegacyBloomBitsReader(contents.data(), num_probes, num_lines,
log2_cache_line_size);
}
// For newer Bloom filter implementations
FilterBitsReader* BloomFilterPolicy::GetBloomBitsReader(
const Slice& contents) const {
uint32_t len_with_meta = static_cast<uint32_t>(contents.size());
uint32_t len = len_with_meta - 5;
assert(len > 0); // precondition
// New Bloom filter data:
// 0 +-----------------------------------+
// | Raw Bloom filter data |
// | ... |
// len +-----------------------------------+
// | char{-1} byte -> new Bloom filter |
// len+1 +-----------------------------------+
// | byte for subimplementation |
// | 0: FastLocalBloom |
// | other: reserved |
// len+2 +-----------------------------------+
// | byte for block_and_probes |
// | 0 in top 3 bits -> 6 -> 64-byte |
// | reserved: |
// | 1 in top 3 bits -> 7 -> 128-byte|
// | 2 in top 3 bits -> 8 -> 256-byte|
// | ... |
// | num_probes in bottom 5 bits, |
// | except 0 and 31 reserved |
// len+3 +-----------------------------------+
// | two bytes reserved |
// | possibly for hash seed |
// len_with_meta +-----------------------------------+
// Read more metadata (see above)
char sub_impl_val = contents.data()[len_with_meta - 4];
char block_and_probes = contents.data()[len_with_meta - 3];
int log2_block_bytes = ((block_and_probes >> 5) & 7) + 6;
int num_probes = (block_and_probes & 31);
if (num_probes < 1 || num_probes > 30) {
// Reserved / future safe
return new AlwaysTrueFilter();
}
uint16_t rest = DecodeFixed16(contents.data() + len_with_meta - 2);
if (rest != 0) {
// Reserved, possibly for hash seed
// Future safe
return new AlwaysTrueFilter();
}
if (sub_impl_val == 0) { // FastLocalBloom
if (log2_block_bytes == 6) { // Only block size supported for now
return new FastLocalBloomBitsReader(contents.data(), num_probes, len);
}
}
// otherwise
// Reserved / future safe
return new AlwaysTrueFilter();
}
const FilterPolicy* NewBloomFilterPolicy(int bits_per_key,
bool use_block_based_builder) {
BloomFilterPolicy::Mode m;
if (use_block_based_builder) {
return new BloomFilterPolicy(bits_per_key, BloomFilterPolicy::kBlock);
m = BloomFilterPolicy::kDeprecatedBlock;
} else {
return new BloomFilterPolicy(bits_per_key, BloomFilterPolicy::kFull);
m = BloomFilterPolicy::kAuto;
}
assert(std::find(BloomFilterPolicy::kAllUserModes.begin(),
BloomFilterPolicy::kAllUserModes.end(),
m) != BloomFilterPolicy::kAllUserModes.end());
return new BloomFilterPolicy(bits_per_key, m);
}
FilterPolicy::~FilterPolicy() { }

@ -13,6 +13,7 @@
#include <vector>
#include "rocksdb/filter_policy.h"
#include "rocksdb/table.h"
namespace rocksdb {
@ -28,27 +29,69 @@ class BuiltinFilterBitsBuilder : public FilterBitsBuilder {
virtual uint32_t CalculateSpace(const int num_entry) = 0;
};
// Current information passed to BloomFilterPolicy at filter building
// time. Subject to change.
struct FilterBuildingContext {
explicit FilterBuildingContext(const BlockBasedTableOptions& table_options)
: table_options_(table_options) {}
// A convenience function to save boilerplate
FilterBitsBuilder* GetBuilder() const {
if (table_options_.filter_policy) {
return table_options_.filter_policy->GetFilterBitsBuilderInternal(*this);
} else {
return nullptr;
}
}
const BlockBasedTableOptions& table_options_;
};
// RocksDB built-in filter policy for Bloom or Bloom-like filters.
// This class is considered internal API and subject to change.
// See NewBloomFilterPolicy.
class BloomFilterPolicy : public FilterPolicy {
public:
// An internal marker for which Bloom filter implementation to use.
// This makes it easier for tests to track or to walk over the built-in
// set of Bloom filter policies. The only variance in BloomFilterPolicy
// by implementation is in GetFilterBitsBuilder(), so an enum is practical
// here vs. subclasses.
enum Impl {
// Implementation of Bloom filter for full and partitioned filters.
// An internal marker for operating modes of BloomFilterPolicy, in terms
// of selecting an implementation. This makes it easier for tests to track
// or to walk over the built-in set of Bloom filter implementations. The
// only variance in BloomFilterPolicy by mode/implementation is in
// GetFilterBitsBuilder(), so an enum is practical here vs. subclasses.
//
// This enum is essentially the union of all the different kinds of return
// value from GetFilterBitsBuilder, or "underlying implementation", and
// higher-level modes that choose an underlying implementation based on
// context information.
enum Mode {
// Legacy implementation of Bloom filter for full and partitioned filters.
// Set to 0 in case of value confusion with bool use_block_based_builder
kFull = 0,
// NOTE: TESTING ONLY as this mode does not use best compatible
// implementation
kLegacyBloom = 0,
// Deprecated block-based Bloom filter implementation.
// Set to 1 in case of value confusion with bool use_block_based_builder
kBlock = 1,
// NOTE: DEPRECATED but user exposed
kDeprecatedBlock = 1,
// A fast, cache-local Bloom filter implementation. See description in
// FastLocalBloomImpl.
// NOTE: TESTING ONLY as this mode does not check format_version
kFastLocalBloom = 2,
// Automatically choose from the above (except kDeprecatedBlock) based on
// context at build time, including compatibility with format_version.
// NOTE: This is currently the only recommended mode that is user exposed.
kAuto = 100,
};
static const std::vector<Impl> kAllImpls;
// All the different underlying implementations that a BloomFilterPolicy
// might use, as a mode that says "always use this implementation."
// Only appropriate for unit tests.
static const std::vector<Mode> kAllFixedImpls;
// All the different modes of BloomFilterPolicy that are exposed from
// user APIs. Only appropriate for higher-level unit tests. Integration
// tests should prefer using NewBloomFilterPolicy (user-exposed).
static const std::vector<Mode> kAllUserModes;
explicit BloomFilterPolicy(int bits_per_key, Impl impl);
explicit BloomFilterPolicy(int bits_per_key, Mode mode);
~BloomFilterPolicy() override;
@ -68,11 +111,23 @@ class BloomFilterPolicy : public FilterPolicy {
// chosen for this BloomFilterPolicy. Not compatible with CreateFilter.
FilterBitsReader* GetFilterBitsReader(const Slice& contents) const override;
protected:
// To use this function, call FilterBuildingContext::GetBuilder().
//
// Neither the context nor any objects therein should be saved beyond
// the call to this function, unless it's shared_ptr.
FilterBitsBuilder* GetFilterBitsBuilderInternal(
const FilterBuildingContext&) const override;
private:
int bits_per_key_;
int num_probes_;
// Selected implementation for building new SST filters
Impl impl_;
// Selected mode (a specific implementation or way of selecting an
// implementation) for building new SST filters.
Mode mode_;
// For newer Bloom filter implementation(s)
FilterBitsReader* GetBloomBitsReader(const Slice& contents) const;
};
} // namespace rocksdb

@ -111,7 +111,7 @@ class PluginFullFilterBlockTest : public mock::MockBlockBasedTableTester,
TEST_F(PluginFullFilterBlockTest, PluginEmptyBuilder) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
nullptr, true, FilterBuildingContext(table_options_).GetBuilder());
Slice slice = builder.Finish();
ASSERT_EQ("", EscapeString(slice));
@ -131,7 +131,7 @@ TEST_F(PluginFullFilterBlockTest, PluginEmptyBuilder) {
TEST_F(PluginFullFilterBlockTest, PluginSingleChunk) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
nullptr, true, FilterBuildingContext(table_options_).GetBuilder());
builder.Add("foo");
builder.Add("bar");
builder.Add("box");
@ -189,7 +189,7 @@ class FullFilterBlockTest : public mock::MockBlockBasedTableTester,
TEST_F(FullFilterBlockTest, EmptyBuilder) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
nullptr, true, FilterBuildingContext(table_options_).GetBuilder());
Slice slice = builder.Finish();
ASSERT_EQ("", EscapeString(slice));
@ -239,7 +239,7 @@ TEST_F(FullFilterBlockTest, DuplicateEntries) {
std::unique_ptr<const SliceTransform> prefix_extractor(
NewFixedPrefixTransform(0));
auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(
table_options_.filter_policy->GetFilterBitsBuilder());
FilterBuildingContext(table_options_).GetBuilder());
const bool WHOLE_KEY = true;
FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
bits_builder);
@ -263,7 +263,7 @@ TEST_F(FullFilterBlockTest, DuplicateEntries) {
std::unique_ptr<const SliceTransform> prefix_extractor(
NewFixedPrefixTransform(7));
auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(
table_options_.filter_policy->GetFilterBitsBuilder());
FilterBuildingContext(table_options_).GetBuilder());
const bool WHOLE_KEY = true;
FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
bits_builder);
@ -280,7 +280,7 @@ TEST_F(FullFilterBlockTest, DuplicateEntries) {
TEST_F(FullFilterBlockTest, SingleChunk) {
FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());
nullptr, true, FilterBuildingContext(table_options_).GetBuilder());
ASSERT_EQ(0, builder.NumAdded());
builder.Add("foo");
builder.Add("bar");

@ -25,7 +25,7 @@ class ParsedFullFilterBlock {
return filter_bits_reader_.get();
}
// TODO: consider memory usage of FullFilterBitsReader
// TODO: consider memory usage of the FilterBitsReader
size_t ApproximateMemoryUsage() const {
return block_contents_.ApproximateMemoryUsage();
}

@ -126,7 +126,7 @@ class PartitionedFilterBlockTest
const bool kValueDeltaEncoded = true;
return new PartitionedFilterBlockBuilder(
prefix_extractor, table_options_.whole_key_filtering,
table_options_.filter_policy->GetFilterBitsBuilder(),
FilterBuildingContext(table_options_).GetBuilder(),
table_options_.index_block_restart_interval, !kValueDeltaEncoded,
p_index_builder, partition_size);
}

@ -117,7 +117,7 @@ inline uint32_t GetCompressFormatForVersion(CompressionType compression_type,
}
inline bool BlockBasedTableSupportedVersion(uint32_t version) {
return version <= 4;
return version <= 5;
}
// Footer encapsulates the fixed information stored at the tail

@ -24,7 +24,7 @@ namespace rocksdb {
namespace test {
const uint32_t kDefaultFormatVersion = BlockBasedTableOptions().format_version;
const uint32_t kLatestFormatVersion = 4u;
const uint32_t kLatestFormatVersion = 5u;
Slice RandomString(Random* rnd, int len, std::string* dst) {
dst->resize(len);

@ -12,18 +12,219 @@
#include <stdint.h>
#include "rocksdb/slice.h"
#include "util/hash.h"
#ifdef HAVE_AVX2
#include <immintrin.h>
#endif
namespace rocksdb {
// A fast, flexible, and accurate cache-local Bloom implementation with
// SIMD-optimized query performance (currently using AVX2 on Intel). Write
// performance and non-SIMD read are very good, benefiting from fastrange32
// used in place of % and single-cycle multiplication on recent processors.
//
// Most other SIMD Bloom implementations sacrifice flexibility and/or
// accuracy by requiring num_probes to be a power of two and restricting
// where each probe can occur in a cache line. This implementation sacrifices
// SIMD-optimization for add (might still be possible, especially with AVX512)
// in favor of allowing any num_probes, not crossing cache line boundary,
// and accuracy close to theoretical best accuracy for a cache-local Bloom.
// E.g. theoretical best for 10 bits/key, num_probes=6, and 512-bit bucket
// (Intel cache line size) is 0.9535% FP rate. This implementation yields
// about 0.957%. (Compare to LegacyLocalityBloomImpl<false> at 1.138%, or
// about 0.951% for 1024-bit buckets, cache line size for some ARM CPUs.)
//
// This implementation can use a 32-bit hash (let h2 be h1 * 0x9e3779b9) or
// a 64-bit hash (split into two uint32s). With many millions of keys, the
// false positive rate associated with using a 32-bit hash can dominate the
// false positive rate of the underlying filter. At 10 bits/key setting, the
// inflection point is about 40 million keys, so 32-bit hash is a bad idea
// with 10s of millions of keys or more.
//
// Despite accepting a 64-bit hash, this implementation uses 32-bit fastrange
// to pick a cache line, which can be faster than 64-bit in some cases.
// This only hurts accuracy as you get into 10s of GB for a single filter,
// and accuracy abruptly breaks down at 256GB (2^32 cache lines). Switch to
// 64-bit fastrange if you need filters so big. ;)
//
// Using only a 32-bit input hash within each cache line has negligible
// impact for any reasonable cache line / bucket size, for arbitrary filter
// size, and potentially saves intermediate data size in some cases vs.
// tracking full 64 bits. (Even in an implementation using 64-bit arithmetic
// to generate indices, I might do the same, as a single multiplication
// suffices to generate a sufficiently mixed 64 bits from 32 bits.)
//
// This implementation is currently tied to Intel cache line size, 64 bytes ==
// 512 bits. If there's sufficient demand for other cache line sizes, this is
// a pretty good implementation to extend, but slight performance enhancements
// are possible with an alternate implementation (probably not very compatible
// with SIMD):
// (1) Use rotation in addition to multiplication for remixing
// (like murmur hash). (Using multiplication alone *slightly* hurts accuracy
// because lower bits never depend on original upper bits.)
// (2) Extract more than one bit index from each re-mix. (Only if rotation
// or similar is part of remix, because otherwise you're making the
// multiplication-only problem worse.)
// (3) Re-mix full 64 bit hash, to get maximum number of bit indices per
// re-mix.
//
class FastLocalBloomImpl {
public:
static inline void AddHash(uint32_t h1, uint32_t h2, uint32_t len_bytes,
int num_probes, char *data) {
uint32_t bytes_to_cache_line = fastrange32(len_bytes >> 6, h1) << 6;
AddHashPrepared(h2, num_probes, data + bytes_to_cache_line);
}
static inline void AddHashPrepared(uint32_t h2, int num_probes,
char *data_at_cache_line) {
uint32_t h = h2;
for (int i = 0; i < num_probes; ++i, h *= uint32_t{0x9e3779b9}) {
// 9-bit address within 512 bit cache line
int bitpos = h >> (32 - 9);
data_at_cache_line[bitpos >> 3] |= (uint8_t{1} << (bitpos & 7));
}
}
static inline void PrepareHash(uint32_t h1, uint32_t len_bytes,
const char *data,
uint32_t /*out*/ *byte_offset) {
uint32_t bytes_to_cache_line = fastrange32(len_bytes >> 6, h1) << 6;
PREFETCH(data + bytes_to_cache_line, 0 /* rw */, 1 /* locality */);
PREFETCH(data + bytes_to_cache_line + 63, 0 /* rw */, 1 /* locality */);
*byte_offset = bytes_to_cache_line;
}
static inline bool HashMayMatch(uint32_t h1, uint32_t h2, uint32_t len_bytes,
int num_probes, const char *data) {
uint32_t bytes_to_cache_line = fastrange32(len_bytes >> 6, h1) << 6;
return HashMayMatchPrepared(h2, num_probes, data + bytes_to_cache_line);
}
static inline bool HashMayMatchPrepared(uint32_t h2, int num_probes,
const char *data_at_cache_line) {
uint32_t h = h2;
#ifdef HAVE_AVX2
int rem_probes = num_probes;
// NOTE: For better performance for num_probes in {1, 2, 9, 10, 17, 18,
// etc.} one can insert specialized code for rem_probes <= 2, bypassing
// the SIMD code in those cases. There is a detectable but minor overhead
// applied to other values of num_probes (when not statically determined),
// but smoother performance curve vs. num_probes. But for now, when
// in doubt, don't add unnecessary code.
// Powers of 32-bit golden ratio, mod 2**32.
const __m256i multipliers =
_mm256_setr_epi32(0x00000001, 0x9e3779b9, 0xe35e67b1, 0x734297e9,
0x35fbe861, 0xdeb7c719, 0x448b211, 0x3459b749);
for (;;) {
// Eight copies of hash
__m256i hash_vector = _mm256_set1_epi32(h);
// Same effect as repeated multiplication by 0x9e3779b9 thanks to
// associativity of multiplication.
hash_vector = _mm256_mullo_epi32(hash_vector, multipliers);
// Now the top 9 bits of each of the eight 32-bit values in
// hash_vector are bit addresses for probes within the cache line.
// While the platform-independent code uses byte addressing (6 bits
// to pick a byte + 3 bits to pick a bit within a byte), here we work
// with 32-bit words (4 bits to pick a word + 5 bits to pick a bit
// within a word) because that works well with AVX2 and is equivalent
// under little-endian.
// Shift each right by 28 bits to get 4-bit word addresses.
const __m256i word_addresses = _mm256_srli_epi32(hash_vector, 28);
// Gather 32-bit values spread over 512 bits by 4-bit address. In
// essence, we are dereferencing eight pointers within the cache
// line.
//
// Option 1: AVX2 gather (seems to be a little slow - understandable)
// const __m256i value_vector =
// _mm256_i32gather_epi32(static_cast<const int
// *>(data_at_cache_line),
// word_addresses,
// /*bytes / i32*/ 4);
// END Option 1
// Potentially unaligned as we're not *always* cache-aligned -> loadu
const __m256i *mm_data =
reinterpret_cast<const __m256i *>(data_at_cache_line);
__m256i lower = _mm256_loadu_si256(mm_data);
__m256i upper = _mm256_loadu_si256(mm_data + 1);
// Option 2: AVX512VL permute hack
// Only negligibly faster than Option 3, so not yet worth supporting
// const __m256i value_vector =
// _mm256_permutex2var_epi32(lower, word_addresses, upper);
// END Option 2
// Option 3: AVX2 permute+blend hack
// Use lowest three bits to order probing values, as if all from same
// 256 bit piece.
lower = _mm256_permutevar8x32_epi32(lower, word_addresses);
upper = _mm256_permutevar8x32_epi32(upper, word_addresses);
// Just top 1 bit of address, to select between lower and upper.
const __m256i upper_lower_selector = _mm256_srai_epi32(hash_vector, 31);
// Finally: the next 8 probed 32-bit values, in probing sequence order.
const __m256i value_vector =
_mm256_blendv_epi8(lower, upper, upper_lower_selector);
// END Option 3
// We might not need to probe all 8, so build a mask for selecting only
// what we need. (The k_selector(s) could be pre-computed but that
// doesn't seem to make a noticeable performance difference.)
const __m256i zero_to_seven = _mm256_setr_epi32(0, 1, 2, 3, 4, 5, 6, 7);
// Subtract rem_probes from each of those constants
__m256i k_selector =
_mm256_sub_epi32(zero_to_seven, _mm256_set1_epi32(rem_probes));
// Negative after subtract -> use/select
// Keep only high bit (logical shift right each by 31).
k_selector = _mm256_srli_epi32(k_selector, 31);
// Strip off the 4 bit word address (shift left)
__m256i bit_addresses = _mm256_slli_epi32(hash_vector, 4);
// And keep only 5-bit (32 - 27) bit-within-32-bit-word addresses.
bit_addresses = _mm256_srli_epi32(bit_addresses, 27);
// Build a bit mask
const __m256i bit_mask = _mm256_sllv_epi32(k_selector, bit_addresses);
// Like ((~value_vector) & bit_mask) == 0)
bool match = _mm256_testc_si256(value_vector, bit_mask) != 0;
// This check first so that it's easy for branch predictor to optimize
// num_probes <= 8 case, making it free of unpredictable branches.
if (rem_probes <= 8) {
return match;
} else if (!match) {
return false;
}
// otherwise
// Need another iteration. 0xab25f4c1 == golden ratio to the 8th power
h *= 0xab25f4c1;
rem_probes -= 8;
}
#else
for (int i = 0; i < num_probes; ++i, h *= uint32_t{0x9e3779b9}) {
// 9-bit address within 512 bit cache line
int bitpos = h >> (32 - 9);
if ((data_at_cache_line[bitpos >> 3] & (char(1) << (bitpos & 7))) == 0) {
return false;
}
}
return true;
#endif
}
};
// A legacy Bloom filter implementation with no locality of probes (slow).
// It uses double hashing to generate a sequence of hash values.
// Asymptotic analysis is in [Kirsch,Mitzenmacher 2006], but known to have
// subtle accuracy flaws for practical sizes [Dillinger,Manolios 2004].
//
// DO NOT REUSE - faster and more predictably accurate implementations
// are available at
// https://github.com/pdillinger/wormhashing/blob/master/bloom_simulation_tests/foo.cc
// See e.g. RocksDB DynamicBloom.
// DO NOT REUSE
//
class LegacyNoLocalityBloomImpl {
public:
@ -64,10 +265,7 @@ class LegacyNoLocalityBloomImpl {
// a 1/2 * 1/512 or absolute 0.1% FP rate penalty. More detail at
// https://github.com/facebook/rocksdb/issues/4120
//
// DO NOT REUSE - faster and more predictably accurate implementations
// are available at
// https://github.com/pdillinger/wormhashing/blob/master/bloom_simulation_tests/foo.cc
// See e.g. RocksDB DynamicBloom.
// DO NOT REUSE
//
template <bool ExtraRotates>
class LegacyLocalityBloomImpl {

@ -62,23 +62,21 @@ class BlockBasedBloomTest : public testing::Test {
std::vector<std::string> keys_;
public:
BlockBasedBloomTest()
: policy_(NewBloomFilterPolicy(FLAGS_bits_per_key, true)) {}
BlockBasedBloomTest() { ResetPolicy(); }
void Reset() {
keys_.clear();
filter_.clear();
}
void ResetPolicy(const FilterPolicy* policy = nullptr) {
if (policy == nullptr) {
policy_.reset(NewBloomFilterPolicy(FLAGS_bits_per_key, true));
} else {
policy_.reset(policy);
}
void ResetPolicy(int bits_per_key) {
policy_.reset(new BloomFilterPolicy(bits_per_key,
BloomFilterPolicy::kDeprecatedBlock));
Reset();
}
void ResetPolicy() { ResetPolicy(FLAGS_bits_per_key); }
void Add(const Slice& s) {
keys_.push_back(s.ToString());
}
@ -189,43 +187,43 @@ TEST_F(BlockBasedBloomTest, VaryingLengths) {
TEST_F(BlockBasedBloomTest, Schema) {
char buffer[sizeof(int)];
ResetPolicy(NewBloomFilterPolicy(8, true)); // num_probes = 5
ResetPolicy(8); // num_probes = 5
for (int key = 0; key < 87; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()), 3589896109U);
ResetPolicy(NewBloomFilterPolicy(9, true)); // num_probes = 6
ResetPolicy(9); // num_probes = 6
for (int key = 0; key < 87; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()), 969445585);
ResetPolicy(NewBloomFilterPolicy(11, true)); // num_probes = 7
ResetPolicy(11); // num_probes = 7
for (int key = 0; key < 87; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()), 1694458207);
ResetPolicy(NewBloomFilterPolicy(10, true)); // num_probes = 6
ResetPolicy(10); // num_probes = 6
for (int key = 0; key < 87; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()), 2373646410U);
ResetPolicy(NewBloomFilterPolicy(10, true));
for (int key = 1; key < 87; key++) {
ResetPolicy(10);
for (int key = /*CHANGED*/ 1; key < 87; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()), 1908442116);
ResetPolicy(NewBloomFilterPolicy(10, true));
for (int key = 1; key < 88; key++) {
ResetPolicy(10);
for (int key = 1; key < /*CHANGED*/ 88; key++) {
Add(Key(key, buffer));
}
Build();
@ -236,19 +234,18 @@ TEST_F(BlockBasedBloomTest, Schema) {
// Different bits-per-byte
class FullBloomTest : public testing::Test {
class FullBloomTest : public testing::TestWithParam<BloomFilterPolicy::Mode> {
private:
std::unique_ptr<const FilterPolicy> policy_;
BlockBasedTableOptions table_options_;
std::shared_ptr<const FilterPolicy>& policy_;
std::unique_ptr<FilterBitsBuilder> bits_builder_;
std::unique_ptr<FilterBitsReader> bits_reader_;
std::unique_ptr<const char[]> buf_;
size_t filter_size_;
public:
FullBloomTest() :
policy_(NewBloomFilterPolicy(FLAGS_bits_per_key, false)),
filter_size_(0) {
Reset();
FullBloomTest() : policy_(table_options_.filter_policy), filter_size_(0) {
ResetPolicy();
}
BuiltinFilterBitsBuilder* GetBuiltinFilterBitsBuilder() {
@ -257,21 +254,19 @@ class FullBloomTest : public testing::Test {
}
void Reset() {
bits_builder_.reset(policy_->GetFilterBitsBuilder());
bits_builder_.reset(FilterBuildingContext(table_options_).GetBuilder());
bits_reader_.reset(nullptr);
buf_.reset(nullptr);
filter_size_ = 0;
}
void ResetPolicy(const FilterPolicy* policy = nullptr) {
if (policy == nullptr) {
policy_.reset(NewBloomFilterPolicy(FLAGS_bits_per_key, false));
} else {
policy_.reset(policy);
}
void ResetPolicy(int bits_per_key) {
policy_.reset(new BloomFilterPolicy(bits_per_key, GetParam()));
Reset();
}
void ResetPolicy() { ResetPolicy(FLAGS_bits_per_key); }
void Add(const Slice& s) {
bits_builder_->AddKey(s);
}
@ -292,6 +287,16 @@ class FullBloomTest : public testing::Test {
Slice FilterData() { return Slice(buf_.get(), filter_size_); }
int GetNumProbesFromFilterData() {
assert(filter_size_ >= 5);
int8_t raw_num_probes = static_cast<int8_t>(buf_.get()[filter_size_ - 5]);
if (raw_num_probes == -1) { // New bloom filter marker
return static_cast<uint8_t>(buf_.get()[filter_size_ - 3]);
} else {
return raw_num_probes;
}
}
bool Matches(const Slice& s) {
if (bits_reader_ == nullptr) {
Build();
@ -299,6 +304,8 @@ class FullBloomTest : public testing::Test {
return bits_reader_->MayMatch(s);
}
// Provides a kind of fingerprint on the Bloom filter's
// behavior, for reasonbly high FP rates.
uint64_t PackedMatches() {
char buffer[sizeof(int)];
uint64_t result = 0;
@ -310,6 +317,26 @@ class FullBloomTest : public testing::Test {
return result;
}
// Provides a kind of fingerprint on the Bloom filter's
// behavior, for lower FP rates.
std::string FirstFPs(int count) {
char buffer[sizeof(int)];
std::string rv;
int fp_count = 0;
for (int i = 0; i < 1000000; i++) {
// Pack four match booleans into each hexadecimal digit
if (Matches(Key(i + 1000000, buffer))) {
++fp_count;
rv += std::to_string(i);
if (fp_count == count) {
break;
}
rv += ',';
}
}
return rv;
}
double FalsePositiveRate() {
char buffer[sizeof(int)];
int result = 0;
@ -320,9 +347,25 @@ class FullBloomTest : public testing::Test {
}
return result / 10000.0;
}
uint32_t SelectByImpl(uint32_t for_legacy_bloom,
uint32_t for_fast_local_bloom) {
switch (GetParam()) {
case BloomFilterPolicy::kLegacyBloom:
return for_legacy_bloom;
case BloomFilterPolicy::kFastLocalBloom:
return for_fast_local_bloom;
case BloomFilterPolicy::kDeprecatedBlock:
case BloomFilterPolicy::kAuto:
/* N/A */;
}
// otherwise
assert(false);
return 0;
}
};
TEST_F(FullBloomTest, FilterSize) {
TEST_P(FullBloomTest, FilterSize) {
auto bits_builder = GetBuiltinFilterBitsBuilder();
for (int n = 1; n < 100; n++) {
auto space = bits_builder->CalculateSpace(n);
@ -333,13 +376,13 @@ TEST_F(FullBloomTest, FilterSize) {
}
}
TEST_F(FullBloomTest, FullEmptyFilter) {
TEST_P(FullBloomTest, FullEmptyFilter) {
// Empty filter is not match, at this level
ASSERT_TRUE(!Matches("hello"));
ASSERT_TRUE(!Matches("world"));
}
TEST_F(FullBloomTest, FullSmall) {
TEST_P(FullBloomTest, FullSmall) {
Add("hello");
Add("world");
ASSERT_TRUE(Matches("hello"));
@ -348,7 +391,7 @@ TEST_F(FullBloomTest, FullSmall) {
ASSERT_TRUE(!Matches("foo"));
}
TEST_F(FullBloomTest, FullVaryingLengths) {
TEST_P(FullBloomTest, FullVaryingLengths) {
char buffer[sizeof(int)];
// Count number of filters that significantly exceed the false positive rate
@ -409,66 +452,175 @@ inline uint32_t SelectByCacheLineSize(uint32_t for64, uint32_t for128,
} // namespace
// Ensure the implementation doesn't accidentally change in an
// incompatible way
TEST_F(FullBloomTest, Schema) {
// incompatible way. This test doesn't check the reading side
// (FirstFPs/PackedMatches) for LegacyBloom because it requires the
// ability to read filters generated using other cache line sizes.
// See RawSchema.
TEST_P(FullBloomTest, Schema) {
char buffer[sizeof(int)];
// Use enough keys so that changing bits / key by 1 is guaranteed to
// change number of allocated cache lines. So keys > max cache line bits.
ResetPolicy(NewBloomFilterPolicy(8)); // num_probes = 5
ResetPolicy(2); // num_probes = 1
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
EXPECT_EQ(GetNumProbesFromFilterData(), 1);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(1567096579, 1964771444, 2659542661U),
3817481309U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("11,13,17,25,29,30,35,37,45,53", FirstFPs(10));
}
ResetPolicy(3); // num_probes = 2
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
EXPECT_EQ(GetNumProbesFromFilterData(), 2);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(2707206547U, 2571983456U, 218344685),
2807269961U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("4,15,17,24,27,28,29,53,63,70", FirstFPs(10));
}
ResetPolicy(5); // num_probes = 3
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(1302145999, 2811644657U, 756553699));
EXPECT_EQ(GetNumProbesFromFilterData(), 3);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(515748486, 94611728, 2436112214U),
204628445));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("15,24,29,39,53,87,89,100,103,104", FirstFPs(10));
}
ResetPolicy(NewBloomFilterPolicy(9)); // num_probes = 6
ResetPolicy(8); // num_probes = 5
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(2092755149, 661139132, 1182970461));
EXPECT_EQ(GetNumProbesFromFilterData(), 5);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(1302145999, 2811644657U, 756553699),
355564975));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("16,60,66,126,220,238,244,256,265,287", FirstFPs(10));
}
ResetPolicy(NewBloomFilterPolicy(11)); // num_probes = 7
ResetPolicy(9); // num_probes = 6
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(3755609649U, 1812694762, 1449142939));
EXPECT_EQ(GetNumProbesFromFilterData(), 6);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(2092755149, 661139132, 1182970461),
2137566013U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("156,367,791,872,945,1015,1139,1159,1265,1435", FirstFPs(10));
}
ResetPolicy(NewBloomFilterPolicy(10)); // num_probes = 6
ResetPolicy(11); // num_probes = 7
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(1478976371, 2910591341U, 1182970461));
EXPECT_EQ(GetNumProbesFromFilterData(), 7);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(3755609649U, 1812694762, 1449142939),
2561502687U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("34,74,130,236,643,882,962,1015,1035,1110", FirstFPs(10));
}
ResetPolicy(NewBloomFilterPolicy(10));
for (int key = 1; key < 2087; key++) {
ResetPolicy(14); // num_probes = 9
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(4205696321U, 1132081253U, 2385981855U));
EXPECT_EQ(GetNumProbesFromFilterData(), 9);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(178861123, 379087593, 2574136516U),
3129678118U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("130,989,2002,3225,3543,4522,4863,5256,5277", FirstFPs(9));
}
ResetPolicy(NewBloomFilterPolicy(10));
for (int key = 1; key < 2088; key++) {
ResetPolicy(16); // num_probes = 11
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
ASSERT_EQ(BloomHash(FilterData()),
SelectByCacheLineSize(2885052954U, 769447944, 4175124908U));
EXPECT_EQ(GetNumProbesFromFilterData(), 11);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(1129406313, 3049154394U, 1727750964),
1262483504));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("240,945,2660,3299,4031,4282,5173,6197,8715", FirstFPs(9));
}
ResetPolicy(10); // num_probes = 6, but different memory ratio vs. 9
for (int key = 0; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
EXPECT_EQ(GetNumProbesFromFilterData(), 6);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(1478976371, 2910591341U, 1182970461),
2498541272U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("16,126,133,422,466,472,813,1002,1035,1159", FirstFPs(10));
}
ResetPolicy(10);
for (int key = /*CHANGED*/ 1; key < 2087; key++) {
Add(Key(key, buffer));
}
Build();
EXPECT_EQ(GetNumProbesFromFilterData(), 6);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(4205696321U, 1132081253U, 2385981855U),
2058382345U));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("16,126,133,422,466,472,813,1002,1035,1159", FirstFPs(10));
}
ResetPolicy(10);
for (int key = 1; key < /*CHANGED*/ 2088; key++) {
Add(Key(key, buffer));
}
Build();
EXPECT_EQ(GetNumProbesFromFilterData(), 6);
EXPECT_EQ(
BloomHash(FilterData()),
SelectByImpl(SelectByCacheLineSize(2885052954U, 769447944, 4175124908U),
23699164));
if (GetParam() == BloomFilterPolicy::kFastLocalBloom) {
EXPECT_EQ("16,126,133,422,466,472,813,1002,1035,1159", FirstFPs(10));
}
ResetPolicy();
}
// A helper class for testing custom or corrupt filter bits as read by
// FullFilterBitsReader.
// built-in FilterBitsReaders.
struct RawFilterTester {
// Buffer, from which we always return a tail Slice, so the
// last five bytes are always the metadata bytes.
@ -502,23 +654,23 @@ struct RawFilterTester {
}
};
TEST_F(FullBloomTest, RawSchema) {
TEST_P(FullBloomTest, RawSchema) {
RawFilterTester cft;
// Two probes, about 3/4 bits set: ~50% "FP" rate
// One 256-byte cache line.
OpenRaw(cft.ResetWeirdFill(256, 1, 2));
ASSERT_EQ(uint64_t{11384799501900898790U}, PackedMatches());
EXPECT_EQ(uint64_t{11384799501900898790U}, PackedMatches());
// Two 128-byte cache lines.
OpenRaw(cft.ResetWeirdFill(256, 2, 2));
ASSERT_EQ(uint64_t{10157853359773492589U}, PackedMatches());
EXPECT_EQ(uint64_t{10157853359773492589U}, PackedMatches());
// Four 64-byte cache lines.
OpenRaw(cft.ResetWeirdFill(256, 4, 2));
ASSERT_EQ(uint64_t{7123594913907464682U}, PackedMatches());
EXPECT_EQ(uint64_t{7123594913907464682U}, PackedMatches());
}
TEST_F(FullBloomTest, CorruptFilters) {
TEST_P(FullBloomTest, CorruptFilters) {
RawFilterTester cft;
for (bool fill : {false, true}) {
@ -634,6 +786,10 @@ TEST_F(FullBloomTest, CorruptFilters) {
}
}
INSTANTIATE_TEST_CASE_P(Full, FullBloomTest,
testing::Values(BloomFilterPolicy::kLegacyBloom,
BloomFilterPolicy::kFastLocalBloom));
} // namespace rocksdb
int main(int argc, char** argv) {

@ -19,7 +19,7 @@ int main() {
#include "memory/arena.h"
#include "port/port.h"
#include "port/stack_trace.h"
#include "rocksdb/filter_policy.h"
#include "table/block_based/filter_policy_internal.h"
#include "table/block_based/full_filter_block.h"
#include "table/block_based/mock_block_based_table.h"
#include "table/plain/plain_table_bloom.h"
@ -93,12 +93,14 @@ void _always_assert_fail(int line, const char *file, const char *expr) {
using rocksdb::Arena;
using rocksdb::BlockContents;
using rocksdb::BloomFilterPolicy;
using rocksdb::BloomHash;
using rocksdb::CachableEntry;
using rocksdb::EncodeFixed32;
using rocksdb::fastrange32;
using rocksdb::FilterBitsBuilder;
using rocksdb::FilterBitsReader;
using rocksdb::FilterBuildingContext;
using rocksdb::FullFilterBlockReader;
using rocksdb::GetSliceHash;
using rocksdb::GetSliceHash64;
@ -240,8 +242,9 @@ struct FilterBench : public MockBlockBasedTableTester {
Arena arena_;
FilterBench()
: MockBlockBasedTableTester(
rocksdb::NewBloomFilterPolicy(FLAGS_bits_per_key)),
: MockBlockBasedTableTester(new BloomFilterPolicy(
FLAGS_bits_per_key,
static_cast<BloomFilterPolicy::Mode>(FLAGS_impl))),
random_(FLAGS_seed) {
for (uint32_t i = 0; i < FLAGS_batch_size; ++i) {
kms_.emplace_back(FLAGS_key_size < 8 ? 8 : FLAGS_key_size);
@ -259,17 +262,25 @@ void FilterBench::Go() {
throw std::runtime_error(
"Can't combine -use_plain_table_bloom and -use_full_block_reader");
}
if (FLAGS_impl > 1) {
throw std::runtime_error("-impl must currently be >= 0 and <= 1");
}
if (!FLAGS_use_plain_table_bloom && FLAGS_impl == 1) {
throw std::runtime_error(
"Block-based filter not currently supported by filter_bench");
if (FLAGS_use_plain_table_bloom) {
if (FLAGS_impl > 1) {
throw std::runtime_error(
"-impl must currently be >= 0 and <= 1 for Plain table");
}
} else {
if (FLAGS_impl == 1) {
throw std::runtime_error(
"Block-based filter not currently supported by filter_bench");
}
if (FLAGS_impl > 2) {
throw std::runtime_error(
"-impl must currently be 0 or 2 for Block-based table");
}
}
std::unique_ptr<FilterBitsBuilder> builder;
if (!FLAGS_use_plain_table_bloom && FLAGS_impl != 1) {
builder.reset(table_options_.filter_policy->GetFilterBitsBuilder());
builder.reset(FilterBuildingContext(table_options_).GetBuilder());
}
uint32_t variance_mask = 1;
@ -350,7 +361,8 @@ void FilterBench::Go() {
std::cout << "----------------------------" << std::endl;
std::cout << "Verifying..." << std::endl;
uint32_t outside_q_per_f = 1000000 / infos_.size();
uint32_t outside_q_per_f =
static_cast<uint32_t>(FLAGS_m_queries * 1000000 / infos_.size());
uint64_t fps = 0;
for (uint32_t i = 0; i < infos_.size(); ++i) {
FilterInfo &info = infos_[i];

Loading…
Cancel
Save