Summary: **Context/Summary:** During prefix scan, prefetched data blocks containing keys not in the same prefix as the `Seek()`'s key will be wasted when `ReadOptions::prefix_same_as_start = true` since they won't be returned to the user. This PR is to exclude those data blocks from being prefetched in a similar manner like trimming according to `ReadOptions::iterate_upper_bound`. Bonus: refactoring to some existing prefetch test so they are easier to extend and read Pull Request resolved: https://github.com/facebook/rocksdb/pull/13040 Test Plan: - New UT, integration to existing UTs - Benchmark to ensure no regression from CPU due to more trimming logic ``` // Build DB with one sorted run under the same prefix ./db_bench --benchmarks=fillrandom --prefix_size=3 --keys_per_prefix=5000000 --num=5000000 --db=/dev/shm/db_bench --disable_auto_compactions=1 ``` ``` // Augment the existing db bench to call `Seek()` instead of `SeekToFirst()` in `void ReadSequential(){..}` to trigger the logic in this PR +++ b/tools/db_bench_tool.cc @@ -5900,7 +5900,12 @@ class Benchmark { Iterator* iter = db->NewIterator(options); int64_t i = 0; int64_t bytes = 0; - for (iter->SeekToFirst(); i < reads_ && iter->Valid(); iter->Next()) { + + iter->SeekToFirst(); + assert(iter->status().ok() && iter->Valid()); + auto prefix = prefix_extractor_->Transform(iter->key()); + + for (iter->Seek(prefix); i < reads_ && iter->Valid(); iter->Next()) { bytes += iter->key().size() + iter->value().size(); thread->stats.FinishedOps(nullptr, db, 1, kRead); ++i; : ``` ``` // Compare prefix scan performance ./db_bench --benchmarks=readseq[-X20] --prefix_size=3 --prefix_same_as_start=1 --auto_readahead_size=1 --cache_size=1 --use_existing_db=1 --db=/dev/shm/db_bench --disable_auto_compactions=1 // Before PR readseq [AVG 20 runs] : 2449011 (± 50238) ops/sec; 270.9 (± 5.6) MB/sec readseq [MEDIAN 20 runs] : 2499167 ops/sec; 276.5 MB/sec // After PR (regress 0.4 %) readseq [AVG 20 runs] : 2439098 (± 42931) ops/sec; 269.8 (± 4.7) MB/sec readseq [MEDIAN 20 runs] : 2460859 ops/sec; 272.2 MB/sec ``` - Stress test: randomly set `prefix_same_as_start` in `TestPrefixScan()`. Run below for a while ``` python3 tools/db_crashtest.py --simple blackbox --prefix_size=5 --prefixpercent=65 --WAL_size_limit_MB=1 --WAL_ttl_seconds=0 --acquire_snapshot_one_in=10000 --adm_policy=1 --advise_random_on_open=1 --allow_data_in_errors=True --allow_fallocate=0 --async_io=0 --avoid_flush_during_recovery=1 --avoid_flush_during_shutdown=1 --avoid_unnecessary_blocking_io=0 --backup_max_size=104857600 --backup_one_in=1000 --batch_protection_bytes_per_key=8 --bgerror_resume_retry_interval=100 --block_align=1 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=-1 --bloom_bits=3 --bottommost_compression_type=none --bottommost_file_compaction_delay=3600 --bytes_per_sync=262144 --cache_index_and_filter_blocks=0 --cache_index_and_filter_blocks_with_high_priority=0 --cache_size=33554432 --cache_type=lru_cache --charge_compression_dictionary_building_buffer=1 --charge_file_metadata=0 --charge_filter_construction=0 --charge_table_reader=0 --check_multiget_consistency=0 --check_multiget_entity_consistency=0 --checkpoint_one_in=10000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000 --compact_range_one_in=1000 --compaction_pri=4 --compaction_readahead_size=0 --compaction_style=2 --compaction_ttl=0 --compress_format_version=2 --compressed_secondary_cache_size=16777216 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=1 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc= --data_block_index_type=0 --db_write_buffer_size=8388608 --decouple_partitioned_filters=1 --default_temperature=kCold --default_write_temperature=kWarm --delete_obsolete_files_period_micros=21600000000 --delpercent=4 --delrangepercent=1 --destroy_db_initially=1 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=10000 --disable_manual_compaction_one_in=1000000 --disable_wal=0 --dump_malloc_stats=1 --enable_checksum_handoff=0 --enable_compaction_filter=0 --enable_custom_split_merge=1 --enable_do_not_compress_roles=1 --enable_index_compression=1 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=1 --enable_sst_partitioner_factory=0 --enable_thread_tracking=1 --enable_write_thread_adaptive_yield=0 --error_recovery_with_no_fault_injection=0 --exclude_wal_from_write_fault_injection=1 --fail_if_options_file_error=0 --fifo_allow_compaction=1 --file_checksum_impl=big --fill_cache=0 --flush_one_in=1000000 --format_version=6 --get_all_column_family_metadata_one_in=10000 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=10000 --get_properties_of_all_tables_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --hard_pending_compaction_bytes_limit=274877906944 --high_pri_pool_ratio=0 --index_block_restart_interval=15 --index_shortening=2 --index_type=2 --ingest_external_file_one_in=0 --inplace_update_support=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=100000 --last_level_temperature=kUnknown --level_compaction_dynamic_level_bytes=0 --lock_wal_one_in=1000000 --log2_keys_per_lock=10 --log_file_time_to_roll=0 --log_readahead_size=0 --long_running_snapshots=0 --low_pri_pool_ratio=0.5 --lowest_used_cache_tier=2 --manifest_preallocation_size=5120 --manual_wal_flush_one_in=1000 --mark_for_compaction_one_file_in=10 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=1000 --max_key_len=3 --max_log_file_size=1048576 --max_manifest_file_size=1073741824 --max_sequential_skip_in_iterations=8 --max_total_wal_size=0 --max_write_batch_group_size_bytes=16777216 --max_write_buffer_number=10 --max_write_buffer_size_to_maintain=4194304 --memtable_insert_hint_per_batch=1 --memtable_max_range_deletions=0 --memtable_prefix_bloom_size_ratio=0.01 --memtable_protection_bytes_per_key=2 --memtable_whole_key_filtering=1 --memtablerep=skip_list --metadata_charge_policy=1 --metadata_read_fault_one_in=32 --metadata_write_fault_one_in=0 --min_write_buffer_number_to_merge=2 --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --open_files=-1 --open_metadata_read_fault_one_in=0 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=0 --open_write_fault_one_in=0 --ops_per_thread=40000 --optimize_filters_for_hits=0 --optimize_filters_for_memory=1 --optimize_multiget_for_io=1 --paranoid_file_checks=1 --paranoid_memory_checks=0 --partition_filters=0 --partition_pinning=3 --pause_background_one_in=10000 --periodic_compaction_seconds=0 --prepopulate_block_cache=1 --preserve_internal_time_seconds=0 --progress_reports=0 --promote_l0_one_in=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=32 --readpercent=10 --recycle_log_file_num=0 --reopen=0 --report_bg_io_stats=0 --reset_stats_one_in=1000000 --sample_for_compression=5 --secondary_cache_fault_one_in=0 --secondary_cache_uri= --skip_stats_update_on_db_open=0 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=1048576 --sqfc_name=foo --sqfc_version=2 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=1048576 --stats_dump_period_sec=10 --stats_history_buffer_size=1048576 --strict_bytes_per_sync=1 --subcompactions=4 --sync=0 --sync_fault_injection=1 --table_cache_numshardbits=-1 --target_file_size_base=524288 --target_file_size_multiplier=2 --test_batches_snapshots=0 --top_level_index_pinning=0 --uncache_aggressiveness=1 --universal_max_read_amp=10 --unpartitioned_pinning=0 --use_adaptive_mutex=1 --use_adaptive_mutex_lru=1 --use_attribute_group=0 --use_delta_encoding=0 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=1 --use_full_merge_v1=1 --use_get_entity=0 --use_merge=1 --use_multi_cf_iterator=1 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=0 --use_sqfc_for_range_queries=1 --use_timed_put_one_in=0 --use_write_buffer_manager=1 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000 --verify_compression=0 --verify_db_one_in=100000 --verify_file_checksums_one_in=1000 --verify_iterator_with_expected_state_one_in=5 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=none --write_buffer_size=1048576 --write_dbid_to_manifest=1 --write_fault_one_in=1000 --writepercent=10 ``` Reviewed By: anand1976 Differential Revision: D64367065 Pulled By: hx235 fbshipit-source-id: 5750c05ccc835c3e9dc81c961b76deaf30bd23c2
476 lines
17 KiB
C++
476 lines
17 KiB
C++
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
// (found in the LICENSE.Apache file in the root directory).
|
|
//
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
#pragma once
|
|
#include <deque>
|
|
|
|
#include "db/seqno_to_time_mapping.h"
|
|
#include "table/block_based/block_based_table_reader.h"
|
|
#include "table/block_based/block_based_table_reader_impl.h"
|
|
#include "table/block_based/block_prefetcher.h"
|
|
#include "table/block_based/reader_common.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
// Iterates over the contents of BlockBasedTable.
|
|
class BlockBasedTableIterator : public InternalIteratorBase<Slice> {
|
|
// compaction_readahead_size: its value will only be used if for_compaction =
|
|
// true
|
|
// @param read_options Must outlive this iterator.
|
|
public:
|
|
BlockBasedTableIterator(
|
|
const BlockBasedTable* table, const ReadOptions& read_options,
|
|
const InternalKeyComparator& icomp,
|
|
std::unique_ptr<InternalIteratorBase<IndexValue>>&& index_iter,
|
|
bool check_filter, bool need_upper_bound_check,
|
|
const SliceTransform* prefix_extractor, TableReaderCaller caller,
|
|
size_t compaction_readahead_size = 0, bool allow_unprepared_value = false)
|
|
: index_iter_(std::move(index_iter)),
|
|
table_(table),
|
|
read_options_(read_options),
|
|
icomp_(icomp),
|
|
user_comparator_(icomp.user_comparator()),
|
|
pinned_iters_mgr_(nullptr),
|
|
prefix_extractor_(prefix_extractor),
|
|
lookup_context_(caller),
|
|
block_prefetcher_(
|
|
compaction_readahead_size,
|
|
table_->get_rep()->table_options.initial_auto_readahead_size),
|
|
allow_unprepared_value_(allow_unprepared_value),
|
|
block_iter_points_to_real_block_(false),
|
|
check_filter_(check_filter),
|
|
need_upper_bound_check_(need_upper_bound_check),
|
|
async_read_in_progress_(false),
|
|
is_last_level_(table->IsLastLevel()) {}
|
|
|
|
~BlockBasedTableIterator() override { ClearBlockHandles(); }
|
|
|
|
void Seek(const Slice& target) override;
|
|
void SeekForPrev(const Slice& target) override;
|
|
void SeekToFirst() override;
|
|
void SeekToLast() override;
|
|
void Next() final override;
|
|
bool NextAndGetResult(IterateResult* result) override;
|
|
void Prev() override;
|
|
bool Valid() const override {
|
|
return !is_out_of_bound_ &&
|
|
(is_at_first_key_from_index_ ||
|
|
(block_iter_points_to_real_block_ && block_iter_.Valid()));
|
|
}
|
|
|
|
// For block cache readahead lookup scenario -
|
|
// If is_at_first_key_from_index_ is true, InitDataBlock hasn't been
|
|
// called. It means block_handles is empty and index_ point to current block.
|
|
// So index_iter_ can be accessed directly.
|
|
Slice key() const override {
|
|
assert(Valid());
|
|
if (is_at_first_key_from_index_) {
|
|
return index_iter_->value().first_internal_key;
|
|
} else {
|
|
return block_iter_.key();
|
|
}
|
|
}
|
|
Slice user_key() const override {
|
|
assert(Valid());
|
|
if (is_at_first_key_from_index_) {
|
|
return ExtractUserKey(index_iter_->value().first_internal_key);
|
|
} else {
|
|
return block_iter_.user_key();
|
|
}
|
|
}
|
|
|
|
bool PrepareValue() override {
|
|
assert(Valid());
|
|
|
|
if (!is_at_first_key_from_index_) {
|
|
return true;
|
|
}
|
|
|
|
return const_cast<BlockBasedTableIterator*>(this)
|
|
->MaterializeCurrentBlock();
|
|
}
|
|
|
|
uint64_t write_unix_time() const override {
|
|
assert(Valid());
|
|
ParsedInternalKey pikey;
|
|
SequenceNumber seqno;
|
|
const SeqnoToTimeMapping& seqno_to_time_mapping =
|
|
table_->GetSeqnoToTimeMapping();
|
|
Status s = ParseInternalKey(key(), &pikey, /*log_err_key=*/false);
|
|
if (!s.ok()) {
|
|
return std::numeric_limits<uint64_t>::max();
|
|
} else if (kUnknownSeqnoBeforeAll == pikey.sequence) {
|
|
return kUnknownTimeBeforeAll;
|
|
} else if (seqno_to_time_mapping.Empty()) {
|
|
return std::numeric_limits<uint64_t>::max();
|
|
} else if (kTypeValuePreferredSeqno == pikey.type) {
|
|
seqno = ParsePackedValueForSeqno(value());
|
|
} else {
|
|
seqno = pikey.sequence;
|
|
}
|
|
return seqno_to_time_mapping.GetProximalTimeBeforeSeqno(seqno);
|
|
}
|
|
|
|
Slice value() const override {
|
|
// PrepareValue() must have been called.
|
|
assert(!is_at_first_key_from_index_);
|
|
assert(Valid());
|
|
|
|
if (seek_stat_state_ & kReportOnUseful) {
|
|
bool filter_used = (seek_stat_state_ & kFilterUsed) != 0;
|
|
RecordTick(
|
|
table_->GetStatistics(),
|
|
filter_used
|
|
? (is_last_level_ ? LAST_LEVEL_SEEK_DATA_USEFUL_FILTER_MATCH
|
|
: NON_LAST_LEVEL_SEEK_DATA_USEFUL_FILTER_MATCH)
|
|
: (is_last_level_ ? LAST_LEVEL_SEEK_DATA_USEFUL_NO_FILTER
|
|
: NON_LAST_LEVEL_SEEK_DATA_USEFUL_NO_FILTER));
|
|
seek_stat_state_ = kDataBlockReadSinceLastSeek;
|
|
}
|
|
|
|
return block_iter_.value();
|
|
}
|
|
Status status() const override {
|
|
// In case of block cache readahead lookup, it won't add the block to
|
|
// block_handles if it's index is invalid. So index_iter_->status check can
|
|
// be skipped.
|
|
// Prefix index set status to NotFound when the prefix does not exist.
|
|
if (IsIndexAtCurr() && !index_iter_->status().ok() &&
|
|
!index_iter_->status().IsNotFound()) {
|
|
return index_iter_->status();
|
|
} else if (block_iter_points_to_real_block_) {
|
|
return block_iter_.status();
|
|
} else if (async_read_in_progress_) {
|
|
return Status::TryAgain("Async read in progress");
|
|
} else {
|
|
return Status::OK();
|
|
}
|
|
}
|
|
|
|
inline IterBoundCheck UpperBoundCheckResult() override {
|
|
if (is_out_of_bound_) {
|
|
return IterBoundCheck::kOutOfBound;
|
|
} else if (block_upper_bound_check_ ==
|
|
BlockUpperBound::kUpperBoundBeyondCurBlock) {
|
|
assert(!is_out_of_bound_);
|
|
return IterBoundCheck::kInbound;
|
|
} else {
|
|
return IterBoundCheck::kUnknown;
|
|
}
|
|
}
|
|
|
|
void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
|
|
pinned_iters_mgr_ = pinned_iters_mgr;
|
|
}
|
|
bool IsKeyPinned() const override {
|
|
// Our key comes either from block_iter_'s current key
|
|
// or index_iter_'s current *value*.
|
|
return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
|
|
((is_at_first_key_from_index_ && index_iter_->IsValuePinned()) ||
|
|
(block_iter_points_to_real_block_ && block_iter_.IsKeyPinned()));
|
|
}
|
|
bool IsValuePinned() const override {
|
|
assert(!is_at_first_key_from_index_);
|
|
assert(Valid());
|
|
|
|
// BlockIter::IsValuePinned() is always true. No need to check
|
|
return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
|
|
block_iter_points_to_real_block_;
|
|
}
|
|
|
|
void ResetDataIter() {
|
|
if (block_iter_points_to_real_block_) {
|
|
if (pinned_iters_mgr_ != nullptr && pinned_iters_mgr_->PinningEnabled()) {
|
|
block_iter_.DelegateCleanupsTo(pinned_iters_mgr_);
|
|
}
|
|
block_iter_.Invalidate(Status::OK());
|
|
block_iter_points_to_real_block_ = false;
|
|
}
|
|
block_upper_bound_check_ = BlockUpperBound::kUnknown;
|
|
}
|
|
|
|
void SavePrevIndexValue() {
|
|
if (block_iter_points_to_real_block_ && IsIndexAtCurr()) {
|
|
// Reseek. If they end up with the same data block, we shouldn't re-fetch
|
|
// the same data block.
|
|
prev_block_offset_ = index_iter_->value().handle.offset();
|
|
}
|
|
}
|
|
|
|
void GetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
|
|
if (block_prefetcher_.prefetch_buffer() != nullptr &&
|
|
read_options_.adaptive_readahead) {
|
|
block_prefetcher_.prefetch_buffer()->GetReadaheadState(
|
|
&(readahead_file_info->data_block_readahead_info));
|
|
if (index_iter_) {
|
|
index_iter_->GetReadaheadState(readahead_file_info);
|
|
}
|
|
}
|
|
}
|
|
|
|
void SetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
|
|
if (read_options_.adaptive_readahead) {
|
|
block_prefetcher_.SetReadaheadState(
|
|
&(readahead_file_info->data_block_readahead_info));
|
|
if (index_iter_) {
|
|
index_iter_->SetReadaheadState(readahead_file_info);
|
|
}
|
|
}
|
|
}
|
|
|
|
FilePrefetchBuffer* prefetch_buffer() {
|
|
return block_prefetcher_.prefetch_buffer();
|
|
}
|
|
|
|
std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter_;
|
|
|
|
private:
|
|
enum class IterDirection {
|
|
kForward,
|
|
kBackward,
|
|
};
|
|
// This enum indicates whether the upper bound falls into current block
|
|
// or beyond.
|
|
// +-------------+
|
|
// | cur block | <-- (1)
|
|
// +-------------+
|
|
// <-- (2)
|
|
// --- <boundary key> ---
|
|
// <-- (3)
|
|
// +-------------+
|
|
// | next block | <-- (4)
|
|
// ......
|
|
//
|
|
// When the block is smaller than <boundary key>, kUpperBoundInCurBlock
|
|
// is the value to use. The examples are (1) or (2) in the graph. It means
|
|
// all keys in the next block or beyond will be out of bound. Keys within
|
|
// the current block may or may not be out of bound.
|
|
// When the block is larger or equal to <boundary key>,
|
|
// kUpperBoundBeyondCurBlock is to be used. The examples are (3) and (4)
|
|
// in the graph. It means that all keys in the current block is within the
|
|
// upper bound and keys in the next block may or may not be within the uppder
|
|
// bound.
|
|
// If the boundary key hasn't been checked against the upper bound,
|
|
// kUnknown can be used.
|
|
enum class BlockUpperBound : uint8_t {
|
|
kUpperBoundInCurBlock,
|
|
kUpperBoundBeyondCurBlock,
|
|
kUnknown,
|
|
};
|
|
|
|
// State bits for collecting stats on seeks and whether they returned useful
|
|
// results.
|
|
enum SeekStatState : uint8_t {
|
|
kNone = 0,
|
|
// Most recent seek checked prefix filter (or similar future feature)
|
|
kFilterUsed = 1 << 0,
|
|
// Already recorded that a data block was accessed since the last seek.
|
|
kDataBlockReadSinceLastSeek = 1 << 1,
|
|
// Have not yet recorded that a value() was accessed.
|
|
kReportOnUseful = 1 << 2,
|
|
};
|
|
|
|
// BlockHandleInfo is used to store the info needed when block cache lookup
|
|
// ahead is enabled to tune readahead_size.
|
|
struct BlockHandleInfo {
|
|
void SetFirstInternalKey(const Slice& key) {
|
|
if (key.empty()) {
|
|
return;
|
|
}
|
|
size_t size = key.size();
|
|
buf_ = std::unique_ptr<char[]>(new char[size]);
|
|
memcpy(buf_.get(), key.data(), size);
|
|
first_internal_key_ = Slice(buf_.get(), size);
|
|
}
|
|
|
|
BlockHandle handle_;
|
|
bool is_cache_hit_ = false;
|
|
CachableEntry<Block> cachable_entry_;
|
|
Slice first_internal_key_;
|
|
std::unique_ptr<char[]> buf_;
|
|
};
|
|
|
|
bool IsIndexAtCurr() const { return is_index_at_curr_block_; }
|
|
|
|
const BlockBasedTable* table_;
|
|
const ReadOptions& read_options_;
|
|
const InternalKeyComparator& icomp_;
|
|
UserComparatorWrapper user_comparator_;
|
|
PinnedIteratorsManager* pinned_iters_mgr_;
|
|
DataBlockIter block_iter_;
|
|
const SliceTransform* prefix_extractor_;
|
|
uint64_t prev_block_offset_ = std::numeric_limits<uint64_t>::max();
|
|
BlockCacheLookupContext lookup_context_;
|
|
|
|
BlockPrefetcher block_prefetcher_;
|
|
|
|
const bool allow_unprepared_value_;
|
|
// True if block_iter_ is initialized and points to the same block
|
|
// as index iterator.
|
|
bool block_iter_points_to_real_block_;
|
|
// See InternalIteratorBase::IsOutOfBound().
|
|
bool is_out_of_bound_ = false;
|
|
// How current data block's boundary key with the next block is compared with
|
|
// iterate upper bound.
|
|
BlockUpperBound block_upper_bound_check_ = BlockUpperBound::kUnknown;
|
|
// True if we're standing at the first key of a block, and we haven't loaded
|
|
// that block yet. A call to PrepareValue() will trigger loading the block.
|
|
bool is_at_first_key_from_index_ = false;
|
|
bool check_filter_;
|
|
// TODO(Zhongyi): pick a better name
|
|
bool need_upper_bound_check_;
|
|
|
|
bool async_read_in_progress_;
|
|
|
|
mutable SeekStatState seek_stat_state_ = SeekStatState::kNone;
|
|
bool is_last_level_;
|
|
|
|
// If set to true, it'll lookup in the cache ahead to estimate the readahead
|
|
// size based on cache hit and miss.
|
|
bool readahead_cache_lookup_ = false;
|
|
|
|
// It stores all the block handles that are lookuped in cache ahead when
|
|
// BlockCacheLookupForReadAheadSize is called. Since index_iter_ may point to
|
|
// different blocks when readahead_size is calculated in
|
|
// BlockCacheLookupForReadAheadSize, to avoid index_iter_ reseek,
|
|
// block_handles_ is used.
|
|
// `block_handles_` is lazily constructed to save CPU when it is unused
|
|
std::unique_ptr<std::deque<BlockHandleInfo>> block_handles_;
|
|
|
|
// During cache lookup to find readahead size, index_iter_ is iterated and it
|
|
// can point to a different block. is_index_at_curr_block_ keeps track of
|
|
// that.
|
|
bool is_index_at_curr_block_ = true;
|
|
bool is_index_out_of_bound_ = false;
|
|
|
|
// Used in case of auto_readahead_size to disable the block_cache lookup if
|
|
// direction is reversed from forward to backward. In case of backward
|
|
// direction, SeekForPrev or Prev might call Seek from db_iter. So direction
|
|
// is used to disable the lookup.
|
|
IterDirection direction_ = IterDirection::kForward;
|
|
|
|
// The prefix of the key called with SeekImpl().
|
|
// This is for readahead trimming so no data blocks containing keys of a
|
|
// different prefix are prefetched
|
|
std::string seek_key_prefix_for_readahead_trimming_ = "";
|
|
|
|
void SeekSecondPass(const Slice* target);
|
|
|
|
// If `target` is null, seek to first.
|
|
void SeekImpl(const Slice* target, bool async_prefetch);
|
|
|
|
void InitDataBlock();
|
|
void AsyncInitDataBlock(bool is_first_pass);
|
|
bool MaterializeCurrentBlock();
|
|
void FindKeyForward();
|
|
void FindBlockForward();
|
|
void FindKeyBackward();
|
|
void CheckOutOfBound();
|
|
|
|
// Check if data block is fully within iterate_upper_bound.
|
|
//
|
|
// Note MyRocks may update iterate bounds between seek. To workaround it,
|
|
// we need to check and update data_block_within_upper_bound_ accordingly.
|
|
void CheckDataBlockWithinUpperBound();
|
|
|
|
bool CheckPrefixMayMatch(const Slice& ikey, IterDirection direction,
|
|
bool* filter_checked) {
|
|
if (need_upper_bound_check_ && direction == IterDirection::kBackward) {
|
|
// Upper bound check isn't sufficient for backward direction to
|
|
// guarantee the same result as total order, so disable prefix
|
|
// check.
|
|
return true;
|
|
}
|
|
if (check_filter_ &&
|
|
!table_->PrefixRangeMayMatch(ikey, read_options_, prefix_extractor_,
|
|
need_upper_bound_check_, &lookup_context_,
|
|
filter_checked)) {
|
|
// TODO remember the iterator is invalidated because of prefix
|
|
// match. This can avoid the upper level file iterator to falsely
|
|
// believe the position is the end of the SST file and move to
|
|
// the first key of the next file.
|
|
ResetDataIter();
|
|
return false;
|
|
}
|
|
return true;
|
|
}
|
|
|
|
// *** BEGIN APIs relevant to auto tuning of readahead_size ***
|
|
|
|
// This API is called to lookup the data blocks ahead in the cache to tune
|
|
// the start and end offsets passed.
|
|
void BlockCacheLookupForReadAheadSize(bool read_curr_block,
|
|
uint64_t& start_offset,
|
|
uint64_t& end_offset);
|
|
|
|
void ResetBlockCacheLookupVar() {
|
|
is_index_out_of_bound_ = false;
|
|
readahead_cache_lookup_ = false;
|
|
ClearBlockHandles();
|
|
}
|
|
|
|
bool IsNextBlockOutOfReadaheadBound() {
|
|
const Slice& index_iter_user_key = index_iter_->user_key();
|
|
// If curr block's index key >= iterate_upper_bound, it means all the keys
|
|
// in next block or above are out of bound.
|
|
bool out_of_upper_bound =
|
|
read_options_.iterate_upper_bound != nullptr &&
|
|
(user_comparator_.CompareWithoutTimestamp(
|
|
index_iter_user_key,
|
|
/*a_has_ts=*/true, *read_options_.iterate_upper_bound,
|
|
/*b_has_ts=*/false) >= 0
|
|
? true
|
|
: false);
|
|
if (out_of_upper_bound) {
|
|
return true;
|
|
}
|
|
|
|
// If curr block's index key has a different prefix from the seek key's, it
|
|
// means all the keys in next block or above has a different prefix from the
|
|
// seek key's.
|
|
bool out_of_prefix_bound =
|
|
(read_options_.prefix_same_as_start &&
|
|
!seek_key_prefix_for_readahead_trimming_.empty() &&
|
|
(prefix_extractor_->InDomain(index_iter_user_key)
|
|
? (prefix_extractor_->Transform(index_iter_user_key)
|
|
.compare(seek_key_prefix_for_readahead_trimming_) != 0)
|
|
: user_comparator_.CompareWithoutTimestamp(
|
|
index_iter_user_key,
|
|
/*a_has_ts=*/true, seek_key_prefix_for_readahead_trimming_,
|
|
/*b_has_ts=*/false) > 0));
|
|
|
|
if (out_of_prefix_bound) {
|
|
return true;
|
|
}
|
|
|
|
return false;
|
|
}
|
|
|
|
void ClearBlockHandles() {
|
|
if (block_handles_ != nullptr) {
|
|
block_handles_->clear();
|
|
}
|
|
}
|
|
|
|
// Reset prev_block_offset_. If index_iter_ has moved ahead, it won't get
|
|
// accurate prev_block_offset_.
|
|
void ResetPreviousBlockOffset() {
|
|
prev_block_offset_ = std::numeric_limits<uint64_t>::max();
|
|
}
|
|
|
|
bool DoesContainBlockHandles() {
|
|
return block_handles_ != nullptr && !block_handles_->empty();
|
|
}
|
|
|
|
void InitializeStartAndEndOffsets(bool read_curr_block,
|
|
bool& found_first_miss_block,
|
|
uint64_t& start_updated_offset,
|
|
uint64_t& end_updated_offset,
|
|
size_t& prev_handles_size);
|
|
// *** END APIs relevant to auto tuning of readahead_size ***
|
|
};
|
|
} // namespace ROCKSDB_NAMESPACE
|