提交 0b3d03d0 编写于 作者: K Kai Liu

Materialize the hash index

Summary:
Materialize the hash index to avoid the soaring cpu/flash usage
when initializing the database.

Test Plan: existing unit tests passed

Reviewers: sdong, haobo

Reviewed By: sdong

CC: leveldb

Differential Revision: https://reviews.facebook.net/D18339
上级 4e0602f9
......@@ -5,6 +5,9 @@
### Public API changes
* Replaced ColumnFamilyOptions::table_properties_collectors with ColumnFamilyOptions::table_properties_collector_factories
### New Features
* Hash index for block-based table will be materialized and reconstructed more efficiently. Previously hash index is constructed by scanning the whole table during every table open.
## 3.0.0 (05/05/2014)
### Public API changes
......
......@@ -15,6 +15,8 @@
#include <map>
#include <memory>
#include <string>
#include <unordered_map>
#include "db/dbformat.h"
......@@ -41,6 +43,8 @@
namespace rocksdb {
extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
namespace {
typedef BlockBasedTableOptions::IndexType IndexType;
......@@ -57,6 +61,14 @@ typedef BlockBasedTableOptions::IndexType IndexType;
// design that just works.
class IndexBuilder {
public:
// Index builder will construct a set of blocks which contain:
// 1. One primary index block.
// 2. (Optional) a set of metablocks that contains the metadata of the
// primary index.
struct IndexBlocks {
Slice index_block_contents;
std::unordered_map<std::string, Slice> meta_blocks;
};
explicit IndexBuilder(const Comparator* comparator)
: comparator_(comparator) {}
......@@ -72,15 +84,19 @@ class IndexBuilder {
// the last one in the table
//
// REQUIRES: Finish() has not yet been called.
virtual void AddEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) = 0;
virtual void AddIndexEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) = 0;
// This method will be called whenever a key is added. The subclasses may
// override OnKeyAdded() if they need to collect additional information.
virtual void OnKeyAdded(const Slice& key) {}
// Inform the index builder that all entries has been written. Block builder
// may therefore perform any operation required for block finalization.
//
// REQUIRES: Finish() has not yet been called.
virtual Slice Finish() = 0;
virtual Status Finish(IndexBlocks* index_blocks) = 0;
// Get the estimated size for index block.
virtual size_t EstimatedSize() const = 0;
......@@ -103,9 +119,9 @@ class ShortenedIndexBuilder : public IndexBuilder {
: IndexBuilder(comparator),
index_block_builder_(1 /* block_restart_interval == 1 */, comparator) {}
virtual void AddEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) override {
virtual void AddIndexEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) override {
if (first_key_in_next_block != nullptr) {
comparator_->FindShortestSeparator(last_key_in_current_block,
*first_key_in_next_block);
......@@ -118,7 +134,10 @@ class ShortenedIndexBuilder : public IndexBuilder {
index_block_builder_.Add(*last_key_in_current_block, handle_encoding);
}
virtual Slice Finish() override { return index_block_builder_.Finish(); }
virtual Status Finish(IndexBlocks* index_blocks) {
index_blocks->index_block_contents = index_block_builder_.Finish();
return Status::OK();
}
virtual size_t EstimatedSize() const {
return index_block_builder_.CurrentSizeEstimate();
......@@ -128,38 +147,125 @@ class ShortenedIndexBuilder : public IndexBuilder {
BlockBuilder index_block_builder_;
};
// FullKeyIndexBuilder is also based on BlockBuilder. It works pretty much like
// ShortenedIndexBuilder, but preserves the full key instead the substitude key.
class FullKeyIndexBuilder : public IndexBuilder {
// HashIndexBuilder contains a binary-searchable primary index and the
// metadata for secondary hash index construction.
// The metadata for hash index consists two parts:
// - a metablock that compactly contains a sequence of prefixes. All prefixes
// are stored consectively without any metadata (like, prefix sizes) being
// stored, which is kept in the other metablock.
// - a metablock contains the metadata of the prefixes, including prefix size,
// restart index and number of block it spans. The format looks like:
//
// +-----------------+---------------------------+---------------------+ <=prefix 1
// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
// +-----------------+---------------------------+---------------------+ <=prefix 2
// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
// +-----------------+---------------------------+---------------------+
// | |
// | .... |
// | |
// +-----------------+---------------------------+---------------------+ <=prefix n
// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
// +-----------------+---------------------------+---------------------+
//
// The reason of separating these two metablocks is to enable the efficiently
// reuse the first metablock during hash index construction without unnecessary
// data copy or small heap allocations for prefixes.
class HashIndexBuilder : public IndexBuilder {
public:
explicit FullKeyIndexBuilder(const Comparator* comparator)
explicit HashIndexBuilder(const Comparator* comparator,
const SliceTransform* hash_key_extractor)
: IndexBuilder(comparator),
index_block_builder_(1 /* block_restart_interval == 1 */, comparator) {}
primary_index_builder(comparator),
hash_key_extractor_(hash_key_extractor) {}
virtual void AddIndexEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) override {
++current_restart_index_;
primary_index_builder.AddIndexEntry(last_key_in_current_block,
first_key_in_next_block, block_handle);
}
virtual void AddEntry(std::string* last_key_in_current_block,
const Slice* first_key_in_next_block,
const BlockHandle& block_handle) override {
std::string handle_encoding;
block_handle.EncodeTo(&handle_encoding);
index_block_builder_.Add(*last_key_in_current_block, handle_encoding);
virtual void OnKeyAdded(const Slice& key) override {
auto key_prefix = hash_key_extractor_->Transform(key);
bool is_first_entry = pending_block_num_ == 0;
// Keys may share the prefix
if (is_first_entry || pending_entry_prefix_ != key_prefix) {
if (!is_first_entry) {
FlushPendingPrefix();
}
// need a hard copy otherwise the underlying data changes all the time.
// TODO(kailiu) ToString() is expensive. We may speed up can avoid data
// copy.
pending_entry_prefix_ = key_prefix.ToString();
pending_block_num_ = 1;
pending_entry_index_ = current_restart_index_;
} else {
// entry number increments when keys share the prefix reside in
// differnt data blocks.
auto last_restart_index = pending_entry_index_ + pending_block_num_ - 1;
assert(last_restart_index <= current_restart_index_);
if (last_restart_index != current_restart_index_) {
++pending_block_num_;
}
}
}
virtual Slice Finish() override { return index_block_builder_.Finish(); }
virtual Status Finish(IndexBlocks* index_blocks) {
FlushPendingPrefix();
primary_index_builder.Finish(index_blocks);
index_blocks->meta_blocks.insert(
{kHashIndexPrefixesBlock.c_str(), prefix_block_});
index_blocks->meta_blocks.insert(
{kHashIndexPrefixesMetadataBlock.c_str(), prefix_meta_block_});
return Status::OK();
}
virtual size_t EstimatedSize() const {
return index_block_builder_.CurrentSizeEstimate();
return primary_index_builder.EstimatedSize() + prefix_block_.size() +
prefix_meta_block_.size();
}
private:
BlockBuilder index_block_builder_;
void FlushPendingPrefix() {
prefix_block_.append(pending_entry_prefix_.data(),
pending_entry_prefix_.size());
PutVarint32(&prefix_meta_block_, pending_entry_prefix_.size());
PutVarint32(&prefix_meta_block_, pending_entry_index_);
PutVarint32(&prefix_meta_block_, pending_block_num_);
}
ShortenedIndexBuilder primary_index_builder;
const SliceTransform* hash_key_extractor_;
// stores a sequence of prefixes
std::string prefix_block_;
// stores the metadata of prefixes
std::string prefix_meta_block_;
// The following 3 variables keeps unflushed prefix and its metadata.
// The details of block_num and entry_index can be found in
// "block_hash_index.{h,cc}"
uint32_t pending_block_num_ = 0;
uint32_t pending_entry_index_ = 0;
std::string pending_entry_prefix_;
uint64_t current_restart_index_ = 0;
};
// Create a index builder based on its type.
IndexBuilder* CreateIndexBuilder(IndexType type, const Comparator* comparator) {
IndexBuilder* CreateIndexBuilder(IndexType type, const Comparator* comparator,
const SliceTransform* prefix_extractor) {
switch (type) {
case BlockBasedTableOptions::kBinarySearch: {
return new ShortenedIndexBuilder(comparator);
}
case BlockBasedTableOptions::kHashSearch: {
return new HashIndexBuilder(comparator, prefix_extractor);
}
default: {
assert(!"Do not recognize the index type ");
return nullptr;
......@@ -249,7 +355,7 @@ extern const uint64_t kLegacyBlockBasedTableMagicNumber = 0xdb4775248b80fb57ull;
class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
: public TablePropertiesCollector {
public:
BlockBasedTablePropertiesCollector(
explicit BlockBasedTablePropertiesCollector(
BlockBasedTableOptions::IndexType index_type)
: index_type_(index_type) {}
......@@ -288,6 +394,8 @@ struct BlockBasedTableBuilder::Rep {
uint64_t offset = 0;
Status status;
BlockBuilder data_block;
InternalKeySliceTransform internal_prefix_transform;
std::unique_ptr<IndexBuilder> index_builder;
std::string last_key;
......@@ -316,8 +424,9 @@ struct BlockBasedTableBuilder::Rep {
internal_comparator(icomparator),
file(f),
data_block(options, &internal_comparator),
index_builder(
CreateIndexBuilder(index_block_type, &internal_comparator)),
internal_prefix_transform(options.prefix_extractor.get()),
index_builder(CreateIndexBuilder(index_block_type, &internal_comparator,
&this->internal_prefix_transform)),
compression_type(compression_type),
checksum_type(checksum_type),
filter_block(opt.filter_policy == nullptr
......@@ -335,16 +444,13 @@ struct BlockBasedTableBuilder::Rep {
}
};
// TODO(sdong): Currently only write out binary search index. In
// BlockBasedTableReader, Hash index will be built using binary search index.
BlockBasedTableBuilder::BlockBasedTableBuilder(
const Options& options, const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator, WritableFile* file,
CompressionType compression_type)
: rep_(new Rep(options, internal_comparator, file,
table_options.flush_block_policy_factory.get(),
compression_type,
BlockBasedTableOptions::IndexType::kBinarySearch,
compression_type, table_options.index_type,
table_options.checksum)) {
if (rep_->filter_block != nullptr) {
rep_->filter_block->StartBlock(0);
......@@ -370,7 +476,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
if (r->props.num_entries > 0) {
assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0);
}
r->index_builder->OnKeyAdded(key);
auto should_flush = r->flush_block_policy->Update(key, value);
if (should_flush) {
assert(!r->data_block.empty());
......@@ -385,7 +491,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
// entries in the first block and < all entries in subsequent
// blocks.
if (ok()) {
r->index_builder->AddEntry(&r->last_key, &key, r->pending_handle);
r->index_builder->AddIndexEntry(&r->last_key, &key, r->pending_handle);
}
}
......@@ -561,24 +667,36 @@ Status BlockBasedTableBuilder::Finish() {
// block, we will finish writing all index entries here and flush them
// to storage after metaindex block is written.
if (ok() && !empty_data_block) {
r->index_builder->AddEntry(&r->last_key, nullptr /* no next data block */,
r->pending_handle);
r->index_builder->AddIndexEntry(
&r->last_key, nullptr /* no next data block */, r->pending_handle);
}
IndexBuilder::IndexBlocks index_blocks;
auto s = r->index_builder->Finish(&index_blocks);
if (!s.ok()) {
return s;
}
// Write meta blocks and metaindex block with the following order.
// 1. [meta block: filter]
// 2. [meta block: properties]
// 3. [metaindex block]
if (ok()) {
MetaIndexBuilder meta_index_builer;
// 2. [other meta blocks]
// 3. [meta block: properties]
// 4. [metaindex block]
// write meta blocks
MetaIndexBuilder meta_index_builder;
for (const auto& item : index_blocks.meta_blocks) {
BlockHandle block_handle;
WriteBlock(item.second, &block_handle);
meta_index_builder.Add(item.first, block_handle);
}
// Write filter block.
if (ok()) {
if (r->filter_block != nullptr) {
// Add mapping from "<filter_block_prefix>.Name" to location
// of filter data.
std::string key = BlockBasedTable::kFilterBlockPrefix;
key.append(r->options.filter_policy->Name());
meta_index_builer.Add(key, filter_block_handle);
meta_index_builder.Add(key, filter_block_handle);
}
// Write properties block.
......@@ -605,20 +723,16 @@ Status BlockBasedTableBuilder::Finish() {
&properties_block_handle
);
meta_index_builer.Add(kPropertiesBlock,
properties_block_handle);
meta_index_builder.Add(kPropertiesBlock, properties_block_handle);
} // end of properties block writing
WriteRawBlock(
meta_index_builer.Finish(),
kNoCompression,
&metaindex_block_handle
);
} // meta blocks and metaindex block.
} // meta blocks
// Write index block
if (ok()) {
WriteBlock(r->index_builder->Finish(), &index_block_handle);
// flush the meta index block
WriteRawBlock(meta_index_builder.Finish(), kNoCompression,
&metaindex_block_handle);
WriteBlock(index_blocks.index_block_contents, &index_block_handle);
}
// Write footer
......@@ -685,7 +799,6 @@ uint64_t BlockBasedTableBuilder::FileSize() const {
return rep_->offset;
}
const std::string BlockBasedTable::kFilterBlockPrefix =
"filter.";
const std::string BlockBasedTable::kFilterBlockPrefix = "filter.";
} // namespace rocksdb
......@@ -56,5 +56,8 @@ TableFactory* NewBlockBasedTableFactory(
const std::string BlockBasedTablePropertyNames::kIndexType =
"rocksdb.block.based.table.index.type";
const std::string kHashIndexPrefixesBlock = "rocksdb.hashindex.prefixes";
const std::string kHashIndexPrefixesMetadataBlock =
"rocksdb.hashindex.metadata";
} // namespace rocksdb
......@@ -8,9 +8,11 @@
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#pragma once
#include <memory>
#include <stdint.h>
#include <memory>
#include <string>
#include "rocksdb/flush_block_policy.h"
#include "rocksdb/options.h"
#include "rocksdb/table.h"
......@@ -45,4 +47,7 @@ class BlockBasedTableFactory : public TableFactory {
BlockBasedTableOptions table_options_;
};
extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
} // namespace rocksdb
......@@ -38,6 +38,8 @@
namespace rocksdb {
extern const uint64_t kBlockBasedTableMagicNumber;
extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
using std::unique_ptr;
typedef BlockBasedTable::IndexReader IndexReader;
......@@ -186,19 +188,13 @@ class BinarySearchIndexReader : public IndexReader {
// Index that leverages an internal hash table to quicken the lookup for a given
// key.
// @param data_iter_gen, equavalent to BlockBasedTable::NewIterator(). But that
// functions requires index to be initalized. To avoid this problem external
// caller will pass a function that can create the iterator over the entries
// without the table to be fully initialized.
class HashIndexReader : public IndexReader {
public:
static Status Create(RandomAccessFile* file, const Footer& footer,
const BlockHandle& index_handle, Env* env,
static Status Create(const SliceTransform* hash_key_extractor,
const Footer& footer, RandomAccessFile* file, Env* env,
const Comparator* comparator,
std::function<Iterator*(Iterator*)> data_iter_gen,
const SliceTransform* prefix_extractor,
IndexReader** index_reader) {
assert(prefix_extractor);
const BlockHandle& index_handle,
Iterator* meta_index_iter, IndexReader** index_reader) {
Block* index_block = nullptr;
auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle,
&index_block, env);
......@@ -207,14 +203,57 @@ class HashIndexReader : public IndexReader {
return s;
}
*index_reader = new HashIndexReader(comparator, index_block);
std::unique_ptr<Iterator> index_iter(index_block->NewIterator(nullptr));
std::unique_ptr<Iterator> data_iter(
data_iter_gen(index_block->NewIterator(nullptr)));
auto hash_index = CreateBlockHashIndex(index_iter.get(), data_iter.get(),
index_block->NumRestarts(),
comparator, prefix_extractor);
index_block->SetBlockHashIndex(hash_index);
// Get prefixes block
BlockHandle prefixes_handle;
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesBlock,
&prefixes_handle);
if (!s.ok()) {
return s;
}
// Get index metadata block
BlockHandle prefixes_meta_handle;
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesMetadataBlock,
&prefixes_meta_handle);
if (!s.ok()) {
return s;
}
// Read contents for the blocks
BlockContents prefixes_contents;
s = ReadBlockContents(file, footer, ReadOptions(), prefixes_handle,
&prefixes_contents, env, true /* do decompression */);
if (!s.ok()) {
return s;
}
BlockContents prefixes_meta_contents;
s = ReadBlockContents(file, footer, ReadOptions(), prefixes_meta_handle,
&prefixes_meta_contents, env,
true /* do decompression */);
if (!s.ok()) {
if (prefixes_contents.heap_allocated) {
delete[] prefixes_contents.data.data();
}
return s;
}
auto new_index_reader =
new HashIndexReader(comparator, index_block, prefixes_contents);
BlockHashIndex* hash_index = nullptr;
s = CreateBlockHashIndex(hash_key_extractor, prefixes_contents.data,
prefixes_meta_contents.data, &hash_index);
if (!s.ok()) {
return s;
}
new_index_reader->index_block_->SetBlockHashIndex(hash_index);
*index_reader = new_index_reader;
// release resources
if (prefixes_meta_contents.heap_allocated) {
delete[] prefixes_meta_contents.data.data();
}
return s;
}
......@@ -225,11 +264,22 @@ class HashIndexReader : public IndexReader {
virtual size_t size() const override { return index_block_->size(); }
private:
HashIndexReader(const Comparator* comparator, Block* index_block)
: IndexReader(comparator), index_block_(index_block) {
HashIndexReader(const Comparator* comparator, Block* index_block,
const BlockContents& prefixes_contents)
: IndexReader(comparator),
index_block_(index_block),
prefixes_contents_(prefixes_contents) {
assert(index_block_ != nullptr);
}
~HashIndexReader() {
if (prefixes_contents_.heap_allocated) {
delete[] prefixes_contents_.data.data();
}
}
std::unique_ptr<Block> index_block_;
BlockContents prefixes_contents_;
};
......@@ -408,7 +458,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
// and with a same life-time as this table object.
IndexReader* index_reader = nullptr;
// TODO: we never really verify check sum for index block
s = new_table->CreateIndexReader(&index_reader);
s = new_table->CreateIndexReader(&index_reader, meta_iter.get());
if (s.ok()) {
rep->index_reader.reset(index_reader);
......@@ -417,10 +467,9 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
if (rep->options.filter_policy) {
std::string key = kFilterBlockPrefix;
key.append(rep->options.filter_policy->Name());
meta_iter->Seek(key);
if (meta_iter->Valid() && meta_iter->key() == Slice(key)) {
rep->filter.reset(ReadFilter(meta_iter->value(), rep));
BlockHandle handle;
if (FindMetaBlock(meta_iter.get(), key, &handle).ok()) {
rep->filter.reset(ReadFilter(handle, rep));
}
}
} else {
......@@ -617,16 +666,9 @@ Status BlockBasedTable::PutDataBlockToCache(
return s;
}
FilterBlockReader* BlockBasedTable::ReadFilter (
const Slice& filter_handle_value,
BlockBasedTable::Rep* rep,
size_t* filter_size) {
Slice v = filter_handle_value;
BlockHandle filter_handle;
if (!filter_handle.DecodeFrom(&v).ok()) {
return nullptr;
}
FilterBlockReader* BlockBasedTable::ReadFilter(const BlockHandle& filter_handle,
BlockBasedTable::Rep* rep,
size_t* filter_size) {
// TODO: We might want to unify with ReadBlockFromFile() if we start
// requiring checksum verification in Table::Open.
ReadOptions opt;
......@@ -687,10 +729,9 @@ BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
if (s.ok()) {
std::string filter_block_key = kFilterBlockPrefix;
filter_block_key.append(rep_->options.filter_policy->Name());
iter->Seek(filter_block_key);
if (iter->Valid() && iter->key() == Slice(filter_block_key)) {
filter = ReadFilter(iter->value(), rep_, &filter_size);
BlockHandle handle;
if (FindMetaBlock(iter.get(), filter_block_key, &handle).ok()) {
filter = ReadFilter(handle, rep_, &filter_size);
assert(filter);
assert(filter_size > 0);
......@@ -1032,7 +1073,8 @@ bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options,
// 3. options
// 4. internal_comparator
// 5. index_type
Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader) {
Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader,
Iterator* preloaded_meta_index_iter) {
// Some old version of block-based tables don't have index type present in
// table properties. If that's the case we can safely use the kBinarySearch.
auto index_type_on_file = BlockBasedTableOptions::kBinarySearch;
......@@ -1045,41 +1087,45 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader) {
}
}
// TODO(sdong): Currently binary index is the only index type we support in
// files. Hash index is built on top of binary index too.
if (index_type_on_file != BlockBasedTableOptions::kBinarySearch) {
return Status::NotSupported("File Contains not supported index type: ",
std::to_string(index_type_on_file));
}
auto file = rep_->file.get();
auto env = rep_->options.env;
auto comparator = &rep_->internal_comparator;
const Footer& footer = rep_->footer;
switch (rep_->index_type) {
switch (index_type_on_file) {
case BlockBasedTableOptions::kBinarySearch: {
return BinarySearchIndexReader::Create(
file, footer, footer.index_handle(), env, comparator, index_reader);
}
case BlockBasedTableOptions::kHashSearch: {
std::unique_ptr<Block> meta_guard;
std::unique_ptr<Iterator> meta_iter_guard;
auto meta_index_iter = preloaded_meta_index_iter;
if (meta_index_iter == nullptr) {
auto s = ReadMetaBlock(rep_, &meta_guard, &meta_iter_guard);
if (!s.ok()) {
return Status::Corruption("Unable to read the metaindex block");
}
meta_index_iter = meta_iter_guard.get();
}
// We need to wrap data with internal_prefix_transform to make sure it can
// handle prefix correctly.
if (rep_->options.prefix_extractor == nullptr) {
return Status::InvalidArgument(
"BlockBasedTableOptions::kHashSearch requires "
"options.prefix_extractor to be set.");
}
rep_->internal_prefix_transform.reset(
new InternalKeySliceTransform(rep_->options.prefix_extractor.get()));
return HashIndexReader::Create(
file, footer, footer.index_handle(), env, comparator,
[&](Iterator* index_iter) {
return NewTwoLevelIterator(new BlockEntryIteratorState(this,
ReadOptions(), nullptr), index_iter);
},
rep_->internal_prefix_transform.get(), index_reader);
rep_->internal_prefix_transform.get(), footer, file, env, comparator,
footer.index_handle(), meta_index_iter, index_reader);
}
default: {
std::string error_message =
"Unrecognized index type: " + std::to_string(rep_->index_type);
// equivalent to assert(false), but more informative.
assert(!error_message.c_str());
return Status::InvalidArgument(error_message.c_str());
}
}
......
......@@ -160,8 +160,13 @@ class BlockBasedTable : public TableReader {
friend class BlockBasedTableBuilder;
void ReadMeta(const Footer& footer);
void ReadFilter(const Slice& filter_handle_value);
Status CreateIndexReader(IndexReader** index_reader);
// Create a index reader based on the index type stored in the table.
// Optionally, user can pass a preloaded meta_index_iter for the index that
// need to access extra meta blocks for index construction. This parameter
// helps avoid re-reading meta index block if caller already created one.
Status CreateIndexReader(IndexReader** index_reader,
Iterator* preloaded_meta_index_iter = nullptr);
// Read the meta block from sst.
static Status ReadMetaBlock(
......@@ -170,10 +175,8 @@ class BlockBasedTable : public TableReader {
std::unique_ptr<Iterator>* iter);
// Create the filter from the filter block.
static FilterBlockReader* ReadFilter(
const Slice& filter_handle_value,
Rep* rep,
size_t* filter_size = nullptr);
static FilterBlockReader* ReadFilter(const BlockHandle& filter_handle,
Rep* rep, size_t* filter_size = nullptr);
static void SetupCacheKeyPrefix(Rep* rep);
......
......@@ -3,21 +3,62 @@
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include "table/block_hash_index.h"
#include <algorithm>
#include "table/block_hash_index.h"
#include "rocksdb/comparator.h"
#include "rocksdb/iterator.h"
#include "rocksdb/slice_transform.h"
#include "util/coding.h"
namespace rocksdb {
BlockHashIndex* CreateBlockHashIndex(Iterator* index_iter, Iterator* data_iter,
const uint32_t num_restarts,
const Comparator* comparator,
const SliceTransform* hash_key_extractor) {
Status CreateBlockHashIndex(const SliceTransform* hash_key_extractor,
const Slice& prefixes, const Slice& prefix_meta,
BlockHashIndex** hash_index) {
uint64_t pos = 0;
auto meta_pos = prefix_meta;
Status s;
*hash_index = new BlockHashIndex(
hash_key_extractor,
false /* external module manages memory space for prefixes */);
while (!meta_pos.empty()) {
uint32_t prefix_size = 0;
uint32_t entry_index = 0;
uint32_t num_blocks = 0;
if (!GetVarint32(&meta_pos, &prefix_size) ||
!GetVarint32(&meta_pos, &entry_index) ||
!GetVarint32(&meta_pos, &num_blocks)) {
s = Status::Corruption(
"Corrupted prefix meta block: unable to read from it.");
break;
}
Slice prefix(prefixes.data() + pos, prefix_size);
(*hash_index)->Add(prefix, entry_index, num_blocks);
pos += prefix_size;
}
if (s.ok() && pos != prefixes.size()) {
s = Status::Corruption("Corrupted prefix meta block");
}
if (!s.ok()) {
delete *hash_index;
}
return s;
}
BlockHashIndex* CreateBlockHashIndexOnTheFly(
Iterator* index_iter, Iterator* data_iter, const uint32_t num_restarts,
const Comparator* comparator, const SliceTransform* hash_key_extractor) {
assert(hash_key_extractor);
auto hash_index = new BlockHashIndex(hash_key_extractor);
auto hash_index = new BlockHashIndex(
hash_key_extractor,
true /* hash_index will copy prefix when Add() is called */);
uint64_t current_restart_index = 0;
std::string pending_entry_prefix;
......@@ -88,12 +129,16 @@ BlockHashIndex* CreateBlockHashIndex(Iterator* index_iter, Iterator* data_iter,
bool BlockHashIndex::Add(const Slice& prefix, uint32_t restart_index,
uint32_t num_blocks) {
auto prefix_ptr = arena_.Allocate(prefix.size());
std::copy(prefix.data() /* begin */, prefix.data() + prefix.size() /* end */,
prefix_ptr /* destination */);
auto result =
restart_indices_.insert({Slice(prefix_ptr, prefix.size()),
RestartIndex(restart_index, num_blocks)});
auto prefix_to_insert = prefix;
if (kOwnPrefixes) {
auto prefix_ptr = arena_.Allocate(prefix.size());
std::copy(prefix.data() /* begin */,
prefix.data() + prefix.size() /* end */,
prefix_ptr /* destination */);
prefix_to_insert = Slice(prefix_ptr, prefix.size());
}
auto result = restart_indices_.insert(
{prefix_to_insert, RestartIndex(restart_index, num_blocks)});
return result.second;
}
......
......@@ -7,6 +7,7 @@
#include <string>
#include <unordered_map>
#include "rocksdb/status.h"
#include "util/arena.h"
#include "util/murmurhash.h"
......@@ -35,8 +36,12 @@ class BlockHashIndex {
uint32_t num_blocks = 1;
};
explicit BlockHashIndex(const SliceTransform* hash_key_extractor)
: hash_key_extractor_(hash_key_extractor) {}
// @params own_prefixes indicate if we should take care the memory space for
// the `key_prefix`
// passed by Add()
explicit BlockHashIndex(const SliceTransform* hash_key_extractor,
bool own_prefixes)
: hash_key_extractor_(hash_key_extractor), kOwnPrefixes(own_prefixes) {}
// Maps a key to its restart first_index.
// Returns nullptr if the restart first_index is found
......@@ -52,9 +57,18 @@ class BlockHashIndex {
private:
const SliceTransform* hash_key_extractor_;
std::unordered_map<Slice, RestartIndex, murmur_hash> restart_indices_;
Arena arena_;
bool kOwnPrefixes;
};
// Create hash index by reading from the metadata blocks.
// @params prefixes: a sequence of prefixes.
// @params prefix_meta: contains the "metadata" to of the prefixes.
Status CreateBlockHashIndex(const SliceTransform* hash_key_extractor,
const Slice& prefixes, const Slice& prefix_meta,
BlockHashIndex** hash_index);
// Create hash index by scanning the entries in index as well as the whole
// dataset.
// @params index_iter: an iterator with the pointer to the first entry in a
......@@ -64,9 +78,8 @@ class BlockHashIndex {
// @params num_restarts: used for correctness verification.
// @params hash_key_extractor: extract the hashable part of a given key.
// On error, nullptr will be returned.
BlockHashIndex* CreateBlockHashIndex(Iterator* index_iter, Iterator* data_iter,
const uint32_t num_restarts,
const Comparator* comparator,
const SliceTransform* hash_key_extractor);
BlockHashIndex* CreateBlockHashIndexOnTheFly(
Iterator* index_iter, Iterator* data_iter, const uint32_t num_restarts,
const Comparator* comparator, const SliceTransform* hash_key_extractor);
} // namespace rocksdb
......@@ -81,9 +81,9 @@ TEST(BlockTest, BasicTest) {
MapIterator index_iter(index_entries);
auto prefix_extractor = NewFixedPrefixTransform(prefix_size);
std::unique_ptr<BlockHashIndex> block_hash_index(
CreateBlockHashIndex(&index_iter, &data_iter, index_entries.size(),
BytewiseComparator(), prefix_extractor));
std::unique_ptr<BlockHashIndex> block_hash_index(CreateBlockHashIndexOnTheFly(
&index_iter, &data_iter, index_entries.size(), BytewiseComparator(),
prefix_extractor));
std::map<std::string, BlockHashIndex::RestartIndex> expected = {
{"01xx", BlockHashIndex::RestartIndex(0, 1)},
......
......@@ -163,9 +163,9 @@ void CheckBlockContents(BlockContents contents, const int max_key,
{
auto iter1 = reader1.NewIterator(nullptr);
auto iter2 = reader1.NewIterator(nullptr);
reader1.SetBlockHashIndex(CreateBlockHashIndex(iter1, iter2, keys.size(),
BytewiseComparator(),
prefix_extractor.get()));
reader1.SetBlockHashIndex(CreateBlockHashIndexOnTheFly(
iter1, iter2, keys.size(), BytewiseComparator(),
prefix_extractor.get()));
delete iter1;
delete iter2;
......
......@@ -254,11 +254,23 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size,
properties);
} else {
s = Status::Corruption("Unable to read the property block.");
Log(WARN_LEVEL, info_log,
"Cannot find Properties block from file.");
Log(WARN_LEVEL, info_log, "Cannot find Properties block from file.");
}
return s;
}
Status FindMetaBlock(Iterator* meta_index_iter,
const std::string& meta_block_name,
BlockHandle* block_handle) {
meta_index_iter->Seek(meta_block_name);
if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
meta_index_iter->key() == meta_block_name) {
Slice v = meta_index_iter->value();
return block_handle->DecodeFrom(&v);
} else {
return Status::Corruption("Cannot find the meta block", meta_block_name);
}
}
} // namespace rocksdb
......@@ -123,4 +123,9 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size,
// set to true.
extern Status SeekToPropertiesBlock(Iterator* meta_iter, bool* is_found);
// Find the meta block from the meta index block.
Status FindMetaBlock(Iterator* meta_index_iter,
const std::string& meta_block_name,
BlockHandle* block_handle);
} // namespace rocksdb
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册