Temporarily disable caching index/filter blocks
Summary: Mixing index/filter blocks with data blocks resulted in some known issues. To make sure in next release our users won't be affected, we added a new option in BlockBasedTableFactory::TableOption to conceal this functionality for now. This patch also introduced a BlockBasedTableReader::OpenOptions, which avoids the "infinite" growth of parameters in BlockBasedTableReader::Open(). Test Plan: make check Reviewers: haobo, sdong, igor, dhruba Reviewed By: igor CC: leveldb, tnovak Differential Revision: https://reviews.facebook.net/D15327
This commit is contained in:
parent
d24961b65e
commit
66dc033af3
@ -17,6 +17,7 @@
|
|||||||
#include "db/filename.h"
|
#include "db/filename.h"
|
||||||
#include "db/version_set.h"
|
#include "db/version_set.h"
|
||||||
#include "db/write_batch_internal.h"
|
#include "db/write_batch_internal.h"
|
||||||
|
#include "table/block_based_table_factory.h"
|
||||||
#include "rocksdb/cache.h"
|
#include "rocksdb/cache.h"
|
||||||
#include "rocksdb/compaction_filter.h"
|
#include "rocksdb/compaction_filter.h"
|
||||||
#include "rocksdb/env.h"
|
#include "rocksdb/env.h"
|
||||||
@ -732,6 +733,9 @@ TEST(DBTest, IndexAndFilterBlocksOfNewTableAddedToCache) {
|
|||||||
options.filter_policy = filter_policy.get();
|
options.filter_policy = filter_policy.get();
|
||||||
options.create_if_missing = true;
|
options.create_if_missing = true;
|
||||||
options.statistics = rocksdb::CreateDBStatistics();
|
options.statistics = rocksdb::CreateDBStatistics();
|
||||||
|
BlockBasedTableOptions table_options;
|
||||||
|
table_options.cache_index_and_filter_blocks = true;
|
||||||
|
options.table_factory.reset(new BlockBasedTableFactory(table_options));
|
||||||
DestroyAndReopen(&options);
|
DestroyAndReopen(&options);
|
||||||
|
|
||||||
ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
|
ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
|
||||||
|
@ -20,10 +20,10 @@ namespace rocksdb {
|
|||||||
|
|
||||||
Status BlockBasedTableFactory::GetTableReader(
|
Status BlockBasedTableFactory::GetTableReader(
|
||||||
const Options& options, const EnvOptions& soptions,
|
const Options& options, const EnvOptions& soptions,
|
||||||
unique_ptr<RandomAccessFile> && file, uint64_t file_size,
|
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
|
||||||
unique_ptr<TableReader>* table_reader) const {
|
unique_ptr<TableReader>* table_reader) const {
|
||||||
return BlockBasedTable::Open(options, soptions, std::move(file), file_size,
|
return BlockBasedTable::Open(options, soptions, table_options_,
|
||||||
table_reader);
|
std::move(file), file_size, table_reader);
|
||||||
}
|
}
|
||||||
|
|
||||||
TableBuilder* BlockBasedTableFactory::GetTableBuilder(
|
TableBuilder* BlockBasedTableFactory::GetTableBuilder(
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
#include "rocksdb/flush_block_policy.h"
|
#include "rocksdb/flush_block_policy.h"
|
||||||
#include "rocksdb/options.h"
|
#include "rocksdb/options.h"
|
||||||
#include "rocksdb/table.h"
|
#include "rocksdb/table.h"
|
||||||
|
#include "table/block_based_table_options.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
@ -30,40 +31,25 @@ class BlockBasedTable;
|
|||||||
class BlockBasedTableBuilder;
|
class BlockBasedTableBuilder;
|
||||||
|
|
||||||
class BlockBasedTableFactory: public TableFactory {
|
class BlockBasedTableFactory: public TableFactory {
|
||||||
public:
|
public:
|
||||||
struct TableOptions {
|
BlockBasedTableFactory() : BlockBasedTableFactory(BlockBasedTableOptions()) {}
|
||||||
// @flush_block_policy_factory creates the instances of flush block policy.
|
explicit BlockBasedTableFactory(const BlockBasedTableOptions& table_options)
|
||||||
// which provides a configurable way to determine when to flush a block in
|
: table_options_(table_options) {}
|
||||||
// the block based tables. If not set, table builder will use the default
|
|
||||||
// block flush policy, which cut blocks by block size (please refer to
|
|
||||||
// `FlushBlockBySizePolicy`).
|
|
||||||
std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
|
|
||||||
};
|
|
||||||
|
|
||||||
BlockBasedTableFactory() : BlockBasedTableFactory(TableOptions()) { }
|
~BlockBasedTableFactory() {}
|
||||||
BlockBasedTableFactory(const TableOptions& table_options):
|
|
||||||
table_options_(table_options) {
|
|
||||||
}
|
|
||||||
|
|
||||||
~BlockBasedTableFactory() {
|
const char* Name() const override { return "BlockBasedTable"; }
|
||||||
}
|
|
||||||
|
|
||||||
const char* Name() const override {
|
|
||||||
return "BlockBasedTable";
|
|
||||||
}
|
|
||||||
|
|
||||||
Status GetTableReader(const Options& options, const EnvOptions& soptions,
|
Status GetTableReader(const Options& options, const EnvOptions& soptions,
|
||||||
unique_ptr<RandomAccessFile> && file,
|
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
|
||||||
uint64_t file_size,
|
|
||||||
unique_ptr<TableReader>* table_reader) const override;
|
unique_ptr<TableReader>* table_reader) const override;
|
||||||
|
|
||||||
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file,
|
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file,
|
||||||
CompressionType compression_type) const
|
CompressionType compression_type)
|
||||||
override;
|
const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
TableOptions table_options_;
|
BlockBasedTableOptions table_options_;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
} // namespace rocksdb
|
} // namespace rocksdb
|
||||||
|
31
table/block_based_table_options.h
Normal file
31
table/block_based_table_options.h
Normal file
@ -0,0 +1,31 @@
|
|||||||
|
// 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.
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
namespace rocksdb {
|
||||||
|
|
||||||
|
class FlushBlockPolicyFactory;
|
||||||
|
|
||||||
|
struct BlockBasedTableOptions {
|
||||||
|
// @flush_block_policy_factory creates the instances of flush block policy.
|
||||||
|
// which provides a configurable way to determine when to flush a block in
|
||||||
|
// the block based tables. If not set, table builder will use the default
|
||||||
|
// block flush policy, which cut blocks by block size (please refer to
|
||||||
|
// `FlushBlockBySizePolicy`).
|
||||||
|
std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
|
||||||
|
|
||||||
|
// TODO(kailiu) Temporarily disable this feature by making the default value
|
||||||
|
// to be false. Also in master branch, this file is non-public so no user
|
||||||
|
// will be able to change the value of `cache_index_and_filter_blocks`.
|
||||||
|
//
|
||||||
|
// Indicating if we'd put index/filter blocks to the block cache.
|
||||||
|
// If not specified, each "table reader" object will pre-load index/filter
|
||||||
|
// block during table initialization.
|
||||||
|
bool cache_index_and_filter_blocks = false;
|
||||||
|
};
|
||||||
|
|
||||||
|
} // namespace rocksdb
|
@ -26,6 +26,7 @@
|
|||||||
#include "util/coding.h"
|
#include "util/coding.h"
|
||||||
#include "util/perf_context_imp.h"
|
#include "util/perf_context_imp.h"
|
||||||
#include "util/stop_watch.h"
|
#include "util/stop_watch.h"
|
||||||
|
#include "table/block_based_table_options.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
@ -45,9 +46,9 @@ struct BlockBasedTable::Rep {
|
|||||||
Status status;
|
Status status;
|
||||||
unique_ptr<RandomAccessFile> file;
|
unique_ptr<RandomAccessFile> file;
|
||||||
char cache_key_prefix[kMaxCacheKeyPrefixSize];
|
char cache_key_prefix[kMaxCacheKeyPrefixSize];
|
||||||
size_t cache_key_prefix_size;
|
size_t cache_key_prefix_size = 0;
|
||||||
char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize];
|
char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize];
|
||||||
size_t compressed_cache_key_prefix_size;
|
size_t compressed_cache_key_prefix_size = 0;
|
||||||
|
|
||||||
// Handle to metaindex_block: saved from footer
|
// Handle to metaindex_block: saved from footer
|
||||||
BlockHandle metaindex_handle;
|
BlockHandle metaindex_handle;
|
||||||
@ -220,20 +221,21 @@ Cache::Handle* GetFromBlockCache(
|
|||||||
|
|
||||||
} // end of anonymous namespace
|
} // end of anonymous namespace
|
||||||
|
|
||||||
Status BlockBasedTable::Open(const Options& options,
|
Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
|
||||||
const EnvOptions& soptions,
|
const BlockBasedTableOptions& table_options,
|
||||||
unique_ptr<RandomAccessFile> && file,
|
unique_ptr<RandomAccessFile>&& file,
|
||||||
uint64_t size,
|
uint64_t file_size,
|
||||||
unique_ptr<TableReader>* table_reader) {
|
unique_ptr<TableReader>* table_reader) {
|
||||||
table_reader->reset();
|
table_reader->reset();
|
||||||
if (size < Footer::kEncodedLength) {
|
|
||||||
|
if (file_size < Footer::kEncodedLength) {
|
||||||
return Status::InvalidArgument("file is too short to be an sstable");
|
return Status::InvalidArgument("file is too short to be an sstable");
|
||||||
}
|
}
|
||||||
|
|
||||||
char footer_space[Footer::kEncodedLength];
|
char footer_space[Footer::kEncodedLength];
|
||||||
Slice footer_input;
|
Slice footer_input;
|
||||||
Status s = file->Read(size - Footer::kEncodedLength, Footer::kEncodedLength,
|
Status s = file->Read(file_size - Footer::kEncodedLength,
|
||||||
&footer_input, footer_space);
|
Footer::kEncodedLength, &footer_input, footer_space);
|
||||||
if (!s.ok()) return s;
|
if (!s.ok()) return s;
|
||||||
|
|
||||||
// Check that we actually read the whole footer from the file. It may be
|
// Check that we actually read the whole footer from the file. It may be
|
||||||
@ -277,11 +279,21 @@ Status BlockBasedTable::Open(const Options& options,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize index/filter blocks. If block cache is not specified,
|
// Will use block cache for index/filter blocks access?
|
||||||
// these blocks will be kept in member variables in Rep, which will
|
if (options.block_cache && table_options.cache_index_and_filter_blocks) {
|
||||||
// reside in the memory as long as this table object is alive; otherwise
|
// Call IndexBlockReader() to implicitly add index to the block_cache
|
||||||
// they will be added to block cache.
|
unique_ptr<Iterator> iter(new_table->IndexBlockReader(ReadOptions()));
|
||||||
if (!options.block_cache) {
|
s = iter->status();
|
||||||
|
|
||||||
|
if (s.ok()) {
|
||||||
|
// Call GetFilter() to implicitly add filter to the block_cache
|
||||||
|
auto filter_entry = new_table->GetFilter();
|
||||||
|
filter_entry.Release(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.
|
||||||
Block* index_block = nullptr;
|
Block* index_block = nullptr;
|
||||||
// TODO: we never really verify check sum for index block
|
// TODO: we never really verify check sum for index block
|
||||||
s = ReadBlockFromFile(
|
s = ReadBlockFromFile(
|
||||||
@ -309,18 +321,7 @@ Status BlockBasedTable::Open(const Options& options,
|
|||||||
} else {
|
} else {
|
||||||
delete index_block;
|
delete index_block;
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
// Call IndexBlockReader() to implicitly add index to the block_cache
|
|
||||||
unique_ptr<Iterator> iter(
|
|
||||||
new_table->IndexBlockReader(ReadOptions())
|
|
||||||
);
|
|
||||||
s = iter->status();
|
|
||||||
|
|
||||||
if (s.ok()) {
|
|
||||||
// Call GetFilter() to implicitly add filter to the block_cache
|
|
||||||
auto filter_entry = new_table->GetFilter();
|
|
||||||
filter_entry.Release(options.block_cache.get());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (s.ok()) {
|
if (s.ok()) {
|
||||||
@ -836,7 +837,6 @@ BlockBasedTable::GetFilter(bool no_io) const {
|
|||||||
// Get the iterator from the index block.
|
// Get the iterator from the index block.
|
||||||
Iterator* BlockBasedTable::IndexBlockReader(const ReadOptions& options) const {
|
Iterator* BlockBasedTable::IndexBlockReader(const ReadOptions& options) const {
|
||||||
if (rep_->index_block) {
|
if (rep_->index_block) {
|
||||||
assert (!rep_->options.block_cache);
|
|
||||||
return rep_->index_block->NewIterator(rep_->options.comparator);
|
return rep_->index_block->NewIterator(rep_->options.comparator);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -29,6 +29,7 @@ struct ReadOptions;
|
|||||||
class TableCache;
|
class TableCache;
|
||||||
class TableReader;
|
class TableReader;
|
||||||
class FilterBlockReader;
|
class FilterBlockReader;
|
||||||
|
struct BlockBasedTableOptions;
|
||||||
|
|
||||||
using std::unique_ptr;
|
using std::unique_ptr;
|
||||||
|
|
||||||
@ -50,10 +51,9 @@ class BlockBasedTable : public TableReader {
|
|||||||
// to nullptr and returns a non-ok status.
|
// to nullptr and returns a non-ok status.
|
||||||
//
|
//
|
||||||
// *file must remain live while this Table is in use.
|
// *file must remain live while this Table is in use.
|
||||||
static Status Open(const Options& options,
|
static Status Open(const Options& db_options, const EnvOptions& env_options,
|
||||||
const EnvOptions& soptions,
|
const BlockBasedTableOptions& table_options,
|
||||||
unique_ptr<RandomAccessFile>&& file,
|
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
|
||||||
uint64_t file_size,
|
|
||||||
unique_ptr<TableReader>* table_reader);
|
unique_ptr<TableReader>* table_reader);
|
||||||
|
|
||||||
bool PrefixMayMatch(const Slice& internal_prefix) override;
|
bool PrefixMayMatch(const Slice& internal_prefix) override;
|
||||||
|
@ -243,13 +243,12 @@ class BlockConstructor: public Constructor {
|
|||||||
|
|
||||||
class BlockBasedTableConstructor: public Constructor {
|
class BlockBasedTableConstructor: public Constructor {
|
||||||
public:
|
public:
|
||||||
explicit BlockBasedTableConstructor(
|
explicit BlockBasedTableConstructor(const Comparator* cmp)
|
||||||
const Comparator* cmp)
|
: Constructor(cmp) {}
|
||||||
: Constructor(cmp) {
|
|
||||||
}
|
|
||||||
~BlockBasedTableConstructor() {
|
~BlockBasedTableConstructor() {
|
||||||
Reset();
|
Reset();
|
||||||
}
|
}
|
||||||
|
|
||||||
virtual Status FinishImpl(const Options& options, const KVMap& data) {
|
virtual Status FinishImpl(const Options& options, const KVMap& data) {
|
||||||
Reset();
|
Reset();
|
||||||
sink_.reset(new StringSink());
|
sink_.reset(new StringSink());
|
||||||
@ -277,7 +276,6 @@ class BlockBasedTableConstructor: public Constructor {
|
|||||||
// Open the table
|
// Open the table
|
||||||
uniq_id_ = cur_uniq_id_++;
|
uniq_id_ = cur_uniq_id_++;
|
||||||
source_.reset(new StringSource(sink_->contents(), uniq_id_));
|
source_.reset(new StringSource(sink_->contents(), uniq_id_));
|
||||||
unique_ptr<TableFactory> table_factory;
|
|
||||||
return options.table_factory->GetTableReader(options, soptions,
|
return options.table_factory->GetTableReader(options, soptions,
|
||||||
std::move(source_),
|
std::move(source_),
|
||||||
sink_->contents().size(),
|
sink_->contents().size(),
|
||||||
@ -979,6 +977,11 @@ TEST(TableTest, BlockCacheTest) {
|
|||||||
options.create_if_missing = true;
|
options.create_if_missing = true;
|
||||||
options.statistics = CreateDBStatistics();
|
options.statistics = CreateDBStatistics();
|
||||||
options.block_cache = NewLRUCache(1024);
|
options.block_cache = NewLRUCache(1024);
|
||||||
|
|
||||||
|
// Enable the cache for index/filter blocks
|
||||||
|
BlockBasedTableOptions table_options;
|
||||||
|
table_options.cache_index_and_filter_blocks = true;
|
||||||
|
options.table_factory.reset(new BlockBasedTableFactory(table_options));
|
||||||
std::vector<std::string> keys;
|
std::vector<std::string> keys;
|
||||||
KVMap kvmap;
|
KVMap kvmap;
|
||||||
|
|
||||||
@ -1292,7 +1295,6 @@ TEST(MemTableTest, Simple) {
|
|||||||
delete memtable->Unref();
|
delete memtable->Unref();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
} // namespace rocksdb
|
} // namespace rocksdb
|
||||||
|
|
||||||
int main(int argc, char** argv) {
|
int main(int argc, char** argv) {
|
||||||
|
Loading…
Reference in New Issue
Block a user