Merge from master
This commit is contained in:
commit
136b8e0cad
@ -360,6 +360,7 @@ set(TESTS
|
||||
db/perf_context_test.cc
|
||||
db/plain_table_db_test.cc
|
||||
db/prefix_test.cc
|
||||
db/repair_test.cc
|
||||
db/skiplist_test.cc
|
||||
db/table_properties_collector_test.cc
|
||||
db/version_builder_test.cc
|
||||
|
3
Makefile
3
Makefile
@ -1047,6 +1047,9 @@ transaction_test: utilities/transactions/transaction_test.o $(LIBOBJECTS) $(TEST
|
||||
sst_dump: tools/sst_dump.o $(LIBOBJECTS)
|
||||
$(AM_LINK)
|
||||
|
||||
repair_test: db/repair_test.o db/db_test_util.o $(LIBOBJECTS) $(TESTHARNESS)
|
||||
$(AM_LINK)
|
||||
|
||||
ldb_cmd_test: tools/ldb_cmd_test.o $(LIBOBJECTS) $(TESTHARNESS)
|
||||
$(AM_LINK)
|
||||
|
||||
|
@ -63,7 +63,7 @@ Status BuildTable(
|
||||
const CompressionType compression,
|
||||
const CompressionOptions& compression_opts, bool paranoid_file_checks,
|
||||
InternalStats* internal_stats, const Env::IOPriority io_priority,
|
||||
TableProperties* table_properties, int level) {
|
||||
TableProperties* table_properties) {
|
||||
// Reports the IOStats for flush for every following bytes.
|
||||
const size_t kReportFlushIOStatsEvery = 1048576;
|
||||
Status s;
|
||||
@ -149,8 +149,7 @@ Status BuildTable(
|
||||
ReadOptions(), env_options, internal_comparator, meta->fd, nullptr,
|
||||
(internal_stats == nullptr) ? nullptr
|
||||
: internal_stats->GetFileReadHist(0),
|
||||
false /* for_compaction */, nullptr /* arena */,
|
||||
false /* skip_filter */, level));
|
||||
false));
|
||||
s = it->status();
|
||||
if (s.ok() && paranoid_file_checks) {
|
||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||
|
@ -61,6 +61,6 @@ extern Status BuildTable(
|
||||
const CompressionOptions& compression_opts, bool paranoid_file_checks,
|
||||
InternalStats* internal_stats,
|
||||
const Env::IOPriority io_priority = Env::IO_HIGH,
|
||||
TableProperties* table_properties = nullptr, int level = -1);
|
||||
TableProperties* table_properties = nullptr);
|
||||
|
||||
} // namespace rocksdb
|
||||
|
5
db/c.cc
5
db/c.cc
@ -1288,11 +1288,6 @@ void rocksdb_block_based_options_set_cache_index_and_filter_blocks(
|
||||
options->rep.cache_index_and_filter_blocks = v;
|
||||
}
|
||||
|
||||
void rocksdb_block_based_options_set_pin_l0_filter_and_index_blocks_in_cache(
|
||||
rocksdb_block_based_table_options_t* options, unsigned char v) {
|
||||
options->rep.pin_l0_filter_and_index_blocks_in_cache = v;
|
||||
}
|
||||
|
||||
void rocksdb_block_based_options_set_skip_table_builder_flush(
|
||||
rocksdb_block_based_table_options_t* options, unsigned char v) {
|
||||
options->rep.skip_table_builder_flush = v;
|
||||
|
@ -465,8 +465,6 @@ class ColumnFamilySet {
|
||||
// Don't call while iterating over ColumnFamilySet
|
||||
void FreeDeadColumnFamilies();
|
||||
|
||||
Cache* get_table_cache() { return table_cache_; }
|
||||
|
||||
private:
|
||||
friend class ColumnFamilyData;
|
||||
// helper function that gets called from cfd destructor
|
||||
|
@ -424,97 +424,6 @@ TEST_F(DBTest, IndexAndFilterBlocksOfNewTableAddedToCache) {
|
||||
TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
}
|
||||
|
||||
TEST_F(DBTest, IndexAndFilterBlocksOfNewTableAddedToCacheWithPinning) {
|
||||
Options options = CurrentOptions();
|
||||
options.create_if_missing = true;
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
BlockBasedTableOptions table_options;
|
||||
table_options.cache_index_and_filter_blocks = true;
|
||||
table_options.pin_l0_filter_and_index_blocks_in_cache = true;
|
||||
table_options.filter_policy.reset(NewBloomFilterPolicy(20));
|
||||
options.table_factory.reset(new BlockBasedTableFactory(table_options));
|
||||
CreateAndReopenWithCF({"pikachu"}, options);
|
||||
|
||||
ASSERT_OK(Put(1, "key", "val"));
|
||||
// Create a new table.
|
||||
ASSERT_OK(Flush(1));
|
||||
|
||||
// index/filter blocks added to block cache right after table creation.
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT));
|
||||
|
||||
// only index/filter were added
|
||||
ASSERT_EQ(2, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_DATA_MISS));
|
||||
|
||||
std::string value;
|
||||
// Miss and hit count should remain the same, they're all pinned.
|
||||
db_->KeyMayExist(ReadOptions(), handles_[1], "key", &value);
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT));
|
||||
|
||||
// Miss and hit count should remain the same, they're all pinned.
|
||||
value = Get(1, "key");
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT));
|
||||
}
|
||||
|
||||
TEST_F(DBTest, MultiLevelIndexAndFilterBlocksCachedWithPinning) {
|
||||
Options options = CurrentOptions();
|
||||
options.create_if_missing = true;
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
BlockBasedTableOptions table_options;
|
||||
table_options.cache_index_and_filter_blocks = true;
|
||||
table_options.pin_l0_filter_and_index_blocks_in_cache = true;
|
||||
table_options.filter_policy.reset(NewBloomFilterPolicy(20));
|
||||
options.table_factory.reset(new BlockBasedTableFactory(table_options));
|
||||
CreateAndReopenWithCF({"pikachu"}, options);
|
||||
|
||||
Put(1, "a", "begin");
|
||||
Put(1, "z", "end");
|
||||
ASSERT_OK(Flush(1));
|
||||
// move this table to L1
|
||||
dbfull()->TEST_CompactRange(0, nullptr, nullptr, handles_[1]);
|
||||
|
||||
// reset block cache
|
||||
table_options.block_cache = NewLRUCache(64 * 1024);
|
||||
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
|
||||
TryReopenWithColumnFamilies({"default", "pikachu"}, options);
|
||||
// create new table at L0
|
||||
Put(1, "a2", "begin2");
|
||||
Put(1, "z2", "end2");
|
||||
ASSERT_OK(Flush(1));
|
||||
|
||||
// get base cache values
|
||||
uint64_t fm = TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS);
|
||||
uint64_t fh = TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT);
|
||||
uint64_t im = TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS);
|
||||
uint64_t ih = TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT);
|
||||
|
||||
std::string value;
|
||||
// this should be read from L0
|
||||
// so cache values don't change
|
||||
value = Get(1, "a2");
|
||||
ASSERT_EQ(fm, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(fh, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(im, TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(ih, TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT));
|
||||
|
||||
// this should be read from L1
|
||||
// the file is opened, prefetching results in a cache filter miss
|
||||
// the block is loaded and added to the cache,
|
||||
// then the get results in a cache hit for L1
|
||||
value = Get(1, "a");
|
||||
ASSERT_EQ(fm + 1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(fh + 1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
}
|
||||
|
||||
TEST_F(DBTest, ParanoidFileChecks) {
|
||||
Options options = CurrentOptions();
|
||||
options.create_if_missing = true;
|
||||
|
@ -652,6 +652,29 @@ TEST_F(DBTest2, WalFilterTestWithColumnFamilies) {
|
||||
ASSERT_TRUE(index == keys_cf.size());
|
||||
}
|
||||
#endif // ROCKSDB_LITE
|
||||
|
||||
TEST_F(DBTest2, DISABLED_FirstSnapshotTest) {
|
||||
Options options;
|
||||
options.write_buffer_size = 100000; // Small write buffer
|
||||
options = CurrentOptions(options);
|
||||
CreateAndReopenWithCF({"pikachu"}, options);
|
||||
|
||||
// This snapshot will have sequence number 0. When compaction encounters
|
||||
// this snapshot, CompactionIterator::findEarliestVisibleSnapshot() will
|
||||
// assert as it expects non-zero snapshots.
|
||||
//
|
||||
// One fix would be to simply remove this assert. However, a better fix
|
||||
// might
|
||||
// be to always have db sequence numbers start from 1 so that no code is
|
||||
// ever
|
||||
// confused by 0.
|
||||
const Snapshot* s1 = db_->GetSnapshot();
|
||||
|
||||
Put(1, "k1", std::string(100000, 'x')); // Fill memtable
|
||||
Put(1, "k2", std::string(100000, 'y')); // Trigger flush
|
||||
|
||||
db_->ReleaseSnapshot(s1);
|
||||
}
|
||||
} // namespace rocksdb
|
||||
|
||||
int main(int argc, char** argv) {
|
||||
|
@ -234,14 +234,14 @@ Status FlushJob::WriteLevel0Table(const autovector<MemTable*>& mems,
|
||||
|
||||
TEST_SYNC_POINT_CALLBACK("FlushJob::WriteLevel0Table:output_compression",
|
||||
&output_compression_);
|
||||
s = BuildTable(
|
||||
dbname_, db_options_.env, *cfd_->ioptions(), env_options_,
|
||||
cfd_->table_cache(), iter.get(), meta, cfd_->internal_comparator(),
|
||||
cfd_->int_tbl_prop_collector_factories(), cfd_->GetID(),
|
||||
existing_snapshots_, earliest_write_conflict_snapshot_,
|
||||
output_compression_, cfd_->ioptions()->compression_opts,
|
||||
mutable_cf_options_.paranoid_file_checks, cfd_->internal_stats(),
|
||||
Env::IO_HIGH, &table_properties_, 0 /* level */);
|
||||
s = BuildTable(dbname_, db_options_.env, *cfd_->ioptions(), env_options_,
|
||||
cfd_->table_cache(), iter.get(), meta,
|
||||
cfd_->internal_comparator(),
|
||||
cfd_->int_tbl_prop_collector_factories(), cfd_->GetID(),
|
||||
existing_snapshots_, earliest_write_conflict_snapshot_,
|
||||
output_compression_, cfd_->ioptions()->compression_opts,
|
||||
mutable_cf_options_.paranoid_file_checks,
|
||||
cfd_->internal_stats(), Env::IO_HIGH, &table_properties_);
|
||||
info.table_properties = table_properties_;
|
||||
LogFlush(db_options_.info_log);
|
||||
}
|
||||
|
177
db/repair_test.cc
Normal file
177
db/repair_test.cc
Normal file
@ -0,0 +1,177 @@
|
||||
// Copyright (c) 2016-present, 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.
|
||||
|
||||
#include <algorithm>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "db/db_impl.h"
|
||||
#include "db/db_test_util.h"
|
||||
#include "rocksdb/db.h"
|
||||
#include "rocksdb/transaction_log.h"
|
||||
#include "util/file_util.h"
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
class RepairTest : public DBTestBase {
|
||||
public:
|
||||
RepairTest() : DBTestBase("/repair_test") {}
|
||||
|
||||
std::string GetFirstSstPath() {
|
||||
uint64_t manifest_size;
|
||||
std::vector<std::string> files;
|
||||
db_->GetLiveFiles(files, &manifest_size);
|
||||
auto sst_iter =
|
||||
std::find_if(files.begin(), files.end(), [](const std::string& file) {
|
||||
uint64_t number;
|
||||
FileType type;
|
||||
bool ok = ParseFileName(file, &number, &type);
|
||||
return ok && type == kTableFile;
|
||||
});
|
||||
return sst_iter == files.end() ? "" : dbname_ + *sst_iter;
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(RepairTest, LostManifest) {
|
||||
// Add a couple SST files, delete the manifest, and verify RepairDB() saves
|
||||
// the day.
|
||||
Put("key", "val");
|
||||
Flush();
|
||||
Put("key2", "val2");
|
||||
Flush();
|
||||
// Need to get path before Close() deletes db_, but delete it after Close() to
|
||||
// ensure Close() didn't change the manifest.
|
||||
std::string manifest_path =
|
||||
DescriptorFileName(dbname_, dbfull()->TEST_Current_Manifest_FileNo());
|
||||
|
||||
Close();
|
||||
ASSERT_OK(env_->FileExists(manifest_path));
|
||||
ASSERT_OK(env_->DeleteFile(manifest_path));
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
ASSERT_EQ(Get("key"), "val");
|
||||
ASSERT_EQ(Get("key2"), "val2");
|
||||
}
|
||||
|
||||
TEST_F(RepairTest, CorruptManifest) {
|
||||
// Manifest is in an invalid format. Expect a full recovery.
|
||||
Put("key", "val");
|
||||
Flush();
|
||||
Put("key2", "val2");
|
||||
Flush();
|
||||
// Need to get path before Close() deletes db_, but overwrite it after Close()
|
||||
// to ensure Close() didn't change the manifest.
|
||||
std::string manifest_path =
|
||||
DescriptorFileName(dbname_, dbfull()->TEST_Current_Manifest_FileNo());
|
||||
|
||||
Close();
|
||||
ASSERT_OK(env_->FileExists(manifest_path));
|
||||
CreateFile(env_, manifest_path, "blah");
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
ASSERT_EQ(Get("key"), "val");
|
||||
ASSERT_EQ(Get("key2"), "val2");
|
||||
}
|
||||
|
||||
TEST_F(RepairTest, IncompleteManifest) {
|
||||
// In this case, the manifest is valid but does not reference all of the SST
|
||||
// files. Expect a full recovery.
|
||||
Put("key", "val");
|
||||
Flush();
|
||||
std::string orig_manifest_path =
|
||||
DescriptorFileName(dbname_, dbfull()->TEST_Current_Manifest_FileNo());
|
||||
CopyFile(orig_manifest_path, orig_manifest_path + ".tmp");
|
||||
Put("key2", "val2");
|
||||
Flush();
|
||||
// Need to get path before Close() deletes db_, but overwrite it after Close()
|
||||
// to ensure Close() didn't change the manifest.
|
||||
std::string new_manifest_path =
|
||||
DescriptorFileName(dbname_, dbfull()->TEST_Current_Manifest_FileNo());
|
||||
|
||||
Close();
|
||||
ASSERT_OK(env_->FileExists(new_manifest_path));
|
||||
// Replace the manifest with one that is only aware of the first SST file.
|
||||
CopyFile(orig_manifest_path + ".tmp", new_manifest_path);
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
ASSERT_EQ(Get("key"), "val");
|
||||
ASSERT_EQ(Get("key2"), "val2");
|
||||
}
|
||||
|
||||
TEST_F(RepairTest, LostSst) {
|
||||
// Delete one of the SST files but preserve the manifest that refers to it,
|
||||
// then verify the DB is still usable for the intact SST.
|
||||
Put("key", "val");
|
||||
Flush();
|
||||
Put("key2", "val2");
|
||||
Flush();
|
||||
auto sst_path = GetFirstSstPath();
|
||||
ASSERT_FALSE(sst_path.empty());
|
||||
ASSERT_OK(env_->DeleteFile(sst_path));
|
||||
|
||||
Close();
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
// Exactly one of the key-value pairs should be in the DB now.
|
||||
ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2"));
|
||||
}
|
||||
|
||||
TEST_F(RepairTest, CorruptSst) {
|
||||
// Corrupt one of the SST files but preserve the manifest that refers to it,
|
||||
// then verify the DB is still usable for the intact SST.
|
||||
Put("key", "val");
|
||||
Flush();
|
||||
Put("key2", "val2");
|
||||
Flush();
|
||||
auto sst_path = GetFirstSstPath();
|
||||
ASSERT_FALSE(sst_path.empty());
|
||||
CreateFile(env_, sst_path, "blah");
|
||||
|
||||
Close();
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
// Exactly one of the key-value pairs should be in the DB now.
|
||||
ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2"));
|
||||
}
|
||||
|
||||
TEST_F(RepairTest, UnflushedSst) {
|
||||
// This test case invokes repair while some data is unflushed, then verifies
|
||||
// that data is in the db.
|
||||
Put("key", "val");
|
||||
VectorLogPtr wal_files;
|
||||
ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files));
|
||||
ASSERT_EQ(wal_files.size(), 1);
|
||||
uint64_t total_ssts_size;
|
||||
GetAllSSTFiles(&total_ssts_size);
|
||||
ASSERT_EQ(total_ssts_size, 0);
|
||||
// Need to get path before Close() deletes db_, but delete it after Close() to
|
||||
// ensure Close() didn't change the manifest.
|
||||
std::string manifest_path =
|
||||
DescriptorFileName(dbname_, dbfull()->TEST_Current_Manifest_FileNo());
|
||||
|
||||
Close();
|
||||
ASSERT_OK(env_->FileExists(manifest_path));
|
||||
ASSERT_OK(env_->DeleteFile(manifest_path));
|
||||
RepairDB(dbname_, CurrentOptions());
|
||||
Reopen(CurrentOptions());
|
||||
|
||||
ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files));
|
||||
ASSERT_EQ(wal_files.size(), 0);
|
||||
GetAllSSTFiles(&total_ssts_size);
|
||||
ASSERT_GT(total_ssts_size, 0);
|
||||
ASSERT_EQ(Get("key"), "val");
|
||||
}
|
||||
|
||||
} // namespace rocksdb
|
||||
|
||||
int main(int argc, char** argv) {
|
||||
::testing::InitGoogleTest(&argc, argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
@ -88,7 +88,7 @@ Status TableCache::GetTableReader(
|
||||
const EnvOptions& env_options,
|
||||
const InternalKeyComparator& internal_comparator, const FileDescriptor& fd,
|
||||
bool sequential_mode, bool record_read_stats, HistogramImpl* file_read_hist,
|
||||
unique_ptr<TableReader>* table_reader, bool skip_filters, int level) {
|
||||
unique_ptr<TableReader>* table_reader, bool skip_filters) {
|
||||
std::string fname =
|
||||
TableFileName(ioptions_.db_paths, fd.GetNumber(), fd.GetPathId());
|
||||
unique_ptr<RandomAccessFile> file;
|
||||
@ -109,7 +109,7 @@ Status TableCache::GetTableReader(
|
||||
file_read_hist));
|
||||
s = ioptions_.table_factory->NewTableReader(
|
||||
TableReaderOptions(ioptions_, env_options, internal_comparator,
|
||||
skip_filters, level),
|
||||
skip_filters),
|
||||
std::move(file_reader), fd.GetFileSize(), table_reader);
|
||||
TEST_SYNC_POINT("TableCache::GetTableReader:0");
|
||||
}
|
||||
@ -120,8 +120,7 @@ Status TableCache::FindTable(const EnvOptions& env_options,
|
||||
const InternalKeyComparator& internal_comparator,
|
||||
const FileDescriptor& fd, Cache::Handle** handle,
|
||||
const bool no_io, bool record_read_stats,
|
||||
HistogramImpl* file_read_hist, bool skip_filters,
|
||||
int level) {
|
||||
HistogramImpl* file_read_hist, bool skip_filters) {
|
||||
PERF_TIMER_GUARD(find_table_nanos);
|
||||
Status s;
|
||||
uint64_t number = fd.GetNumber();
|
||||
@ -137,7 +136,7 @@ Status TableCache::FindTable(const EnvOptions& env_options,
|
||||
unique_ptr<TableReader> table_reader;
|
||||
s = GetTableReader(env_options, internal_comparator, fd,
|
||||
false /* sequential mode */, record_read_stats,
|
||||
file_read_hist, &table_reader, skip_filters, level);
|
||||
file_read_hist, &table_reader, skip_filters);
|
||||
if (!s.ok()) {
|
||||
assert(table_reader == nullptr);
|
||||
RecordTick(ioptions_.statistics, NO_FILE_ERRORS);
|
||||
@ -159,7 +158,7 @@ InternalIterator* TableCache::NewIterator(
|
||||
const ReadOptions& options, const EnvOptions& env_options,
|
||||
const InternalKeyComparator& icomparator, const FileDescriptor& fd,
|
||||
TableReader** table_reader_ptr, HistogramImpl* file_read_hist,
|
||||
bool for_compaction, Arena* arena, bool skip_filters, int level) {
|
||||
bool for_compaction, Arena* arena, bool skip_filters) {
|
||||
PERF_TIMER_GUARD(new_table_iterator_nanos);
|
||||
|
||||
if (table_reader_ptr != nullptr) {
|
||||
@ -174,8 +173,7 @@ InternalIterator* TableCache::NewIterator(
|
||||
unique_ptr<TableReader> table_reader_unique_ptr;
|
||||
Status s = GetTableReader(
|
||||
env_options, icomparator, fd, /* sequential mode */ true,
|
||||
/* record stats */ false, nullptr, &table_reader_unique_ptr,
|
||||
false /* skip_filters */, level);
|
||||
/* record stats */ false, nullptr, &table_reader_unique_ptr);
|
||||
if (!s.ok()) {
|
||||
return NewErrorInternalIterator(s, arena);
|
||||
}
|
||||
@ -186,7 +184,7 @@ InternalIterator* TableCache::NewIterator(
|
||||
Status s = FindTable(env_options, icomparator, fd, &handle,
|
||||
options.read_tier == kBlockCacheTier /* no_io */,
|
||||
!for_compaction /* record read_stats */,
|
||||
file_read_hist, skip_filters, level);
|
||||
file_read_hist, skip_filters);
|
||||
if (!s.ok()) {
|
||||
return NewErrorInternalIterator(s, arena);
|
||||
}
|
||||
@ -218,7 +216,7 @@ Status TableCache::Get(const ReadOptions& options,
|
||||
const InternalKeyComparator& internal_comparator,
|
||||
const FileDescriptor& fd, const Slice& k,
|
||||
GetContext* get_context, HistogramImpl* file_read_hist,
|
||||
bool skip_filters, int level) {
|
||||
bool skip_filters) {
|
||||
TableReader* t = fd.table_reader;
|
||||
Status s;
|
||||
Cache::Handle* handle = nullptr;
|
||||
@ -267,8 +265,7 @@ Status TableCache::Get(const ReadOptions& options,
|
||||
if (!t) {
|
||||
s = FindTable(env_options_, internal_comparator, fd, &handle,
|
||||
options.read_tier == kBlockCacheTier /* no_io */,
|
||||
true /* record_read_stats */, file_read_hist, skip_filters,
|
||||
level);
|
||||
true /* record_read_stats */, file_read_hist, skip_filters);
|
||||
if (s.ok()) {
|
||||
t = GetTableReaderFromHandle(handle);
|
||||
}
|
||||
|
@ -45,37 +45,34 @@ class TableCache {
|
||||
// the cache and should not be deleted, and is valid for as long as the
|
||||
// returned iterator is live.
|
||||
// @param skip_filters Disables loading/accessing the filter block
|
||||
// @param level The level this table is at, -1 for "not set / don't know"
|
||||
InternalIterator* NewIterator(
|
||||
const ReadOptions& options, const EnvOptions& toptions,
|
||||
const InternalKeyComparator& internal_comparator,
|
||||
const FileDescriptor& file_fd, TableReader** table_reader_ptr = nullptr,
|
||||
HistogramImpl* file_read_hist = nullptr, bool for_compaction = false,
|
||||
Arena* arena = nullptr, bool skip_filters = false, int level = -1);
|
||||
Arena* arena = nullptr, bool skip_filters = false);
|
||||
|
||||
// If a seek to internal key "k" in specified file finds an entry,
|
||||
// call (*handle_result)(arg, found_key, found_value) repeatedly until
|
||||
// it returns false.
|
||||
// @param skip_filters Disables loading/accessing the filter block
|
||||
// @param level The level this table is at, -1 for "not set / don't know"
|
||||
Status Get(const ReadOptions& options,
|
||||
const InternalKeyComparator& internal_comparator,
|
||||
const FileDescriptor& file_fd, const Slice& k,
|
||||
GetContext* get_context, HistogramImpl* file_read_hist = nullptr,
|
||||
bool skip_filters = false, int level = -1);
|
||||
bool skip_filters = false);
|
||||
|
||||
// Evict any entry for the specified file number
|
||||
static void Evict(Cache* cache, uint64_t file_number);
|
||||
|
||||
// Find table reader
|
||||
// @param skip_filters Disables loading/accessing the filter block
|
||||
// @param level == -1 means not specified
|
||||
Status FindTable(const EnvOptions& toptions,
|
||||
const InternalKeyComparator& internal_comparator,
|
||||
const FileDescriptor& file_fd, Cache::Handle**,
|
||||
const bool no_io = false, bool record_read_stats = true,
|
||||
HistogramImpl* file_read_hist = nullptr,
|
||||
bool skip_filters = false, int level = -1);
|
||||
bool skip_filters = false);
|
||||
|
||||
// Get TableReader from a cache handle.
|
||||
TableReader* GetTableReaderFromHandle(Cache::Handle* handle);
|
||||
@ -109,7 +106,7 @@ class TableCache {
|
||||
const FileDescriptor& fd, bool sequential_mode,
|
||||
bool record_read_stats, HistogramImpl* file_read_hist,
|
||||
unique_ptr<TableReader>* table_reader,
|
||||
bool skip_filters = false, int level = -1);
|
||||
bool skip_filters = false);
|
||||
|
||||
const ImmutableCFOptions& ioptions_;
|
||||
const EnvOptions& env_options_;
|
||||
|
@ -91,7 +91,6 @@ class FilePicker {
|
||||
const InternalKeyComparator* internal_comparator)
|
||||
: num_levels_(num_levels),
|
||||
curr_level_(static_cast<unsigned int>(-1)),
|
||||
returned_file_level_(static_cast<unsigned int>(-1)),
|
||||
hit_file_level_(static_cast<unsigned int>(-1)),
|
||||
search_left_bound_(0),
|
||||
search_right_bound_(FileIndexer::kLevelMaxIndex),
|
||||
@ -118,8 +117,6 @@ class FilePicker {
|
||||
}
|
||||
}
|
||||
|
||||
int GetCurrentLevel() { return returned_file_level_; }
|
||||
|
||||
FdWithKeyRange* GetNextFile() {
|
||||
while (!search_ended_) { // Loops over different levels.
|
||||
while (curr_index_in_curr_level_ < curr_file_level_->num_files) {
|
||||
@ -192,7 +189,6 @@ class FilePicker {
|
||||
}
|
||||
prev_file_ = f;
|
||||
#endif
|
||||
returned_file_level_ = curr_level_;
|
||||
if (curr_level_ > 0 && cmp_largest < 0) {
|
||||
// No more files to search in this level.
|
||||
search_ended_ = !PrepareNextLevel();
|
||||
@ -219,7 +215,6 @@ class FilePicker {
|
||||
private:
|
||||
unsigned int num_levels_;
|
||||
unsigned int curr_level_;
|
||||
unsigned int returned_file_level_;
|
||||
unsigned int hit_file_level_;
|
||||
int32_t search_left_bound_;
|
||||
int32_t search_right_bound_;
|
||||
@ -490,7 +485,7 @@ class LevelFileIteratorState : public TwoLevelIteratorState {
|
||||
const EnvOptions& env_options,
|
||||
const InternalKeyComparator& icomparator,
|
||||
HistogramImpl* file_read_hist, bool for_compaction,
|
||||
bool prefix_enabled, bool skip_filters, int level)
|
||||
bool prefix_enabled, bool skip_filters)
|
||||
: TwoLevelIteratorState(prefix_enabled),
|
||||
table_cache_(table_cache),
|
||||
read_options_(read_options),
|
||||
@ -498,8 +493,7 @@ class LevelFileIteratorState : public TwoLevelIteratorState {
|
||||
icomparator_(icomparator),
|
||||
file_read_hist_(file_read_hist),
|
||||
for_compaction_(for_compaction),
|
||||
skip_filters_(skip_filters),
|
||||
level_(level) {}
|
||||
skip_filters_(skip_filters) {}
|
||||
|
||||
InternalIterator* NewSecondaryIterator(const Slice& meta_handle) override {
|
||||
if (meta_handle.size() != sizeof(FileDescriptor)) {
|
||||
@ -511,7 +505,7 @@ class LevelFileIteratorState : public TwoLevelIteratorState {
|
||||
return table_cache_->NewIterator(
|
||||
read_options_, env_options_, icomparator_, *fd,
|
||||
nullptr /* don't need reference to table*/, file_read_hist_,
|
||||
for_compaction_, nullptr /* arena */, skip_filters_, level_);
|
||||
for_compaction_, nullptr /* arena */, skip_filters_);
|
||||
}
|
||||
}
|
||||
|
||||
@ -527,7 +521,6 @@ class LevelFileIteratorState : public TwoLevelIteratorState {
|
||||
HistogramImpl* file_read_hist_;
|
||||
bool for_compaction_;
|
||||
bool skip_filters_;
|
||||
int level_;
|
||||
};
|
||||
|
||||
// A wrapper of version builder which references the current version in
|
||||
@ -795,8 +788,7 @@ void Version::AddIterators(const ReadOptions& read_options,
|
||||
const auto& file = storage_info_.LevelFilesBrief(0).files[i];
|
||||
merge_iter_builder->AddIterator(cfd_->table_cache()->NewIterator(
|
||||
read_options, soptions, cfd_->internal_comparator(), file.fd, nullptr,
|
||||
cfd_->internal_stats()->GetFileReadHist(0), false, arena,
|
||||
false /* skip_filters */, 0 /* level */));
|
||||
cfd_->internal_stats()->GetFileReadHist(0), false, arena));
|
||||
}
|
||||
|
||||
// For levels > 0, we can use a concatenating iterator that sequentially
|
||||
@ -811,7 +803,7 @@ void Version::AddIterators(const ReadOptions& read_options,
|
||||
cfd_->internal_stats()->GetFileReadHist(level),
|
||||
false /* for_compaction */,
|
||||
cfd_->ioptions()->prefix_extractor != nullptr,
|
||||
IsFilterSkipped(level), level);
|
||||
IsFilterSkipped(level));
|
||||
mem = arena->AllocateAligned(sizeof(LevelFileNumIterator));
|
||||
auto* first_level_iter = new (mem) LevelFileNumIterator(
|
||||
cfd_->internal_comparator(), &storage_info_.LevelFilesBrief(level));
|
||||
@ -916,8 +908,7 @@ void Version::Get(const ReadOptions& read_options, const LookupKey& k,
|
||||
read_options, *internal_comparator(), f->fd, ikey, &get_context,
|
||||
cfd_->internal_stats()->GetFileReadHist(fp.GetHitFileLevel()),
|
||||
IsFilterSkipped(static_cast<int>(fp.GetHitFileLevel()),
|
||||
fp.IsHitFileLastInLevel()),
|
||||
fp.GetCurrentLevel());
|
||||
fp.IsHitFileLastInLevel()));
|
||||
// TODO: examine the behavior for corrupted key
|
||||
if (!status->ok()) {
|
||||
return;
|
||||
@ -2063,16 +2054,9 @@ VersionSet::VersionSet(const std::string& dbname, const DBOptions* db_options,
|
||||
env_options_(storage_options),
|
||||
env_options_compactions_(env_options_) {}
|
||||
|
||||
void CloseTables(void* ptr, size_t) {
|
||||
TableReader* table_reader = reinterpret_cast<TableReader*>(ptr);
|
||||
table_reader->Close();
|
||||
}
|
||||
|
||||
VersionSet::~VersionSet() {
|
||||
// we need to delete column_family_set_ because its destructor depends on
|
||||
// VersionSet
|
||||
column_family_set_->get_table_cache()->ApplyToAllCacheEntries(&CloseTables,
|
||||
false);
|
||||
column_family_set_.reset();
|
||||
for (auto file : obsolete_files_) {
|
||||
delete file;
|
||||
@ -3283,8 +3267,7 @@ InternalIterator* VersionSet::MakeInputIterator(Compaction* c) {
|
||||
read_options, env_options_compactions_,
|
||||
cfd->internal_comparator(), flevel->files[i].fd, nullptr,
|
||||
nullptr, /* no per level latency histogram*/
|
||||
true /* for_compaction */, nullptr /* arena */,
|
||||
false /* skip_filters */, (int)which /* level */);
|
||||
true /* for compaction */);
|
||||
}
|
||||
} else {
|
||||
// Create concatenating iterator for the files from this level
|
||||
@ -3294,7 +3277,7 @@ InternalIterator* VersionSet::MakeInputIterator(Compaction* c) {
|
||||
cfd->internal_comparator(),
|
||||
nullptr /* no per level latency histogram */,
|
||||
true /* for_compaction */, false /* prefix enabled */,
|
||||
false /* skip_filters */, (int)which /* level */),
|
||||
false /* skip_filters */),
|
||||
new LevelFileNumIterator(cfd->internal_comparator(),
|
||||
c->input_levels(which)));
|
||||
}
|
||||
|
@ -138,7 +138,6 @@
|
||||
block_size=8192
|
||||
block_restart_interval=16
|
||||
cache_index_and_filter_blocks=false
|
||||
pin_l0_filter_and_index_blocks_in_cache=false
|
||||
index_type=kBinarySearch
|
||||
hash_index_allow_collision=true
|
||||
flush_block_policy_factory=FlushBlockBySizePolicyFactory
|
||||
|
@ -451,9 +451,6 @@ extern ROCKSDB_LIBRARY_API void
|
||||
rocksdb_block_based_options_set_cache_index_and_filter_blocks(
|
||||
rocksdb_block_based_table_options_t*, unsigned char);
|
||||
extern ROCKSDB_LIBRARY_API void
|
||||
rocksdb_block_based_options_set_pin_l0_filter_and_index_blocks_in_cache(
|
||||
rocksdb_block_based_table_options_t*, unsigned char);
|
||||
extern ROCKSDB_LIBRARY_API void
|
||||
rocksdb_block_based_options_set_skip_table_builder_flush(
|
||||
rocksdb_block_based_table_options_t* options, unsigned char);
|
||||
extern ROCKSDB_LIBRARY_API void rocksdb_options_set_block_based_table_factory(
|
||||
|
@ -64,12 +64,6 @@ struct BlockBasedTableOptions {
|
||||
// block during table initialization.
|
||||
bool cache_index_and_filter_blocks = false;
|
||||
|
||||
// if cache_index_and_filter_blocks is true and the below is true, then
|
||||
// filter and index blocks are stored in the cache, but a reference is
|
||||
// held in the "table reader" object so the blocks are pinned and only
|
||||
// evicted from cache when the table reader is freed.
|
||||
bool pin_l0_filter_and_index_blocks_in_cache = false;
|
||||
|
||||
// The index type that will be used for this table.
|
||||
enum IndexType : char {
|
||||
// A space efficient index block that is optimized for
|
||||
|
@ -38,14 +38,13 @@ jlong Java_org_rocksdb_PlainTableConfig_newTableFactoryHandle(
|
||||
/*
|
||||
* Class: org_rocksdb_BlockBasedTableConfig
|
||||
* Method: newTableFactoryHandle
|
||||
* Signature: (ZJIJIIZIZZZJIBBI)J
|
||||
* Signature: (ZJIJIIZIZZJIBBI)J
|
||||
*/
|
||||
jlong Java_org_rocksdb_BlockBasedTableConfig_newTableFactoryHandle(
|
||||
JNIEnv* env, jobject jobj, jboolean no_block_cache, jlong block_cache_size,
|
||||
jint block_cache_num_shardbits, jlong block_size, jint block_size_deviation,
|
||||
jint block_restart_interval, jboolean whole_key_filtering,
|
||||
jlong jfilterPolicy, jboolean cache_index_and_filter_blocks,
|
||||
jboolean pin_l0_filter_and_index_blocks_in_cache,
|
||||
jboolean hash_index_allow_collision, jlong block_cache_compressed_size,
|
||||
jint block_cache_compressd_num_shard_bits, jbyte jchecksum_type,
|
||||
jbyte jindex_type, jint jformat_version) {
|
||||
@ -71,8 +70,6 @@ jlong Java_org_rocksdb_BlockBasedTableConfig_newTableFactoryHandle(
|
||||
options.filter_policy = *pFilterPolicy;
|
||||
}
|
||||
options.cache_index_and_filter_blocks = cache_index_and_filter_blocks;
|
||||
options.pin_l0_filter_and_index_blocks_in_cache =
|
||||
pin_l0_filter_and_index_blocks_in_cache;
|
||||
options.hash_index_allow_collision = hash_index_allow_collision;
|
||||
if (block_cache_compressed_size > 0) {
|
||||
if (block_cache_compressd_num_shard_bits > 0) {
|
||||
|
@ -64,7 +64,7 @@ Status BlockBasedTableFactory::NewTableReader(
|
||||
table_reader_options.ioptions, table_reader_options.env_options,
|
||||
table_options_, table_reader_options.internal_comparator, std::move(file),
|
||||
file_size, table_reader, prefetch_enabled,
|
||||
table_reader_options.skip_filters, table_reader_options.level);
|
||||
table_reader_options.skip_filters);
|
||||
}
|
||||
|
||||
TableBuilder* BlockBasedTableFactory::NewTableBuilder(
|
||||
@ -94,12 +94,6 @@ Status BlockBasedTableFactory::SanitizeOptions(
|
||||
return Status::InvalidArgument("Enable cache_index_and_filter_blocks, "
|
||||
", but block cache is disabled");
|
||||
}
|
||||
if (table_options_.pin_l0_filter_and_index_blocks_in_cache &&
|
||||
table_options_.no_block_cache) {
|
||||
return Status::InvalidArgument(
|
||||
"Enable pin_l0_filter_and_index_blocks_in_cache, "
|
||||
", but block cache is disabled");
|
||||
}
|
||||
if (!BlockBasedTableSupportedVersion(table_options_.format_version)) {
|
||||
return Status::InvalidArgument(
|
||||
"Unsupported BlockBasedTable format_version. Please check "
|
||||
@ -121,10 +115,6 @@ std::string BlockBasedTableFactory::GetPrintableTableOptions() const {
|
||||
snprintf(buffer, kBufferSize, " cache_index_and_filter_blocks: %d\n",
|
||||
table_options_.cache_index_and_filter_blocks);
|
||||
ret.append(buffer);
|
||||
snprintf(buffer, kBufferSize,
|
||||
" pin_l0_filter_and_index_blocks_in_cache: %d\n",
|
||||
table_options_.pin_l0_filter_and_index_blocks_in_cache);
|
||||
ret.append(buffer);
|
||||
snprintf(buffer, kBufferSize, " index_type: %d\n",
|
||||
table_options_.index_type);
|
||||
ret.append(buffer);
|
||||
|
@ -340,28 +340,6 @@ class HashIndexReader : public IndexReader {
|
||||
BlockContents prefixes_contents_;
|
||||
};
|
||||
|
||||
// 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;
|
||||
}
|
||||
}
|
||||
bool IsSet() const { return cache_handle != nullptr; }
|
||||
|
||||
TValue* value = nullptr;
|
||||
// if the entry is from the cache, cache_handle will be populated.
|
||||
Cache::Handle* cache_handle = nullptr;
|
||||
};
|
||||
|
||||
struct BlockBasedTable::Rep {
|
||||
Rep(const ImmutableCFOptions& _ioptions, const EnvOptions& _env_options,
|
||||
@ -416,21 +394,34 @@ struct BlockBasedTable::Rep {
|
||||
// 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;
|
||||
|
||||
// only used in level 0 files:
|
||||
// when pin_l0_filter_and_index_blocks_in_cache is true, we do use the
|
||||
// LRU cache, but we always keep the filter & idndex block's handle checked
|
||||
// out here (=we don't call Release()), plus the parsed out objects
|
||||
// the LRU cache will never push flush them out, hence they're pinned
|
||||
CachableEntry<FilterBlockReader> filter_entry;
|
||||
CachableEntry<IndexReader> index_entry;
|
||||
};
|
||||
|
||||
BlockBasedTable::~BlockBasedTable() {
|
||||
Close();
|
||||
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, uint64_t file_size) {
|
||||
assert(kMaxCacheKeyPrefixSize >= 10);
|
||||
@ -507,7 +498,7 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
|
||||
uint64_t file_size,
|
||||
unique_ptr<TableReader>* table_reader,
|
||||
const bool prefetch_index_and_filter,
|
||||
const bool skip_filters, const int level) {
|
||||
const bool skip_filters) {
|
||||
table_reader->reset();
|
||||
|
||||
Footer footer;
|
||||
@ -603,33 +594,14 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
|
||||
assert(table_options.block_cache != nullptr);
|
||||
// Hack: Call NewIndexIterator() to implicitly add index to the
|
||||
// block_cache
|
||||
|
||||
// if pin_l0_filter_and_index_blocks_in_cache is true and this is
|
||||
// a level0 file, then we will pass in this pointer to rep->index
|
||||
// to NewIndexIterator(), which will save the index block in there
|
||||
// else it's a nullptr and nothing special happens
|
||||
CachableEntry<IndexReader>* index_entry = nullptr;
|
||||
if (rep->table_options.pin_l0_filter_and_index_blocks_in_cache &&
|
||||
level == 0) {
|
||||
index_entry = &rep->index_entry;
|
||||
}
|
||||
unique_ptr<InternalIterator> iter(
|
||||
new_table->NewIndexIterator(ReadOptions(), nullptr, index_entry));
|
||||
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();
|
||||
// if pin_l0_filter_and_index_blocks_in_cache is true, and this is
|
||||
// a level0 file, then save it in rep_->filter_entry; it will be
|
||||
// released in the destructor only, hence it will be pinned in the
|
||||
// cache until this reader is alive
|
||||
if (rep->table_options.pin_l0_filter_and_index_blocks_in_cache &&
|
||||
level == 0) {
|
||||
rep->filter_entry = filter_entry;
|
||||
} else {
|
||||
filter_entry.Release(table_options.block_cache.get());
|
||||
}
|
||||
filter_entry.Release(table_options.block_cache.get());
|
||||
}
|
||||
} else {
|
||||
// If we don't use block cache for index/filter blocks access, we'll
|
||||
@ -914,11 +886,6 @@ BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
|
||||
return {rep_->filter.get(), nullptr /* cache handle */};
|
||||
}
|
||||
|
||||
// we have a pinned filter block
|
||||
if (rep_->filter_entry.IsSet()) {
|
||||
return rep_->filter_entry;
|
||||
}
|
||||
|
||||
PERF_TIMER_GUARD(read_filter_block_nanos);
|
||||
|
||||
Cache* block_cache = rep_->table_options.block_cache.get();
|
||||
@ -968,19 +935,12 @@ BlockBasedTable::CachableEntry<FilterBlockReader> BlockBasedTable::GetFilter(
|
||||
}
|
||||
|
||||
InternalIterator* BlockBasedTable::NewIndexIterator(
|
||||
const ReadOptions& read_options, BlockIter* input_iter,
|
||||
CachableEntry<IndexReader>* index_entry) {
|
||||
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);
|
||||
}
|
||||
// we have a pinned index block
|
||||
if (rep_->index_entry.IsSet()) {
|
||||
return rep_->index_entry.value->NewIterator(input_iter,
|
||||
read_options.total_order_seek);
|
||||
}
|
||||
|
||||
PERF_TIMER_GUARD(read_index_block_nanos);
|
||||
|
||||
bool no_io = read_options.read_tier == kBlockCacheTier;
|
||||
@ -1036,15 +996,7 @@ InternalIterator* BlockBasedTable::NewIndexIterator(
|
||||
assert(cache_handle);
|
||||
auto* iter = index_reader->NewIterator(
|
||||
input_iter, read_options.total_order_seek);
|
||||
|
||||
// the caller would like to take ownership of the index block
|
||||
// don't call RegisterCleanup() in this case, the caller will take care of it
|
||||
if (index_entry != nullptr) {
|
||||
*index_entry = {index_reader, cache_handle};
|
||||
} else {
|
||||
iter->RegisterCleanup(&ReleaseCachedEntry, block_cache, cache_handle);
|
||||
}
|
||||
|
||||
iter->RegisterCleanup(&ReleaseCachedEntry, block_cache, cache_handle);
|
||||
return iter;
|
||||
}
|
||||
|
||||
@ -1272,13 +1224,7 @@ bool BlockBasedTable::PrefixMayMatch(const Slice& internal_key) {
|
||||
RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL);
|
||||
}
|
||||
|
||||
// if rep_->filter_entry is not set, we should call Release(); otherwise
|
||||
// don't call, in this case we have a local copy in rep_->filter_entry,
|
||||
// it's pinned to the cache and will be released in the destructor
|
||||
if (!rep_->filter_entry.IsSet()) {
|
||||
filter_entry.Release(rep_->table_options.block_cache.get());
|
||||
}
|
||||
|
||||
filter_entry.Release(rep_->table_options.block_cache.get());
|
||||
return may_match;
|
||||
}
|
||||
|
||||
@ -1378,12 +1324,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
|
||||
}
|
||||
}
|
||||
|
||||
// if rep_->filter_entry is not set, we should call Release(); otherwise
|
||||
// don't call, in this case we have a local copy in rep_->filter_entry,
|
||||
// it's pinned to the cache and will be released in the destructor
|
||||
if (!rep_->filter_entry.IsSet()) {
|
||||
filter_entry.Release(rep_->table_options.block_cache.get());
|
||||
}
|
||||
filter_entry.Release(rep_->table_options.block_cache.get());
|
||||
return s;
|
||||
}
|
||||
|
||||
@ -1671,11 +1612,6 @@ Status BlockBasedTable::DumpTable(WritableFile* out_file) {
|
||||
return s;
|
||||
}
|
||||
|
||||
void BlockBasedTable::Close() {
|
||||
rep_->filter_entry.Release(rep_->table_options.block_cache.get());
|
||||
rep_->index_entry.Release(rep_->table_options.block_cache.get());
|
||||
}
|
||||
|
||||
Status BlockBasedTable::DumpIndexBlock(WritableFile* out_file) {
|
||||
out_file->Append(
|
||||
"Index Details:\n"
|
||||
|
@ -76,7 +76,7 @@ class BlockBasedTable : public TableReader {
|
||||
unique_ptr<RandomAccessFileReader>&& file,
|
||||
uint64_t file_size, unique_ptr<TableReader>* table_reader,
|
||||
bool prefetch_index_and_filter = true,
|
||||
bool skip_filters = false, int level = -1);
|
||||
bool skip_filters = false);
|
||||
|
||||
bool PrefixMayMatch(const Slice& internal_key);
|
||||
|
||||
@ -119,8 +119,6 @@ class BlockBasedTable : public TableReader {
|
||||
// convert SST file to a human readable form
|
||||
Status DumpTable(WritableFile* out_file) override;
|
||||
|
||||
void Close() override;
|
||||
|
||||
~BlockBasedTable();
|
||||
|
||||
bool TEST_filter_block_preloaded() const;
|
||||
@ -157,9 +155,8 @@ class BlockBasedTable : public TableReader {
|
||||
// 2. index is not present in block cache.
|
||||
// 3. We disallowed any io to be performed, that is, read_options ==
|
||||
// kBlockCacheTier
|
||||
InternalIterator* NewIndexIterator(
|
||||
const ReadOptions& read_options, BlockIter* input_iter = nullptr,
|
||||
CachableEntry<IndexReader>* index_entry = nullptr);
|
||||
InternalIterator* NewIndexIterator(const ReadOptions& read_options,
|
||||
BlockIter* input_iter = nullptr);
|
||||
|
||||
// Read block cache from block caches (if set): block_cache and
|
||||
// block_cache_compressed.
|
||||
|
@ -29,20 +29,17 @@ struct TableReaderOptions {
|
||||
TableReaderOptions(const ImmutableCFOptions& _ioptions,
|
||||
const EnvOptions& _env_options,
|
||||
const InternalKeyComparator& _internal_comparator,
|
||||
bool _skip_filters = false, int _level = -1)
|
||||
bool _skip_filters = false)
|
||||
: ioptions(_ioptions),
|
||||
env_options(_env_options),
|
||||
internal_comparator(_internal_comparator),
|
||||
skip_filters(_skip_filters),
|
||||
level(_level) {}
|
||||
skip_filters(_skip_filters) {}
|
||||
|
||||
const ImmutableCFOptions& ioptions;
|
||||
const EnvOptions& env_options;
|
||||
const InternalKeyComparator& internal_comparator;
|
||||
// This is only used for BlockBasedTable (reader)
|
||||
bool skip_filters;
|
||||
// what level this table/file is on, -1 for "not set, don't know"
|
||||
int level;
|
||||
};
|
||||
|
||||
struct TableBuilderOptions {
|
||||
|
@ -91,8 +91,6 @@ class TableReader {
|
||||
virtual Status DumpTable(WritableFile* out_file) {
|
||||
return Status::NotSupported("DumpTable() not supported");
|
||||
}
|
||||
|
||||
virtual void Close() {}
|
||||
};
|
||||
|
||||
} // namespace rocksdb
|
||||
|
@ -1715,7 +1715,7 @@ TEST_F(BlockBasedTableTest, FilterBlockInBlockCache) {
|
||||
ImmutableCFOptions ioptions3(options);
|
||||
// Generate table without filter policy
|
||||
c3.Finish(options, ioptions3, table_options,
|
||||
GetPlainInternalComparator(options.comparator), &keys, &kvmap);
|
||||
GetPlainInternalComparator(options.comparator), &keys, &kvmap);
|
||||
// Open table with filter policy
|
||||
table_options.filter_policy.reset(NewBloomFilterPolicy(1));
|
||||
options.table_factory.reset(new BlockBasedTableFactory(table_options));
|
||||
|
@ -74,7 +74,6 @@ const_params="
|
||||
--level_compaction_dynamic_level_bytes=true \
|
||||
--bytes_per_sync=$((8 * M)) \
|
||||
--cache_index_and_filter_blocks=0 \
|
||||
--pin_l0_filter_and_index_blocks_in_cache=1 \
|
||||
--benchmark_write_rate_limit=$(( 1024 * 1024 * $mb_written_per_sec )) \
|
||||
\
|
||||
--hard_rate_limit=3 \
|
||||
|
@ -43,8 +43,8 @@ done
|
||||
|
||||
# v2.1 or older doesn't pass the debug build but OK with release build
|
||||
declare -a need_release_tags=("v1.5.7" "v2.1")
|
||||
declare -a tags=("v2.5" "v2.4" "v2.3" "v2.2" "v2.8" "v3.0" "v3.1" "v3.2" "v3.3" "v3.4" "rocksdb-3.5.1" "rocksdb-3.6.2" "rocksdb-3.7" "rocksdb-3.8" "rocksdb-3.9" "v3.10")
|
||||
declare -a forward_compatible_tags=("rocksdb-3.8" "rocksdb-3.9" "v3.10")
|
||||
declare -a tags=("v2.5" "v2.4" "v2.3" "v2.2" "v2.8" "v3.0" "v3.1" "v3.2" "v3.3" "v3.4" "rocksdb-3.5.1" "rocksdb-3.6.2" "rocksdb-3.7" "rocksdb-3.8" "rocksdb-3.9" "v3.10" "v3.11" "v3.12.1" "v3.13.1" "v4.0" "v4.1" "v4.2" "v4.3" "v4.3.1" "v4.4")
|
||||
declare -a forward_compatible_tags=("v3.10" "v3.11" "v3.12.1" "v3.13.1" "v4.0" "v4.1" "v4.2" "v4.3.1" "v4.4")
|
||||
|
||||
generate_db()
|
||||
{
|
||||
|
@ -340,9 +340,6 @@ DEFINE_int64(cache_size, -1, "Number of bytes to use as a cache of uncompressed"
|
||||
DEFINE_bool(cache_index_and_filter_blocks, false,
|
||||
"Cache index/filter blocks in block cache.");
|
||||
|
||||
DEFINE_bool(pin_l0_filter_and_index_blocks_in_cache, false,
|
||||
"Pin index/filter blocks of L0 files in block cache.");
|
||||
|
||||
DEFINE_int32(block_size,
|
||||
static_cast<int32_t>(rocksdb::BlockBasedTableOptions().block_size),
|
||||
"Number of bytes in a block.");
|
||||
@ -2514,8 +2511,6 @@ class Benchmark {
|
||||
}
|
||||
block_based_options.cache_index_and_filter_blocks =
|
||||
FLAGS_cache_index_and_filter_blocks;
|
||||
block_based_options.pin_l0_filter_and_index_blocks_in_cache =
|
||||
FLAGS_pin_l0_filter_and_index_blocks_in_cache;
|
||||
block_based_options.block_cache = cache_;
|
||||
block_based_options.block_cache_compressed = compressed_cache_;
|
||||
block_based_options.block_size = FLAGS_block_size;
|
||||
|
@ -14,10 +14,6 @@
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
namespace {
|
||||
const HistogramBucketMapper bucketMapper;
|
||||
}
|
||||
|
||||
HistogramWindowingImpl::HistogramWindowingImpl() {
|
||||
env_ = Env::Default();
|
||||
window_stats_.reset(new HistogramStat[num_windows_]);
|
||||
@ -36,8 +32,7 @@ HistogramWindowingImpl::HistogramWindowingImpl(
|
||||
Clear();
|
||||
}
|
||||
|
||||
HistogramWindowingImpl::~HistogramWindowingImpl(){
|
||||
window_stats_.release();
|
||||
HistogramWindowingImpl::~HistogramWindowingImpl() {
|
||||
}
|
||||
|
||||
void HistogramWindowingImpl::Clear() {
|
||||
@ -85,7 +80,7 @@ void HistogramWindowingImpl::Merge(const HistogramWindowingImpl& other) {
|
||||
uint64_t cur_window = current_window();
|
||||
uint64_t other_cur_window = other.current_window();
|
||||
// going backwards for alignment
|
||||
for (unsigned int i = 0;
|
||||
for (unsigned int i = 0;
|
||||
i < std::min(num_windows_, other.num_windows_); i++) {
|
||||
uint64_t window_index =
|
||||
(cur_window + num_windows_ - i) % num_windows_;
|
||||
@ -147,7 +142,7 @@ void HistogramWindowingImpl::SwapHistoryBucket() {
|
||||
last_swap_time_.store(env_->NowMicros(), std::memory_order_relaxed);
|
||||
|
||||
uint64_t curr_window = current_window();
|
||||
uint64_t next_window = (curr_window == num_windows_ - 1) ?
|
||||
uint64_t next_window = (curr_window == num_windows_ - 1) ?
|
||||
0 : curr_window + 1;
|
||||
|
||||
// subtract next buckets from totals and swap to next buckets
|
||||
@ -160,7 +155,7 @@ void HistogramWindowingImpl::SwapHistoryBucket() {
|
||||
}
|
||||
|
||||
if (stats_.min() == stats_to_drop.min()) {
|
||||
uint64_t new_min = bucketMapper.LastValue();
|
||||
uint64_t new_min = std::numeric_limits<uint64_t>::max();
|
||||
for (unsigned int i = 0; i < num_windows_; i++) {
|
||||
if (i != next_window) {
|
||||
uint64_t m = window_stats_[i].min();
|
||||
|
@ -491,10 +491,6 @@ static std::unordered_map<std::string,
|
||||
{"cache_index_and_filter_blocks",
|
||||
{offsetof(struct BlockBasedTableOptions, cache_index_and_filter_blocks),
|
||||
OptionType::kBoolean, OptionVerificationType::kNormal}},
|
||||
{"pin_l0_filter_and_index_blocks_in_cache",
|
||||
{offsetof(struct BlockBasedTableOptions,
|
||||
pin_l0_filter_and_index_blocks_in_cache),
|
||||
OptionType::kBoolean, OptionVerificationType::kNormal}},
|
||||
{"index_type",
|
||||
{offsetof(struct BlockBasedTableOptions, index_type),
|
||||
OptionType::kBlockBasedTableIndexType, OptionVerificationType::kNormal}},
|
||||
|
@ -1582,9 +1582,7 @@ TEST_F(OptionsParserTest, BlockBasedTableOptionsAllFieldsSettable) {
|
||||
// Need to update the option string if a new option is added.
|
||||
ASSERT_OK(GetBlockBasedTableOptionsFromString(
|
||||
*bbto,
|
||||
"cache_index_and_filter_blocks=1;"
|
||||
"pin_l0_filter_and_index_blocks_in_cache=1;"
|
||||
"index_type=kHashSearch;"
|
||||
"cache_index_and_filter_blocks=1;index_type=kHashSearch;"
|
||||
"checksum=kxxHash;hash_index_allow_collision=1;no_block_cache=1;"
|
||||
"block_cache=1M;block_cache_compressed=1k;block_size=1024;"
|
||||
"block_size_deviation=8;block_restart_interval=4; "
|
||||
|
@ -193,7 +193,6 @@ const SliceTransform* RandomSliceTransform(Random* rnd, int pre_defined) {
|
||||
BlockBasedTableOptions RandomBlockBasedTableOptions(Random* rnd) {
|
||||
BlockBasedTableOptions opt;
|
||||
opt.cache_index_and_filter_blocks = rnd->Uniform(2);
|
||||
opt.pin_l0_filter_and_index_blocks_in_cache = rnd->Uniform(2);
|
||||
opt.index_type = rnd->Uniform(2) ? BlockBasedTableOptions::kBinarySearch
|
||||
: BlockBasedTableOptions::kHashSearch;
|
||||
opt.hash_index_allow_collision = rnd->Uniform(2);
|
||||
|
Loading…
Reference in New Issue
Block a user