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: 7f015f0098632c65979a22898a50424384730b10main
parent
ea212e5316
commit
8805ec2f49
@ -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
|
@ -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
|
@ -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(); |
||||||
|
} |
Loading…
Reference in new issue