Ignore `total_order_seek` in DB::Get (#9427)

Summary:
Apparently setting total_order_seek=true for DB::Get was
intended to allow accurate read semantics if the current prefix
extractor doesn't match what was used to generate SST files on
disk. But since prefix_extractor was made a mutable option in 5.14.0, we
have been able to detect this case and provide the correct semantics
regardless of the total_order_seek option. Since that time, the option
has only made Get() slower in a reasonably common case: prefix_extractor
unchanged and whole_key_filtering=false.

So this change primarily removes unnecessary effect of
total_order_seek on Get. Also cleans up some related comments.

Also adds a -total_order_seek option to db_bench and canonicalizes
handling of ReadOptions in db_bench so that command line options have
the expected association with library features. (There is potential
for change in regression test behavior, but the old behavior is likely
indefensible, or some other inconsistency would need to be fixed.)

TODO in follow-up work: there should be no reason for Get() to depend on
current prefix extractor at all.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/9427

Test Plan:
Unit tests updated.

Performance (using db_bench update)

Create DB with `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -benchmarks=fillrandom -num=10000000 -disable_wal=1 -write_buffer_size=10000000 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12 -whole_key_filtering=0`

Test with and without `-total_order_seek` on `TEST_TMPDIR=/dev/shm/rocksdb ./db_bench -use_existing_db -readonly -benchmarks=readrandom -num=10000000 -duration=40 -disable_wal=1 -bloom_bits=16 -compaction_style=2 -fifo_compaction_max_table_files_size_mb=10000 -fifo_compaction_allow_compaction=0 -prefix_size=12`

Before this change, total_order_seek=false: 25188 ops/sec
Before this change, total_order_seek=true:   1222 ops/sec (~20x slower)

After this change, total_order_seek=false: 24570 ops/sec
After this change, total_order_seek=true:  25012 ops/sec (indistinguishable)

Reviewed By: siying

Differential Revision: D33753458

Pulled By: pdillinger

fbshipit-source-id: bf892f34907a5e407d9c40bd4d42f0adbcbe0014
main
Peter Dillinger 2 years ago committed by Facebook GitHub Bot
parent c7ce03dce1
commit f6d7ec1d02
  1. 1
      HISTORY.md
  2. 36
      db/db_bloom_filter_test.cc
  3. 3
      include/rocksdb/options.h
  4. 47
      table/block_based/block_based_table_reader.cc
  5. 10
      table/block_based/block_based_table_reader.h
  6. 86
      tools/db_bench_tool.cc

@ -23,6 +23,7 @@
### Behavior Changes
* Disallow the combination of DBOptions.use_direct_io_for_flush_and_compaction == true and DBOptions.writable_file_max_buffer_size == 0. This combination can cause WritableFileWriter::Append() to loop forever, and it does not make much sense in direct IO.
* `ReadOptions::total_order_seek` no longer affects `DB::Get()`. The original motivation for this interaction has been obsolete since RocksDB has been able to detect whether the current prefix extractor is compatible with that used to generate table files, probably RocksDB 5.14.0.
## 6.29.0 (01/21/2022)
Note: The next release will be major release 7.0. See https://github.com/facebook/rocksdb/issues/9390 for more info.

@ -217,11 +217,23 @@ TEST_F(DBBloomFilterTest, GetFilterByPrefixBloomCustomPrefixExtractor) {
(*(get_perf_context()->level_to_perf_context))[0].bloom_filter_useful);
ro.total_order_seek = true;
ASSERT_TRUE(db_->Get(ro, "foobarbar", &value).IsNotFound());
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 2);
// NOTE: total_order_seek no longer affects Get()
ASSERT_EQ("NOT_FOUND", Get("foobarbar"));
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 3);
ASSERT_EQ(
2,
3,
(*(get_perf_context()->level_to_perf_context))[0].bloom_filter_useful);
// No bloom on extractor changed
#ifndef ROCKSDB_LITE
ASSERT_OK(db_->SetOptions({{"prefix_extractor", "capped:10"}}));
ASSERT_EQ("NOT_FOUND", Get("foobarbar"));
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 3);
ASSERT_EQ(
3,
(*(get_perf_context()->level_to_perf_context))[0].bloom_filter_useful);
#endif // ROCKSDB_LITE
get_perf_context()->Reset();
}
}
@ -268,11 +280,23 @@ TEST_F(DBBloomFilterTest, GetFilterByPrefixBloom) {
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 2);
ro.total_order_seek = true;
ASSERT_TRUE(db_->Get(ro, "foobarbar", &value).IsNotFound());
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 2);
// NOTE: total_order_seek no longer affects Get()
ASSERT_EQ("NOT_FOUND", Get("foobarbar"));
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 3);
ASSERT_EQ(
2,
3,
(*(get_perf_context()->level_to_perf_context))[0].bloom_filter_useful);
// No bloom on extractor changed
#ifndef ROCKSDB_LITE
ASSERT_OK(db_->SetOptions({{"prefix_extractor", "capped:10"}}));
ASSERT_EQ("NOT_FOUND", Get("foobarbar"));
ASSERT_EQ(TestGetTickerCount(options, BLOOM_FILTER_USEFUL), 3);
ASSERT_EQ(
3,
(*(get_perf_context()->level_to_perf_context))[0].bloom_filter_useful);
#endif // ROCKSDB_LITE
get_perf_context()->Reset();
}
}

