You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 
 
 
 
 
rocksdb/table/block_based_table_reader.cc

1648 lines
57 KiB

// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// 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.
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "table/block_based_table_reader.h"
#include <string>
#include <utility>
#include "db/dbformat.h"
#include "rocksdb/cache.h"
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/filter_policy.h"
#include "rocksdb/iterator.h"
#include "rocksdb/options.h"
#include "rocksdb/statistics.h"
#include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "table/block.h"
#include "table/filter_block.h"
#include "table/block_based_filter_block.h"
#include "table/block_based_table_factory.h"
#include "table/full_filter_block.h"
#include "table/block_hash_index.h"
#include "table/block_prefix_index.h"
#include "table/format.h"
#include "table/meta_blocks.h"
#include "table/two_level_iterator.h"
#include "table/get_context.h"
#include "util/coding.h"
#include "util/perf_context_imp.h"
#include "util/stop_watch.h"
#include "util/string_util.h"
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;
namespace {
// The longest the prefix of the cache key used to identify blocks can be.
// We are using the fact that we know for Posix files the unique ID is three
// varints.
// For some reason, compiling for iOS complains that this variable is unused
const size_t kMaxCacheKeyPrefixSize __attribute__((unused)) =
kMaxVarint64Length * 3 + 1;
// Read the block identified by "handle" from "file".
// The only relevant option is options.verify_checksums for now.
// On failure return non-OK.
// On success fill *result and return OK - caller owns *result
Status ReadBlockFromFile(RandomAccessFile* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
std::unique_ptr<Block>* result, Env* env,
bool do_uncompress = true) {
BlockContents contents;
Status s = ReadBlockContents(file, footer, options, handle, &contents, env,
do_uncompress);
if (s.ok()) {
result->reset(new Block(std::move(contents)));
}
return s;
}
// Delete the resource that is held by the iterator.
template <class ResourceType>
void DeleteHeldResource(void* arg, void* ignored) {
delete reinterpret_cast<ResourceType*>(arg);
}
// Delete the entry resided in the cache.
template <class Entry>
void DeleteCachedEntry(const Slice& key, void* value) {
auto entry = reinterpret_cast<Entry*>(value);
delete entry;
}
// Release the cached entry and decrement its ref count.
void ReleaseCachedEntry(void* arg, void* h) {
Cache* cache = reinterpret_cast<Cache*>(arg);
Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h);
cache->Release(handle);
}
Slice GetCacheKey(const char* cache_key_prefix, size_t cache_key_prefix_size,
const BlockHandle& handle, char* cache_key) {
assert(cache_key != nullptr);
assert(cache_key_prefix_size != 0);
assert(cache_key_prefix_size <= kMaxCacheKeyPrefixSize);
memcpy(cache_key, cache_key_prefix, cache_key_prefix_size);
char* end =
EncodeVarint64(cache_key + cache_key_prefix_size, handle.offset());
return Slice(cache_key, static_cast<size_t>(end - cache_key));
}
Cache::Handle* GetEntryFromCache(Cache* block_cache, const Slice& key,
Tickers block_cache_miss_ticker,
Tickers block_cache_hit_ticker,
Statistics* statistics) {
auto cache_handle = block_cache->Lookup(key);
if (cache_handle != nullptr) {
PERF_COUNTER_ADD(block_cache_hit_count, 1);
// overall cache hit
RecordTick(statistics, BLOCK_CACHE_HIT);
// block-type specific cache hit
RecordTick(statistics, block_cache_hit_ticker);
} else {
// overall cache miss
RecordTick(statistics, BLOCK_CACHE_MISS);
// block-type specific cache miss
RecordTick(statistics, block_cache_miss_ticker);
}
return cache_handle;
}
} // namespace
// -- IndexReader and its subclasses
// IndexReader is the interface that provide the functionality for index access.
class BlockBasedTable::IndexReader {
public:
explicit IndexReader(const Comparator* comparator)
: comparator_(comparator) {}
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
virtual Iterator* NewIterator(
BlockIter* iter = nullptr, bool total_order_seek = true) = 0;
// The size of the index.
virtual size_t size() const = 0;
// Report an approximation of how much memory has been used other than memory
// that was allocated in block cache.
virtual size_t ApproximateMemoryUsage() const = 0;
protected:
const Comparator* comparator_;
};
// 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.
class BinarySearchIndexReader : public IndexReader {
public:
// Read index from the file and create an intance for
// `BinarySearchIndexReader`.
// On success, index_reader will be populated; otherwise it will remain
// unmodified.
static Status Create(RandomAccessFile* file, const Footer& footer,
const BlockHandle& index_handle, Env* env,
const Comparator* comparator,
IndexReader** index_reader) {
std::unique_ptr<Block> index_block;
auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle,
&index_block, env);
if (s.ok()) {
*index_reader =
new BinarySearchIndexReader(comparator, std::move(index_block));
}
return s;
}
virtual Iterator* NewIterator(
BlockIter* iter = nullptr, bool dont_care = true) override {
return index_block_->NewIterator(comparator_, iter, true);
}
virtual size_t size() const override { return index_block_->size(); }
virtual size_t ApproximateMemoryUsage() const override {
assert(index_block_);
return index_block_->ApproximateMemoryUsage();
}
private:
BinarySearchIndexReader(const Comparator* comparator,
std::unique_ptr<Block>&& index_block)
: IndexReader(comparator), index_block_(std::move(index_block)) {
assert(index_block_ != nullptr);
}
std::unique_ptr<Block> index_block_;
};
// Index that leverages an internal hash table to quicken the lookup for a given
// key.
class HashIndexReader : public IndexReader {
public:
static Status Create(const SliceTransform* hash_key_extractor,
const Footer& footer, RandomAccessFile* file, Env* env,
const Comparator* comparator,
const BlockHandle& index_handle,
Iterator* meta_index_iter, IndexReader** index_reader,
bool hash_index_allow_collision) {
std::unique_ptr<Block> index_block;
auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle,
&index_block, env);
if (!s.ok()) {
return s;
}
// Note, failure to create prefix hash index does not need to be a
// hard error. We can still fall back to the original binary search index.
// So, Create will succeed regardless, from this point on.
auto new_index_reader =
new HashIndexReader(comparator, std::move(index_block));
*index_reader = new_index_reader;
// Get prefixes block
BlockHandle prefixes_handle;
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesBlock,
&prefixes_handle);
if (!s.ok()) {
// TODO: log error
return Status::OK();
}
// Get index metadata block
BlockHandle prefixes_meta_handle;
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesMetadataBlock,
&prefixes_meta_handle);
if (!s.ok()) {
// TODO: log error
return Status::OK();
}
// 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()) {
// TODO: log error
return Status::OK();
}
if (!hash_index_allow_collision) {
// TODO: deprecate once hash_index_allow_collision proves to be stable.
BlockHashIndex* hash_index = nullptr;
s = CreateBlockHashIndex(hash_key_extractor,
prefixes_contents.data,
prefixes_meta_contents.data,
&hash_index);
// TODO: log error
if (s.ok()) {
new_index_reader->index_block_->SetBlockHashIndex(hash_index);
new_index_reader->OwnPrefixesContents(std::move(prefixes_contents));
}
} else {
BlockPrefixIndex* prefix_index = nullptr;
s = BlockPrefixIndex::Create(hash_key_extractor,
prefixes_contents.data,
prefixes_meta_contents.data,
&prefix_index);
// TODO: log error
if (s.ok()) {
new_index_reader->index_block_->SetBlockPrefixIndex(prefix_index);
}
}
return Status::OK();
}
virtual Iterator* NewIterator(
BlockIter* iter = nullptr, bool total_order_seek = true) override {
return index_block_->NewIterator(comparator_, iter, total_order_seek);
}
virtual size_t size() const override { return index_block_->size(); }
virtual size_t ApproximateMemoryUsage() const override {
assert(index_block_);
return index_block_->ApproximateMemoryUsage() +
prefixes_contents_.data.size();
}
private:
HashIndexReader(const Comparator* comparator,
std::unique_ptr<Block>&& index_block)
: IndexReader(comparator), index_block_(std::move(index_block)) {
assert(index_block_ != nullptr);
}
~HashIndexReader() {
}
void OwnPrefixesContents(BlockContents&& prefixes_contents) {
prefixes_contents_ = std::move(prefixes_contents);
}
std::unique_ptr<Block> index_block_;
BlockContents prefixes_contents_;
};
struct BlockBasedTable::Rep {
Rep(const ImmutableCFOptions& _ioptions, const EnvOptions& _env_options,
const BlockBasedTableOptions& _table_opt,
const InternalKeyComparator& _internal_comparator)
: ioptions(_ioptions),
env_options(_env_options),
table_options(_table_opt),
filter_policy(_table_opt.filter_policy.get()),
internal_comparator(_internal_comparator),
whole_key_filtering(_table_opt.whole_key_filtering),
prefix_filtering(true) {}
const ImmutableCFOptions& ioptions;
const EnvOptions& env_options;
const BlockBasedTableOptions& table_options;
const FilterPolicy* const filter_policy;
const InternalKeyComparator& internal_comparator;
Status status;
unique_ptr<RandomAccessFile> file;
char cache_key_prefix[kMaxCacheKeyPrefixSize];
size_t cache_key_prefix_size = 0;
char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize];
size_t compressed_cache_key_prefix_size = 0;
// Footer contains the fixed table information
Footer footer;
// index_reader and filter will be populated and used only when
// options.block_cache is nullptr; otherwise we will get the index block via
// the block cache.
unique_ptr<IndexReader> index_reader;
unique_ptr<FilterBlockReader> filter;
std::shared_ptr<const TableProperties> table_properties;
BlockBasedTableOptions::IndexType index_type;
bool hash_index_allow_collision;
bool whole_key_filtering;
bool prefix_filtering;
// TODO(kailiu) It is very ugly to use internal key in table, since table
// module should not be relying on db module. However to make things easier
// and compatible with existing code, we introduce a wrapper that allows
// block to extract prefix without knowing if a key is internal or not.
unique_ptr<SliceTransform> internal_prefix_transform;
};
BlockBasedTable::~BlockBasedTable() {
delete rep_;
}
// CachableEntry represents the entries that *may* be fetched from block cache.
// field `value` is the item we want to get.
// field `cache_handle` is the cache handle to the block cache. If the value
// was not read from cache, `cache_handle` will be nullptr.
template <class TValue>
struct BlockBasedTable::CachableEntry {
CachableEntry(TValue* _value, Cache::Handle* _cache_handle)
: value(_value), cache_handle(_cache_handle) {}
CachableEntry() : CachableEntry(nullptr, nullptr) {}
void Release(Cache* cache) {
if (cache_handle) {
cache->Release(cache_handle);
value = nullptr;
cache_handle = nullptr;
}
}
TValue* value = nullptr;
// if the entry is from the cache, cache_handle will be populated.
Cache::Handle* cache_handle = nullptr;
};
// Helper function to setup the cache key's prefix for the Table.
void BlockBasedTable::SetupCacheKeyPrefix(Rep* rep) {
assert(kMaxCacheKeyPrefixSize >= 10);
rep->cache_key_prefix_size = 0;
rep->compressed_cache_key_prefix_size = 0;
if (rep->table_options.block_cache != nullptr) {
GenerateCachePrefix(rep->table_options.block_cache.get(), rep->file.get(),
&rep->cache_key_prefix[0],
&rep->cache_key_prefix_size);
}
if (rep->table_options.block_cache_compressed != nullptr) {
GenerateCachePrefix(rep->table_options.block_cache_compressed.get(),
rep->file.get(), &rep->compressed_cache_key_prefix[0],
&rep->compressed_cache_key_prefix_size);
}
}
void BlockBasedTable::GenerateCachePrefix(Cache* cc,
RandomAccessFile* file, char* buffer, size_t* size) {
// generate an id from the file
*size = file->GetUniqueId(buffer, kMaxCacheKeyPrefixSize);
// If the prefix wasn't generated or was too long,
// create one from the cache.
if (*size == 0) {
char* end = EncodeVarint64(buffer, cc->NewId());
*size = static_cast<size_t>(end - buffer);
}
}
void BlockBasedTable::GenerateCachePrefix(Cache* cc,
WritableFile* file, char* buffer, size_t* size) {
// generate an id from the file
*size = file->GetUniqueId(buffer, kMaxCacheKeyPrefixSize);
// If the prefix wasn't generated or was too long,
// create one from the cache.
if (*size == 0) {
char* end = EncodeVarint64(buffer, cc->NewId());
*size = static_cast<size_t>(end - buffer);
}
}
namespace {
// Return True if table_properties has `user_prop_name` has a `true` value
// or it doesn't contain this property (for backward compatible).
bool IsFeatureSupported(const TableProperties& table_properties,
const std::string& user_prop_name, Logger* info_log) {
auto& props = table_properties.user_collected_properties;
auto pos = props.find(user_prop_name);
// Older version doesn't have this value set. Skip this check.
if (pos != props.end()) {
if (pos->second == kPropFalse) {
return false;
} else if (pos->second != kPropTrue) {
Log(InfoLogLevel::WARN_LEVEL, info_log,
"Property %s has invalidate value %s", user_prop_name.c_str(),
pos->second.c_str());
}
}
return true;
}
} // namespace
Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator,
unique_ptr<RandomAccessFile>&& file,
uint64_t file_size,
unique_ptr<TableReader>* table_reader,
const bool prefetch_index_and_filter) {
table_reader->reset();
Footer footer;
auto s = ReadFooterFromFile(file.get(), file_size, &footer,
kBlockBasedTableMagicNumber);
if (!s.ok()) {
return s;
}
if (!BlockBasedTableSupportedVersion(footer.version())) {
return Status::Corruption(
"Unknown Footer version. Maybe this file was created with newer "
"version of RocksDB?");
}
// We've successfully read the footer and the index block: we're
// ready to serve requests.
Rep* rep = new BlockBasedTable::Rep(
ioptions, env_options, table_options, internal_comparator);
rep->file = std::move(file);
rep->footer = footer;
rep->index_type = table_options.index_type;
rep->hash_index_allow_collision = table_options.hash_index_allow_collision;
SetupCacheKeyPrefix(rep);
unique_ptr<BlockBasedTable> new_table(new BlockBasedTable(rep));
// Read meta index
std::unique_ptr<Block> meta;
std::unique_ptr<Iterator> meta_iter;
s = ReadMetaBlock(rep, &meta, &meta_iter);
if (!s.ok()) {
return s;
}
// Read the properties
bool found_properties_block = true;
s = SeekToPropertiesBlock(meta_iter.get(), &found_properties_block);
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Cannot seek to properties block from file: %s",
s.ToString().c_str());
} else if (found_properties_block) {
s = meta_iter->status();
TableProperties* table_properties = nullptr;
if (s.ok()) {
s = ReadProperties(meta_iter->value(), rep->file.get(), rep->footer,
rep->ioptions.env, rep->ioptions.info_log,
&table_properties);
}
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Encountered error while reading data from properties "
"block %s", s.ToString().c_str());
} else {
rep->table_properties.reset(table_properties);
}
} else {
Log(InfoLogLevel::ERROR_LEVEL, rep->ioptions.info_log,
"Cannot find Properties block from file.");
}
// Determine whether whole key filtering is supported.
if (rep->table_properties) {
rep->whole_key_filtering &=
IsFeatureSupported(*(rep->table_properties),
BlockBasedTablePropertyNames::kWholeKeyFiltering,
rep->ioptions.info_log);
rep->prefix_filtering &= IsFeatureSupported(
*(rep->table_properties),
BlockBasedTablePropertyNames::kPrefixFiltering, rep->ioptions.info_log);
}
if (prefetch_index_and_filter) {
// pre-fetching of blocks is turned on
// Will use block cache for index/filter blocks access?
if (table_options.cache_index_and_filter_blocks) {
assert(table_options.block_cache != nullptr);
// Hack: Call NewIndexIterator() to implicitly add index to the
// block_cache
unique_ptr<Iterator> iter(new_table->NewIndexIterator(ReadOptions()));
s = iter->status();
if (s.ok()) {
// Hack: Call GetFilter() to implicitly add filter to the block_cache
auto filter_entry = new_table->GetFilter();
filter_entry.Release(table_options.block_cache.get());
}
} else {
// If we don't use block cache for index/filter blocks access, we'll
// pre-load these blocks, which will kept in member variables in Rep
// and with a same life-time as this table object.
IndexReader* index_reader = nullptr;
s = new_table->CreateIndexReader(&index_reader, meta_iter.get());
if (s.ok()) {
rep->index_reader.reset(index_reader);
// Set filter block
if (rep->filter_policy) {
rep->filter.reset(ReadFilter(rep, meta_iter.get(), nullptr));
}
} else {
delete index_reader;
}
}
}
if (s.ok()) {
*table_reader = std::move(new_table);
}
return s;
}
void BlockBasedTable::SetupForCompaction() {
switch (rep_->ioptions.access_hint_on_compaction_start) {
case Options::NONE:
break;
case Options::NORMAL:
rep_->file->Hint(RandomAccessFile::NORMAL);
break;
case Options::SEQUENTIAL:
rep_->file->Hint(RandomAccessFile::SEQUENTIAL);
break;
case Options::WILLNEED:
rep_->file->Hint(RandomAccessFile::WILLNEED);
break;
default:
assert(false);
}
compaction_optimized_ = true;
}
std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties()
const {
return rep_->table_properties;
}
size_t BlockBasedTable::ApproximateMemoryUsage() const {
size_t usage = 0;
if (rep_->filter) {
usage += rep_->filter->ApproximateMemoryUsage();
}
if (rep_->index_reader) {
usage += rep_->index_reader->ApproximateMemoryUsage();
}
return usage;
}
// Load the meta-block from the file. On success, return the loaded meta block
// and its iterator.
Status BlockBasedTable::ReadMetaBlock(
Rep* rep,
std::unique_ptr<Block>* meta_block,
std::unique_ptr<Iterator>* iter) {
// TODO(sanjay): Skip this if footer.metaindex_handle() size indicates
// it is an empty block.
// TODO: we never really verify check sum for meta index block
std::unique_ptr<Block> meta;
Status s = ReadBlockFromFile(
rep->file.get(),
rep->footer,
ReadOptions(),
rep->footer.metaindex_handle(),
&meta,
rep->ioptions.env);
if (!s.ok()) {
Log(InfoLogLevel::ERROR_LEVEL, rep->ioptions.info_log,
"Encountered error while reading data from properties"
" block %s", s.ToString().c_str());
return s;
}
*meta_block = std::move(meta);
// meta block uses bytewise comparator.
iter->reset(meta_block->get()->NewIterator(BytewiseComparator()));
return Status::OK();
}
Status BlockBasedTable::GetDataBlockFromCache(
const Slice& block_cache_key, const Slice& compressed_block_cache_key,
Cache* block_cache, Cache* block_cache_compressed, Statistics* statistics,
const ReadOptions& read_options,
BlockBasedTable::CachableEntry<Block>* block, uint32_t format_version) {
Status s;
Block* compressed_block = nullptr;
Cache::Handle* block_cache_compressed_handle = nullptr;
// Lookup uncompressed cache first
if (block_cache != nullptr) {
block->cache_handle =
GetEntryFromCache(block_cache, block_cache_key, BLOCK_CACHE_DATA_MISS,
BLOCK_CACHE_DATA_HIT, statistics);
if (block->cache_handle != nullptr) {
block->value =
reinterpret_cast<Block*>(block_cache->Value(block->cache_handle));
return s;
}
}
// If not found, search from the compressed block cache.
assert(block->cache_handle == nullptr && block->value == nullptr);
if (block_cache_compressed == nullptr) {
return s;
}
assert(!compressed_block_cache_key.empty());
block_cache_compressed_handle =
block_cache_compressed->Lookup(compressed_block_cache_key);
// if we found in the compressed cache, then uncompress and insert into
// uncompressed cache
if (block_cache_compressed_handle == nullptr) {
RecordTick(statistics, BLOCK_CACHE_COMPRESSED_MISS);
return s;
}
// found compressed block
RecordTick(statistics, BLOCK_CACHE_COMPRESSED_HIT);
compressed_block = reinterpret_cast<Block*>(
block_cache_compressed->Value(block_cache_compressed_handle));
assert(compressed_block->compression_type() != kNoCompression);
// Retrieve the uncompressed contents into a new buffer
BlockContents contents;
s = UncompressBlockContents(compressed_block->data(),
compressed_block->size(), &contents,
format_version);
// Insert uncompressed block into block cache
if (s.ok()) {
block->value = new Block(std::move(contents)); // uncompressed block
assert(block->value->compression_type() == kNoCompression);
if (block_cache != nullptr && block->value->cachable() &&
read_options.fill_cache) {
block->cache_handle =
block_cache->Insert(block_cache_key, block->value,
block->value->size(), &DeleteCachedEntry<Block>);
assert(reinterpret_cast<Block*>(
block_cache->Value(block->cache_handle)) == block->value);
}
}
// Release hold on compressed cache entry
block_cache_compressed->Release(block_cache_compressed_handle);
return s;
}
Status BlockBasedTable::PutDataBlockToCache(
const Slice& block_cache_key, const Slice& compressed_block_cache_key,
Cache* block_cache, Cache* block_cache_compressed,
const ReadOptions& read_options, Statistics* statistics,
CachableEntry<Block>* block, Block* raw_block, uint32_t format_version) {
assert(raw_block->compression_type() == kNoCompression ||
block_cache_compressed != nullptr);
Status s;
// Retrieve the uncompressed contents into a new buffer
BlockContents contents;
if (raw_block->compression_type() != kNoCompression) {
s = UncompressBlockContents(raw_block->data(), raw_block->size(), &contents,
format_version);
}
if (!s.ok()) {
delete raw_block;
return s;
}
if (raw_block->compression_type() != kNoCompression) {
block->value = new Block(std::move(contents)); // uncompressed block
} else {
block->value = raw_block;
raw_block = nullptr;
}
// Insert compressed block into compressed block cache.
// Release the hold on the compressed cache entry immediately.
if (block_cache_compressed != nullptr && raw_block != nullptr &&
raw_block->cachable()) {
auto cache_handle = block_cache_compressed->Insert(
compressed_block_cache_key, raw_block, raw_block->size(),
&DeleteCachedEntry<Block>);
block_cache_compressed->Release(cache_handle);
RecordTick(statistics, BLOCK_CACHE_COMPRESSED_MISS);
// Avoid the following code to delete this cached block.
raw_block = nullptr;
}
delete raw_block;
// insert into uncompressed block cache
assert((block->value->compression_type() == kNoCompression));
if (block_cache != nullptr && block->value->cachable()) {
block->cache_handle =
block_cache->Insert(block_cache_key, block->value, block->value->size(),
&DeleteCachedEntry<Block>);
RecordTick(statistics, BLOCK_CACHE_ADD);
assert(reinterpret_cast<Block*>(block_cache->Value(block->cache_handle)) ==
block->value);
}
return s;
}
FilterBlockReader* BlockBasedTable::ReadFilter(
Rep* rep, Iterator* meta_index_iter, size_t* filter_size) {
// TODO: We might want to unify with ReadBlockFromFile() if we start
// requiring checksum verification in Table::Open.
for (auto prefix : {kFullFilterBlockPrefix, kFilterBlockPrefix}) {
std::string filter_block_key = prefix;
filter_block_key.append(rep->filter_policy->Name());
BlockHandle handle;
if (FindMetaBlock(meta_index_iter, filter_block_key, &handle).ok()) {
BlockContents block;
if (!ReadBlockContents(rep->file.get(), rep->footer, ReadOptions(),
handle, &block, rep->ioptions.env, false).ok()) {
// Error reading the block
return nullptr;
}
if (filter_size) {
*filter_size = block.data.size();
}
assert(rep->filter_policy);
if (kFilterBlockPrefix == prefix) {
return new BlockBasedFilterBlockReader(
rep->prefix_filtering ? rep->ioptions.prefix_extractor : nullptr,
rep->table_options, rep->whole_key_filtering, std::move(block));
} else if (kFullFilterBlockPrefix == prefix) {
auto filter_bits_reader = rep->filter_policy->
GetFilterBitsReader(block.data);
if (filter_bits_reader != nullptr) {
return new FullFilterBlockReader(
rep->prefix_filtering ? rep->ioptions.prefix_extractor : nullptr,
rep->whole_key_filtering, std::move(block), filter_bits_reader);
}
} else {
assert(false);
return nullptr;
}
}
}
return nullptr;
}
BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
bool no_io) const {
// If cache_index_and_filter_blocks is false, filter should be pre-populated.
// We will return rep_->filter anyway. rep_->filter can be nullptr if filter
// read fails at Open() time. We don't want to reload again since it will
// most probably fail again.
if (!rep_->table_options.cache_index_and_filter_blocks) {
return {rep_->filter.get(), nullptr /* cache handle */};
}
Cache* block_cache = rep_->table_options.block_cache.get();
if (rep_->filter_policy == nullptr /* do not use filter */ ||
block_cache == nullptr /* no block cache at all */) {
return {nullptr /* filter */, nullptr /* cache handle */};
}
// Fetching from the cache
char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size,
rep_->footer.metaindex_handle(),
cache_key);
Statistics* statistics = rep_->ioptions.statistics;
auto cache_handle =
GetEntryFromCache(block_cache, key, BLOCK_CACHE_FILTER_MISS,
BLOCK_CACHE_FILTER_HIT, statistics);
FilterBlockReader* filter = nullptr;
if (cache_handle != nullptr) {
filter = reinterpret_cast<FilterBlockReader*>(
block_cache->Value(cache_handle));
} else if (no_io) {
// Do not invoke any io.
return CachableEntry<FilterBlockReader>();
} else {
size_t filter_size = 0;
std::unique_ptr<Block> meta;
std::unique_ptr<Iterator> iter;
auto s = ReadMetaBlock(rep_, &meta, &iter);
if (s.ok()) {
filter = ReadFilter(rep_, iter.get(), &filter_size);
if (filter != nullptr) {
assert(filter_size > 0);
cache_handle = block_cache->Insert(
key, filter, filter_size, &DeleteCachedEntry<FilterBlockReader>);
RecordTick(statistics, BLOCK_CACHE_ADD);
}
}
}
return { filter, cache_handle };
}
Iterator* BlockBasedTable::NewIndexIterator(const ReadOptions& read_options,
BlockIter* input_iter) {
// index reader has already been pre-populated.
if (rep_->index_reader) {
return rep_->index_reader->NewIterator(
input_iter, read_options.total_order_seek);
}
bool no_io = read_options.read_tier == kBlockCacheTier;
Cache* block_cache = rep_->table_options.block_cache.get();
char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
auto key = GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size,
rep_->footer.index_handle(), cache_key);
Statistics* statistics = rep_->ioptions.statistics;
auto cache_handle =
GetEntryFromCache(block_cache, key, BLOCK_CACHE_INDEX_MISS,
BLOCK_CACHE_INDEX_HIT, statistics);
if (cache_handle == nullptr && no_io) {
if (input_iter != nullptr) {
input_iter->SetStatus(Status::Incomplete("no blocking io"));
return input_iter;
} else {
return NewErrorIterator(Status::Incomplete("no blocking io"));
}
}
IndexReader* index_reader = nullptr;
if (cache_handle != nullptr) {
index_reader =
reinterpret_cast<IndexReader*>(block_cache->Value(cache_handle));
} else {
// Create index reader and put it in the cache.
Status s;
s = CreateIndexReader(&index_reader);
if (!s.ok()) {
// make sure if something goes wrong, index_reader shall remain intact.
assert(index_reader == nullptr);
if (input_iter != nullptr) {
input_iter->SetStatus(s);
return input_iter;
} else {
return NewErrorIterator(s);
}
}
cache_handle = block_cache->Insert(key, index_reader, index_reader->size(),
&DeleteCachedEntry<IndexReader>);
RecordTick(statistics, BLOCK_CACHE_ADD);
}
assert(cache_handle);
auto* iter = index_reader->NewIterator(
input_iter, read_options.total_order_seek);
iter->RegisterCleanup(&ReleaseCachedEntry, block_cache, cache_handle);
return iter;
}
// Convert an index iterator value (i.e., an encoded BlockHandle)
// into an iterator over the contents of the corresponding block.
// If input_iter is null, new a iterator
// If input_iter is not null, update this iter and return it
Iterator* BlockBasedTable::NewDataBlockIterator(Rep* rep,
const ReadOptions& ro, const Slice& index_value,
BlockIter* input_iter) {
const bool no_io = (ro.read_tier == kBlockCacheTier);
Cache* block_cache = rep->table_options.block_cache.get();
Cache* block_cache_compressed =
rep->table_options.block_cache_compressed.get();
CachableEntry<Block> block;
BlockHandle handle;
Slice input = index_value;
// We intentionally allow extra stuff in index_value so that we
// can add more features in the future.
Status s = handle.DecodeFrom(&input);
if (!s.ok()) {
if (input_iter != nullptr) {
input_iter->SetStatus(s);
return input_iter;
} else {
return NewErrorIterator(s);
}
}
// If either block cache is enabled, we'll try to read from it.
if (block_cache != nullptr || block_cache_compressed != nullptr) {
Statistics* statistics = rep->ioptions.statistics;
char cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
char compressed_cache_key[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
Slice key, /* key to the block cache */
ckey /* key to the compressed block cache */;
// create key for block cache
if (block_cache != nullptr) {
key = GetCacheKey(rep->cache_key_prefix, rep->cache_key_prefix_size,
handle, cache_key);
}
if (block_cache_compressed != nullptr) {
ckey = GetCacheKey(rep->compressed_cache_key_prefix,
rep->compressed_cache_key_prefix_size, handle,
compressed_cache_key);
}
s = GetDataBlockFromCache(key, ckey, block_cache, block_cache_compressed,
statistics, ro, &block,
rep->table_options.format_version);
if (block.value == nullptr && !no_io && ro.fill_cache) {
std::unique_ptr<Block> raw_block;
{
StopWatch sw(rep->ioptions.env, statistics, READ_BLOCK_GET_MICROS);
s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle,
&raw_block, rep->ioptions.env,
block_cache_compressed == nullptr);
}
if (s.ok()) {
s = PutDataBlockToCache(key, ckey, block_cache, block_cache_compressed,
ro, statistics, &block, raw_block.release(),
rep->table_options.format_version);
}
}
}
// Didn't get any data from block caches.
if (block.value == nullptr) {
if (no_io) {
// Could not read from block_cache and can't do IO
if (input_iter != nullptr) {
input_iter->SetStatus(Status::Incomplete("no blocking io"));
return input_iter;
} else {
return NewErrorIterator(Status::Incomplete("no blocking io"));
}
}
std::unique_ptr<Block> block_value;
s = ReadBlockFromFile(rep->file.get(), rep->footer, ro, handle,
&block_value, rep->ioptions.env);
if (s.ok()) {
block.value = block_value.release();
}
}
Iterator* iter;
if (block.value != nullptr) {
iter = block.value->NewIterator(&rep->internal_comparator, input_iter);
if (block.cache_handle != nullptr) {
iter->RegisterCleanup(&ReleaseCachedEntry, block_cache,
block.cache_handle);
} else {
iter->RegisterCleanup(&DeleteHeldResource<Block>, block.value, nullptr);
}
} else {
if (input_iter != nullptr) {
input_iter->SetStatus(s);
iter = input_iter;
} else {
iter = NewErrorIterator(s);
}
}
return iter;
}
class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState {
public:
BlockEntryIteratorState(BlockBasedTable* table,
const ReadOptions& read_options)
: TwoLevelIteratorState(
table->rep_->ioptions.prefix_extractor != nullptr),
table_(table),
read_options_(read_options) {}
Iterator* NewSecondaryIterator(const Slice& index_value) override {
return NewDataBlockIterator(table_->rep_, read_options_, index_value);
}
bool PrefixMayMatch(const Slice& internal_key) override {
if (read_options_.total_order_seek) {
return true;
}
return table_->PrefixMayMatch(internal_key);
}
private:
// Don't own table_
BlockBasedTable* table_;
const ReadOptions read_options_;
};
// This will be broken if the user specifies an unusual implementation
// of Options.comparator, or if the user specifies an unusual
// definition of prefixes in BlockBasedTableOptions.filter_policy.
// In particular, we require the following three properties:
//
// 1) key.starts_with(prefix(key))
// 2) Compare(prefix(key), key) <= 0.
// 3) If Compare(key1, key2) <= 0, then Compare(prefix(key1), prefix(key2)) <= 0
//
// Otherwise, this method guarantees no I/O will be incurred.
//
// REQUIRES: this method shouldn't be called while the DB lock is held.
bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) {
if (!rep_->filter_policy) {
return true;
}
assert(rep_->ioptions.prefix_extractor != nullptr);
auto prefix = rep_->ioptions.prefix_extractor->Transform(
ExtractUserKey(internal_key));
InternalKey internal_key_prefix(prefix, 0, kTypeValue);
auto internal_prefix = internal_key_prefix.Encode();
bool may_match = true;
Status s;
// To prevent any io operation in this method, we set `read_tier` to make
// sure we always read index or filter only when they have already been
// loaded to memory.
ReadOptions no_io_read_options;
no_io_read_options.read_tier = kBlockCacheTier;
// First, try check with full filter
auto filter_entry = GetFilter(true /* no io */);
FilterBlockReader* filter = filter_entry.value;
if (filter != nullptr && !filter->IsBlockBased()) {
may_match = filter->PrefixMayMatch(prefix);
}
// Then, try find it within each block
if (may_match) {
unique_ptr<Iterator> iiter(NewIndexIterator(no_io_read_options));
iiter->Seek(internal_prefix);
if (!iiter->Valid()) {
// we're past end of file
// if it's incomplete, it means that we avoided I/O
// and we're not really sure that we're past the end
// of the file
may_match = iiter->status().IsIncomplete();
} else if (ExtractUserKey(iiter->key()).starts_with(
ExtractUserKey(internal_prefix))) {
// we need to check for this subtle case because our only
// guarantee is that "the key is a string >= last key in that data
// block" according to the doc/table_format.txt spec.
//
// Suppose iiter->key() starts with the desired prefix; it is not
// necessarily the case that the corresponding data block will
// contain the prefix, since iiter->key() need not be in the
// block. However, the next data block may contain the prefix, so
// we return true to play it safe.
may_match = true;
} else if (filter != nullptr && filter->IsBlockBased()) {
// iiter->key() does NOT start with the desired prefix. Because
// Seek() finds the first key that is >= the seek target, this
// means that iiter->key() > prefix. Thus, any data blocks coming
// after the data block corresponding to iiter->key() cannot
// possibly contain the key. Thus, the corresponding data block
// is the only on could potentially contain the prefix.
Slice handle_value = iiter->value();
BlockHandle handle;
s = handle.DecodeFrom(&handle_value);
assert(s.ok());
may_match = filter->PrefixMayMatch(prefix, handle.offset());
}
}
Statistics* statistics = rep_->ioptions.statistics;
RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED);
if (!may_match) {
RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL);
}
filter_entry.Release(rep_->table_options.block_cache.get());
return may_match;
}
Iterator* BlockBasedTable::NewIterator(const ReadOptions& read_options,
Arena* arena) {
return NewTwoLevelIterator(new BlockEntryIteratorState(this, read_options),
NewIndexIterator(read_options), arena);
}
bool BlockBasedTable::FullFilterKeyMayMatch(FilterBlockReader* filter,
const Slice& internal_key) const {
if (filter == nullptr || filter->IsBlockBased()) {
return true;
}
Slice user_key = ExtractUserKey(internal_key);
if (!filter->KeyMayMatch(user_key)) {
return false;
}
if (rep_->ioptions.prefix_extractor &&
!filter->PrefixMayMatch(
rep_->ioptions.prefix_extractor->Transform(user_key))) {
return false;
}
return true;
}
Status BlockBasedTable::Get(
const ReadOptions& read_options, const Slice& key,
GetContext* get_context) {
Status s;
auto filter_entry = GetFilter(read_options.read_tier == kBlockCacheTier);
FilterBlockReader* filter = filter_entry.value;
// First check the full filter
// If full filter not useful, Then go into each block
if (!FullFilterKeyMayMatch(filter, key)) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
} else {
BlockIter iiter;
NewIndexIterator(read_options, &iiter);
bool done = false;
for (iiter.Seek(key); iiter.Valid() && !done; iiter.Next()) {
Slice handle_value = iiter.value();
BlockHandle handle;
bool not_exist_in_filter =
filter != nullptr && filter->IsBlockBased() == true &&
handle.DecodeFrom(&handle_value).ok() &&
!filter->KeyMayMatch(ExtractUserKey(key), handle.offset());
if (not_exist_in_filter) {
// Not found
// TODO: think about interaction with Merge. If a user key cannot
// cross one data block, we should be fine.
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_USEFUL);
break;
} else {
BlockIter biter;
NewDataBlockIterator(rep_, read_options, iiter.value(), &biter);
if (read_options.read_tier && biter.status().IsIncomplete()) {
// couldn't get block from block_cache
// Update Saver.state to Found because we are only looking for whether
// we can guarantee the key is not there when "no_io" is set
get_context->MarkKeyMayExist();
break;
}
if (!biter.status().ok()) {
s = biter.status();
break;
}
// Call the *saver function on each entry/block until it returns false
for (biter.Seek(key); biter.Valid(); biter.Next()) {
ParsedInternalKey parsed_key;
if (!ParseInternalKey(biter.key(), &parsed_key)) {
s = Status::Corruption(Slice());
}
if (!get_context->SaveValue(parsed_key, biter.value())) {
done = true;
break;
}
}
s = biter.status();
}
}
if (s.ok()) {
s = iiter.status();
}
}
filter_entry.Release(rep_->table_options.block_cache.get());
return s;
}
Status BlockBasedTable::Prefetch(const Slice* const begin,
const Slice* const end) {
auto& comparator = rep_->internal_comparator;
// pre-condition
if (begin && end && comparator.Compare(*begin, *end) > 0) {
return Status::InvalidArgument(*begin, *end);
}
BlockIter iiter;
NewIndexIterator(ReadOptions(), &iiter);
if (!iiter.status().ok()) {
// error opening index iterator
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();
if (end && comparator.Compare(iiter.key(), *end) >= 0) {
if (prefetching_boundary_page) {
break;
}
// The index entry represents the last key in the data block.
// We should load this page into memory as well, but no more
prefetching_boundary_page = true;
}
// Load the block specified by the block_handle into the block cache
BlockIter biter;
NewDataBlockIterator(rep_, ReadOptions(), block_handle, &biter);
if (!biter.status().ok()) {
// there was an unexpected error while pre-fetching
return biter.status();
}
}
return Status::OK();
}
bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options,
const Slice& key) {
std::unique_ptr<Iterator> iiter(NewIndexIterator(options));
iiter->Seek(key);
assert(iiter->Valid());
CachableEntry<Block> block;
BlockHandle handle;
Slice input = iiter->value();
Status s = handle.DecodeFrom(&input);
assert(s.ok());
Cache* block_cache = rep_->table_options.block_cache.get();
assert(block_cache != nullptr);
char cache_key_storage[kMaxCacheKeyPrefixSize + kMaxVarint64Length];
Slice cache_key =
GetCacheKey(rep_->cache_key_prefix, rep_->cache_key_prefix_size,
handle, cache_key_storage);
Slice ckey;
s = GetDataBlockFromCache(cache_key, ckey, block_cache, nullptr, nullptr,
options, &block,
rep_->table_options.format_version);
assert(s.ok());
bool in_cache = block.value != nullptr;
if (in_cache) {
ReleaseCachedEntry(block_cache, block.cache_handle);
}
return in_cache;
}
// REQUIRES: The following fields of rep_ should have already been populated:
// 1. file
// 2. index_handle,
// 3. options
// 4. internal_comparator
// 5. index_type
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;
if (rep_->table_properties) {
auto& props = rep_->table_properties->user_collected_properties;
auto pos = props.find(BlockBasedTablePropertyNames::kIndexType);
if (pos != props.end()) {
index_type_on_file = static_cast<BlockBasedTableOptions::IndexType>(
DecodeFixed32(pos->second.c_str()));
}
}
auto file = rep_->file.get();
auto env = rep_->ioptions.env;
auto comparator = &rep_->internal_comparator;
const Footer& footer = rep_->footer;
if (index_type_on_file == BlockBasedTableOptions::kHashSearch &&
rep_->ioptions.prefix_extractor == nullptr) {
Log(InfoLogLevel::WARN_LEVEL, rep_->ioptions.info_log,
"BlockBasedTableOptions::kHashSearch requires "
"options.prefix_extractor to be set."
" Fall back to binary search index.");
index_type_on_file = BlockBasedTableOptions::kBinarySearch;
}
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()) {
// we simply fall back to binary search in case there is any
// problem with prefix hash index loading.
Log(InfoLogLevel::WARN_LEVEL, rep_->ioptions.info_log,
"Unable to read the metaindex block."
" Fall back to binary search index.");
return BinarySearchIndexReader::Create(
file, footer, footer.index_handle(), env, comparator, index_reader);
}
meta_index_iter = meta_iter_guard.get();
}
// We need to wrap data with internal_prefix_transform to make sure it can
// handle prefix correctly.
rep_->internal_prefix_transform.reset(
new InternalKeySliceTransform(rep_->ioptions.prefix_extractor));
return HashIndexReader::Create(
rep_->internal_prefix_transform.get(), footer, file, env, comparator,
footer.index_handle(), meta_index_iter, index_reader,
rep_->hash_index_allow_collision);
}
default: {
std::string error_message =
"Unrecognized index type: " + ToString(rep_->index_type);
return Status::InvalidArgument(error_message.c_str());
}
}
}
uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key) {
unique_ptr<Iterator> index_iter(NewIndexIterator(ReadOptions()));
index_iter->Seek(key);
uint64_t result;
if (index_iter->Valid()) {
BlockHandle handle;
Slice input = index_iter->value();
Status s = handle.DecodeFrom(&input);
if (s.ok()) {
result = handle.offset();
} else {
// Strange: we can't decode the block handle in the index block.
// We'll just return the offset of the metaindex block, which is
// close to the whole file size for this case.
result = rep_->footer.metaindex_handle().offset();
}
} else {
// key is past the last key in the file. If table_properties is not
// available, approximate the offset by returning the offset of the
// metaindex block (which is right near the end of the file).
result = 0;
if (rep_->table_properties) {
result = rep_->table_properties->data_size;
}
// table_properties is not present in the table.
if (result == 0) {
result = rep_->footer.metaindex_handle().offset();
}
}
return result;
}
bool BlockBasedTable::TEST_filter_block_preloaded() const {
return rep_->filter != nullptr;
}
bool BlockBasedTable::TEST_index_reader_preloaded() const {
return rep_->index_reader != nullptr;
}
Status BlockBasedTable::DumpTable(WritableFile* out_file) {
// Output Footer
out_file->Append(
"Footer Details:\n"
"--------------------------------------\n"
" ");
out_file->Append(rep_->footer.ToString().c_str());
out_file->Append("\n");
// Output MetaIndex
out_file->Append(
"Metaindex Details:\n"
"--------------------------------------\n");
std::unique_ptr<Block> meta;
std::unique_ptr<Iterator> meta_iter;
Status s = ReadMetaBlock(rep_, &meta, &meta_iter);
if (s.ok()) {
for (meta_iter->SeekToFirst(); meta_iter->Valid(); meta_iter->Next()) {
s = meta_iter->status();
if (!s.ok()) {
return s;
}
if (meta_iter->key() == rocksdb::kPropertiesBlock) {
out_file->Append(" Properties block handle: ");
out_file->Append(meta_iter->value().ToString(true).c_str());
out_file->Append("\n");
} else if (strstr(meta_iter->key().ToString().c_str(),
"filter.rocksdb.") != nullptr) {
out_file->Append(" Filter block handle: ");
out_file->Append(meta_iter->value().ToString(true).c_str());
out_file->Append("\n");
}
}
out_file->Append("\n");
} else {
return s;
}
// Output TableProperties
const rocksdb::TableProperties* table_properties;
table_properties = rep_->table_properties.get();
if (table_properties != nullptr) {
out_file->Append(
"Table Properties:\n"
"--------------------------------------\n"
" ");
out_file->Append(table_properties->ToString("\n ", ": ").c_str());
out_file->Append("\n");
}
// Output Filter blocks
if (!rep_->filter && !table_properties->filter_policy_name.empty()) {
// Support only BloomFilter as off now
rocksdb::BlockBasedTableOptions table_options;
table_options.filter_policy.reset(rocksdb::NewBloomFilterPolicy(1));
if (table_properties->filter_policy_name.compare(
table_options.filter_policy->Name()) == 0) {
std::string filter_block_key = kFilterBlockPrefix;
filter_block_key.append(table_properties->filter_policy_name);
BlockHandle handle;
if (FindMetaBlock(meta_iter.get(), filter_block_key, &handle).ok()) {
BlockContents block;
if (ReadBlockContents(rep_->file.get(), rep_->footer, ReadOptions(),
handle, &block, rep_->ioptions.env, false).ok()) {
rep_->filter.reset(new BlockBasedFilterBlockReader(
rep_->ioptions.prefix_extractor, table_options,
table_options.whole_key_filtering, std::move(block)));
}
}
}
}
if (rep_->filter) {
out_file->Append(
"Filter Details:\n"
"--------------------------------------\n"
" ");
out_file->Append(rep_->filter->ToString().c_str());
out_file->Append("\n");
}
// Output Index block
s = DumpIndexBlock(out_file);
if (!s.ok()) {
return s;
}
// Output Data blocks
s = DumpDataBlocks(out_file);
return s;
}
Status BlockBasedTable::DumpIndexBlock(WritableFile* out_file) {
out_file->Append(
"Index Details:\n"
"--------------------------------------\n");
std::unique_ptr<Iterator> blockhandles_iter(NewIndexIterator(ReadOptions()));
Status s = blockhandles_iter->status();
if (!s.ok()) {
out_file->Append("Can not read Index Block \n\n");
return s;
}
out_file->Append(" Block key hex dump: Data block handle\n");
out_file->Append(" Block key ascii\n\n");
for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
blockhandles_iter->Next()) {
s = blockhandles_iter->status();
if (!s.ok()) {
break;
}
Slice key = blockhandles_iter->key();
InternalKey ikey;
ikey.DecodeFrom(key);
out_file->Append(" HEX ");
out_file->Append(ikey.user_key().ToString(true).c_str());
out_file->Append(": ");
out_file->Append(blockhandles_iter->value().ToString(true).c_str());
out_file->Append("\n");
std::string str_key = ikey.user_key().ToString();
std::string res_key("");
char cspace = ' ';
for (size_t i = 0; i < str_key.size(); i++) {
res_key.append(&str_key[i], 1);
res_key.append(1, cspace);
}
out_file->Append(" ASCII ");
out_file->Append(res_key.c_str());
out_file->Append("\n ------\n");
}
out_file->Append("\n");
return Status::OK();
}
Status BlockBasedTable::DumpDataBlocks(WritableFile* out_file) {
std::unique_ptr<Iterator> blockhandles_iter(NewIndexIterator(ReadOptions()));
Status s = blockhandles_iter->status();
if (!s.ok()) {
out_file->Append("Can not read Index Block \n\n");
return s;
}
size_t block_id = 1;
for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
block_id++, blockhandles_iter->Next()) {
s = blockhandles_iter->status();
if (!s.ok()) {
break;
}
out_file->Append("Data Block # ");
out_file->Append(rocksdb::ToString(block_id));
out_file->Append(" @ ");
out_file->Append(blockhandles_iter->value().ToString(true).c_str());
out_file->Append("\n");
out_file->Append("--------------------------------------\n");
std::unique_ptr<Iterator> datablock_iter;
datablock_iter.reset(
NewDataBlockIterator(rep_, ReadOptions(), blockhandles_iter->value()));
s = datablock_iter->status();
if (!s.ok()) {
out_file->Append("Error reading the block - Skipped \n\n");
continue;
}
for (datablock_iter->SeekToFirst(); datablock_iter->Valid();
datablock_iter->Next()) {
s = datablock_iter->status();
if (!s.ok()) {
out_file->Append("Error reading the block - Skipped \n");
break;
}
Slice key = datablock_iter->key();
Slice value = datablock_iter->value();
InternalKey ikey, iValue;
ikey.DecodeFrom(key);
iValue.DecodeFrom(value);
out_file->Append(" HEX ");
out_file->Append(ikey.user_key().ToString(true).c_str());
out_file->Append(": ");
out_file->Append(iValue.user_key().ToString(true).c_str());
out_file->Append("\n");
std::string str_key = ikey.user_key().ToString();
std::string str_value = iValue.user_key().ToString();
std::string res_key(""), res_value("");
char cspace = ' ';
for (size_t i = 0; i < str_key.size(); i++) {
res_key.append(&str_key[i], 1);
res_key.append(1, cspace);
}
for (size_t i = 0; i < str_value.size(); i++) {
res_value.append(&str_value[i], 1);
res_value.append(1, cspace);
}
out_file->Append(" ASCII ");
out_file->Append(res_key.c_str());
out_file->Append(": ");
out_file->Append(res_value.c_str());
out_file->Append("\n ------\n");
}
out_file->Append("\n");
}
return Status::OK();
}
} // namespace rocksdb