DataBlockHashIndex: Standalone Implementation with Unit Test (#4139)

Summary:
The first step of the `DataBlockHashIndex` implementation. A string based hash table is implemented and unit-tested.

`DataBlockHashIndexBuilder`: `Add()` takes pairs of `<key, restart_index>`, and formats it into a string when `Finish()` is called.
`DataBlockHashIndex`: initialized by the formatted string, and can interpret it as a hash table. Lookup for a key is supported by iterator operation.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4139

Reviewed By: sagar0

Differential Revision: D8866764

Pulled By: fgwu

fbshipit-source-id: 7f015f0098632c65979a22898a50424384730b10
This commit is contained in:
Fenggang Wu 2018-07-24 11:42:19 -07:00 committed by Facebook Github Bot
parent ea212e5316
commit 8805ec2f49
7 changed files with 455 additions and 1 deletions

View File

@ -549,6 +549,7 @@ set(SOURCES
table/cuckoo_table_builder.cc
table/cuckoo_table_factory.cc
table/cuckoo_table_reader.cc
table/data_block_hash_index.cc
table/flush_block_policy.cc
table/format.cc
table/full_filter_block.cc
@ -913,6 +914,7 @@ if(WITH_TESTS)
table/cleanable_test.cc
table/cuckoo_table_builder_test.cc
table/cuckoo_table_reader_test.cc
table/data_block_hash_index_test.cc
table/full_filter_block_test.cc
table/merger_test.cc
table/table_test.cc

View File

@ -437,6 +437,7 @@ TESTS = \
table_properties_collector_test \
arena_test \
block_test \
data_block_hash_index_test \
cache_test \
corruption_test \
slice_transform_test \
@ -1350,6 +1351,9 @@ table_test: table/table_test.o $(LIBOBJECTS) $(TESTHARNESS)
block_test: table/block_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)
data_block_hash_index_test: table/data_block_hash_index_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)
inlineskiplist_test: memtable/inlineskiplist_test.o $(LIBOBJECTS) $(TESTHARNESS)
$(AM_LINK)

View File

