@ -160,6 +160,13 @@ bool PrefixExtractorChanged(const TableProperties* table_properties,
}
}
}
}
CacheAllocationPtr CopyBufferToHeap ( MemoryAllocator * allocator , Slice & buf ) {
CacheAllocationPtr heap_buf ;
heap_buf = AllocateBlock ( buf . size ( ) , allocator ) ;
memcpy ( heap_buf . get ( ) , buf . data ( ) , buf . size ( ) ) ;
return heap_buf ;
}
} // namespace
} // namespace
// Encapsulates common functionality for the various index reader
// Encapsulates common functionality for the various index reader
@ -421,7 +428,8 @@ class PartitionIndexReader : public BlockBasedTable::IndexReaderCommon {
// filter blocks
// filter blocks
s = table ( ) - > MaybeReadBlockAndLoadToCache (
s = table ( ) - > MaybeReadBlockAndLoadToCache (
prefetch_buffer . get ( ) , ro , handle , UncompressionDict : : GetEmptyDict ( ) ,
prefetch_buffer . get ( ) , ro , handle , UncompressionDict : : GetEmptyDict ( ) ,
& block , BlockType : : kIndex , /*get_context=*/ nullptr , & lookup_context ) ;
& block , BlockType : : kIndex , /*get_context=*/ nullptr , & lookup_context ,
/*contents=*/ nullptr ) ;
assert ( s . ok ( ) | | block . GetValue ( ) = = nullptr ) ;
assert ( s . ok ( ) | | block . GetValue ( ) = = nullptr ) ;
if ( s . ok ( ) & & block . GetValue ( ) ! = nullptr ) {
if ( s . ok ( ) & & block . GetValue ( ) ! = nullptr ) {
@ -1745,8 +1753,6 @@ Status BlockBasedTable::PutDataBlockToCache(
: Cache : : Priority : : LOW ;
: Cache : : Priority : : LOW ;
assert ( cached_block ) ;
assert ( cached_block ) ;
assert ( cached_block - > IsEmpty ( ) ) ;
assert ( cached_block - > IsEmpty ( ) ) ;
assert ( raw_block_comp_type = = kNoCompression | |
block_cache_compressed ! = nullptr ) ;
Status s ;
Status s ;
Statistics * statistics = ioptions . statistics ;
Statistics * statistics = ioptions . statistics ;
@ -2195,11 +2201,105 @@ IndexBlockIter* BlockBasedTable::InitBlockIterator<IndexBlockIter>(
rep - > index_value_is_full , block_contents_pinned ) ;
rep - > index_value_is_full , block_contents_pinned ) ;
}
}
// Convert an uncompressed data block (i.e CachableEntry<Block>)
// into an iterator over the contents of the corresponding block.
// If input_iter is null, new a iterator
// If input_iter is not null, update this iter and return it
template < typename TBlockIter >
TBlockIter * BlockBasedTable : : NewDataBlockIterator (
const ReadOptions & ro , CachableEntry < Block > & block , TBlockIter * input_iter ,
Status s ) const {
PERF_TIMER_GUARD ( new_table_block_iter_nanos ) ;
TBlockIter * iter = input_iter ! = nullptr ? input_iter : new TBlockIter ;
if ( ! s . ok ( ) ) {
iter - > Invalidate ( s ) ;
return iter ;
}
assert ( block . GetValue ( ) ! = nullptr ) ;
// Block contents are pinned and it is still pinned after the iterator
// is destroyed as long as cleanup functions are moved to another object,
// when:
// 1. block cache handle is set to be released in cleanup function, or
// 2. it's pointing to immortal source. If own_bytes is true then we are
// not reading data from the original source, whether immortal or not.
// Otherwise, the block is pinned iff the source is immortal.
const bool block_contents_pinned =
block . IsCached ( ) | |
( ! block . GetValue ( ) - > own_bytes ( ) & & rep_ - > immortal_table ) ;
iter = InitBlockIterator < TBlockIter > ( rep_ , block . GetValue ( ) , iter ,
block_contents_pinned ) ;
if ( ! block . IsCached ( ) ) {
if ( ! ro . fill_cache & & rep_ - > cache_key_prefix_size ! = 0 ) {
// insert a dummy record to block cache to track the memory usage
Cache * const block_cache = rep_ - > table_options . block_cache . get ( ) ;
Cache : : Handle * cache_handle = nullptr ;
// There are two other types of cache keys: 1) SST cache key added in
// `MaybeReadBlockAndLoadToCache` 2) dummy cache key added in
// `write_buffer_manager`. Use longer prefix (41 bytes) to differentiate
// from SST cache key(31 bytes), and use non-zero prefix to
// differentiate from `write_buffer_manager`
const size_t kExtraCacheKeyPrefix = kMaxVarint64Length * 4 + 1 ;
char cache_key [ kExtraCacheKeyPrefix + kMaxVarint64Length ] ;
// Prefix: use rep_->cache_key_prefix padded by 0s
memset ( cache_key , 0 , kExtraCacheKeyPrefix + kMaxVarint64Length ) ;
assert ( rep_ - > cache_key_prefix_size ! = 0 ) ;
assert ( rep_ - > cache_key_prefix_size < = kExtraCacheKeyPrefix ) ;
memcpy ( cache_key , rep_ - > cache_key_prefix , rep_ - > cache_key_prefix_size ) ;
char * end = EncodeVarint64 ( cache_key + kExtraCacheKeyPrefix ,
next_cache_key_id_ + + ) ;
assert ( end - cache_key < =
static_cast < int > ( kExtraCacheKeyPrefix + kMaxVarint64Length ) ) ;
const Slice unique_key ( cache_key , static_cast < size_t > ( end - cache_key ) ) ;
s = block_cache - > Insert ( unique_key , nullptr ,
block . GetValue ( ) - > ApproximateMemoryUsage ( ) ,
nullptr , & cache_handle ) ;
if ( s . ok ( ) ) {
assert ( cache_handle ! = nullptr ) ;
iter - > RegisterCleanup ( & ForceReleaseCachedEntry , block_cache ,
cache_handle ) ;
}
}
} else {
iter - > SetCacheHandle ( block . GetCacheHandle ( ) ) ;
}
block . TransferTo ( iter ) ;
return iter ;
}
// 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.
Status BlockBasedTable : : GetDataBlockFromCache (
const ReadOptions & ro , const BlockHandle & handle ,
const UncompressionDict & uncompression_dict ,
CachableEntry < Block > * block , BlockType block_type ,
GetContext * get_context ) const {
BlockCacheLookupContext lookup_data_block_context (
TableReaderCaller : : kUserMultiGet ) ;
Status s = RetrieveBlock ( nullptr , ro , handle , uncompression_dict , block ,
block_type , get_context , & lookup_data_block_context ) ;
if ( s . IsIncomplete ( ) ) {
s = Status : : OK ( ) ;
}
return s ;
}
// If contents is nullptr, this function looks up the block caches for the
// data block referenced by handle, and read the block from disk if necessary.
// If contents is non-null, it skips the cache lookup and disk read, since
// the caller has already read it. In both cases, if ro.fill_cache is true,
// it inserts the block into the block cache.
Status BlockBasedTable : : MaybeReadBlockAndLoadToCache (
Status BlockBasedTable : : MaybeReadBlockAndLoadToCache (
FilePrefetchBuffer * prefetch_buffer , const ReadOptions & ro ,
FilePrefetchBuffer * prefetch_buffer , const ReadOptions & ro ,
const BlockHandle & handle , const UncompressionDict & uncompression_dict ,
const BlockHandle & handle , const UncompressionDict & uncompression_dict ,
CachableEntry < Block > * block_entry , BlockType block_type ,
CachableEntry < Block > * block_entry , BlockType block_type ,
GetContext * get_context , BlockCacheLookupContext * lookup_context ) const {
GetContext * get_context , BlockCacheLookupContext * lookup_context ,
BlockContents * contents ) const {
assert ( block_entry ! = nullptr ) ;
assert ( block_entry ! = nullptr ) ;
const bool no_io = ( ro . read_tier = = kBlockCacheTier ) ;
const bool no_io = ( ro . read_tier = = kBlockCacheTier ) ;
Cache * block_cache = rep_ - > table_options . block_cache . get ( ) ;
Cache * block_cache = rep_ - > table_options . block_cache . get ( ) ;
@ -2231,14 +2331,17 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
compressed_cache_key ) ;
compressed_cache_key ) ;
}
}
s = GetDataBlockFromCache ( key , ckey , block_cache , block_cache_compressed ,
if ( ! contents ) {
ro , block_entry , uncompression_dict , block_type ,
s = GetDataBlockFromCache ( key , ckey , block_cache , block_cache_compressed ,
get_context ) ;
ro , block_entry , uncompression_dict , block_type ,
if ( block_entry - > GetValue ( ) ) {
get_context ) ;
// TODO(haoyu): Differentiate cache hit on uncompressed block cache and
if ( block_entry - > GetValue ( ) ) {
// compressed block cache.
// TODO(haoyu): Differentiate cache hit on uncompressed block cache and
is_cache_hit = true ;
// compressed block cache.
is_cache_hit = true ;
}
}
}
// Can't find the block from the cache. If I/O is allowed, read from the
// Can't find the block from the cache. If I/O is allowed, read from the
// file.
// file.
if ( block_entry - > GetValue ( ) = = nullptr & & ! no_io & & ro . fill_cache ) {
if ( block_entry - > GetValue ( ) = = nullptr & & ! no_io & & ro . fill_cache ) {
@ -2248,7 +2351,7 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
block_cache_compressed = = nullptr & & rep_ - > blocks_maybe_compressed ;
block_cache_compressed = = nullptr & & rep_ - > blocks_maybe_compressed ;
CompressionType raw_block_comp_type ;
CompressionType raw_block_comp_type ;
BlockContents raw_block_contents ;
BlockContents raw_block_contents ;
{
if ( ! contents ) {
StopWatch sw ( rep_ - > ioptions . env , statistics , READ_BLOCK_GET_MICROS ) ;
StopWatch sw ( rep_ - > ioptions . env , statistics , READ_BLOCK_GET_MICROS ) ;
BlockFetcher block_fetcher (
BlockFetcher block_fetcher (
rep_ - > file . get ( ) , prefetch_buffer , rep_ - > footer , ro , handle ,
rep_ - > file . get ( ) , prefetch_buffer , rep_ - > footer , ro , handle ,
@ -2259,6 +2362,9 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
GetMemoryAllocatorForCompressedBlock ( rep_ - > table_options ) ) ;
GetMemoryAllocatorForCompressedBlock ( rep_ - > table_options ) ) ;
s = block_fetcher . ReadBlockContents ( ) ;
s = block_fetcher . ReadBlockContents ( ) ;
raw_block_comp_type = block_fetcher . get_compression_type ( ) ;
raw_block_comp_type = block_fetcher . get_compression_type ( ) ;
contents = & raw_block_contents ;
} else {
raw_block_comp_type = contents - > get_compression_type ( ) ;
}
}
if ( s . ok ( ) ) {
if ( s . ok ( ) ) {
@ -2266,7 +2372,7 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
// If filling cache is allowed and a cache is configured, try to put the
// If filling cache is allowed and a cache is configured, try to put the
// block to the cache.
// block to the cache.
s = PutDataBlockToCache ( key , ckey , block_cache , block_cache_compressed ,
s = PutDataBlockToCache ( key , ckey , block_cache , block_cache_compressed ,
block_entry , & raw_block_ contents,
block_entry , contents ,
raw_block_comp_type , uncompression_dict , seq_no ,
raw_block_comp_type , uncompression_dict , seq_no ,
GetMemoryAllocator ( rep_ - > table_options ) ,
GetMemoryAllocator ( rep_ - > table_options ) ,
block_type , get_context ) ;
block_type , get_context ) ;
@ -2331,6 +2437,172 @@ Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
return s ;
return s ;
}
}
// 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.
// 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
void BlockBasedTable : : MaybeLoadBlocksToCache (
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 ImmutableCFOptions & ioptions = rep_ - > ioptions ;
SequenceNumber global_seqno = rep_ - > get_global_seqno ( BlockType : : kData ) ;
size_t read_amp_bytes_per_bit = rep_ - > table_options . read_amp_bytes_per_bit ;
MemoryAllocator * memory_allocator = GetMemoryAllocator ( rep_ - > table_options ) ;
if ( file - > use_direct_io ( ) | | 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 ) ;
}
return ;
}
autovector < ReadRequest , MultiGetContext : : MAX_BATCH_SIZE > read_reqs ;
size_t buf_offset = 0 ;
size_t idx_in_batch = 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 ;
}
ReadRequest req ;
req . len = handle . size ( ) + kBlockTrailerSize ;
if ( scratch = = nullptr ) {
req . scratch = new char [ req . len ] ;
} else {
req . scratch = scratch + buf_offset ;
buf_offset + = req . len ;
}
req . offset = handle . offset ( ) ;
req . status = Status : : OK ( ) ;
read_reqs . emplace_back ( req ) ;
}
file - > MultiRead ( & read_reqs [ 0 ] , read_reqs . size ( ) ) ;
size_t read_req_idx = 0 ;
idx_in_batch = 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 ;
}
ReadRequest & req = read_reqs [ read_req_idx + + ] ;
Status s = req . status ;
if ( s . ok ( ) ) {
if ( req . result . size ( ) ! = handle . size ( ) + kBlockTrailerSize ) {
s = Status : : Corruption ( " truncated block read from " +
rep_ - > file - > file_name ( ) + " offset " +
ToString ( handle . offset ( ) ) + " , expected " +
ToString ( handle . size ( ) + kBlockTrailerSize ) +
" bytes, got " + ToString ( req . result . size ( ) ) ) ;
}
}
BlockContents raw_block_contents ;
if ( s . ok ( ) ) {
if ( scratch = = nullptr ) {
// 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 ) ;
std : : unique_ptr < char [ ] > raw_block ( req . scratch ) ;
raw_block_contents = BlockContents ( std : : move ( raw_block ) ,
handle . size ( ) ) ;
} else {
// We used the scratch buffer, so no need to free anything
raw_block_contents = BlockContents ( Slice ( req . scratch ,
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 ( ) ;
uint32_t expected = DecodeFixed32 ( data + handle . size ( ) + 1 ) ;
s = rocksdb : : VerifyChecksum ( footer . checksum ( ) , req . result . data ( ) ,
handle . size ( ) + 1 , expected ) ;
}
}
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 , block_entry , BlockType : : kData ,
mget_iter - > get_context , & lookup_data_block_context ,
& raw_block_contents ) ;
} else {
CompressionType compression_type =
raw_block_contents . get_compression_type ( ) ;
BlockContents contents ;
if ( compression_type ! = kNoCompression ) {
UncompressionContext context ( compression_type ) ;
UncompressionInfo info ( context , uncompression_dict , compression_type ) ;
s = UncompressBlockContents ( info , req . result . data ( ) , handle . size ( ) ,
& contents , footer . version ( ) , rep_ - > ioptions ,
memory_allocator ) ;
} else {
if ( scratch ! = nullptr ) {
// If we used the scratch buffer, then the contents need to be
// copied to heap
Slice raw = Slice ( req . result . data ( ) , handle . size ( ) ) ;
contents = BlockContents ( CopyBufferToHeap (
GetMemoryAllocator ( rep_ - > table_options ) , raw ) ,
handle . size ( ) ) ;
} else {
contents = std : : move ( raw_block_contents ) ;
}
}
if ( s . ok ( ) ) {
( * results ) [ idx_in_batch ] . SetOwnedValue ( new Block ( std : : move ( contents ) ,
global_seqno , read_amp_bytes_per_bit , ioptions . statistics ) ) ;
}
}
}
( * statuses ) [ idx_in_batch ] = s ;
}
}
Status BlockBasedTable : : RetrieveBlock (
Status BlockBasedTable : : RetrieveBlock (
FilePrefetchBuffer * prefetch_buffer , const ReadOptions & ro ,
FilePrefetchBuffer * prefetch_buffer , const ReadOptions & ro ,
const BlockHandle & handle , const UncompressionDict & uncompression_dict ,
const BlockHandle & handle , const UncompressionDict & uncompression_dict ,
@ -2347,7 +2619,8 @@ Status BlockBasedTable::RetrieveBlock(
block_type ! = BlockType : : kIndex ) ) {
block_type ! = BlockType : : kIndex ) ) {
s = MaybeReadBlockAndLoadToCache ( prefetch_buffer , ro , handle ,
s = MaybeReadBlockAndLoadToCache ( prefetch_buffer , ro , handle ,
uncompression_dict , block_entry ,
uncompression_dict , block_entry ,
block_type , get_context , lookup_context ) ;
block_type , get_context , lookup_context ,
/*contents=*/ nullptr ) ;
if ( ! s . ok ( ) ) {
if ( ! s . ok ( ) ) {
return s ;
return s ;
@ -3248,8 +3521,101 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
iiter_unique_ptr . reset ( iiter ) ;
iiter_unique_ptr . reset ( iiter ) ;
}
}
DataBlockIter biter ;
uint64_t offset = std : : numeric_limits < uint64_t > : : max ( ) ;
uint64_t 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 ;
static const size_t kMultiGetReadStackBufSize = 8192 ;
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 ( ) ) ;
BlockCacheLookupContext lookup_compression_dict_context (
TableReaderCaller : : kUserMultiGet ) ;
auto uncompression_dict_storage = GetUncompressionDict ( nullptr , no_io ,
sst_file_range . begin ( ) - > get_context ,
& lookup_compression_dict_context ) ;
const UncompressionDict & uncompression_dict =
uncompression_dict_storage . GetValue ( ) = = nullptr
? UncompressionDict : : GetEmptyDict ( )
: * uncompression_dict_storage . GetValue ( ) ;
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 ( ) )
. Compare ( ExtractUserKey ( key ) ,
ExtractUserKey ( v . first_internal_key ) ) < 0 ) ) {
// The requested key falls between highest key in previous block and
// lowest key in current block.
* ( miter - > s ) = iiter - > status ( ) ;
data_block_range . SkipKey ( miter ) ;
sst_file_range . SkipKey ( miter ) ;
continue ;
}
statuses . emplace_back ( ) ;
results . emplace_back ( ) ;
if ( v . handle . offset ( ) = = offset ) {
// We're going to reuse the block for this key later on. No need to
// look it up now. Place a null handle
block_handles . emplace_back ( BlockHandle : : NullBlockHandle ( ) ) ;
continue ;
}
offset = v . handle . offset ( ) ;
BlockHandle handle = v . handle ;
Status s = GetDataBlockFromCache ( ro , handle , uncompression_dict ,
& ( results . back ( ) ) , BlockType : : kData , miter - > get_context ) ;
if ( s . ok ( ) & & ! results . back ( ) . IsEmpty ( ) ) {
// Found it in the cache. Add NULL handle to indicate there is
// nothing to read from disk
block_handles . emplace_back ( BlockHandle : : NullBlockHandle ( ) ) ;
} else {
block_handles . emplace_back ( handle ) ;
total_len + = handle . size ( ) ;
}
}
if ( total_len ) {
char * scratch = 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_ - > 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 ) ;
}
}
MaybeLoadBlocksToCache ( read_options ,
& data_block_range , & block_handles , & statuses , & results ,
scratch , uncompression_dict ) ;
}
}
DataBlockIter first_biter ;
DataBlockIter next_biter ;
size_t idx_in_batch = 0 ;
for ( auto miter = sst_file_range . begin ( ) ; miter ! = sst_file_range . end ( ) ;
for ( auto miter = sst_file_range . begin ( ) ; miter ! = sst_file_range . end ( ) ;
+ + miter ) {
+ + miter ) {
Status s ;
Status s ;
@ -3257,83 +3623,97 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
const Slice & key = miter - > ikey ;
const Slice & key = miter - > ikey ;
bool matched = false ; // if such user key matched a key in SST
bool matched = false ; // if such user key matched a key in SST
bool done = false ;
bool done = false ;
for ( iiter - > Seek ( key ) ; iiter - > Valid ( ) & & ! done ; iiter - > Next ( ) ) {
bool first_block = true ;
IndexValue v = iiter - > value ( ) ;
do {
if ( ! v . first_internal_key . empty ( ) & & ! skip_filters & &
DataBlockIter * biter = nullptr ;
UserComparatorWrapper ( rep_ - > internal_comparator . user_comparator ( ) )
. Compare ( 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 ;
}
bool reusing_block = true ;
bool reusing_block = true ;
uint64_t referenced_data_size = 0 ;
uint64_t referenced_data_size = 0 ;
bool does_referenced_key_exist = false ;
bool does_referenced_key_exist = false ;
BlockCacheLookupContext lookup_data_block_context (
BlockCacheLookupContext lookup_data_block_context (
TableReaderCaller : : kUserMultiGet ) ;
TableReaderCaller : : kUserMultiGet ) ;
if ( iiter - > value ( ) . handle . offset ( ) ! = offset ) {
if ( first_block ) {
offset = iiter - > value ( ) . handle . offset ( ) ;
if ( ! block_handles [ idx_in_batch ] . IsNull ( ) | |
biter . Invalidate ( Status : : OK ( ) ) ;
! 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_block = false ;
}
biter = & first_biter ;
idx_in_batch + + ;
} else {
IndexValue v = iiter - > value ( ) ;
if ( ! v . first_internal_key . empty ( ) & & ! skip_filters & &
UserComparatorWrapper ( rep_ - > internal_comparator . user_comparator ( ) )
. Compare ( 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 > (
NewDataBlockIterator < DataBlockIter > (
read_options , v . handle , & biter , BlockType : : kData , get_context ,
read_options , iiter - > value ( ) . handle , & next_biter ,
& lookup_data_block_context , Status ( ) , nullptr ) ;
BlockType : : kData , get_context , & lookup_data_block_context ,
Status ( ) , nullptr ) ;
biter = & next_biter ;
reusing_block = false ;
reusing_block = false ;
}
}
if ( read_options . read_tier = = kBlockCacheTier & &
if ( read_options . read_tier = = kBlockCacheTier & &
biter . status ( ) . IsIncomplete ( ) ) {
biter - > status ( ) . IsIncomplete ( ) ) {
// couldn't get block from block_cache
// couldn't get block from block_cache
// Update Saver.state to Found because we are only looking for
// 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
// whether we can guarantee the key is not there when "no_io" is set
get_context - > MarkKeyMayExist ( ) ;
get_context - > MarkKeyMayExist ( ) ;
break ;
break ;
}
}
if ( ! biter . status ( ) . ok ( ) ) {
if ( ! biter - > status ( ) . ok ( ) ) {
s = biter . status ( ) ;
s = biter - > status ( ) ;
break ;
break ;
}
}
bool may_exist = biter . SeekForGet ( key ) ;
bool may_exist = biter - > SeekForGet ( key ) ;
if ( ! may_exist ) {
if ( ! may_exist ) {
// HashSeek cannot find the key this block and the the iter is not
// 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
// 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
// either. In this case, the seek_key cannot be found, so we break
// from the top level for-loop.
// from the top level for-loop.
done = true ;
break ;
} else {
}
// Call the *saver function on each entry/block until it returns false
for ( ; biter . Valid ( ) ; biter . Next ( ) ) {
ParsedInternalKey parsed_key ;
Cleanable dummy ;
Cleanable * value_pinner = nullptr ;
if ( ! ParseInternalKey ( biter . key ( ) , & parsed_key ) ) {
s = Status : : Corruption ( Slice ( ) ) ;
}
if ( biter . IsValuePinned ( ) ) {
if ( reusing_block ) {
Cache * block_cache = rep_ - > table_options . block_cache . get ( ) ;
assert ( biter . cache_handle ( ) ! = nullptr ) ;
block_cache - > Ref ( biter . cache_handle ( ) ) ;
dummy . RegisterCleanup ( & ReleaseCachedEntry , block_cache ,
biter . cache_handle ( ) ) ;
value_pinner = & dummy ;
} else {
value_pinner = & biter ;
}
}
if ( ! get_context - > SaveValue ( parsed_key , biter . value ( ) , & matched ,
// Call the *saver function on each entry/block until it returns false
value_pinner ) ) {
for ( ; biter - > Valid ( ) ; biter - > Next ( ) ) {
does_referenced_key_exist = true ;
ParsedInternalKey parsed_key ;
referenced_data_size = biter . key ( ) . size ( ) + biter . value ( ) . size ( ) ;
Cleanable dummy ;
done = true ;
Cleanable * value_pinner = nullptr ;
break ;
if ( ! ParseInternalKey ( biter - > key ( ) , & parsed_key ) ) {
s = Status : : Corruption ( Slice ( ) ) ;
}
if ( biter - > IsValuePinned ( ) ) {
if ( reusing_block ) {
Cache * block_cache = rep_ - > table_options . block_cache . get ( ) ;
assert ( biter - > cache_handle ( ) ! = nullptr ) ;
block_cache - > Ref ( biter - > cache_handle ( ) ) ;
dummy . RegisterCleanup ( & ReleaseCachedEntry , block_cache ,
biter - > cache_handle ( ) ) ;
value_pinner = & dummy ;
} else {
value_pinner = biter ;
}
}
}
}
s = biter . status ( ) ;
if ( ! get_context - > SaveValue (
parsed_key , biter - > value ( ) , & matched , value_pinner ) ) {
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.
// Write the block cache access.
if ( block_cache_tracer_ & & block_cache_tracer_ - > is_tracing_enabled ( ) ) {
if ( block_cache_tracer_ & & block_cache_tracer_ - > is_tracing_enabled ( ) ) {
@ -3354,11 +3734,18 @@ void BlockBasedTable::MultiGet(const ReadOptions& read_options,
access_record , lookup_data_block_context . block_key ,
access_record , lookup_data_block_context . block_key ,
rep_ - > cf_name_for_tracing ( ) , key ) ;
rep_ - > cf_name_for_tracing ( ) , key ) ;
}
}
s = biter - > status ( ) ;
if ( done ) {
if ( done ) {
// Avoid the extra Next which is expensive in two-level indexes
// Avoid the extra Next which is expensive in two-level indexes
break ;
break ;
}
}
}
if ( first_block ) {
iiter - > Seek ( key ) ;
}
first_block = false ;
iiter - > Next ( ) ;
} while ( iiter - > Valid ( ) ) ;
if ( matched & & filter ! = nullptr & & ! filter - > IsBlockBased ( ) ) {
if ( matched & & filter ! = nullptr & & ! filter - > IsBlockBased ( ) ) {
RecordTick ( rep_ - > ioptions . statistics , BLOOM_FILTER_FULL_TRUE_POSITIVE ) ;
RecordTick ( rep_ - > ioptions . statistics , BLOOM_FILTER_FULL_TRUE_POSITIVE ) ;
PERF_COUNTER_BY_LEVEL_ADD ( bloom_filter_full_true_positive , 1 ,
PERF_COUNTER_BY_LEVEL_ADD ( bloom_filter_full_true_positive , 1 ,