Multi file concurrency in MultiGet using coroutines and async IO (#9968)
Summary: This PR implements a coroutine version of batched MultiGet in order to concurrently read from multiple SST files in a level using async IO, thus reducing the latency of the MultiGet. The API from the user perspective is still synchronous and single threaded, with the RocksDB part of the processing happening in the context of the caller's thread. In Version::MultiGet, the decision is made whether to call synchronous or coroutine code. A good way to review this PR is to review the first 4 commits in order - de773b3, 70c2f70, 10b50e1, and 377a597 - before reviewing the rest. TODO: 1. Figure out how to build it in CircleCI (requires some dependencies to be installed) 2. Do some stress testing with coroutines enabled No regression in synchronous MultiGet between this branch and main - ``` ./db_bench -use_existing_db=true --db=/data/mysql/rocksdb/prefix_scan -benchmarks="readseq,multireadrandom" -key_size=32 -value_size=512 -num=5000000 -batch_size=64 -multiread_batched=true -use_direct_reads=false -duration=60 -ops_between_duration_checks=1 -readonly=true -adaptive_readahead=true -threads=16 -cache_size=10485760000 -async_io=false -multiread_stride=40000 -statistics ``` Branch - ```multireadrandom : 4.025 micros/op 3975111 ops/sec 60.001 seconds 238509056 operations; 2062.3 MB/s (14767808 of 14767808 found)``` Main - ```multireadrandom : 3.987 micros/op 4013216 ops/sec 60.001 seconds 240795392 operations; 2082.1 MB/s (15231040 of 15231040 found)``` More benchmarks in various scenarios are given below. The measurements were taken with ```async_io=false``` (no coroutines) and ```async_io=true``` (use coroutines). For an IO bound workload (with every key requiring an IO), the coroutines version shows a clear benefit, being ~2.6X faster. For CPU bound workloads, the coroutines version has ~6-15% higher CPU utilization, depending on how many keys overlap an SST file. 1. Single thread IO bound workload on remote storage with sparse MultiGet batch keys (~1 key overlap/file) - No coroutines - ```multireadrandom : 831.774 micros/op 1202 ops/sec 60.001 seconds 72136 operations; 0.6 MB/s (72136 of 72136 found)``` Using coroutines - ```multireadrandom : 318.742 micros/op 3137 ops/sec 60.003 seconds 188248 operations; 1.6 MB/s (188248 of 188248 found)``` 2. Single thread CPU bound workload (all data cached) with ~1 key overlap/file - No coroutines - ```multireadrandom : 4.127 micros/op 242322 ops/sec 60.000 seconds 14539384 operations; 125.7 MB/s (14539384 of 14539384 found)``` Using coroutines - ```multireadrandom : 4.741 micros/op 210935 ops/sec 60.000 seconds 12656176 operations; 109.4 MB/s (12656176 of 12656176 found)``` 3. Single thread CPU bound workload with ~2 key overlap/file - No coroutines - ```multireadrandom : 3.717 micros/op 269000 ops/sec 60.000 seconds 16140024 operations; 139.6 MB/s (16140024 of 16140024 found)``` Using coroutines - ```multireadrandom : 4.146 micros/op 241204 ops/sec 60.000 seconds 14472296 operations; 125.1 MB/s (14472296 of 14472296 found)``` 4. CPU bound multi-threaded (16 threads) with ~4 key overlap/file - No coroutines - ```multireadrandom : 4.534 micros/op 3528792 ops/sec 60.000 seconds 211728728 operations; 1830.7 MB/s (12737024 of 12737024 found) ``` Using coroutines - ```multireadrandom : 4.872 micros/op 3283812 ops/sec 60.000 seconds 197030096 operations; 1703.6 MB/s (12548032 of 12548032 found) ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/9968 Reviewed By: akankshamahajan15 Differential Revision: D36348563 Pulled By: anand1976 fbshipit-source-id: c0ce85a505fd26ebfbb09786cbd7f25202038696main
parent
5be1579ead
commit
57997ddaaf
@ -0,0 +1,140 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
|
||||
#include "util/coro_utils.h" |
||||
|
||||
#if defined(WITHOUT_COROUTINES) || \ |
||||
(defined(USE_COROUTINES) && defined(WITH_COROUTINES)) |
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
#if defined(WITHOUT_COROUTINES) |
||||
#endif |
||||
|
||||
// Batched version of TableCache::MultiGet.
|
||||
DEFINE_SYNC_AND_ASYNC(Status, TableCache::MultiGet) |
||||
(const ReadOptions& options, const InternalKeyComparator& internal_comparator, |
||||
const FileMetaData& file_meta, const MultiGetContext::Range* mget_range, |
||||
const std::shared_ptr<const SliceTransform>& prefix_extractor, |
||||
HistogramImpl* file_read_hist, bool skip_filters, int level) { |
||||
auto& fd = file_meta.fd; |
||||
Status s; |
||||
TableReader* t = fd.table_reader; |
||||
Cache::Handle* handle = nullptr; |
||||
MultiGetRange table_range(*mget_range, mget_range->begin(), |
||||
mget_range->end()); |
||||
#ifndef ROCKSDB_LITE |
||||
autovector<std::string, MultiGetContext::MAX_BATCH_SIZE> row_cache_entries; |
||||
IterKey row_cache_key; |
||||
size_t row_cache_key_prefix_size = 0; |
||||
KeyContext& first_key = *table_range.begin(); |
||||
bool lookup_row_cache = |
||||
ioptions_.row_cache && !first_key.get_context->NeedToReadSequence(); |
||||
|
||||
// Check row cache if enabled. Since row cache does not currently store
|
||||
// sequence numbers, we cannot use it if we need to fetch the sequence.
|
||||
if (lookup_row_cache) { |
||||
GetContext* first_context = first_key.get_context; |
||||
CreateRowCacheKeyPrefix(options, fd, first_key.ikey, first_context, |
||||
row_cache_key); |
||||
row_cache_key_prefix_size = row_cache_key.Size(); |
||||
|
||||
for (auto miter = table_range.begin(); miter != table_range.end(); |
||||
++miter) { |
||||
const Slice& user_key = miter->ukey_with_ts; |
||||
|
||||
GetContext* get_context = miter->get_context; |
||||
|
||||
if (GetFromRowCache(user_key, row_cache_key, row_cache_key_prefix_size, |
||||
get_context)) { |
||||
table_range.SkipKey(miter); |
||||
} else { |
||||
row_cache_entries.emplace_back(); |
||||
get_context->SetReplayLog(&(row_cache_entries.back())); |
||||
} |
||||
} |
||||
} |
||||
#endif // ROCKSDB_LITE
|
||||
|
||||
// Check that table_range is not empty. Its possible all keys may have been
|
||||
// found in the row cache and thus the range may now be empty
|
||||
if (s.ok() && !table_range.empty()) { |
||||
if (t == nullptr) { |
||||
s = FindTable(options, file_options_, internal_comparator, fd, &handle, |
||||
prefix_extractor, |
||||
options.read_tier == kBlockCacheTier /* no_io */, |
||||
true /* record_read_stats */, file_read_hist, skip_filters, |
||||
level, true /* prefetch_index_and_filter_in_cache */, |
||||
0 /*max_file_size_for_l0_meta_pin*/, file_meta.temperature); |
||||
TEST_SYNC_POINT_CALLBACK("TableCache::MultiGet:FindTable", &s); |
||||
if (s.ok()) { |
||||
t = GetTableReaderFromHandle(handle); |
||||
assert(t); |
||||
} |
||||
} |
||||
if (s.ok() && !options.ignore_range_deletions) { |
||||
std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter( |
||||
t->NewRangeTombstoneIterator(options)); |
||||
if (range_del_iter != nullptr) { |
||||
for (auto iter = table_range.begin(); iter != table_range.end(); |
||||
++iter) { |
||||
SequenceNumber* max_covering_tombstone_seq = |
||||
iter->get_context->max_covering_tombstone_seq(); |
||||
*max_covering_tombstone_seq = std::max( |
||||
*max_covering_tombstone_seq, |
||||
range_del_iter->MaxCoveringTombstoneSeqnum(iter->ukey_with_ts)); |
||||
} |
||||
} |
||||
} |
||||
if (s.ok()) { |
||||
CO_AWAIT(t->MultiGet) |
||||
(options, &table_range, prefix_extractor.get(), skip_filters); |
||||
} else if (options.read_tier == kBlockCacheTier && s.IsIncomplete()) { |
||||
for (auto iter = table_range.begin(); iter != table_range.end(); ++iter) { |
||||
Status* status = iter->s; |
||||
if (status->IsIncomplete()) { |
||||
// Couldn't find Table in cache but treat as kFound if no_io set
|
||||
iter->get_context->MarkKeyMayExist(); |
||||
s = Status::OK(); |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
if (lookup_row_cache) { |
||||
size_t row_idx = 0; |
||||
|
||||
for (auto miter = table_range.begin(); miter != table_range.end(); |
||||
++miter) { |
||||
std::string& row_cache_entry = row_cache_entries[row_idx++]; |
||||
const Slice& user_key = miter->ukey_with_ts; |
||||
; |
||||
GetContext* get_context = miter->get_context; |
||||
|
||||
get_context->SetReplayLog(nullptr); |
||||
// Compute row cache key.
|
||||
row_cache_key.TrimAppend(row_cache_key_prefix_size, user_key.data(), |
||||
user_key.size()); |
||||
// Put the replay log in row cache only if something was found.
|
||||
if (s.ok() && !row_cache_entry.empty()) { |
||||
size_t charge = row_cache_entry.capacity() + sizeof(std::string); |
||||
void* row_ptr = new std::string(std::move(row_cache_entry)); |
||||
// If row cache is full, it's OK.
|
||||
ioptions_.row_cache |
||||
->Insert(row_cache_key.GetUserKey(), row_ptr, charge, |
||||
&DeleteEntry<std::string>) |
||||
.PermitUncheckedError(); |
||||
} |
||||
} |
||||
} |
||||
#endif // ROCKSDB_LITE
|
||||
|
||||
if (handle != nullptr) { |
||||
ReleaseHandle(handle); |
||||
} |
||||
CO_RETURN s; |
||||
} |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif |
@ -0,0 +1,154 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
|
||||
#include "util/coro_utils.h" |
||||
|
||||
#if defined(WITHOUT_COROUTINES) || \ |
||||
(defined(USE_COROUTINES) && defined(WITH_COROUTINES)) |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
// Lookup a batch of keys in a single SST file
|
||||
DEFINE_SYNC_AND_ASYNC(Status, Version::MultiGetFromSST) |
||||
(const ReadOptions& read_options, MultiGetRange file_range, int hit_file_level, |
||||
bool is_hit_file_last_in_level, FdWithKeyRange* f, |
||||
std::unordered_map<uint64_t, BlobReadRequests>& blob_rqs, |
||||
uint64_t& num_filter_read, uint64_t& num_index_read, uint64_t& num_data_read, |
||||
uint64_t& num_sst_read) { |
||||
bool timer_enabled = GetPerfLevel() >= PerfLevel::kEnableTimeExceptForMutex && |
||||
get_perf_context()->per_level_perf_context_enabled; |
||||
|
||||
Status s; |
||||
StopWatchNano timer(clock_, timer_enabled /* auto_start */); |
||||
s = CO_AWAIT(table_cache_->MultiGet)( |
||||
read_options, *internal_comparator(), *f->file_metadata, &file_range, |
||||
mutable_cf_options_.prefix_extractor, |
||||
cfd_->internal_stats()->GetFileReadHist(hit_file_level), |
||||
IsFilterSkipped(static_cast<int>(hit_file_level), |
||||
is_hit_file_last_in_level), |
||||
hit_file_level); |
||||
// TODO: examine the behavior for corrupted key
|
||||
if (timer_enabled) { |
||||
PERF_COUNTER_BY_LEVEL_ADD(get_from_table_nanos, timer.ElapsedNanos(), |
||||
hit_file_level); |
||||
} |
||||
if (!s.ok()) { |
||||
// TODO: Set status for individual keys appropriately
|
||||
for (auto iter = file_range.begin(); iter != file_range.end(); ++iter) { |
||||
*iter->s = s; |
||||
file_range.MarkKeyDone(iter); |
||||
} |
||||
CO_RETURN s; |
||||
} |
||||
uint64_t batch_size = 0; |
||||
for (auto iter = file_range.begin(); s.ok() && iter != file_range.end(); |
||||
++iter) { |
||||
GetContext& get_context = *iter->get_context; |
||||
Status* status = iter->s; |
||||
// The Status in the KeyContext takes precedence over GetContext state
|
||||
// Status may be an error if there were any IO errors in the table
|
||||
// reader. We never expect Status to be NotFound(), as that is
|
||||
// determined by get_context
|
||||
assert(!status->IsNotFound()); |
||||
if (!status->ok()) { |
||||
file_range.MarkKeyDone(iter); |
||||
continue; |
||||
} |
||||
|
||||
if (get_context.sample()) { |
||||
sample_file_read_inc(f->file_metadata); |
||||
} |
||||
batch_size++; |
||||
num_index_read += get_context.get_context_stats_.num_index_read; |
||||
num_filter_read += get_context.get_context_stats_.num_filter_read; |
||||
num_data_read += get_context.get_context_stats_.num_data_read; |
||||
num_sst_read += get_context.get_context_stats_.num_sst_read; |
||||
// Reset these stats since they're specific to a level
|
||||
get_context.get_context_stats_.num_index_read = 0; |
||||
get_context.get_context_stats_.num_filter_read = 0; |
||||
get_context.get_context_stats_.num_data_read = 0; |
||||
get_context.get_context_stats_.num_sst_read = 0; |
||||
|
||||
// report the counters before returning
|
||||
if (get_context.State() != GetContext::kNotFound && |
||||
get_context.State() != GetContext::kMerge && |
||||
db_statistics_ != nullptr) { |
||||
get_context.ReportCounters(); |
||||
} else { |
||||
if (iter->max_covering_tombstone_seq > 0) { |
||||
// The remaining files we look at will only contain covered keys, so
|
||||
// we stop here for this key
|
||||
file_range.SkipKey(iter); |
||||
} |
||||
} |
||||
switch (get_context.State()) { |
||||
case GetContext::kNotFound: |
||||
// Keep searching in other files
|
||||
break; |
||||
case GetContext::kMerge: |
||||
// TODO: update per-level perfcontext user_key_return_count for kMerge
|
||||
break; |
||||
case GetContext::kFound: |
||||
if (hit_file_level == 0) { |
||||
RecordTick(db_statistics_, GET_HIT_L0); |
||||
} else if (hit_file_level == 1) { |
||||
RecordTick(db_statistics_, GET_HIT_L1); |
||||
} else if (hit_file_level >= 2) { |
||||
RecordTick(db_statistics_, GET_HIT_L2_AND_UP); |
||||
} |
||||
|
||||
PERF_COUNTER_BY_LEVEL_ADD(user_key_return_count, 1, hit_file_level); |
||||
|
||||
file_range.MarkKeyDone(iter); |
||||
|
||||
if (iter->is_blob_index) { |
||||
if (iter->value) { |
||||
TEST_SYNC_POINT_CALLBACK("Version::MultiGet::TamperWithBlobIndex", |
||||
&(*iter)); |
||||
|
||||
const Slice& blob_index_slice = *(iter->value); |
||||
BlobIndex blob_index; |
||||
Status tmp_s = blob_index.DecodeFrom(blob_index_slice); |
||||
if (tmp_s.ok()) { |
||||
const uint64_t blob_file_num = blob_index.file_number(); |
||||
blob_rqs[blob_file_num].emplace_back( |
||||
std::make_pair(blob_index, std::cref(*iter))); |
||||
} else { |
||||
*(iter->s) = tmp_s; |
||||
} |
||||
} |
||||
} else { |
||||
file_range.AddValueSize(iter->value->size()); |
||||
if (file_range.GetValueSize() > read_options.value_size_soft_limit) { |
||||
s = Status::Aborted(); |
||||
break; |
||||
} |
||||
} |
||||
continue; |
||||
case GetContext::kDeleted: |
||||
// Use empty error message for speed
|
||||
*status = Status::NotFound(); |
||||
file_range.MarkKeyDone(iter); |
||||
continue; |
||||
case GetContext::kCorrupt: |
||||
*status = |
||||
Status::Corruption("corrupted key for ", iter->lkey->user_key()); |
||||
file_range.MarkKeyDone(iter); |
||||
continue; |
||||
case GetContext::kUnexpectedBlobIndex: |
||||
ROCKS_LOG_ERROR(info_log_, "Encounter unexpected blob index."); |
||||
*status = Status::NotSupported( |
||||
"Encounter unexpected blob index. Please open DB with " |
||||
"ROCKSDB_NAMESPACE::blob_db::BlobDB instead."); |
||||
file_range.MarkKeyDone(iter); |
||||
continue; |
||||
} |
||||
} |
||||
|
||||
RecordInHistogram(db_statistics_, SST_BATCH_SIZE, batch_size); |
||||
CO_RETURN s; |
||||
} |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif |
@ -0,0 +1,748 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
|
||||
#include "util/async_file_reader.h" |
||||
#include "util/coro_utils.h" |
||||
|
||||
#if defined(WITHOUT_COROUTINES) || \ |
||||
(defined(USE_COROUTINES) && defined(WITH_COROUTINES)) |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
// This function reads multiple data blocks from disk using Env::MultiRead()
|
||||
// and optionally inserts them into the block cache. It uses the scratch
|
||||
// buffer provided by the caller, which is contiguous. If scratch is a nullptr
|
||||
// it allocates a separate buffer for each block. Typically, if the blocks
|
||||
// need to be uncompressed and there is no compressed block cache, callers
|
||||
// can allocate a temporary scratch buffer in order to minimize memory
|
||||
// allocations.
|
||||
// If options.fill_cache is true, it inserts the blocks into cache. If its
|
||||
// false and scratch is non-null and the blocks are uncompressed, it copies
|
||||
// the buffers to heap. In any case, the CachableEntry<Block> returned will
|
||||
// own the data bytes.
|
||||
// If compression is enabled and also there is no compressed block cache,
|
||||
// the adjacent blocks are read out in one IO (combined read)
|
||||
// batch - A MultiGetRange with only those keys with unique data blocks not
|
||||
// found in cache
|
||||
// handles - A vector of block handles. Some of them me be NULL handles
|
||||
// scratch - An optional contiguous buffer to read compressed blocks into
|
||||
DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) |
||||
(const ReadOptions& options, const MultiGetRange* batch, |
||||
const autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE>* handles, |
||||
autovector<Status, MultiGetContext::MAX_BATCH_SIZE>* statuses, |
||||
autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>* results, |
||||
char* scratch, const UncompressionDict& uncompression_dict) const { |
||||
RandomAccessFileReader* file = rep_->file.get(); |
||||
const Footer& footer = rep_->footer; |
||||
const ImmutableOptions& ioptions = rep_->ioptions; |
||||
size_t read_amp_bytes_per_bit = rep_->table_options.read_amp_bytes_per_bit; |
||||
MemoryAllocator* memory_allocator = GetMemoryAllocator(rep_->table_options); |
||||
|
||||
if (ioptions.allow_mmap_reads) { |
||||
size_t idx_in_batch = 0; |
||||
for (auto mget_iter = batch->begin(); mget_iter != batch->end(); |
||||
++mget_iter, ++idx_in_batch) { |
||||
BlockCacheLookupContext lookup_data_block_context( |
||||
TableReaderCaller::kUserMultiGet); |
||||
const BlockHandle& handle = (*handles)[idx_in_batch]; |
||||
if (handle.IsNull()) { |
||||
continue; |
||||
} |
||||
|
||||
(*statuses)[idx_in_batch] = |
||||
RetrieveBlock(nullptr, options, handle, uncompression_dict, |
||||
&(*results)[idx_in_batch], BlockType::kData, |
||||
mget_iter->get_context, &lookup_data_block_context, |
||||
/* for_compaction */ false, /* use_cache */ true, |
||||
/* wait_for_cache */ true); |
||||
} |
||||
CO_RETURN; |
||||
} |
||||
|
||||
// In direct IO mode, blocks share the direct io buffer.
|
||||
// Otherwise, blocks share the scratch buffer.
|
||||
const bool use_shared_buffer = file->use_direct_io() || scratch != nullptr; |
||||
|
||||
autovector<FSReadRequest, MultiGetContext::MAX_BATCH_SIZE> read_reqs; |
||||
size_t buf_offset = 0; |
||||
size_t idx_in_batch = 0; |
||||
|
||||
uint64_t prev_offset = 0; |
||||
size_t prev_len = 0; |
||||
autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_idx_for_block; |
||||
autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_offset_for_block; |
||||
for (auto mget_iter = batch->begin(); mget_iter != batch->end(); |
||||
++mget_iter, ++idx_in_batch) { |
||||
const BlockHandle& handle = (*handles)[idx_in_batch]; |
||||
if (handle.IsNull()) { |
||||
continue; |
||||
} |
||||
|
||||
size_t prev_end = static_cast<size_t>(prev_offset) + prev_len; |
||||
|
||||
// If current block is adjacent to the previous one, at the same time,
|
||||
// compression is enabled and there is no compressed cache, we combine
|
||||
// the two block read as one.
|
||||
// We don't combine block reads here in direct IO mode, because when doing
|
||||
// direct IO read, the block requests will be realigned and merged when
|
||||
// necessary.
|
||||
if (use_shared_buffer && !file->use_direct_io() && |
||||
prev_end == handle.offset()) { |
||||
req_offset_for_block.emplace_back(prev_len); |
||||
prev_len += BlockSizeWithTrailer(handle); |
||||
} else { |
||||
// No compression or current block and previous one is not adjacent:
|
||||
// Step 1, create a new request for previous blocks
|
||||
if (prev_len != 0) { |
||||
FSReadRequest req; |
||||
req.offset = prev_offset; |
||||
req.len = prev_len; |
||||
if (file->use_direct_io()) { |
||||
req.scratch = nullptr; |
||||
} else if (use_shared_buffer) { |
||||
req.scratch = scratch + buf_offset; |
||||
buf_offset += req.len; |
||||
} else { |
||||
req.scratch = new char[req.len]; |
||||
} |
||||
read_reqs.emplace_back(req); |
||||
} |
||||
|
||||
// Step 2, remeber the previous block info
|
||||
prev_offset = handle.offset(); |
||||
prev_len = BlockSizeWithTrailer(handle); |
||||
req_offset_for_block.emplace_back(0); |
||||
} |
||||
req_idx_for_block.emplace_back(read_reqs.size()); |
||||
|
||||
PERF_COUNTER_ADD(block_read_count, 1); |
||||
PERF_COUNTER_ADD(block_read_byte, BlockSizeWithTrailer(handle)); |
||||
} |
||||
// Handle the last block and process the pending last request
|
||||
if (prev_len != 0) { |
||||
FSReadRequest req; |
||||
req.offset = prev_offset; |
||||
req.len = prev_len; |
||||
if (file->use_direct_io()) { |
||||
req.scratch = nullptr; |
||||
} else if (use_shared_buffer) { |
||||
req.scratch = scratch + buf_offset; |
||||
} else { |
||||
req.scratch = new char[req.len]; |
||||
} |
||||
read_reqs.emplace_back(req); |
||||
} |
||||
|
||||
AlignedBuf direct_io_buf; |
||||
{ |
||||
IOOptions opts; |
||||
IOStatus s = file->PrepareIOOptions(options, opts); |
||||
if (s.ok()) { |
||||
#if defined(WITHOUT_COROUTINES) |
||||
s = file->MultiRead(opts, &read_reqs[0], read_reqs.size(), &direct_io_buf, |
||||
options.rate_limiter_priority); |
||||
#else // WITH_COROUTINES
|
||||
co_await batch->context()->reader().MultiReadAsync( |
||||
file, opts, &read_reqs[0], read_reqs.size(), &direct_io_buf); |
||||
#endif // WITH_COROUTINES
|
||||
} |
||||
if (!s.ok()) { |
||||
// Discard all the results in this batch if there is any time out
|
||||
// or overall MultiRead error
|
||||
for (FSReadRequest& req : read_reqs) { |
||||
req.status = s; |
||||
} |
||||
} |
||||
} |
||||
|
||||
idx_in_batch = 0; |
||||
size_t valid_batch_idx = 0; |
||||
for (auto mget_iter = batch->begin(); mget_iter != batch->end(); |
||||
++mget_iter, ++idx_in_batch) { |
||||
const BlockHandle& handle = (*handles)[idx_in_batch]; |
||||
|
||||
if (handle.IsNull()) { |
||||
continue; |
||||
} |
||||
|
||||
assert(valid_batch_idx < req_idx_for_block.size()); |
||||
assert(valid_batch_idx < req_offset_for_block.size()); |
||||
assert(req_idx_for_block[valid_batch_idx] < read_reqs.size()); |
||||
size_t& req_idx = req_idx_for_block[valid_batch_idx]; |
||||
size_t& req_offset = req_offset_for_block[valid_batch_idx]; |
||||
valid_batch_idx++; |
||||
if (mget_iter->get_context) { |
||||
++(mget_iter->get_context->get_context_stats_.num_data_read); |
||||
} |
||||
FSReadRequest& req = read_reqs[req_idx]; |
||||
Status s = req.status; |
||||
if (s.ok()) { |
||||
if ((req.result.size() != req.len) || |
||||
(req_offset + BlockSizeWithTrailer(handle) > req.result.size())) { |
||||
s = Status::Corruption("truncated block read from " + |
||||
rep_->file->file_name() + " offset " + |
||||
std::to_string(handle.offset()) + ", expected " + |
||||
std::to_string(req.len) + " bytes, got " + |
||||
std::to_string(req.result.size())); |
||||
} |
||||
} |
||||
|
||||
BlockContents raw_block_contents; |
||||
if (s.ok()) { |
||||
if (!use_shared_buffer) { |
||||
// We allocated a buffer for this block. Give ownership of it to
|
||||
// BlockContents so it can free the memory
|
||||
assert(req.result.data() == req.scratch); |
||||
assert(req.result.size() == BlockSizeWithTrailer(handle)); |
||||
assert(req_offset == 0); |
||||
std::unique_ptr<char[]> raw_block(req.scratch); |
||||
raw_block_contents = BlockContents(std::move(raw_block), handle.size()); |
||||
} else { |
||||
// We used the scratch buffer or direct io buffer
|
||||
// which are shared by the blocks.
|
||||
// raw_block_contents does not have the ownership.
|
||||
raw_block_contents = |
||||
BlockContents(Slice(req.result.data() + req_offset, handle.size())); |
||||
} |
||||
#ifndef NDEBUG |
||||
raw_block_contents.is_raw_block = true; |
||||
#endif |
||||
|
||||
if (options.verify_checksums) { |
||||
PERF_TIMER_GUARD(block_checksum_time); |
||||
const char* data = req.result.data(); |
||||
// Since the scratch might be shared, the offset of the data block in
|
||||
// the buffer might not be 0. req.result.data() only point to the
|
||||
// begin address of each read request, we need to add the offset
|
||||
// in each read request. Checksum is stored in the block trailer,
|
||||
// beyond the payload size.
|
||||
s = VerifyBlockChecksum(footer.checksum_type(), data + req_offset, |
||||
handle.size(), rep_->file->file_name(), |
||||
handle.offset()); |
||||
TEST_SYNC_POINT_CALLBACK("RetrieveMultipleBlocks:VerifyChecksum", &s); |
||||
} |
||||
} else if (!use_shared_buffer) { |
||||
// Free the allocated scratch buffer.
|
||||
delete[] req.scratch; |
||||
} |
||||
|
||||
if (s.ok()) { |
||||
// When the blocks share the same underlying buffer (scratch or direct io
|
||||
// buffer), we may need to manually copy the block into heap if the raw
|
||||
// block has to be inserted into a cache. That falls into th following
|
||||
// cases -
|
||||
// 1. Raw block is not compressed, it needs to be inserted into the
|
||||
// uncompressed block cache if there is one
|
||||
// 2. If the raw block is compressed, it needs to be inserted into the
|
||||
// compressed block cache if there is one
|
||||
//
|
||||
// In all other cases, the raw block is either uncompressed into a heap
|
||||
// buffer or there is no cache at all.
|
||||
CompressionType compression_type = |
||||
GetBlockCompressionType(raw_block_contents); |
||||
if (use_shared_buffer && (compression_type == kNoCompression || |
||||
(compression_type != kNoCompression && |
||||
rep_->table_options.block_cache_compressed))) { |
||||
Slice raw = |
||||
Slice(req.result.data() + req_offset, BlockSizeWithTrailer(handle)); |
||||
raw_block_contents = BlockContents( |
||||
CopyBufferToHeap(GetMemoryAllocator(rep_->table_options), raw), |
||||
handle.size()); |
||||
#ifndef NDEBUG |
||||
raw_block_contents.is_raw_block = true; |
||||
#endif |
||||
} |
||||
} |
||||
|
||||
if (s.ok()) { |
||||
if (options.fill_cache) { |
||||
BlockCacheLookupContext lookup_data_block_context( |
||||
TableReaderCaller::kUserMultiGet); |
||||
CachableEntry<Block>* block_entry = &(*results)[idx_in_batch]; |
||||
// MaybeReadBlockAndLoadToCache will insert into the block caches if
|
||||
// necessary. Since we're passing the raw block contents, it will
|
||||
// avoid looking up the block cache
|
||||
s = MaybeReadBlockAndLoadToCache( |
||||
nullptr, options, handle, uncompression_dict, /*wait=*/true, |
||||
/*for_compaction=*/false, block_entry, BlockType::kData, |
||||
mget_iter->get_context, &lookup_data_block_context, |
||||
&raw_block_contents); |
||||
|
||||
// block_entry value could be null if no block cache is present, i.e
|
||||
// BlockBasedTableOptions::no_block_cache is true and no compressed
|
||||
// block cache is configured. In that case, fall
|
||||
// through and set up the block explicitly
|
||||
if (block_entry->GetValue() != nullptr) { |
||||
s.PermitUncheckedError(); |
||||
continue; |
||||
} |
||||
} |
||||
|
||||
CompressionType compression_type = |
||||
GetBlockCompressionType(raw_block_contents); |
||||
BlockContents contents; |
||||
if (compression_type != kNoCompression) { |
||||
UncompressionContext context(compression_type); |
||||
UncompressionInfo info(context, uncompression_dict, compression_type); |
||||
s = UncompressBlockContents( |
||||
info, req.result.data() + req_offset, handle.size(), &contents, |
||||
footer.format_version(), rep_->ioptions, memory_allocator); |
||||
} else { |
||||
// There are two cases here:
|
||||
// 1) caller uses the shared buffer (scratch or direct io buffer);
|
||||
// 2) we use the requst buffer.
|
||||
// If scratch buffer or direct io buffer is used, we ensure that
|
||||
// all raw blocks are copyed to the heap as single blocks. If scratch
|
||||
// buffer is not used, we also have no combined read, so the raw
|
||||
// block can be used directly.
|
||||
contents = std::move(raw_block_contents); |
||||
} |
||||
if (s.ok()) { |
||||
(*results)[idx_in_batch].SetOwnedValue(new Block( |
||||
std::move(contents), read_amp_bytes_per_bit, ioptions.stats)); |
||||
} |
||||
} |
||||
(*statuses)[idx_in_batch] = s; |
||||
} |
||||
} |
||||
|
||||
using MultiGetRange = MultiGetContext::Range; |
||||
DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) |
||||
(const ReadOptions& read_options, const MultiGetRange* mget_range, |
||||
const SliceTransform* prefix_extractor, bool skip_filters) { |
||||
if (mget_range->empty()) { |
||||
// Caller should ensure non-empty (performance bug)
|
||||
assert(false); |
||||
CO_RETURN; // Nothing to do
|
||||
} |
||||
|
||||
FilterBlockReader* const filter = |
||||
!skip_filters ? rep_->filter.get() : nullptr; |
||||
MultiGetRange sst_file_range(*mget_range, mget_range->begin(), |
||||
mget_range->end()); |
||||
|
||||
// First check the full filter
|
||||
// If full filter not useful, Then go into each block
|
||||
const bool no_io = read_options.read_tier == kBlockCacheTier; |
||||
uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId; |
||||
if (sst_file_range.begin()->get_context) { |
||||
tracing_mget_id = sst_file_range.begin()->get_context->get_tracing_get_id(); |
||||
} |
||||
BlockCacheLookupContext lookup_context{ |
||||
TableReaderCaller::kUserMultiGet, tracing_mget_id, |
||||
/*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; |
||||
FullFilterKeysMayMatch(filter, &sst_file_range, no_io, prefix_extractor, |
||||
&lookup_context); |
||||
|
||||
if (!sst_file_range.empty()) { |
||||
IndexBlockIter iiter_on_stack; |
||||
// if prefix_extractor found in block differs from options, disable
|
||||
// BlockPrefixIndex. Only do this check when index_type is kHashSearch.
|
||||
bool need_upper_bound_check = false; |
||||
if (rep_->index_type == BlockBasedTableOptions::kHashSearch) { |
||||
need_upper_bound_check = PrefixExtractorChanged(prefix_extractor); |
||||
} |
||||
auto iiter = |
||||
NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack, |
||||
sst_file_range.begin()->get_context, &lookup_context); |
||||
std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; |
||||
if (iiter != &iiter_on_stack) { |
||||
iiter_unique_ptr.reset(iiter); |
||||
} |
||||
|
||||
uint64_t prev_offset = std::numeric_limits<uint64_t>::max(); |
||||
autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE> block_handles; |
||||
autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE> results; |
||||
autovector<Status, MultiGetContext::MAX_BATCH_SIZE> statuses; |
||||
MultiGetContext::Mask reused_mask = 0; |
||||
char stack_buf[kMultiGetReadStackBufSize]; |
||||
std::unique_ptr<char[]> block_buf; |
||||
{ |
||||
MultiGetRange data_block_range(sst_file_range, sst_file_range.begin(), |
||||
sst_file_range.end()); |
||||
std::vector<Cache::Handle*> cache_handles; |
||||
bool wait_for_cache_results = false; |
||||
|
||||
CachableEntry<UncompressionDict> uncompression_dict; |
||||
Status uncompression_dict_status; |
||||
uncompression_dict_status.PermitUncheckedError(); |
||||
bool uncompression_dict_inited = false; |
||||
size_t total_len = 0; |
||||
ReadOptions ro = read_options; |
||||
ro.read_tier = kBlockCacheTier; |
||||
|
||||
for (auto miter = data_block_range.begin(); |
||||
miter != data_block_range.end(); ++miter) { |
||||
const Slice& key = miter->ikey; |
||||
iiter->Seek(miter->ikey); |
||||
|
||||
IndexValue v; |
||||
if (iiter->Valid()) { |
||||
v = iiter->value(); |
||||
} |
||||
if (!iiter->Valid() || |
||||
(!v.first_internal_key.empty() && !skip_filters && |
||||
UserComparatorWrapper(rep_->internal_comparator.user_comparator()) |
||||
.CompareWithoutTimestamp( |
||||
ExtractUserKey(key), |
||||
ExtractUserKey(v.first_internal_key)) < 0)) { |
||||
// The requested key falls between highest key in previous block and
|
||||
// lowest key in current block.
|
||||
if (!iiter->status().IsNotFound()) { |
||||
*(miter->s) = iiter->status(); |
||||
} |
||||
data_block_range.SkipKey(miter); |
||||
sst_file_range.SkipKey(miter); |
||||
continue; |
||||
} |
||||
|
||||
if (!uncompression_dict_inited && rep_->uncompression_dict_reader) { |
||||
uncompression_dict_status = |
||||
rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( |
||||
nullptr /* prefetch_buffer */, no_io, |
||||
read_options.verify_checksums, |
||||
sst_file_range.begin()->get_context, &lookup_context, |
||||
&uncompression_dict); |
||||
uncompression_dict_inited = true; |
||||
} |
||||
|
||||
if (!uncompression_dict_status.ok()) { |
||||
assert(!uncompression_dict_status.IsNotFound()); |
||||
*(miter->s) = uncompression_dict_status; |
||||
data_block_range.SkipKey(miter); |
||||
sst_file_range.SkipKey(miter); |
||||
continue; |
||||
} |
||||
|
||||
statuses.emplace_back(); |
||||
results.emplace_back(); |
||||
if (v.handle.offset() == prev_offset) { |
||||
// This key can reuse the previous block (later on).
|
||||
// Mark previous as "reused"
|
||||
reused_mask |= MultiGetContext::Mask{1} << (block_handles.size() - 1); |
||||
// Use null handle to indicate this one reuses same block as
|
||||
// previous.
|
||||
block_handles.emplace_back(BlockHandle::NullBlockHandle()); |
||||
continue; |
||||
} |
||||
// Lookup the cache for the given data block referenced by an index
|
||||
// iterator value (i.e BlockHandle). If it exists in the cache,
|
||||
// initialize block to the contents of the data block.
|
||||
prev_offset = v.handle.offset(); |
||||
BlockHandle handle = v.handle; |
||||
BlockCacheLookupContext lookup_data_block_context( |
||||
TableReaderCaller::kUserMultiGet); |
||||
const UncompressionDict& dict = uncompression_dict.GetValue() |
||||
? *uncompression_dict.GetValue() |
||||
: UncompressionDict::GetEmptyDict(); |
||||
Status s = RetrieveBlock( |
||||
nullptr, ro, handle, dict, &(results.back()), BlockType::kData, |
||||
miter->get_context, &lookup_data_block_context, |
||||
/* for_compaction */ false, /* use_cache */ true, |
||||
/* wait_for_cache */ false); |
||||
if (s.IsIncomplete()) { |
||||
s = Status::OK(); |
||||
} |
||||
if (s.ok() && !results.back().IsEmpty()) { |
||||
// Since we have a valid handle, check the value. If its nullptr,
|
||||
// it means the cache is waiting for the final result and we're
|
||||
// supposed to call WaitAll() to wait for the result.
|
||||
if (results.back().GetValue() != nullptr) { |
||||
// Found it in the cache. Add NULL handle to indicate there is
|
||||
// nothing to read from disk.
|
||||
if (results.back().GetCacheHandle()) { |
||||
results.back().UpdateCachedValue(); |
||||
} |
||||
block_handles.emplace_back(BlockHandle::NullBlockHandle()); |
||||
} else { |
||||
// We have to wait for the cache lookup to finish in the
|
||||
// background, and then we may have to read the block from disk
|
||||
// anyway
|
||||
assert(results.back().GetCacheHandle()); |
||||
wait_for_cache_results = true; |
||||
block_handles.emplace_back(handle); |
||||
cache_handles.emplace_back(results.back().GetCacheHandle()); |
||||
} |
||||
} else { |
||||
block_handles.emplace_back(handle); |
||||
total_len += BlockSizeWithTrailer(handle); |
||||
} |
||||
} |
||||
|
||||
if (wait_for_cache_results) { |
||||
Cache* block_cache = rep_->table_options.block_cache.get(); |
||||
block_cache->WaitAll(cache_handles); |
||||
for (size_t i = 0; i < block_handles.size(); ++i) { |
||||
// If this block was a success or failure or not needed because
|
||||
// the corresponding key is in the same block as a prior key, skip
|
||||
if (block_handles[i] == BlockHandle::NullBlockHandle() || |
||||
results[i].IsEmpty()) { |
||||
continue; |
||||
} |
||||
results[i].UpdateCachedValue(); |
||||
void* val = results[i].GetValue(); |
||||
if (!val) { |
||||
// The async cache lookup failed - could be due to an error
|
||||
// or a false positive. We need to read the data block from
|
||||
// the SST file
|
||||
results[i].Reset(); |
||||
total_len += BlockSizeWithTrailer(block_handles[i]); |
||||
} else { |
||||
block_handles[i] = BlockHandle::NullBlockHandle(); |
||||
} |
||||
} |
||||
} |
||||
|
||||
if (total_len) { |
||||
char* scratch = nullptr; |
||||
const UncompressionDict& dict = uncompression_dict.GetValue() |
||||
? *uncompression_dict.GetValue() |
||||
: UncompressionDict::GetEmptyDict(); |
||||
assert(uncompression_dict_inited || !rep_->uncompression_dict_reader); |
||||
assert(uncompression_dict_status.ok()); |
||||
// If using direct IO, then scratch is not used, so keep it nullptr.
|
||||
// If the blocks need to be uncompressed and we don't need the
|
||||
// compressed blocks, then we can use a contiguous block of
|
||||
// memory to read in all the blocks as it will be temporary
|
||||
// storage
|
||||
// 1. If blocks are compressed and compressed block cache is there,
|
||||
// alloc heap bufs
|
||||
// 2. If blocks are uncompressed, alloc heap bufs
|
||||
// 3. If blocks are compressed and no compressed block cache, use
|
||||
// stack buf
|
||||
if (!rep_->file->use_direct_io() && |
||||
rep_->table_options.block_cache_compressed == nullptr && |
||||
rep_->blocks_maybe_compressed) { |
||||
if (total_len <= kMultiGetReadStackBufSize) { |
||||
scratch = stack_buf; |
||||
} else { |
||||
scratch = new char[total_len]; |
||||
block_buf.reset(scratch); |
||||
} |
||||
} |
||||
CO_AWAIT(RetrieveMultipleBlocks) |
||||
(read_options, &data_block_range, &block_handles, &statuses, &results, |
||||
scratch, dict); |
||||
if (sst_file_range.begin()->get_context) { |
||||
++(sst_file_range.begin() |
||||
->get_context->get_context_stats_.num_sst_read); |
||||
} |
||||
} |
||||
} |
||||
|
||||
DataBlockIter first_biter; |
||||
DataBlockIter next_biter; |
||||
size_t idx_in_batch = 0; |
||||
SharedCleanablePtr shared_cleanable; |
||||
for (auto miter = sst_file_range.begin(); miter != sst_file_range.end(); |
||||
++miter) { |
||||
Status s; |
||||
GetContext* get_context = miter->get_context; |
||||
const Slice& key = miter->ikey; |
||||
bool matched = false; // if such user key matched a key in SST
|
||||
bool done = false; |
||||
bool first_block = true; |
||||
do { |
||||
DataBlockIter* biter = nullptr; |
||||
bool reusing_prev_block; |
||||
bool later_reused; |
||||
uint64_t referenced_data_size = 0; |
||||
bool does_referenced_key_exist = false; |
||||
BlockCacheLookupContext lookup_data_block_context( |
||||
TableReaderCaller::kUserMultiGet, tracing_mget_id, |
||||
/*_get_from_user_specified_snapshot=*/read_options.snapshot != |
||||
nullptr); |
||||
if (first_block) { |
||||
if (!block_handles[idx_in_batch].IsNull() || |
||||
!results[idx_in_batch].IsEmpty()) { |
||||
first_biter.Invalidate(Status::OK()); |
||||
NewDataBlockIterator<DataBlockIter>( |
||||
read_options, results[idx_in_batch], &first_biter, |
||||
statuses[idx_in_batch]); |
||||
reusing_prev_block = false; |
||||
} else { |
||||
// If handler is null and result is empty, then the status is never
|
||||
// set, which should be the initial value: ok().
|
||||
assert(statuses[idx_in_batch].ok()); |
||||
reusing_prev_block = true; |
||||
} |
||||
biter = &first_biter; |
||||
later_reused = |
||||
(reused_mask & (MultiGetContext::Mask{1} << idx_in_batch)) != 0; |
||||
idx_in_batch++; |
||||
} else { |
||||
IndexValue v = iiter->value(); |
||||
if (!v.first_internal_key.empty() && !skip_filters && |
||||
UserComparatorWrapper(rep_->internal_comparator.user_comparator()) |
||||
.CompareWithoutTimestamp( |
||||
ExtractUserKey(key), |
||||
ExtractUserKey(v.first_internal_key)) < 0) { |
||||
// The requested key falls between highest key in previous block and
|
||||
// lowest key in current block.
|
||||
break; |
||||
} |
||||
|
||||
next_biter.Invalidate(Status::OK()); |
||||
NewDataBlockIterator<DataBlockIter>( |
||||
read_options, iiter->value().handle, &next_biter, |
||||
BlockType::kData, get_context, &lookup_data_block_context, |
||||
Status(), nullptr); |
||||
biter = &next_biter; |
||||
reusing_prev_block = false; |
||||
later_reused = false; |
||||
} |
||||
|
||||
if (read_options.read_tier == kBlockCacheTier && |
||||
biter->status().IsIncomplete()) { |
||||
// couldn't get block from block_cache
|
||||
// Update Saver.state to Found because we are only looking for
|
||||
// whether we can guarantee the key is not there when "no_io" is set
|
||||
get_context->MarkKeyMayExist(); |
||||
break; |
||||
} |
||||
if (!biter->status().ok()) { |
||||
s = biter->status(); |
||||
break; |
||||
} |
||||
|
||||
bool may_exist = biter->SeekForGet(key); |
||||
if (!may_exist) { |
||||
// HashSeek cannot find the key this block and the the iter is not
|
||||
// the end of the block, i.e. cannot be in the following blocks
|
||||
// either. In this case, the seek_key cannot be found, so we break
|
||||
// from the top level for-loop.
|
||||
break; |
||||
} |
||||
|
||||
// Reusing blocks complicates pinning/Cleanable, because the cache
|
||||
// entry referenced by biter can only be released once all returned
|
||||
// pinned values are released. This code previously did an extra
|
||||
// block_cache Ref for each reuse, but that unnecessarily increases
|
||||
// block cache contention. Instead we can use a variant of shared_ptr
|
||||
// to release in block cache only once.
|
||||
//
|
||||
// Although the biter loop below might SaveValue multiple times for
|
||||
// merges, just one value_pinner suffices, as MultiGet will merge
|
||||
// the operands before returning to the API user.
|
||||
Cleanable* value_pinner; |
||||
if (biter->IsValuePinned()) { |
||||
if (reusing_prev_block) { |
||||
// Note that we don't yet know if the MultiGet results will need
|
||||
// to pin this block, so we might wrap a block for sharing and
|
||||
// still end up with 1 (or 0) pinning ref. Not ideal but OK.
|
||||
//
|
||||
// Here we avoid adding redundant cleanups if we didn't end up
|
||||
// delegating the cleanup from last time around.
|
||||
if (!biter->HasCleanups()) { |
||||
assert(shared_cleanable.get()); |
||||
if (later_reused) { |
||||
shared_cleanable.RegisterCopyWith(biter); |
||||
} else { |
||||
shared_cleanable.MoveAsCleanupTo(biter); |
||||
} |
||||
} |
||||
} else if (later_reused) { |
||||
assert(biter->HasCleanups()); |
||||
// Make the existing cleanups on `biter` sharable:
|
||||
shared_cleanable.Allocate(); |
||||
// Move existing `biter` cleanup(s) to `shared_cleanable`
|
||||
biter->DelegateCleanupsTo(&*shared_cleanable); |
||||
// Reference `shared_cleanable` as new cleanup for `biter`
|
||||
shared_cleanable.RegisterCopyWith(biter); |
||||
} |
||||
assert(biter->HasCleanups()); |
||||
value_pinner = biter; |
||||
} else { |
||||
value_pinner = nullptr; |
||||
} |
||||
|
||||
// Call the *saver function on each entry/block until it returns false
|
||||
for (; biter->Valid(); biter->Next()) { |
||||
ParsedInternalKey parsed_key; |
||||
Status pik_status = ParseInternalKey( |
||||
biter->key(), &parsed_key, false /* log_err_key */); // TODO
|
||||
if (!pik_status.ok()) { |
||||
s = pik_status; |
||||
} |
||||
if (!get_context->SaveValue(parsed_key, biter->value(), &matched, |
||||
value_pinner)) { |
||||
if (get_context->State() == GetContext::GetState::kFound) { |
||||
does_referenced_key_exist = true; |
||||
referenced_data_size = |
||||
biter->key().size() + biter->value().size(); |
||||
} |
||||
done = true; |
||||
break; |
||||
} |
||||
s = biter->status(); |
||||
} |
||||
// Write the block cache access.
|
||||
// XXX: There appear to be 'break' statements above that bypass this
|
||||
// writing of the block cache trace record
|
||||
if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && |
||||
!reusing_prev_block) { |
||||
// Avoid making copy of block_key, cf_name, and referenced_key when
|
||||
// constructing the access record.
|
||||
Slice referenced_key; |
||||
if (does_referenced_key_exist) { |
||||
referenced_key = biter->key(); |
||||
} else { |
||||
referenced_key = key; |
||||
} |
||||
BlockCacheTraceRecord access_record( |
||||
rep_->ioptions.clock->NowMicros(), |
||||
/*_block_key=*/"", lookup_data_block_context.block_type, |
||||
lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), |
||||
/*_cf_name=*/"", rep_->level_for_tracing(), |
||||
rep_->sst_number_for_tracing(), lookup_data_block_context.caller, |
||||
lookup_data_block_context.is_cache_hit, |
||||
lookup_data_block_context.no_insert, |
||||
lookup_data_block_context.get_id, |
||||
lookup_data_block_context.get_from_user_specified_snapshot, |
||||
/*_referenced_key=*/"", referenced_data_size, |
||||
lookup_data_block_context.num_keys_in_block, |
||||
does_referenced_key_exist); |
||||
// TODO: Should handle status here?
|
||||
block_cache_tracer_ |
||||
->WriteBlockAccess(access_record, |
||||
lookup_data_block_context.block_key, |
||||
rep_->cf_name_for_tracing(), referenced_key) |
||||
.PermitUncheckedError(); |
||||
} |
||||
s = biter->status(); |
||||
if (done) { |
||||
// Avoid the extra Next which is expensive in two-level indexes
|
||||
break; |
||||
} |
||||
if (first_block) { |
||||
iiter->Seek(key); |
||||
if (!iiter->Valid()) { |
||||
break; |
||||
} |
||||
} |
||||
first_block = false; |
||||
iiter->Next(); |
||||
} while (iiter->Valid()); |
||||
|
||||
if (matched && filter != nullptr && !filter->IsBlockBased()) { |
||||
RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE); |
||||
PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1, |
||||
rep_->level); |
||||
} |
||||
if (s.ok() && !iiter->status().IsNotFound()) { |
||||
s = iiter->status(); |
||||
} |
||||
*(miter->s) = s; |
||||
} |
||||
#ifdef ROCKSDB_ASSERT_STATUS_CHECKED |
||||
// Not sure why we need to do it. Should investigate more.
|
||||
for (auto& st : statuses) { |
||||
st.PermitUncheckedError(); |
||||
} |
||||
#endif // ROCKSDB_ASSERT_STATUS_CHECKED
|
||||
} |
||||
} |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif |
@ -0,0 +1,72 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
//
|
||||
#if USE_COROUTINES |
||||
#include "util/async_file_reader.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
bool AsyncFileReader::MultiReadAsyncImpl(ReadAwaiter* awaiter) { |
||||
if (tail_) { |
||||
tail_->next_ = awaiter; |
||||
} |
||||
tail_ = awaiter; |
||||
if (!head_) { |
||||
head_ = awaiter; |
||||
} |
||||
num_reqs_ += awaiter->num_reqs_; |
||||
awaiter->io_handle_.resize(awaiter->num_reqs_); |
||||
awaiter->del_fn_.resize(awaiter->num_reqs_); |
||||
for (size_t i = 0; i < awaiter->num_reqs_; ++i) { |
||||
awaiter->file_ |
||||
->ReadAsync( |
||||
awaiter->read_reqs_[i], awaiter->opts_, |
||||
[](const FSReadRequest& req, void* cb_arg) { |
||||
FSReadRequest* read_req = static_cast<FSReadRequest*>(cb_arg); |
||||
read_req->status = req.status; |
||||
read_req->result = req.result; |
||||
}, |
||||
&awaiter->read_reqs_[i], &awaiter->io_handle_[i], |
||||
&awaiter->del_fn_[i], Env::IOPriority::IO_TOTAL) |
||||
.PermitUncheckedError(); |
||||
} |
||||
return true; |
||||
} |
||||
|
||||
void AsyncFileReader::Wait() { |
||||
if (!head_) { |
||||
return; |
||||
} |
||||
ReadAwaiter* waiter; |
||||
std::vector<void*> io_handles; |
||||
io_handles.reserve(num_reqs_); |
||||
waiter = head_; |
||||
do { |
||||
for (size_t i = 0; i < waiter->num_reqs_; ++i) { |
||||
if (waiter->io_handle_[i]) { |
||||
io_handles.push_back(waiter->io_handle_[i]); |
||||
} |
||||
} |
||||
} while (waiter != tail_ && (waiter = waiter->next_)); |
||||
if (io_handles.size() > 0) { |
||||
StopWatch sw(SystemClock::Default().get(), stats_, POLL_WAIT_MICROS); |
||||
fs_->Poll(io_handles, io_handles.size()).PermitUncheckedError(); |
||||
} |
||||
do { |
||||
waiter = head_; |
||||
head_ = waiter->next_; |
||||
|
||||
for (size_t i = 0; i < waiter->num_reqs_; ++i) { |
||||
if (waiter->io_handle_[i] && waiter->del_fn_[i]) { |
||||
waiter->del_fn_[i](waiter->io_handle_[i]); |
||||
} |
||||
} |
||||
waiter->awaiting_coro_.resume(); |
||||
} while (waiter != tail_); |
||||
head_ = tail_ = nullptr; |
||||
RecordInHistogram(stats_, MULTIGET_IO_BATCH_SIZE, num_reqs_); |
||||
num_reqs_ = 0; |
||||
} |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // USE_COROUTINES
|
@ -0,0 +1,143 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
//
|
||||
#pragma once |
||||
|
||||
#if USE_COROUTINES |
||||
#include "file/random_access_file_reader.h" |
||||
#include "folly/experimental/coro/ViaIfAsync.h" |
||||
#include "port/port.h" |
||||
#include "rocksdb/file_system.h" |
||||
#include "rocksdb/statistics.h" |
||||
#include "util/autovector.h" |
||||
#include "util/stop_watch.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
class SingleThreadExecutor; |
||||
|
||||
// AsyncFileReader implements the Awaitable concept, which allows calling
|
||||
// coroutines to co_await it. When the AsyncFileReader Awaitable is
|
||||
// resumed, it initiates the fie reads requested by the awaiting caller
|
||||
// by calling RandomAccessFileReader's ReadAsync. It then suspends the
|
||||
// awaiting coroutine. The suspended awaiter is later resumed by Wait().
|
||||
class AsyncFileReader { |
||||
class ReadAwaiter; |
||||
template <typename Awaiter> |
||||
class ReadOperation; |
||||
|
||||
public: |
||||
AsyncFileReader(FileSystem* fs, Statistics* stats) : fs_(fs), stats_(stats) {} |
||||
|
||||
~AsyncFileReader() {} |
||||
|
||||
ReadOperation<ReadAwaiter> MultiReadAsync(RandomAccessFileReader* file, |
||||
const IOOptions& opts, |
||||
FSReadRequest* read_reqs, |
||||
size_t num_reqs, |
||||
AlignedBuf* aligned_buf) noexcept { |
||||
return ReadOperation<ReadAwaiter>{*this, file, opts, |
||||
read_reqs, num_reqs, aligned_buf}; |
||||
} |
||||
|
||||
private: |
||||
friend SingleThreadExecutor; |
||||
|
||||
// Implementation of the Awaitable concept
|
||||
class ReadAwaiter { |
||||
public: |
||||
explicit ReadAwaiter(AsyncFileReader& reader, RandomAccessFileReader* file, |
||||
const IOOptions& opts, FSReadRequest* read_reqs, |
||||
size_t num_reqs, AlignedBuf* /*aligned_buf*/) noexcept |
||||
: reader_(reader), |
||||
file_(file), |
||||
opts_(opts), |
||||
read_reqs_(read_reqs), |
||||
num_reqs_(num_reqs) {} |
||||
|
||||
bool await_ready() noexcept { return false; } |
||||
|
||||
// A return value of true means suspend the awaiter (calling coroutine). The
|
||||
// awaiting_coro parameter is the handle of the awaiter. The handle can be
|
||||
// resumed later, so we cache it here.
|
||||
bool await_suspend( |
||||
std::experimental::coroutine_handle<> awaiting_coro) noexcept { |
||||
awaiting_coro_ = awaiting_coro; |
||||
// MultiReadAsyncImpl always returns true, so caller will be suspended
|
||||
return reader_.MultiReadAsyncImpl(this); |
||||
} |
||||
|
||||
void await_resume() noexcept {} |
||||
|
||||
private: |
||||
friend AsyncFileReader; |
||||
|
||||
// The parameters passed to MultiReadAsync are cached here when the caller
|
||||
// calls MultiReadAsync. Later, when the execution of this awaitable is
|
||||
// started, these are used to do the actual IO
|
||||
AsyncFileReader& reader_; |
||||
RandomAccessFileReader* file_; |
||||
const IOOptions& opts_; |
||||
FSReadRequest* read_reqs_; |
||||
size_t num_reqs_; |
||||
autovector<void*, 32> io_handle_; |
||||
autovector<IOHandleDeleter, 32> del_fn_; |
||||
std::experimental::coroutine_handle<> awaiting_coro_; |
||||
// Use this to link to the next ReadAwaiter in the suspended coroutine
|
||||
// list. The head and tail of the list are tracked by AsyncFileReader.
|
||||
// We use this approach rather than an STL container in order to avoid
|
||||
// extra memory allocations. The coroutine call already allocates a
|
||||
// ReadAwaiter object.
|
||||
ReadAwaiter* next_; |
||||
}; |
||||
|
||||
// An instance of ReadOperation is returned to the caller of MultiGetAsync.
|
||||
// This represents an awaitable that can be started later.
|
||||
template <typename Awaiter> |
||||
class ReadOperation { |
||||
public: |
||||
explicit ReadOperation(AsyncFileReader& reader, |
||||
RandomAccessFileReader* file, const IOOptions& opts, |
||||
FSReadRequest* read_reqs, size_t num_reqs, |
||||
AlignedBuf* aligned_buf) noexcept |
||||
: reader_(reader), |
||||
file_(file), |
||||
opts_(opts), |
||||
read_reqs_(read_reqs), |
||||
num_reqs_(num_reqs), |
||||
aligned_buf_(aligned_buf) {} |
||||
|
||||
auto viaIfAsync(folly::Executor::KeepAlive<> executor) const { |
||||
return folly::coro::co_viaIfAsync( |
||||
std::move(executor), |
||||
Awaiter{reader_, file_, opts_, read_reqs_, num_reqs_, aligned_buf_}); |
||||
} |
||||
|
||||
private: |
||||
AsyncFileReader& reader_; |
||||
RandomAccessFileReader* file_; |
||||
const IOOptions& opts_; |
||||
FSReadRequest* read_reqs_; |
||||
size_t num_reqs_; |
||||
AlignedBuf* aligned_buf_; |
||||
}; |
||||
|
||||
// This function does the actual work when this awaitable starts execution
|
||||
bool MultiReadAsyncImpl(ReadAwaiter* awaiter); |
||||
|
||||
// Called by the SingleThreadExecutor to poll for async IO completion.
|
||||
// This also resumes the awaiting coroutines.
|
||||
void Wait(); |
||||
|
||||
// Head of the queue of awaiters waiting for async IO completion
|
||||
ReadAwaiter* head_ = nullptr; |
||||
// Tail of the awaiter queue
|
||||
ReadAwaiter* tail_ = nullptr; |
||||
// Total number of pending async IOs
|
||||
size_t num_reqs_ = 0; |
||||
FileSystem* fs_; |
||||
Statistics* stats_; |
||||
}; |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // USE_COROUTINES
|
@ -0,0 +1,111 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
|
||||
#if defined(USE_COROUTINES) |
||||
#include "folly/experimental/coro/Coroutine.h" |
||||
#include "folly/experimental/coro/Task.h" |
||||
#endif |
||||
#include "rocksdb/rocksdb_namespace.h" |
||||
|
||||
// This file has two sctions. The first section applies to all instances of
|
||||
// header file inclusion and has an include guard. The second section is
|
||||
// meant for multiple inclusions in the same source file, and is idempotent.
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
|
||||
#ifndef UTIL_CORO_UTILS_H_ |
||||
#define UTIL_CORO_UTILS_H_ |
||||
|
||||
#if defined(USE_COROUTINES) |
||||
|
||||
// The follwoing macros expand to regular and coroutine function
|
||||
// declarations for a given function
|
||||
#define DECLARE_SYNC_AND_ASYNC(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__); \
|
||||
folly::coro::Task<__ret_type__> __func_name__##Coroutine(__VA_ARGS__); |
||||
|
||||
#define DECLARE_SYNC_AND_ASYNC_OVERRIDE(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__) override; \
|
||||
folly::coro::Task<__ret_type__> __func_name__##Coroutine(__VA_ARGS__) \
|
||||
override; |
||||
|
||||
#define DECLARE_SYNC_AND_ASYNC_CONST(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__) const; \
|
||||
folly::coro::Task<__ret_type__> __func_name__##Coroutine(__VA_ARGS__) const; |
||||
|
||||
constexpr bool using_coroutines() { return true; } |
||||
#else // !USE_COROUTINES
|
||||
|
||||
// The follwoing macros expand to a regular function declaration for a given
|
||||
// function
|
||||
#define DECLARE_SYNC_AND_ASYNC(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__); |
||||
|
||||
#define DECLARE_SYNC_AND_ASYNC_OVERRIDE(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__) override; |
||||
|
||||
#define DECLARE_SYNC_AND_ASYNC_CONST(__ret_type__, __func_name__, ...) \ |
||||
__ret_type__ __func_name__(__VA_ARGS__) const; |
||||
|
||||
constexpr bool using_coroutines() { return false; } |
||||
#endif // USE_COROUTINES
|
||||
#endif // UTIL_CORO_UTILS_H_
|
||||
|
||||
// The following section of the file is meant to be included twice in a
|
||||
// source file - once defining WITH_COROUTINES and once defining
|
||||
// WITHOUT_COROUTINES
|
||||
#undef DEFINE_SYNC_AND_ASYNC |
||||
#undef CO_AWAIT |
||||
#undef CO_RETURN |
||||
|
||||
#if defined(WITH_COROUTINES) && defined(USE_COROUTINES) |
||||
|
||||
// This macro should be used in the beginning of the function
|
||||
// definition. The declaration should have been done using one of the
|
||||
// DECLARE_SYNC_AND_ASYNC* macros. It expands to the return type and
|
||||
// the function name with the Coroutine suffix. For example -
|
||||
// DEFINE_SYNC_AND_ASYNC(int, foo)(bool bar) {}
|
||||
// would expand to -
|
||||
// folly::coro::Task<int> fooCoroutine(bool bar) {}
|
||||
#define DEFINE_SYNC_AND_ASYNC(__ret_type__, __func_name__) \ |
||||
folly::coro::Task<__ret_type__> __func_name__##Coroutine |
||||
|
||||
// This macro should be used to call a function that might be a
|
||||
// coroutine. It expands to the correct function name and prefixes
|
||||
// the co_await operator if necessary. For example -
|
||||
// s = CO_AWAIT(foo)(true);
|
||||
// if the code is compiled WITH_COROUTINES, would expand to
|
||||
// s = co_await fooCoroutine(true);
|
||||
// if compiled WITHOUT_COROUTINES, would expand to
|
||||
// s = foo(true);
|
||||
#define CO_AWAIT(__func_name__) co_await __func_name__##Coroutine |
||||
|
||||
#define CO_RETURN co_return |
||||
|
||||
#elif defined(WITHOUT_COROUTINES) |
||||
|
||||
// This macro should be used in the beginning of the function
|
||||
// definition. The declaration should have been done using one of the
|
||||
// DECLARE_SYNC_AND_ASYNC* macros. It expands to the return type and
|
||||
// the function name without the Coroutine suffix. For example -
|
||||
// DEFINE_SYNC_AND_ASYNC(int, foo)(bool bar) {}
|
||||
// would expand to -
|
||||
// int foo(bool bar) {}
|
||||
#define DEFINE_SYNC_AND_ASYNC(__ret_type__, __func_name__) \ |
||||
__ret_type__ __func_name__ |
||||
|
||||
// This macro should be used to call a function that might be a
|
||||
// coroutine. It expands to the correct function name and prefixes
|
||||
// the co_await operator if necessary. For example -
|
||||
// s = CO_AWAIT(foo)(true);
|
||||
// if the code is compiled WITH_COROUTINES, would expand to
|
||||
// s = co_await fooCoroutine(true);
|
||||
// if compiled WITHOUT_COROUTINES, would expand to
|
||||
// s = foo(true);
|
||||
#define CO_AWAIT(__func_name__) __func_name__ |
||||
|
||||
#define CO_RETURN return |
||||
|
||||
#endif // DO_NOT_USE_COROUTINES
|
||||
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,55 @@ |
||||
// Copyright (c) Meta Platforms, Inc. and its affiliates. 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).
|
||||
//
|
||||
#pragma once |
||||
|
||||
#if USE_COROUTINES |
||||
#include <atomic> |
||||
|
||||
#include "folly/CPortability.h" |
||||
#include "folly/CppAttributes.h" |
||||
#include "folly/Executor.h" |
||||
#include "util/async_file_reader.h" |
||||
|
||||
namespace ROCKSDB_NAMESPACE { |
||||
// Implements a simple executor that runs callback functions in the same
|
||||
// thread, unlike CPUThreadExecutor which may schedule the callback on
|
||||
// another thread. Runs in a tight loop calling the queued callbacks,
|
||||
// and polls for async IO completions when idle. The completions will
|
||||
// resume suspended coroutines and they get added to the queue, which
|
||||
// will get picked up by this loop.
|
||||
// Any possibility of deadlock is precluded because the file system
|
||||
// guarantees that async IO completion callbacks will not be scheduled
|
||||
// to run in this thread or this executor.
|
||||
class SingleThreadExecutor : public folly::Executor { |
||||
public: |
||||
explicit SingleThreadExecutor(AsyncFileReader& reader) |
||||
: reader_(reader), busy_(false) {} |
||||
|
||||
void add(folly::Func callback) override { |
||||
auto& q = q_; |
||||
q.push(std::move(callback)); |
||||
if (q.size() == 1 && !busy_) { |
||||
while (!q.empty()) { |
||||
q.front()(); |
||||
q.pop(); |
||||
|
||||
if (q.empty()) { |
||||
// Prevent recursion, as the Wait may queue resumed coroutines
|
||||
busy_ = true; |
||||
reader_.Wait(); |
||||
busy_ = false; |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
private: |
||||
std::queue<folly::Func> q_; |
||||
AsyncFileReader& reader_; |
||||
bool busy_; |
||||
}; |
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
#endif // USE_COROUTINES
|
Loading…
Reference in new issue