@ -171,6 +171,7 @@ cpp_library(
"table/cuckoo_table_builder.cc",
"table/cuckoo_table_factory.cc",
"table/cuckoo_table_reader.cc",
"table/data_block_hash_index.cc",
"table/flush_block_policy.cc",
"table/format.cc",
"table/full_filter_block.cc",
@ -372,6 +373,11 @@ ROCKS_TESTS = [
"table/block_based_filter_block_test.cc",
"serial",
],
[
"data_block_hash_index_test",
"table/data_block_hash_index_test.cc",
"serial",
],
[
"block_test",
"table/block_test.cc",
@ -1080,4 +1086,3 @@ if not is_opt_mode:
deps = [":" + test_bin],
command = [TEST_RUNNER, BUCK_BINS + test_bin]
)

2
src.mk
View File

@ -103,6 +103,7 @@ LIB_SOURCES = \
table/cuckoo_table_builder.cc \
table/cuckoo_table_factory.cc \
table/cuckoo_table_reader.cc \
table/data_block_hash_index.cc \
table/flush_block_policy.cc \
table/format.cc \
table/full_filter_block.cc \
@ -346,6 +347,7 @@ MAIN_SOURCES = \
table/cleanable_test.cc \
table/cuckoo_table_builder_test.cc \
table/cuckoo_table_reader_test.cc \
table/data_block_hash_index_test.cc \
table/full_filter_block_test.cc \
table/merger_test.cc \
table/table_reader_bench.cc \

View File

@ -0,0 +1,126 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include <string>
#include <vector>
#include "rocksdb/slice.h"
#include "table/data_block_hash_index.h"
#include "util/coding.h"
#include "util/hash.h"
namespace rocksdb {
const uint32_t kSeed = 2018;
const uint32_t kSeed_tag = 214; /* second hash seed */
inline uint16_t HashToBucket(const Slice& s, uint16_t num_buckets) {
return static_cast<uint16_t>(
rocksdb::Hash(s.data(), s.size(), kSeed) % num_buckets);
}
void DataBlockHashIndexBuilder::Add(const Slice& key,
const uint16_t& restart_index) {
uint16_t idx = HashToBucket(key, num_buckets_);
/* push a TAG to avoid false postive */
/* the TAG is the hash function value of another seed */
uint16_t tag = static_cast<uint16_t>(
rocksdb::Hash(key.data(), key.size(), kSeed_tag));
buckets_[idx].push_back(tag);
buckets_[idx].push_back(restart_index);
estimate_ += 2 * sizeof(uint16_t);
}
void DataBlockHashIndexBuilder::Finish(std::string& buffer) {
// offset is the byte offset within the buffer
std::vector<uint16_t> bucket_offsets(num_buckets_, 0);
uint16_t map_start = static_cast<uint16_t>(buffer.size());
// write each bucket to the string
for (uint16_t i = 0; i < num_buckets_; i++) {
// remember the start offset of the buckets in bucket_offsets
bucket_offsets[i] = static_cast<uint16_t>(buffer.size());
for (uint16_t elem : buckets_[i]) {
// the elem is alternative "TAG" and "offset"
PutFixed16(&buffer, elem);
}
}
// write the bucket_offsets
for (uint16_t i = 0; i < num_buckets_; i++) {
PutFixed16(&buffer, bucket_offsets[i]);
}
// write NUM_BUCK
PutFixed16(&buffer, num_buckets_);
// write MAP_START
PutFixed16(&buffer, map_start);
// Because we use uint16_t address, we only support block less than 64KB
assert(buffer.size() < (1 << 16));
}
void DataBlockHashIndexBuilder::Reset() {
// buckets_.clear();
std::fill(buckets_.begin(), buckets_.end(), std::vector<uint16_t>());
estimate_ = 0;
}
DataBlockHashIndex::DataBlockHashIndex(Slice block_content) {
assert(block_content.size() >=
2 * sizeof(uint16_t)); // NUM_BUCK and MAP_START
data_ = block_content.data();
size_ = static_cast<uint16_t>(block_content.size());
map_start_ = data_ + DecodeFixed16(data_ + size_ - sizeof(uint16_t));
assert(map_start_ < data_ + size_);
num_buckets_ = DecodeFixed16(data_ + size_ - 2 * sizeof(uint16_t));
assert(num_buckets_ > 0);
assert(size_ >= sizeof(uint16_t) * (2 + num_buckets_));
bucket_table_ = data_ + size_ - sizeof(uint16_t) * (2 + num_buckets_);
assert(map_start_ < bucket_table_);
}
DataBlockHashIndexIterator* DataBlockHashIndex::NewIterator(
const Slice& key) const {
uint16_t idx = HashToBucket(key, num_buckets_);
uint16_t bucket_off = DecodeFixed16(bucket_table_ + idx * sizeof(uint16_t));
const char* limit;
if (idx < num_buckets_ - 1) {
// limited by the start offset of the next bucket
limit = data_ + DecodeFixed16(bucket_table_ + (idx + 1) * sizeof(uint16_t));
} else {
// limited by the location of the NUM_BUCK
limit = data_ + (size_ - 2 * sizeof(uint16_t));
}
uint16_t tag = (uint16_t)rocksdb::Hash(key.data(), key.size(), kSeed_tag);
return new DataBlockHashIndexIterator(data_ + bucket_off, limit, tag);
}
bool DataBlockHashIndexIterator::Valid() {
return current_ < end_;
}
void DataBlockHashIndexIterator::Next() {
for (current_ += 2 * sizeof(uint16_t); current_ < end_;
current_ += 2 * sizeof(uint16_t)) {
// stop at a offset that match the tag, i.e. a possible match
uint16_t tag_found = DecodeFixed16(current_);
if (tag_found == tag_) {
break;
}
}
}
uint16_t DataBlockHashIndexIterator::Value() {
return DecodeFixed16(current_ + sizeof(uint16_t));
}
} // namespace rocksdb

View File

@ -0,0 +1,138 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#pragma once
#include <string>
#include <vector>
namespace rocksdb {
// This is an experimental feature aiming to reduce the CPU utilization of
// point-lookup within a data-block. It is not used in per-table index-blocks.
// It supports Get(), but not Seek() or Scan(). If the key does not exist,
// the iterator is set to invalid.
//
// A serialized hash index is appended to the data-block. The new block data
// format is as follows:
//
// DATA_BLOCK: [RI RI RI ... RI RI_IDX HASH_IDX FOOTER]
//
// RI: Restart Interval (the same as the default data-block format)
// RI_IDX: Restart Interval index (the same as the default data-block format)
// HASH_IDX: The new data-block hash index feature.
// FOOTER: A 32bit block footer, which is the NUM_RESTARTS with the MSB as
// the flag indicating if this hash index is in use. Note that
// given a data block < 32KB, the MSB is never used. So we can
// borrow the MSB as the hash index flag. Besides, this format is
// compatible with the legacy data-blocks < 32KB, as the MSB is 0.
//
// If we zoom in the HASH_IDX, the format of the data-block hash index is as
// follows:
//
// HASH_IDX: [B B B ... B IDX NUM_BUCK MAP_START]
//
// B: B = bucket, an array of pairs <TAG, restart index>.
// TAG is the second hash value of the string. It is used to flag a
// matching entry among different keys that are hashed to the same
// bucket. A similar tagging idea is used in [Lim et. al, SOSP'11].
// However we have a differnet hash design that is not based on cuckoo
// hashing as Lim's paper is.
// We do not have to store the length of individual buckets, as they
// are delimited by the next bucket offset.
// IDX: Array of offsets of the index hash bucket (relative to MAP_START)
// NUM_BUCK: Number of buckets, which is the length of the IDX array.
// MAP_START: the starting offset of the data-block hash index.
//
// Each bucket B has the following structure:
// [TAG RESTART_INDEX][TAG RESTART_INDEX]...[TAG RESTART_INDEX]
// where TAG is the hash value of the second hash funtion.
//
// pairs of <key, restart index> are inserted to the hash index. Queries will
// first lookup this hash index to find the restart index, then go to the
// corresponding restart interval to search linearly for the key.
//
// For a point-lookup for a key K:
//
// Hash1()
// 1) K ===========> bucket_id
//
// 2) Look up this bucket_id in the IDX table to find the offset of the bucket
//
// Hash2()
// 3) K ============> TAG
// 3) examine the first field (which is TAG) of each entry within this bucket,
// skip those without a matching TAG.
// 4) for the entries matching the TAG, get the restart interval index from the
// second field.
//
// (following step are implemented in block.cc)
// 5) lookup the restart index table (refer to the traditional block format),
// use the restart interval index to find the offset of the restart interval.
// 6) linearly search the restart interval for the key.
//
class DataBlockHashIndexBuilder {
public:
explicit DataBlockHashIndexBuilder(uint16_t n)
: num_buckets_(n),
buckets_(n),
estimate_((n + 2) *
sizeof(uint16_t) /* n buckets, 2 num at the end */) {}
void Add(const Slice& key, const uint16_t& restart_index);
void Finish(std::string& buffer);
void Reset();
inline size_t EstimateSize() { return estimate_; }
private:
uint16_t num_buckets_;
std::vector<std::vector<uint16_t>> buckets_;
size_t estimate_;
};
class DataBlockHashIndexIterator;
class DataBlockHashIndex {
public:
explicit DataBlockHashIndex(Slice block_content);
inline uint16_t DataBlockHashMapStart() const {
return static_cast<uint16_t>(map_start_ - data_);
}
DataBlockHashIndexIterator* NewIterator(const Slice& key) const;
private:
const char *data_;
// To make the serialized hash index compact and to save the space overhead,
// here all the data fields persisted in the block are in uint16 format.
// We find that a uint16 is large enough to index every offset of a 64KiB
// block.
// So in other words, DataBlockHashIndex does not support block size equal
// or greater then 64KiB.
uint16_t size_;
uint16_t num_buckets_;
const char *map_start_; // start of the map
const char *bucket_table_; // start offset of the bucket index table
};
class DataBlockHashIndexIterator {
public:
DataBlockHashIndexIterator(const char* start, const char* end,
const uint16_t tag)
: end_(end), tag_(tag) {
current_ = start - 2 * sizeof(uint16_t);
Next();
}
bool Valid();
void Next();
uint16_t Value();
private:
const char* end_; // the end of the bucket
const uint16_t tag_; // the fingerprint (2nd hash value) of the searching key
const char* current_;
};
} // namespace rocksdb

View File

@ -0,0 +1,177 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#include <cstdlib>
#include <string>
#include <unordered_map>
#include "rocksdb/slice.h"
#include "table/data_block_hash_index.h"
#include "util/testharness.h"
#include "util/testutil.h"
namespace rocksdb {
bool SearchForOffset(DataBlockHashIndex& index, const Slice& key,
uint16_t& restart_point) {
std::unique_ptr<DataBlockHashIndexIterator> iter;
iter.reset(index.NewIterator(key));
for (; iter->Valid(); iter->Next()) {
if (iter->Value() == restart_point) {
return true;
}
}
return false;
}
TEST(DataBlockHashIndex, DataBlockHashTestSmall) {
// bucket_num = 5, #keys = 2. 40% utilization
DataBlockHashIndexBuilder builder(5);
for (uint16_t i = 0; i < 2; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
builder.Add(key, restart_point);
}
size_t estimated_size = builder.EstimateSize();
std::string buffer("fake"), buffer2;
size_t original_size = buffer.size();
estimated_size += original_size;
builder.Finish(buffer);
ASSERT_EQ(buffer.size(), estimated_size);
buffer2 = buffer; // test for the correctness of relative offset
Slice s(buffer2);
DataBlockHashIndex index(s);
// the additional hash map should start at the end of the buffer
ASSERT_EQ(original_size, index.DataBlockHashMapStart());
for (uint16_t i = 0; i < 2; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
ASSERT_TRUE(SearchForOffset(index, key, restart_point));
}
}
TEST(DataBlockHashIndex, DataBlockHashTest) {
// bucket_num = 200, #keys = 100. 50% utilization
DataBlockHashIndexBuilder builder(200);
for (uint16_t i = 0; i < 100; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
builder.Add(key, restart_point);
}
size_t estimated_size = builder.EstimateSize();
std::string buffer("fake content"), buffer2;
size_t original_size = buffer.size();
estimated_size += original_size;
builder.Finish(buffer);
ASSERT_EQ(buffer.size(), estimated_size);
buffer2 = buffer; // test for the correctness of relative offset
Slice s(buffer2);
DataBlockHashIndex index(s);
// the additional hash map should start at the end of the buffer
ASSERT_EQ(original_size, index.DataBlockHashMapStart());
for (uint16_t i = 0; i < 100; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
ASSERT_TRUE(SearchForOffset(index, key, restart_point));
}
}
TEST(DataBlockHashIndex, DataBlockHashTestCollision) {
// bucket_num = 2. There will be intense hash collisions
DataBlockHashIndexBuilder builder(2);
for (uint16_t i = 0; i < 100; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
builder.Add(key, restart_point);
}
size_t estimated_size = builder.EstimateSize();
std::string buffer("some other fake content to take up space"), buffer2;
size_t original_size = buffer.size();
estimated_size += original_size;
builder.Finish(buffer);
ASSERT_EQ(buffer.size(), estimated_size);
buffer2 = buffer; // test for the correctness of relative offset
Slice s(buffer2);
DataBlockHashIndex index(s);
// the additional hash map should start at the end of the buffer
ASSERT_EQ(original_size, index.DataBlockHashMapStart());
for (uint16_t i = 0; i < 100; i++) {
std::string key("key" + std::to_string(i));
uint16_t restart_point = i;
ASSERT_TRUE(SearchForOffset(index, key, restart_point));
}
}
TEST(DataBlockHashIndex, DataBlockHashTestLarge) {
DataBlockHashIndexBuilder builder(1000);
std::unordered_map<std::string, uint16_t> m;
for (uint16_t i = 0; i < 10000; i++) {
if (i % 2) {
continue; // leave half of the keys out
}
std::string key = "key" + std::to_string(i);
uint16_t restart_point = i;
builder.Add(key, restart_point);
m[key] = restart_point;
}
size_t estimated_size = builder.EstimateSize();
std::string buffer("filling stuff"), buffer2;
size_t original_size = buffer.size();
estimated_size += original_size;
builder.Finish(buffer);
ASSERT_EQ(buffer.size(), estimated_size);
buffer2 = buffer; // test for the correctness of relative offset
Slice s(buffer2);
DataBlockHashIndex index(s);
// the additional hash map should start at the end of the buffer
ASSERT_EQ(original_size, index.DataBlockHashMapStart());
for (uint16_t i = 0; i < 100; i++) {
std::string key = "key" + std::to_string(i);
uint16_t restart_point = i;
if (m.count(key)) {
ASSERT_TRUE(m[key] == restart_point);
ASSERT_TRUE(SearchForOffset(index, key, restart_point));
} else {
// we allow false positve, so don't test the nonexisting keys.
// when false positive happens, the search will continue to the
// restart intervals to see if the key really exist.
}
}
}
} // namespace rocksdb
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}