@ -1499,8 +1499,7 @@ struct ReadOptions {
// used in the table. Some table format (e.g. plain table) may not support
// this option.
// If true when calling Get(), we also skip prefix bloom when reading from
// block based table. It provides a way to read existing data after
// changing implementation of prefix extractor.
// block based table, which only affects Get() performance.
// Default: false
bool total_order_seek;

@ -122,9 +122,8 @@ void ReleaseCachedEntry(void* arg, void* h) {
cache->Release(handle, false /* force_erase */);
}
// For hash based index, return true if prefix_extractor and
// prefix_extractor_block mismatch, false otherwise. This flag will be used
// as total_order_seek via NewIndexIterator
// For hash based index, return false if table_properties->prefix_extractor_name
// and prefix_extractor both exist and match, otherwise true.
inline bool PrefixExtractorChangedHelper(
const TableProperties* table_properties,
const SliceTransform* prefix_extractor) {
@ -616,10 +615,6 @@ Status BlockBasedTable::Open(
"version of RocksDB?");
}
// We've successfully read the footer. We are ready to serve requests.
// Better not mutate rep_ after the creation. eg. internal_prefix_transform
// raw pointer will be used to create HashIndexReader, whose reset may
// access a dangling pointer.
BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
Rep* rep = new BlockBasedTable::Rep(ioptions, env_options, table_options,
internal_comparator, skip_filters,
@ -627,8 +622,14 @@ Status BlockBasedTable::Open(
rep->file = std::move(file);
rep->footer = footer;
rep->hash_index_allow_collision = table_options.hash_index_allow_collision;
// We've successfully read the footer. We are ready to serve requests.
// Better not mutate rep_ after the creation. eg. internal_prefix_transform
// raw pointer will be used to create HashIndexReader, whose reset may
// access a dangling pointer.
// We need to wrap data with internal_prefix_transform to make sure it can
// handle prefix correctly.
// FIXME: is changed prefix_extractor handled anywhere for hash index?
if (prefix_extractor != nullptr) {
rep->internal_prefix_transform.reset(
new InternalKeySliceTransform(prefix_extractor.get()));
@ -2212,8 +2213,7 @@ FragmentedRangeTombstoneIterator* BlockBasedTable::NewRangeTombstoneIterator(
}
bool BlockBasedTable::FullFilterKeyMayMatch(
const ReadOptions& read_options, FilterBlockReader* filter,
const Slice& internal_key, const bool no_io,
FilterBlockReader* filter, const Slice& internal_key, const bool no_io,
const SliceTransform* prefix_extractor, GetContext* get_context,
BlockCacheLookupContext* lookup_context) const {
if (filter == nullptr || filter->IsBlockBased()) {
@ -2228,13 +2228,14 @@ bool BlockBasedTable::FullFilterKeyMayMatch(
may_match =
filter->KeyMayMatch(user_key_without_ts, prefix_extractor, kNotValid,
no_io, const_ikey_ptr, get_context, lookup_context);
} else if (!read_options.total_order_seek &&
!PrefixExtractorChanged(prefix_extractor) &&
} else if (!PrefixExtractorChanged(prefix_extractor) &&
prefix_extractor->InDomain(user_key_without_ts) &&
!filter->PrefixMayMatch(
prefix_extractor->Transform(user_key_without_ts),
prefix_extractor, kNotValid, no_io, const_ikey_ptr,
get_context, lookup_context)) {
// FIXME ^^^: there should be no reason for Get() to depend on current
// prefix_extractor at all. It should always use table_prefix_extractor.
may_match = false;
}
if (may_match) {
@ -2245,8 +2246,7 @@ bool BlockBasedTable::FullFilterKeyMayMatch(
}
void BlockBasedTable::FullFilterKeysMayMatch(
const ReadOptions& read_options, FilterBlockReader* filter,
MultiGetRange* range, const bool no_io,
FilterBlockReader* filter, MultiGetRange* range, const bool no_io,
const SliceTransform* prefix_extractor,
BlockCacheLookupContext* lookup_context) const {
if (filter == nullptr || filter->IsBlockBased()) {
@ -2269,8 +2269,9 @@ void BlockBasedTable::FullFilterKeysMayMatch(
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
rep_->level);
}
} else if (!read_options.total_order_seek &&
!PrefixExtractorChanged(prefix_extractor)) {
} else if (!PrefixExtractorChanged(prefix_extractor)) {
// FIXME ^^^: there should be no reason for MultiGet() to depend on current
// prefix_extractor at all. It should always use table_prefix_extractor.
filter->PrefixesMayMatch(range, prefix_extractor, kNotValid, false,
lookup_context);
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED, before_keys);
@ -2308,9 +2309,8 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
read_options.snapshot != nullptr;
}
TEST_SYNC_POINT("BlockBasedTable::Get:BeforeFilterMatch");
const bool may_match =
FullFilterKeyMayMatch(read_options, filter, key, no_io, prefix_extractor,
get_context, &lookup_context);
const bool may_match = FullFilterKeyMayMatch(
filter, key, no_io, prefix_extractor, get_context, &lookup_context);
TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch");
if (!may_match) {
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL);
@ -2496,8 +2496,8 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
BlockCacheLookupContext lookup_context{
TableReaderCaller::kUserMultiGet, tracing_mget_id,
/*get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
FullFilterKeysMayMatch(read_options, filter, &sst_file_range, no_io,
prefix_extractor, &lookup_context);
FullFilterKeysMayMatch(filter, &sst_file_range, no_io, prefix_extractor,
&lookup_context);
if (!sst_file_range.empty()) {
IndexBlockIter iiter_on_stack;
@ -3110,12 +3110,6 @@ Status BlockBasedTable::CreateIndexReader(
InternalIterator* meta_iter, bool use_cache, bool prefetch, bool pin,
BlockCacheLookupContext* lookup_context,
std::unique_ptr<IndexReader>* index_reader) {
// kHashSearch requires non-empty prefix_extractor but bypass checking
// prefix_extractor here since we have no access to MutableCFOptions.
// Add need_upper_bound_check flag in BlockBasedTable::NewIndexIterator.
// If prefix_extractor does not match prefix_extractor_name from table
// properties, turn off Hash Index by setting total_order_seek to true
switch (rep_->index_type) {
case BlockBasedTableOptions::kTwoLevelIndexSearch: {
return PartitionIndexReader::Create(this, ro, prefetch_buffer, use_cache,
@ -3133,6 +3127,7 @@ Status BlockBasedTable::CreateIndexReader(
std::unique_ptr<Block> metaindex_guard;
std::unique_ptr<InternalIterator> metaindex_iter_guard;
bool should_fallback = false;
// FIXME: is changed prefix_extractor handled anywhere for hash index?
if (rep_->internal_prefix_transform.get() == nullptr) {
ROCKS_LOG_WARN(rep_->ioptions.logger,
"No prefix extractor passed in. Fall back to binary"

@ -439,15 +439,13 @@ class BlockBasedTable : public TableReader {
BlockCacheLookupContext* lookup_context,
std::unique_ptr<IndexReader>* index_reader);
bool FullFilterKeyMayMatch(const ReadOptions& read_options,
FilterBlockReader* filter, const Slice& user_key,
bool FullFilterKeyMayMatch(FilterBlockReader* filter, const Slice& user_key,
const bool no_io,
const SliceTransform* prefix_extractor,
GetContext* get_context,
BlockCacheLookupContext* lookup_context) const;
void FullFilterKeysMayMatch(const ReadOptions& read_options,
FilterBlockReader* filter, MultiGetRange* range,
void FullFilterKeysMayMatch(FilterBlockReader* filter, MultiGetRange* range,
const bool no_io,
const SliceTransform* prefix_extractor,
BlockCacheLookupContext* lookup_context) const;
@ -505,8 +503,8 @@ class BlockBasedTable : public TableReader {
void DumpKeyValue(const Slice& key, const Slice& value,
std::ostream& out_stream);
// Returns true if prefix_extractor is compatible with that used in building
// the table file.
// Returns false if prefix_extractor exists and is compatible with that used
// in building the table file, otherwise true.
bool PrefixExtractorChanged(const SliceTransform* prefix_extractor) const;
// A cumulative data block file read in MultiGet lower than this size will

@ -688,6 +688,10 @@ DEFINE_bool(memtable_whole_key_filtering, false,
DEFINE_bool(memtable_use_huge_page, false,
"Try to use huge page in memtables.");
DEFINE_bool(whole_key_filtering,
ROCKSDB_NAMESPACE::BlockBasedTableOptions().whole_key_filtering,
"Use whole keys (in addition to prefixes) in SST bloom filter.");
DEFINE_bool(use_existing_db, false, "If true, do not destroy the existing"
" database. If you set this flag and also specify a benchmark that"
" wants a fresh database, that benchmark will fail.");
@ -2622,6 +2626,7 @@ class Benchmark {
int64_t writes_per_range_tombstone_;
int64_t range_tombstone_width_;
int64_t max_num_range_tombstones_;
ReadOptions read_options_;
WriteOptions write_options_;
Options open_options_; // keep options around to properly destroy db later
#ifndef ROCKSDB_LITE
@ -3245,6 +3250,12 @@ class Benchmark {
write_options_.sync = true;
}
write_options_.disableWAL = FLAGS_disable_wal;
read_options_ = ReadOptions(FLAGS_verify_checksum, true);
read_options_.total_order_seek = FLAGS_total_order_seek;
read_options_.prefix_same_as_start = FLAGS_prefix_same_as_start;
read_options_.tailing = FLAGS_use_tailing_iterator;
read_options_.readahead_size = FLAGS_readahead_size;
read_options_.adaptive_readahead = FLAGS_adaptive_readahead;
void (Benchmark::*method)(ThreadState*) = nullptr;
void (Benchmark::*post_process_method)() = nullptr;
@ -4163,6 +4174,7 @@ class Benchmark {
block_based_options.enable_index_compression =
FLAGS_enable_index_compression;
block_based_options.block_align = FLAGS_block_align;
block_based_options.whole_key_filtering = FLAGS_whole_key_filtering;
BlockBasedTableOptions::PrepopulateBlockCache prepopulate_block_cache =
block_based_options.prepopulate_block_cache;
switch (FLAGS_prepopulate_block_cache) {
@ -4484,7 +4496,7 @@ class Benchmark {
if (FLAGS_use_existing_keys) {
// Only work on single database
assert(db_.db != nullptr);
ReadOptions read_opts;
ReadOptions read_opts; // before read_options_ initialized
read_opts.total_order_seek = true;
Iterator* iter = db_.db->NewIterator(read_opts);
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
@ -5483,8 +5495,7 @@ class Benchmark {
}
void ReadSequential(ThreadState* thread, DB* db) {
ReadOptions options(FLAGS_verify_checksum, true);
options.tailing = FLAGS_use_tailing_iterator;
ReadOptions options = read_options_;
std::unique_ptr<char[]> ts_guard;
Slice ts;
if (user_timestamp_size_ > 0) {
@ -5523,7 +5534,6 @@ class Benchmark {
int64_t found = 0;
int64_t bytes = 0;
int64_t key_rand = 0;
ReadOptions options(FLAGS_verify_checksum, true);
std::unique_ptr<const char[]> key_guard;
Slice key = AllocateKey(&key_guard);
PinnableSlice pinnable_val;
@ -5538,11 +5548,11 @@ class Benchmark {
read++;
Status s;
if (FLAGS_num_column_families > 1) {
s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
&pinnable_val);
s = db_with_cfh->db->Get(read_options_, db_with_cfh->GetCfh(key_rand),
key, &pinnable_val);
} else {
pinnable_val.Reset();
s = db_with_cfh->db->Get(options,
s = db_with_cfh->db->Get(read_options_,
db_with_cfh->db->DefaultColumnFamily(), key,
&pinnable_val);
}
@ -5588,9 +5598,7 @@ class Benchmark {
}
void ReadReverse(ThreadState* thread, DB* db) {
ReadOptions options(FLAGS_verify_checksum, true);
options.adaptive_readahead = FLAGS_adaptive_readahead;
Iterator* iter = db->NewIterator(options);
Iterator* iter = db->NewIterator(read_options_);
int64_t i = 0;
int64_t bytes = 0;
for (iter->SeekToLast(); i < reads_ && iter->Valid(); iter->Prev()) {
@ -5612,7 +5620,7 @@ class Benchmark {
int64_t read = 0;
int64_t found = 0;
int64_t nonexist = 0;
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
std::unique_ptr<const char[]> key_guard;
Slice key = AllocateKey(&key_guard);
std::string value;
@ -5702,7 +5710,7 @@ class Benchmark {
int64_t bytes = 0;
int num_keys = 0;
int64_t key_rand = 0;
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
std::unique_ptr<const char[]> key_guard;
Slice key = AllocateKey(&key_guard);
PinnableSlice pinnable_val;
@ -5788,7 +5796,7 @@ class Benchmark {
int64_t bytes = 0;
int64_t num_multireads = 0;
int64_t found = 0;
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
std::vector<Slice> keys;
std::vector<std::unique_ptr<const char[]> > key_guards;
std::vector<std::string> values(entries_per_batch_);
@ -6170,7 +6178,6 @@ class Benchmark {
value_max = FLAGS_mix_max_value_size;
}
ReadOptions options(FLAGS_verify_checksum, true);
std::unique_ptr<const char[]> key_guard;
Slice key = AllocateKey(&key_guard);
PinnableSlice pinnable_val;
@ -6253,11 +6260,11 @@ class Benchmark {
gets++;
read++;
if (FLAGS_num_column_families > 1) {
s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
&pinnable_val);
s = db_with_cfh->db->Get(read_options_, db_with_cfh->GetCfh(key_rand),
key, &pinnable_val);
} else {
pinnable_val.Reset();
s = db_with_cfh->db->Get(options,
s = db_with_cfh->db->Get(read_options_,
db_with_cfh->db->DefaultColumnFamily(), key,
&pinnable_val);
}
@ -6302,7 +6309,7 @@ class Benchmark {
// Seek query
if (db_with_cfh->db != nullptr) {
Iterator* single_iter = nullptr;
single_iter = db_with_cfh->db->NewIterator(options);
single_iter = db_with_cfh->db->NewIterator(read_options_);
if (single_iter != nullptr) {
single_iter->Seek(key);
seek++;
@ -6345,7 +6352,7 @@ class Benchmark {
void IteratorCreation(ThreadState* thread) {
Duration duration(FLAGS_duration, reads_);
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
std::unique_ptr<char[]> ts_guard;
if (user_timestamp_size_ > 0) {
ts_guard.reset(new char[user_timestamp_size_]);
@ -6375,12 +6382,7 @@ class Benchmark {
int64_t read = 0;
int64_t found = 0;
int64_t bytes = 0;
ReadOptions options(FLAGS_verify_checksum, true);
options.total_order_seek = FLAGS_total_order_seek;
options.prefix_same_as_start = FLAGS_prefix_same_as_start;
options.tailing = FLAGS_use_tailing_iterator;
options.readahead_size = FLAGS_readahead_size;
options.adaptive_readahead = FLAGS_adaptive_readahead;
ReadOptions options = read_options_;
std::unique_ptr<char[]> ts_guard;
Slice ts;
if (user_timestamp_size_ > 0) {
@ -6669,7 +6671,7 @@ class Benchmark {
abort();
}
assert(db_.db != nullptr);
ReadOptions read_options;
ReadOptions read_options = read_options_;
std::unique_ptr<char[]> ts_guard;
Slice ts;
if (user_timestamp_size_ > 0) {
@ -6677,7 +6679,6 @@ class Benchmark {
ts = mock_app_clock_->GetTimestampForRead(thread->rand, ts_guard.get());
read_options.timestamp = &ts;
}
read_options.adaptive_readahead = FLAGS_adaptive_readahead;
Iterator* iter = db_.db->NewIterator(read_options);
fprintf(stderr, "num reads to do %" PRIu64 "\n", reads_);
@ -6767,13 +6768,12 @@ class Benchmark {
// Given a key K and value V, this gets values for K+"0", K+"1" and K+"2"
// in the same snapshot, and verifies that all the values are identical.
// ASSUMES that PutMany was used to put (K, V) into the DB.
Status GetMany(DB* db, const ReadOptions& readoptions, const Slice& key,
std::string* value) {
Status GetMany(DB* db, const Slice& key, std::string* value) {
std::string suffixes[3] = {"0", "1", "2"};
std::string keys[3];
Slice key_slices[3];
std::string values[3];
ReadOptions readoptionscopy = readoptions;
ReadOptions readoptionscopy = read_options_;
std::unique_ptr<char[]> ts_guard;
Slice ts;
@ -6821,7 +6821,6 @@ class Benchmark {
// FLAGS_numdistinct distinct keys instead of FLAGS_num distinct keys.
// (d) Does not have a MultiGet option.
void RandomWithVerify(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
RandomGenerator gen;
std::string value;
int64_t found = 0;
@ -6848,7 +6847,7 @@ class Benchmark {
FLAGS_numdistinct, &key);
if (get_weight > 0) {
// do all the gets first
Status s = GetMany(db, options, key, &value);
Status s = GetMany(db, key, &value);
if (!s.ok() && !s.IsNotFound()) {
fprintf(stderr, "getmany error: %s\n", s.ToString().c_str());
// we continue after error rather than exiting so that we can
@ -6892,7 +6891,7 @@ class Benchmark {
// This is different from ReadWhileWriting because it does not use
// an extra thread.
void ReadRandomWriteRandom(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
RandomGenerator gen;
std::string value;
int64_t found = 0;
@ -6966,7 +6965,7 @@ class Benchmark {
//
// Read-modify-write for random keys
void UpdateRandom(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
RandomGenerator gen;
std::string value;
int64_t found = 0;
@ -7031,7 +7030,7 @@ class Benchmark {
// representing the existing value, we generate an array B of the same size,
// then compute C = A^B as C[i]=A[i]^B[i], and store C
void XORUpdateRandom(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
RandomGenerator gen;
std::string existing_value;
int64_t found = 0;
@ -7096,7 +7095,7 @@ class Benchmark {
// Each operation causes the key grow by value_size (simulating an append).
// Generally used for benchmarking against merges of similar type
void AppendRandom(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
ReadOptions options = read_options_;
RandomGenerator gen;
std::string value;
int64_t found = 0;
@ -7219,7 +7218,6 @@ class Benchmark {
// As with MergeRandom, the merge operator to use should be defined by
// FLAGS_merge_operator.
void ReadRandomMergeRandom(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
RandomGenerator gen;
std::string value;
int64_t num_hits = 0;
@ -7246,7 +7244,7 @@ class Benchmark {
num_merges++;
thread->stats.FinishedOps(nullptr, db, 1, kMerge);
} else {
Status s = db->Get(options, key, &value);
Status s = db->Get(read_options_, key, &value);
if (value.length() > max_length)
max_length = value.length();
@ -7277,8 +7275,7 @@ class Benchmark {
thread->stats.Start(thread->tid);
DB* db = SelectDB(thread);
ReadOptions read_opts(FLAGS_verify_checksum, true);
read_opts.adaptive_readahead = FLAGS_adaptive_readahead;
ReadOptions read_opts = read_options_;
std::unique_ptr<char[]> ts_guard;
Slice ts;
if (user_timestamp_size_ > 0) {
@ -7418,9 +7415,7 @@ class Benchmark {
// RandomTransactionVerify() will then validate the correctness of the results
// by checking if the sum of all keys in each set is the same.
void RandomTransaction(ThreadState* thread) {
ReadOptions options(FLAGS_verify_checksum, true);
Duration duration(FLAGS_duration, readwrites_);
ReadOptions read_options(FLAGS_verify_checksum, true);
uint16_t num_prefix_ranges = static_cast<uint16_t>(FLAGS_transaction_sets);
uint64_t transactions_done = 0;
@ -7434,7 +7429,7 @@ class Benchmark {
txn_options.set_snapshot = FLAGS_transaction_set_snapshot;
RandomTransactionInserter inserter(&thread->rand, write_options_,
read_options, FLAGS_num,
read_options_, FLAGS_num,
num_prefix_ranges);
if (FLAGS_num_multi_db > 1) {
@ -7585,7 +7580,6 @@ class Benchmark {
}
void TimeSeriesReadOrDelete(ThreadState* thread, bool do_deletion) {
ReadOptions options(FLAGS_verify_checksum, true);
int64_t read = 0;
int64_t found = 0;
int64_t bytes = 0;
@ -7593,7 +7587,7 @@ class Benchmark {
Iterator* iter = nullptr;
// Only work on single database
assert(db_.db != nullptr);
iter = db_.db->NewIterator(options);
iter = db_.db->NewIterator(read_options_);
std::unique_ptr<const char[]> key_guard;
Slice key = AllocateKey(&key_guard);
@ -7609,7 +7603,7 @@ class Benchmark {
}
if (!FLAGS_use_tailing_iterator) {
delete iter;
iter = db_.db->NewIterator(options);
iter = db_.db->NewIterator(read_options_);
}
// Pick a Iterator to use

Loading…
Cancel
Save