Two-level Indexes

Summary:
Partition Index blocks and use a Partition-index as a 2nd level index.

The two-level index can be used by setting
BlockBasedTableOptions::kTwoLevelIndexSearch as the index type and
configuring BlockBasedTableOptions::index_per_partition

t15539501
Closes https://github.com/facebook/rocksdb/pull/1814

Differential Revision: D4473535

Pulled By: maysamyabandeh

fbshipit-source-id: bffb87e
main
Maysam Yabandeh 7 years ago committed by Facebook Github Bot
parent 0a4cdde50a
commit 69d5262c81
  1. 5
      db/db_test_util.cc
  2. 1
      db/db_test_util.h
  3. 10
      include/rocksdb/table.h
  4. 6
      java/src/main/java/org/rocksdb/IndexType.java
  5. 2
      java/src/test/java/org/rocksdb/BlockBasedTableConfigTest.java
  6. 183
      table/block_based_table_builder.cc
  7. 150
      table/block_based_table_reader.cc
  8. 19
      table/block_based_table_reader.h
  9. 77
      table/table_test.cc
  10. 7
      util/options_helper.h
  11. 1
      util/options_settable_test.cc

@ -353,6 +353,11 @@ Options DBTestBase::CurrentOptions(
options.prefix_extractor.reset(NewNoopTransform());
break;
}
case kBlockBasedTableWithPartitionedIndex: {
table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
options.prefix_extractor.reset(NewNoopTransform());
break;
}
case kBlockBasedTableWithIndexRestartInterval: {
table_options.index_block_restart_interval = 8;
break;

@ -594,6 +594,7 @@ class DBTestBase : public testing::Test {
kLevelSubcompactions = 31,
kUniversalSubcompactions = 32,
kBlockBasedTableWithIndexRestartInterval = 33,
kBlockBasedTableWithPartitionedIndex = 34,
};
int option_config_;

@ -86,6 +86,12 @@ struct BlockBasedTableOptions {
// The hash index, if enabled, will do the hash lookup when
// `Options.prefix_extractor` is provided.
kHashSearch,
// TODO(myabandeh): this feature is in experimental phase and shall not be
// used in production; either remove the feature or remove this comment if
// it is ready to be used in production.
// A two-level index implementation. Both levels are binary search indexes.
kTwoLevelIndexSearch,
};
IndexType index_type = kBinarySearch;
@ -138,6 +144,10 @@ struct BlockBasedTableOptions {
// Same as block_restart_interval but used for the index block.
int index_block_restart_interval = 1;
// Number of index keys per partition of indexes in a multi-level index
// i.e., the number of data blocks covered by each index partition
uint64_t index_per_partition = 2;
// Use delta encoding to compress keys in blocks.
// ReadOptions::pin_data requires this option to be disabled.
//

@ -18,7 +18,11 @@ public enum IndexType {
* The hash index, if enabled, will do the hash lookup when
* {@code Options.prefix_extractor} is provided.
*/
kHashSearch((byte) 1);
kHashSearch((byte) 1),
/**
* A two-level index implementation. Both levels are binary search indexes.
*/
kTwoLevelIndexSearch((byte) 2);
/**
* Returns the byte value of the enumerations value

@ -95,7 +95,7 @@ public class BlockBasedTableConfigTest {
@Test
public void indexType() {
BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig();
assertThat(IndexType.values().length).isEqualTo(2);
assertThat(IndexType.values().length).isEqualTo(3);
blockBasedTableConfig.setIndexType(IndexType.kHashSearch);
assertThat(blockBasedTableConfig.indexType().equals(
IndexType.kHashSearch));

@ -13,6 +13,7 @@
#include <inttypes.h>
#include <stdio.h>
#include <list>
#include <map>
#include <memory>
#include <string>
@ -53,6 +54,14 @@ extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
typedef BlockBasedTableOptions::IndexType IndexType;
class IndexBuilder;
namespace {
rocksdb::IndexBuilder* CreateIndexBuilder(
IndexType index_type, const InternalKeyComparator* comparator,
const SliceTransform* prefix_extractor, int index_block_restart_interval,
uint64_t index_per_partition);
}
// The interface for building index.
// Instruction for adding a new concrete IndexBuilder:
@ -101,7 +110,26 @@ class IndexBuilder {
// may therefore perform any operation required for block finalization.
//
// REQUIRES: Finish() has not yet been called.
virtual Status Finish(IndexBlocks* index_blocks) = 0;
inline Status Finish(IndexBlocks* index_blocks) {
// Throw away the changes to last_partition_block_handle. It has no effect
// on the first call to Finish anyway.
BlockHandle last_partition_block_handle;
return Finish(index_blocks, last_partition_block_handle);
}
// This override of Finish can be utilized to build the 2nd level index in
// PartitionIndexBuilder.
//
// index_blocks will be filled with the resulting index data. If the return
// value is Status::InComplete() then it means that the index is partitioned
// and the callee should keep calling Finish until Status::OK() is returned.
// In that case, last_partition_block_handle is pointer to the block written
// with the result of the last call to Finish. This can be utilized to build
// the second level index pointing to each block of partitioned indexes. The
// last call to Finish() that returns Status::OK() populates index_blocks with
// the 2nd level index content.
virtual Status Finish(IndexBlocks* index_blocks,
const BlockHandle& last_partition_block_handle) = 0;
// Get the estimated size for index block.
virtual size_t EstimatedSize() const = 0;
@ -141,7 +169,9 @@ class ShortenedIndexBuilder : public IndexBuilder {
index_block_builder_.Add(*last_key_in_current_block, handle_encoding);
}
virtual Status Finish(IndexBlocks* index_blocks) override {
virtual Status Finish(
IndexBlocks* index_blocks,
const BlockHandle& last_partition_block_handle) override {
index_blocks->index_block_contents = index_block_builder_.Finish();
return Status::OK();
}
@ -154,6 +184,108 @@ class ShortenedIndexBuilder : public IndexBuilder {
BlockBuilder index_block_builder_;
};
/**
* IndexBuilder for two-level indexing. Internally it creates a new index for
* each partition and Finish then in order when Finish is called on it
* continiously until Status::OK() is returned.
*
* The format on the disk would be I I I I I I IP where I is block containing a
* partition of indexes built using ShortenedIndexBuilder and IP is a block
* containing a secondary index on the partitions, built using
* ShortenedIndexBuilder.
*/
class PartitionIndexBuilder : public IndexBuilder {
public:
explicit PartitionIndexBuilder(const InternalKeyComparator* comparator,
const SliceTransform* prefix_extractor,
const uint64_t index_per_partition,
int index_block_restart_interval)
: IndexBuilder(comparator),
prefix_extractor_(prefix_extractor),
index_block_builder_(index_block_restart_interval),
index_per_partition_(index_per_partition),
index_block_restart_interval_(index_block_restart_interval) {
sub_index_builder_ =
CreateIndexBuilder(sub_type_, comparator_, prefix_extractor_,
index_block_restart_interval_, index_per_partition_);
}
virtual ~PartitionIndexBuilder() { delete sub_index_builder_; }
virtual void AddIndexEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) override {
sub_index_builder_->AddIndexEntry(last_key_in_current_block,
first_key_in_next_block, block_handle);
num_indexes++;
if (UNLIKELY(first_key_in_next_block == nullptr)) { // no more keys
entries_.push_back({std::string(*last_key_in_current_block),
std::unique_ptr<IndexBuilder>(sub_index_builder_)});
sub_index_builder_ = nullptr;
} else if (num_indexes % index_per_partition_ == 0) {
entries_.push_back({std::string(*last_key_in_current_block),
std::unique_ptr<IndexBuilder>(sub_index_builder_)});
sub_index_builder_ = CreateIndexBuilder(
sub_type_, comparator_, prefix_extractor_,
index_block_restart_interval_, index_per_partition_);
}
}
virtual Status Finish(
IndexBlocks* index_blocks,
const BlockHandle& last_partition_block_handle) override {
assert(!entries_.empty());
// It must be set to null after last key is added
assert(sub_index_builder_ == nullptr);
if (finishing == true) {
Entry& last_entry = entries_.front();
std::string handle_encoding;
last_partition_block_handle.EncodeTo(&handle_encoding);
index_block_builder_.Add(last_entry.key, handle_encoding);
entries_.pop_front();
}
// If there is no sub_index left, then return the 2nd level index.
if (UNLIKELY(entries_.empty())) {
index_blocks->index_block_contents = index_block_builder_.Finish();
return Status::OK();
} else {
// Finish the next partition index in line and Incomplete() to indicate we
// expect more calls to Finish
Entry& entry = entries_.front();
auto s = entry.value->Finish(index_blocks);
finishing = true;
return s.ok() ? Status::Incomplete() : s;
}
}
virtual size_t EstimatedSize() const override {
size_t total = 0;
for (auto it = entries_.begin(); it != entries_.end(); ++it) {
total += it->value->EstimatedSize();
}
total += index_block_builder_.CurrentSizeEstimate();
total +=
sub_index_builder_ == nullptr ? 0 : sub_index_builder_->EstimatedSize();
return total;
}
private:
static const IndexType sub_type_ = BlockBasedTableOptions::kBinarySearch;
struct Entry {
std::string key;
std::unique_ptr<IndexBuilder> value;
};
std::list<Entry> entries_; // list of partitioned indexes and their keys
const SliceTransform* prefix_extractor_;
BlockBuilder index_block_builder_; // top-level index builder
IndexBuilder* sub_index_builder_; // the active partition index builder
uint64_t index_per_partition_;
int index_block_restart_interval_;
uint64_t num_indexes = 0;
bool finishing =
false; // true if Finish is called once but not complete yet.
};
// HashIndexBuilder contains a binary-searchable primary index and the
// metadata for secondary hash index construction.
// The metadata for hash index consists two parts:
@ -222,9 +354,11 @@ class HashIndexBuilder : public IndexBuilder {
}
}
virtual Status Finish(IndexBlocks* index_blocks) override {
virtual Status Finish(
IndexBlocks* index_blocks,
const BlockHandle& last_partition_block_handle) override {
FlushPendingPrefix();
primary_index_builder_.Finish(index_blocks);
primary_index_builder_.Finish(index_blocks, last_partition_block_handle);
index_blocks->meta_blocks.insert(
{kHashIndexPrefixesBlock.c_str(), prefix_block_});
index_blocks->meta_blocks.insert(
@ -269,11 +403,12 @@ class HashIndexBuilder : public IndexBuilder {
namespace {
// Create a index builder based on its type.
IndexBuilder* CreateIndexBuilder(IndexType type,
IndexBuilder* CreateIndexBuilder(IndexType index_type,
const InternalKeyComparator* comparator,
const SliceTransform* prefix_extractor,
int index_block_restart_interval) {
switch (type) {
int index_block_restart_interval,
uint64_t index_per_partition) {
switch (index_type) {
case BlockBasedTableOptions::kBinarySearch: {
return new ShortenedIndexBuilder(comparator,
index_block_restart_interval);
@ -282,6 +417,11 @@ IndexBuilder* CreateIndexBuilder(IndexType type,
return new HashIndexBuilder(comparator, prefix_extractor,
index_block_restart_interval);
}
case BlockBasedTableOptions::kTwoLevelIndexSearch: {
return new PartitionIndexBuilder(comparator, prefix_extractor,
index_per_partition,
index_block_restart_interval);
}
default: {
assert(!"Do not recognize the index type ");
return nullptr;
@ -512,7 +652,8 @@ struct BlockBasedTableBuilder::Rep {
index_builder(
CreateIndexBuilder(table_options.index_type, &internal_comparator,
&this->internal_prefix_transform,
table_options.index_block_restart_interval)),
table_options.index_block_restart_interval,
table_options.index_per_partition)),
compression_type(_compression_type),
compression_opts(_compression_opts),
compression_dict(_compression_dict),
@ -852,9 +993,14 @@ Status BlockBasedTableBuilder::Finish() {
}
IndexBuilder::IndexBlocks index_blocks;
auto s = r->index_builder->Finish(&index_blocks);
if (!s.ok()) {
return s;
auto index_builder_status = r->index_builder->Finish(&index_blocks);
if (index_builder_status.IsIncomplete()) {
// We we have more than one index partition then meta_blocks are not
// supported for the index. Currently meta_blocks are used only by
// HashIndexBuilder which is not multi-partition.
assert(index_blocks.meta_blocks.empty());
} else if (!index_builder_status.ok()) {
return index_builder_status;
}
// Write meta blocks and metaindex block with the following order.
@ -956,8 +1102,21 @@ Status BlockBasedTableBuilder::Finish() {
// flush the meta index block
WriteRawBlock(meta_index_builder.Finish(), kNoCompression,
&metaindex_block_handle);
const bool is_data_block = true;
WriteBlock(index_blocks.index_block_contents, &index_block_handle,
false /* is_data_block */);
!is_data_block);
// If there are more index partitions, finish them and write them out
Status& s = index_builder_status;
while (s.IsIncomplete()) {
s = r->index_builder->Finish(&index_blocks, index_block_handle);
if (!s.ok() && !s.IsIncomplete()) {
return s;
}
WriteBlock(index_blocks.index_block_contents, &index_block_handle,
!is_data_block);
// The last index_block_handle will be for the partition index block
}
}
// Write footer

@ -153,8 +153,12 @@ class BlockBasedTable::IndexReader {
virtual ~IndexReader() {}
// Create an iterator for index access.
// An iter is passed in, if it is not null, update this one and return it
// If it is null, create a new Iterator
// If iter is null then a new object is created on heap and the callee will
// have the ownership. If a non-null iter is passed in it will be used, and
// the returned value is either the same as iter or a new on-heap object that
// wrapps the passed iter. In the latter case the return value would point to
// a different object then iter and the callee has the ownership of the
// returned object.
virtual InternalIterator* NewIterator(BlockIter* iter = nullptr,
bool total_order_seek = true) = 0;
@ -175,6 +179,63 @@ class BlockBasedTable::IndexReader {
Statistics* statistics_;
};
// Index that allows binary search lookup in a two-level index structure.
class PartitionIndexReader : public IndexReader {
public:
// Read the partition index from the file and create an instance for
// `PartitionIndexReader`.
// On success, index_reader will be populated; otherwise it will remain
// unmodified.
static Status Create(BlockBasedTable* table, RandomAccessFileReader* file,
const Footer& footer, const BlockHandle& index_handle,
const ImmutableCFOptions& ioptions,
const Comparator* comparator, IndexReader** index_reader,
const PersistentCacheOptions& cache_options) {
std::unique_ptr<Block> index_block;
auto s = ReadBlockFromFile(
file, footer, ReadOptions(), index_handle, &index_block, ioptions,
true /* decompress */, Slice() /*compression dict*/, cache_options,
kDisableGlobalSequenceNumber, 0 /* read_amp_bytes_per_bit */);
if (s.ok()) {
*index_reader = new PartitionIndexReader(
table, comparator, std::move(index_block), ioptions.statistics);
}
return s;
}
// return a two-level iterator: first level is on the partition index
virtual InternalIterator* NewIterator(BlockIter* iter = nullptr,
bool dont_care = true) override {
return NewTwoLevelIterator(
new BlockBasedTable::BlockEntryIteratorState(table_, ReadOptions(),
false),
index_block_->NewIterator(comparator_, iter, true));
}
virtual size_t size() const override { return index_block_->size(); }
virtual size_t usable_size() const override {
return index_block_->usable_size();
}
virtual size_t ApproximateMemoryUsage() const override {
assert(index_block_);
return index_block_->ApproximateMemoryUsage();
}
private:
PartitionIndexReader(BlockBasedTable* table, const Comparator* comparator,
std::unique_ptr<Block>&& index_block, Statistics* stats)
: IndexReader(comparator, stats),
table_(table),
index_block_(std::move(index_block)) {
assert(index_block_ != nullptr);
}
BlockBasedTable* table_;
std::unique_ptr<Block> index_block_;
};
// Index that allows binary search lookup for the first key of each block.
// This class can be viewed as a thin wrapper for `Block` class which already
// supports binary search.
@ -1352,33 +1413,27 @@ Status BlockBasedTable::MaybeLoadDataBlockToCache(
return s;
}
class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
public:
BlockEntryIteratorState(BlockBasedTable* table,
const ReadOptions& read_options, bool skip_filters)
: TwoLevelIteratorState(table->rep_->ioptions.prefix_extractor !=
nullptr),
table_(table),
read_options_(read_options),
skip_filters_(skip_filters) {}
InternalIterator* NewSecondaryIterator(const Slice& index_value) override {
return NewDataBlockIterator(table_->rep_, read_options_, index_value);
}
BlockBasedTable::BlockEntryIteratorState::BlockEntryIteratorState(
BlockBasedTable* table, const ReadOptions& read_options, bool skip_filters)
: TwoLevelIteratorState(table->rep_->ioptions.prefix_extractor != nullptr),
table_(table),
read_options_(read_options),
skip_filters_(skip_filters) {}
InternalIterator*
BlockBasedTable::BlockEntryIteratorState::NewSecondaryIterator(
const Slice& index_value) {
// Return a block iterator on the index partition
return NewDataBlockIterator(table_->rep_, read_options_, index_value);
}
bool PrefixMayMatch(const Slice& internal_key) override {
if (read_options_.total_order_seek || skip_filters_) {
return true;
}
return table_->PrefixMayMatch(internal_key);
bool BlockBasedTable::BlockEntryIteratorState::PrefixMayMatch(
const Slice& internal_key) {
if (read_options_.total_order_seek || skip_filters_) {
return true;
}
private:
// Don't own table_
BlockBasedTable* table_;
const ReadOptions read_options_;
bool skip_filters_;
};
return table_->PrefixMayMatch(internal_key);
}
// This will be broken if the user specifies an unusual implementation
// of Options.comparator, or if the user specifies an unusual
@ -1551,15 +1606,19 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
if (!FullFilterKeyMayMatch(read_options, filter, key)) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
} else {
BlockIter iiter;
NewIndexIterator(read_options, &iiter);
BlockIter iiter_on_stack;
auto iiter = NewIndexIterator(read_options, &iiter_on_stack);
std::unique_ptr<InternalIterator> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
iiter_unique_ptr = std::unique_ptr<InternalIterator>(iiter);
}
PinnedIteratorsManager* pinned_iters_mgr = get_context->pinned_iters_mgr();
bool pin_blocks = pinned_iters_mgr && pinned_iters_mgr->PinningEnabled();
bool done = false;
for (iiter.Seek(key); iiter.Valid() && !done; iiter.Next()) {
Slice handle_value = iiter.value();
for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) {
Slice handle_value = iiter->value();
BlockHandle handle;
bool not_exist_in_filter =
@ -1575,7 +1634,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
break;
} else {
BlockIter biter;
NewDataBlockIterator(rep_, read_options, iiter.value(), &biter);
NewDataBlockIterator(rep_, read_options, iiter->value(), &biter);
if (read_options.read_tier == kBlockCacheTier &&
biter.status().IsIncomplete()) {
@ -1611,7 +1670,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
}
}
if (s.ok()) {
s = iiter.status();
s = iiter->status();
}
}
@ -1632,22 +1691,26 @@ Status BlockBasedTable::Prefetch(const Slice* const begin,
return Status::InvalidArgument(*begin, *end);
}
BlockIter iiter;
NewIndexIterator(ReadOptions(), &iiter);
BlockIter iiter_on_stack;
auto iiter = NewIndexIterator(ReadOptions(), &iiter_on_stack);
std::unique_ptr<InternalIterator> iiter_unique_ptr;
if (iiter != &iiter_on_stack) {
iiter_unique_ptr = std::unique_ptr<InternalIterator>(iiter);
}
if (!iiter.status().ok()) {
if (!iiter->status().ok()) {
// error opening index iterator
return iiter.status();
return iiter->status();
}
// indicates if we are on the last page that need to be pre-fetched
bool prefetching_boundary_page = false;
for (begin ? iiter.Seek(*begin) : iiter.SeekToFirst(); iiter.Valid();
iiter.Next()) {
Slice block_handle = iiter.value();
for (begin ? iiter->Seek(*begin) : iiter->SeekToFirst(); iiter->Valid();
iiter->Next()) {
Slice block_handle = iiter->value();
if (end && comparator.Compare(iiter.key(), *end) >= 0) {
if (end && comparator.Compare(iiter->key(), *end) >= 0) {
if (prefetching_boundary_page) {
break;
}
@ -1737,6 +1800,11 @@ Status BlockBasedTable::CreateIndexReader(
}
switch (index_type_on_file) {
case BlockBasedTableOptions::kTwoLevelIndexSearch: {
return PartitionIndexReader::Create(
this, file, footer, footer.index_handle(), rep_->ioptions, comparator,
index_reader, rep_->persistent_cache_options);
}
case BlockBasedTableOptions::kBinarySearch: {
return BinarySearchIndexReader::Create(
file, footer, footer.index_handle(), rep_->ioptions, comparator,

@ -22,6 +22,7 @@
#include "rocksdb/table.h"
#include "table/table_properties_internal.h"
#include "table/table_reader.h"
#include "table/two_level_iterator.h"
#include "util/cf_options.h"
#include "util/coding.h"
#include "util/file_reader_writer.h"
@ -149,6 +150,8 @@ class BlockBasedTable : public TableReader {
// The key retrieved are internal keys.
Status GetKVPairsFromDataBlocks(std::vector<KVPairBlock>* kv_pair_blocks);
class BlockEntryIteratorState;
private:
template <class TValue>
struct CachableEntry;
@ -157,7 +160,6 @@ class BlockBasedTable : public TableReader {
Rep* rep_;
bool compaction_optimized_;
class BlockEntryIteratorState;
// input_iter: if it is not null, update this one and return it as Iterator
static InternalIterator* NewDataBlockIterator(
Rep* rep, const ReadOptions& ro, const Slice& index_value,
@ -273,4 +275,19 @@ class BlockBasedTable : public TableReader {
void operator=(const TableReader&) = delete;
};
// Maitaning state of a two-level iteration on a partitioned index structure
class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
public:
BlockEntryIteratorState(BlockBasedTable* table,
const ReadOptions& read_options, bool skip_filters);
InternalIterator* NewSecondaryIterator(const Slice& index_value) override;
bool PrefixMayMatch(const Slice& internal_key) override;
private:
// Don't own table_
BlockBasedTable* table_;
const ReadOptions read_options_;
bool skip_filters_;
};
} // namespace rocksdb

@ -996,6 +996,7 @@ class TableTest : public testing::Test {
}
return *plain_internal_comparator;
}
void IndexTest(BlockBasedTableOptions table_options);
private:
std::unique_ptr<InternalKeyComparator> plain_internal_comparator;
@ -1383,13 +1384,18 @@ TEST_F(BlockBasedTableTest, TotalOrderSeekOnHashIndex) {
options.prefix_extractor.reset(NewFixedPrefixTransform(4));
break;
case 3:
default:
// Hash search index with filter policy
table_options.index_type = BlockBasedTableOptions::kHashSearch;
table_options.filter_policy.reset(NewBloomFilterPolicy(10));
options.table_factory.reset(new BlockBasedTableFactory(table_options));
options.prefix_extractor.reset(NewFixedPrefixTransform(4));
break;
case 4:
default:
// Binary search index
table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
options.table_factory.reset(new BlockBasedTableFactory(table_options));
break;
}
TableConstructor c(BytewiseComparator(),
@ -1528,7 +1534,7 @@ void AddInternalKey(TableConstructor* c, const std::string& prefix,
c->Add(k.Encode().ToString(), "v");
}
TEST_F(TableTest, HashIndexTest) {
void TableTest::IndexTest(BlockBasedTableOptions table_options) {
TableConstructor c(BytewiseComparator());
// keys with prefix length 3, make sure the key/value is big enough to fill
@ -1552,9 +1558,6 @@ TEST_F(TableTest, HashIndexTest) {
stl_wrappers::KVMap kvmap;
Options options;
options.prefix_extractor.reset(NewFixedPrefixTransform(3));
BlockBasedTableOptions table_options;
table_options.index_type = BlockBasedTableOptions::kHashSearch;
table_options.hash_index_allow_collision = true;
table_options.block_size = 1700;
table_options.block_cache = NewLRUCache(1024, 4);
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
@ -1568,7 +1571,7 @@ TEST_F(TableTest, HashIndexTest) {
auto props = reader->GetTableProperties();
ASSERT_EQ(5u, props->num_data_blocks);
std::unique_ptr<InternalIterator> hash_iter(
std::unique_ptr<InternalIterator> index_iter(
reader->NewIterator(ReadOptions()));
// -- Find keys do not exist, but have common prefix.
@ -1578,13 +1581,13 @@ TEST_F(TableTest, HashIndexTest) {
// find the lower bound of the prefix
for (size_t i = 0; i < prefixes.size(); ++i) {
hash_iter->Seek(InternalKey(prefixes[i], 0, kTypeValue).Encode());
ASSERT_OK(hash_iter->status());
ASSERT_TRUE(hash_iter->Valid());
index_iter->Seek(InternalKey(prefixes[i], 0, kTypeValue).Encode());
ASSERT_OK(index_iter->status());
ASSERT_TRUE(index_iter->Valid());
// seek the first element in the block
ASSERT_EQ(lower_bound[i], hash_iter->key().ToString());
ASSERT_EQ("v", hash_iter->value().ToString());
ASSERT_EQ(lower_bound[i], index_iter->key().ToString());
ASSERT_EQ("v", index_iter->value().ToString());
}
// find the upper bound of prefixes
@ -1593,46 +1596,46 @@ TEST_F(TableTest, HashIndexTest) {
// find existing keys
for (const auto& item : kvmap) {
auto ukey = ExtractUserKey(item.first).ToString();
hash_iter->Seek(ukey);
index_iter->Seek(ukey);
// ASSERT_OK(regular_iter->status());
ASSERT_OK(hash_iter->status());
ASSERT_OK(index_iter->status());
// ASSERT_TRUE(regular_iter->Valid());
ASSERT_TRUE(hash_iter->Valid());
ASSERT_TRUE(index_iter->Valid());
ASSERT_EQ(item.first, hash_iter->key().ToString());
ASSERT_EQ(item.second, hash_iter->value().ToString());
ASSERT_EQ(item.first, index_iter->key().ToString());
ASSERT_EQ(item.second, index_iter->value().ToString());
}
for (size_t i = 0; i < prefixes.size(); ++i) {
// the key is greater than any existing keys.
auto key = prefixes[i] + "9";
hash_iter->Seek(InternalKey(key, 0, kTypeValue).Encode());
index_iter->Seek(InternalKey(key, 0, kTypeValue).Encode());
ASSERT_OK(hash_iter->status());
ASSERT_OK(index_iter->status());
if (i == prefixes.size() - 1) {
// last key
ASSERT_TRUE(!hash_iter->Valid());
ASSERT_TRUE(!index_iter->Valid());
} else {
ASSERT_TRUE(hash_iter->Valid());
ASSERT_TRUE(index_iter->Valid());
// seek the first element in the block
ASSERT_EQ(upper_bound[i], hash_iter->key().ToString());
ASSERT_EQ("v", hash_iter->value().ToString());
ASSERT_EQ(upper_bound[i], index_iter->key().ToString());
ASSERT_EQ("v", index_iter->value().ToString());
}
}
// find keys with prefix that don't match any of the existing prefixes.
std::vector<std::string> non_exist_prefixes = {"002", "004", "006", "008"};
for (const auto& prefix : non_exist_prefixes) {
hash_iter->Seek(InternalKey(prefix, 0, kTypeValue).Encode());
index_iter->Seek(InternalKey(prefix, 0, kTypeValue).Encode());
// regular_iter->Seek(prefix);
ASSERT_OK(hash_iter->status());
ASSERT_OK(index_iter->status());
// Seek to non-existing prefixes should yield either invalid, or a
// key with prefix greater than the target.
if (hash_iter->Valid()) {
Slice ukey = ExtractUserKey(hash_iter->key());
if (index_iter->Valid()) {
Slice ukey = ExtractUserKey(index_iter->key());
Slice ukey_prefix = options.prefix_extractor->Transform(ukey);
ASSERT_TRUE(BytewiseComparator()->Compare(prefix, ukey_prefix) < 0);
}
@ -1640,6 +1643,28 @@ TEST_F(TableTest, HashIndexTest) {
c.ResetTableReader();
}
TEST_F(TableTest, BinaryIndexTest) {
BlockBasedTableOptions table_options;
table_options.index_type = BlockBasedTableOptions::kBinarySearch;
IndexTest(table_options);
}
TEST_F(TableTest, HashIndexTest) {
BlockBasedTableOptions table_options;
table_options.index_type = BlockBasedTableOptions::kHashSearch;
IndexTest(table_options);
}
TEST_F(TableTest, PartitionIndexTest) {
const int max_index_keys = 5;
for (int i = 1; i <= max_index_keys + 1; i++) {
BlockBasedTableOptions table_options;
table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
table_options.index_per_partition = i;
IndexTest(table_options);
}
}
// It's very hard to figure out the index block size of a block accurately.
// To make sure we get the index size, we just make sure as key number
// grows, the filter block size also grows.

@ -611,6 +611,9 @@ static std::unordered_map<std::string, OptionTypeInfo>
{"index_block_restart_interval",
{offsetof(struct BlockBasedTableOptions, index_block_restart_interval),
OptionType::kInt, OptionVerificationType::kNormal, false, 0}},
{"index_per_partition",
{offsetof(struct BlockBasedTableOptions, index_per_partition),
OptionType::kUInt64T, OptionVerificationType::kNormal, false, 0}},
{"filter_policy",
{offsetof(struct BlockBasedTableOptions, filter_policy),
OptionType::kFilterPolicy, OptionVerificationType::kByName, false,
@ -673,7 +676,9 @@ static std::unordered_map<std::string, CompressionType>
static std::unordered_map<std::string, BlockBasedTableOptions::IndexType>
block_base_table_index_type_string_map = {
{"kBinarySearch", BlockBasedTableOptions::IndexType::kBinarySearch},
{"kHashSearch", BlockBasedTableOptions::IndexType::kHashSearch}};
{"kHashSearch", BlockBasedTableOptions::IndexType::kHashSearch},
{"kTwoLevelIndexSearch",
BlockBasedTableOptions::IndexType::kHashSearch}};
static std::unordered_map<std::string, EncodingType> encoding_type_string_map =
{{"kPlain", kPlain}, {"kPrefix", kPrefix}};

@ -155,6 +155,7 @@ TEST_F(OptionsSettableTest, BlockBasedTableOptionsAllFieldsSettable) {
"checksum=kxxHash;hash_index_allow_collision=1;no_block_cache=1;"
"block_cache=1M;block_cache_compressed=1k;block_size=1024;"
"block_size_deviation=8;block_restart_interval=4; "
"index_per_partition=4;"
"index_block_restart_interval=4;"
"filter_policy=bloomfilter:4:true;whole_key_filtering=1;"
"skip_table_builder_flush=1;format_version=1;"

Loading…
Cancel
Save