1 //  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
2 //  This source code is licensed under both the GPLv2 (found in the
3 //  COPYING file in the root directory) and Apache 2.0 License
4 //  (found in the LICENSE.Apache file in the root directory).
5 //
6 // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
7 // Use of this source code is governed by a BSD-style license that can be
8 // found in the LICENSE file. See the AUTHORS file for names of contributors.
9 
10 #pragma once
11 #include <stddef.h>
12 #include <stdint.h>
13 #include <string>
14 #include <vector>
15 
16 #include "db/dbformat.h"
17 #include "db/pinned_iterators_manager.h"
18 #include "port/malloc.h"
19 #include "rocksdb/iterator.h"
20 #include "rocksdb/options.h"
21 #include "rocksdb/statistics.h"
22 #include "rocksdb/table.h"
23 #include "table/block_based/block_prefix_index.h"
24 #include "table/block_based/data_block_hash_index.h"
25 #include "table/format.h"
26 #include "table/internal_iterator.h"
27 #include "test_util/sync_point.h"
28 #include "util/random.h"
29 
30 namespace ROCKSDB_NAMESPACE {
31 
32 struct BlockContents;
33 class Comparator;
34 template <class TValue>
35 class BlockIter;
36 class DataBlockIter;
37 class IndexBlockIter;
38 class BlockPrefixIndex;
39 
40 // BlockReadAmpBitmap is a bitmap that map the ROCKSDB_NAMESPACE::Block data
41 // bytes to a bitmap with ratio bytes_per_bit. Whenever we access a range of
42 // bytes in the Block we update the bitmap and increment
43 // READ_AMP_ESTIMATE_USEFUL_BYTES.
44 class BlockReadAmpBitmap {
45  public:
BlockReadAmpBitmap(size_t block_size,size_t bytes_per_bit,Statistics * statistics)46   explicit BlockReadAmpBitmap(size_t block_size, size_t bytes_per_bit,
47                               Statistics* statistics)
48       : bitmap_(nullptr),
49         bytes_per_bit_pow_(0),
50         statistics_(statistics),
51         rnd_(Random::GetTLSInstance()->Uniform(
52             static_cast<int>(bytes_per_bit))) {
53     TEST_SYNC_POINT_CALLBACK("BlockReadAmpBitmap:rnd", &rnd_);
54     assert(block_size > 0 && bytes_per_bit > 0);
55 
56     // convert bytes_per_bit to be a power of 2
57     while (bytes_per_bit >>= 1) {
58       bytes_per_bit_pow_++;
59     }
60 
61     // num_bits_needed = ceil(block_size / bytes_per_bit)
62     size_t num_bits_needed = ((block_size - 1) >> bytes_per_bit_pow_) + 1;
63     assert(num_bits_needed > 0);
64 
65     // bitmap_size = ceil(num_bits_needed / kBitsPerEntry)
66     size_t bitmap_size = (num_bits_needed - 1) / kBitsPerEntry + 1;
67 
68     // Create bitmap and set all the bits to 0
69     bitmap_ = new std::atomic<uint32_t>[bitmap_size]();
70 
71     RecordTick(GetStatistics(), READ_AMP_TOTAL_READ_BYTES, block_size);
72   }
73 
~BlockReadAmpBitmap()74   ~BlockReadAmpBitmap() { delete[] bitmap_; }
75 
Mark(uint32_t start_offset,uint32_t end_offset)76   void Mark(uint32_t start_offset, uint32_t end_offset) {
77     assert(end_offset >= start_offset);
78     // Index of first bit in mask
79     uint32_t start_bit =
80         (start_offset + (1 << bytes_per_bit_pow_) - rnd_ - 1) >>
81         bytes_per_bit_pow_;
82     // Index of last bit in mask + 1
83     uint32_t exclusive_end_bit =
84         (end_offset + (1 << bytes_per_bit_pow_) - rnd_) >> bytes_per_bit_pow_;
85     if (start_bit >= exclusive_end_bit) {
86       return;
87     }
88     assert(exclusive_end_bit > 0);
89 
90     if (GetAndSet(start_bit) == 0) {
91       uint32_t new_useful_bytes = (exclusive_end_bit - start_bit)
92                                   << bytes_per_bit_pow_;
93       RecordTick(GetStatistics(), READ_AMP_ESTIMATE_USEFUL_BYTES,
94                  new_useful_bytes);
95     }
96   }
97 
GetStatistics()98   Statistics* GetStatistics() {
99     return statistics_.load(std::memory_order_relaxed);
100   }
101 
SetStatistics(Statistics * stats)102   void SetStatistics(Statistics* stats) { statistics_.store(stats); }
103 
GetBytesPerBit()104   uint32_t GetBytesPerBit() { return 1 << bytes_per_bit_pow_; }
105 
ApproximateMemoryUsage()106   size_t ApproximateMemoryUsage() const {
107 #ifdef ROCKSDB_MALLOC_USABLE_SIZE
108     return malloc_usable_size((void*)this);
109 #endif  // ROCKSDB_MALLOC_USABLE_SIZE
110     return sizeof(*this);
111   }
112 
113  private:
114   // Get the current value of bit at `bit_idx` and set it to 1
GetAndSet(uint32_t bit_idx)115   inline bool GetAndSet(uint32_t bit_idx) {
116     const uint32_t byte_idx = bit_idx / kBitsPerEntry;
117     const uint32_t bit_mask = 1 << (bit_idx % kBitsPerEntry);
118 
119     return bitmap_[byte_idx].fetch_or(bit_mask, std::memory_order_relaxed) &
120            bit_mask;
121   }
122 
123   const uint32_t kBytesPersEntry = sizeof(uint32_t);   // 4 bytes
124   const uint32_t kBitsPerEntry = kBytesPersEntry * 8;  // 32 bits
125 
126   // Bitmap used to record the bytes that we read, use atomic to protect
127   // against multiple threads updating the same bit
128   std::atomic<uint32_t>* bitmap_;
129   // (1 << bytes_per_bit_pow_) is bytes_per_bit. Use power of 2 to optimize
130   // muliplication and division
131   uint8_t bytes_per_bit_pow_;
132   // Pointer to DB Statistics object, Since this bitmap may outlive the DB
133   // this pointer maybe invalid, but the DB will update it to a valid pointer
134   // by using SetStatistics() before calling Mark()
135   std::atomic<Statistics*> statistics_;
136   uint32_t rnd_;
137 };
138 
139 // This Block class is not for any old block: it is designed to hold only
140 // uncompressed blocks containing sorted key-value pairs. It is thus
141 // suitable for storing uncompressed data blocks, index blocks (including
142 // partitions), range deletion blocks, properties blocks, metaindex blocks,
143 // as well as the top level of the partitioned filter structure (which is
144 // actually an index of the filter partitions). It is NOT suitable for
145 // compressed blocks in general, filter blocks/partitions, or compression
146 // dictionaries (since the latter do not contain sorted key-value pairs).
147 // Use BlockContents directly for those.
148 //
149 // See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format
150 // for details of the format and the various block types.
151 class Block {
152  public:
153   // Initialize the block with the specified contents.
154   explicit Block(BlockContents&& contents, size_t read_amp_bytes_per_bit = 0,
155                  Statistics* statistics = nullptr);
156   // No copying allowed
157   Block(const Block&) = delete;
158   void operator=(const Block&) = delete;
159 
160   ~Block();
161 
size()162   size_t size() const { return size_; }
data()163   const char* data() const { return data_; }
164   // The additional memory space taken by the block data.
usable_size()165   size_t usable_size() const { return contents_.usable_size(); }
166   uint32_t NumRestarts() const;
own_bytes()167   bool own_bytes() const { return contents_.own_bytes(); }
168 
169   BlockBasedTableOptions::DataBlockIndexType IndexType() const;
170 
171   // If comparator is InternalKeyComparator, user_comparator is its user
172   // comparator; they are equal otherwise.
173   //
174   // If iter is null, return new Iterator
175   // If iter is not null, update this one and return it as Iterator*
176   //
177   // Updates read_amp_bitmap_ if it is not nullptr.
178   //
179   // If `block_contents_pinned` is true, the caller will guarantee that when
180   // the cleanup functions are transferred from the iterator to other
181   // classes, e.g. PinnableSlice, the pointer to the bytes will still be
182   // valid. Either the iterator holds cache handle or ownership of some resource
183   // and release them in a release function, or caller is sure that the data
184   // will not go away (for example, it's from mmapped file which will not be
185   // closed).
186   //
187   // NOTE: for the hash based lookup, if a key prefix doesn't match any key,
188   // the iterator will simply be set as "invalid", rather than returning
189   // the key that is just pass the target key.
190   DataBlockIter* NewDataIterator(const Comparator* comparator,
191                                  const Comparator* user_comparator,
192                                  SequenceNumber global_seqno,
193                                  DataBlockIter* iter = nullptr,
194                                  Statistics* stats = nullptr,
195                                  bool block_contents_pinned = false);
196 
197   // key_includes_seq, default true, means that the keys are in internal key
198   // format.
199   // value_is_full, default true, means that no delta encoding is
200   // applied to values.
201   //
202   // If `prefix_index` is not nullptr this block will do hash lookup for the key
203   // prefix. If total_order_seek is true, prefix_index_ is ignored.
204   //
205   // `have_first_key` controls whether IndexValue will contain
206   // first_internal_key. It affects data serialization format, so the same value
207   // have_first_key must be used when writing and reading index.
208   // It is determined by IndexType property of the table.
209   IndexBlockIter* NewIndexIterator(const Comparator* comparator,
210                                    const Comparator* user_comparator,
211                                    SequenceNumber global_seqno,
212                                    IndexBlockIter* iter, Statistics* stats,
213                                    bool total_order_seek, bool have_first_key,
214                                    bool key_includes_seq, bool value_is_full,
215                                    bool block_contents_pinned = false,
216                                    BlockPrefixIndex* prefix_index = nullptr);
217 
218   // Report an approximation of how much memory has been used.
219   size_t ApproximateMemoryUsage() const;
220 
221  private:
222   BlockContents contents_;
223   const char* data_;         // contents_.data.data()
224   size_t size_;              // contents_.data.size()
225   uint32_t restart_offset_;  // Offset in data_ of restart array
226   uint32_t num_restarts_;
227   std::unique_ptr<BlockReadAmpBitmap> read_amp_bitmap_;
228   DataBlockHashIndex data_block_hash_index_;
229 };
230 
231 template <class TValue>
232 class BlockIter : public InternalIteratorBase<TValue> {
233  public:
InitializeBase(const Comparator * comparator,const char * data,uint32_t restarts,uint32_t num_restarts,SequenceNumber global_seqno,bool block_contents_pinned)234   void InitializeBase(const Comparator* comparator, const char* data,
235                       uint32_t restarts, uint32_t num_restarts,
236                       SequenceNumber global_seqno, bool block_contents_pinned) {
237     assert(data_ == nullptr);  // Ensure it is called only once
238     assert(num_restarts > 0);  // Ensure the param is valid
239 
240     comparator_ = comparator;
241     data_ = data;
242     restarts_ = restarts;
243     num_restarts_ = num_restarts;
244     current_ = restarts_;
245     restart_index_ = num_restarts_;
246     global_seqno_ = global_seqno;
247     block_contents_pinned_ = block_contents_pinned;
248     cache_handle_ = nullptr;
249   }
250 
251   // Makes Valid() return false, status() return `s`, and Seek()/Prev()/etc do
252   // nothing. Calls cleanup functions.
InvalidateBase(Status s)253   void InvalidateBase(Status s) {
254     // Assert that the BlockIter is never deleted while Pinning is Enabled.
255     assert(!pinned_iters_mgr_ ||
256            (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled()));
257 
258     data_ = nullptr;
259     current_ = restarts_;
260     status_ = s;
261 
262     // Call cleanup callbacks.
263     Cleanable::Reset();
264   }
265 
Valid()266   bool Valid() const override { return current_ < restarts_; }
status()267   Status status() const override { return status_; }
key()268   Slice key() const override {
269     assert(Valid());
270     return key_.GetKey();
271   }
272 
273 #ifndef NDEBUG
~BlockIter()274   ~BlockIter() override {
275     // Assert that the BlockIter is never deleted while Pinning is Enabled.
276     assert(!pinned_iters_mgr_ ||
277            (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled()));
278   }
SetPinnedItersMgr(PinnedIteratorsManager * pinned_iters_mgr)279   void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
280     pinned_iters_mgr_ = pinned_iters_mgr;
281   }
282   PinnedIteratorsManager* pinned_iters_mgr_ = nullptr;
283 #endif
284 
IsKeyPinned()285   bool IsKeyPinned() const override {
286     return block_contents_pinned_ && key_pinned_;
287   }
288 
IsValuePinned()289   bool IsValuePinned() const override { return block_contents_pinned_; }
290 
TEST_CurrentEntrySize()291   size_t TEST_CurrentEntrySize() { return NextEntryOffset() - current_; }
292 
ValueOffset()293   uint32_t ValueOffset() const {
294     return static_cast<uint32_t>(value_.data() - data_);
295   }
296 
SetCacheHandle(Cache::Handle * handle)297   void SetCacheHandle(Cache::Handle* handle) { cache_handle_ = handle; }
298 
cache_handle()299   Cache::Handle* cache_handle() { return cache_handle_; }
300 
301  protected:
302   // Note: The type could be changed to InternalKeyComparator but we see a weird
303   // performance drop by that.
304   const Comparator* comparator_;
305   const char* data_;       // underlying block contents
306   uint32_t num_restarts_;  // Number of uint32_t entries in restart array
307 
308   // Index of restart block in which current_ or current_-1 falls
309   uint32_t restart_index_;
310   uint32_t restarts_;  // Offset of restart array (list of fixed32)
311   // current_ is offset in data_ of current entry.  >= restarts_ if !Valid
312   uint32_t current_;
313   IterKey key_;
314   Slice value_;
315   Status status_;
316   bool key_pinned_;
317   // Whether the block data is guaranteed to outlive this iterator, and
318   // as long as the cleanup functions are transferred to another class,
319   // e.g. PinnableSlice, the pointer to the bytes will still be valid.
320   bool block_contents_pinned_;
321   SequenceNumber global_seqno_;
322 
323  private:
324   // Store the cache handle, if the block is cached. We need this since the
325   // only other place the handle is stored is as an argument to the Cleanable
326   // function callback, which is hard to retrieve. When multiple value
327   // PinnableSlices reference the block, they need the cache handle in order
328   // to bump up the ref count
329   Cache::Handle* cache_handle_;
330 
331  public:
332   // Return the offset in data_ just past the end of the current entry.
NextEntryOffset()333   inline uint32_t NextEntryOffset() const {
334     // NOTE: We don't support blocks bigger than 2GB
335     return static_cast<uint32_t>((value_.data() + value_.size()) - data_);
336   }
337 
GetRestartPoint(uint32_t index)338   uint32_t GetRestartPoint(uint32_t index) {
339     assert(index < num_restarts_);
340     return DecodeFixed32(data_ + restarts_ + index * sizeof(uint32_t));
341   }
342 
SeekToRestartPoint(uint32_t index)343   void SeekToRestartPoint(uint32_t index) {
344     key_.Clear();
345     restart_index_ = index;
346     // current_ will be fixed by ParseNextKey();
347 
348     // ParseNextKey() starts at the end of value_, so set value_ accordingly
349     uint32_t offset = GetRestartPoint(index);
350     value_ = Slice(data_ + offset, 0);
351   }
352 
353   void CorruptionError();
354 
355   template <typename DecodeKeyFunc>
356   inline bool BinarySeek(const Slice& target, uint32_t left, uint32_t right,
357                          uint32_t* index, const Comparator* comp);
358 };
359 
360 class DataBlockIter final : public BlockIter<Slice> {
361  public:
DataBlockIter()362   DataBlockIter()
363       : BlockIter(), read_amp_bitmap_(nullptr), last_bitmap_offset_(0) {}
DataBlockIter(const Comparator * comparator,const Comparator * user_comparator,const char * data,uint32_t restarts,uint32_t num_restarts,SequenceNumber global_seqno,BlockReadAmpBitmap * read_amp_bitmap,bool block_contents_pinned,DataBlockHashIndex * data_block_hash_index)364   DataBlockIter(const Comparator* comparator, const Comparator* user_comparator,
365                 const char* data, uint32_t restarts, uint32_t num_restarts,
366                 SequenceNumber global_seqno,
367                 BlockReadAmpBitmap* read_amp_bitmap, bool block_contents_pinned,
368                 DataBlockHashIndex* data_block_hash_index)
369       : DataBlockIter() {
370     Initialize(comparator, user_comparator, data, restarts, num_restarts,
371                global_seqno, read_amp_bitmap, block_contents_pinned,
372                data_block_hash_index);
373   }
Initialize(const Comparator * comparator,const Comparator * user_comparator,const char * data,uint32_t restarts,uint32_t num_restarts,SequenceNumber global_seqno,BlockReadAmpBitmap * read_amp_bitmap,bool block_contents_pinned,DataBlockHashIndex * data_block_hash_index)374   void Initialize(const Comparator* comparator,
375                   const Comparator* user_comparator, const char* data,
376                   uint32_t restarts, uint32_t num_restarts,
377                   SequenceNumber global_seqno,
378                   BlockReadAmpBitmap* read_amp_bitmap,
379                   bool block_contents_pinned,
380                   DataBlockHashIndex* data_block_hash_index) {
381     InitializeBase(comparator, data, restarts, num_restarts, global_seqno,
382                    block_contents_pinned);
383     user_comparator_ = user_comparator;
384     key_.SetIsUserKey(false);
385     read_amp_bitmap_ = read_amp_bitmap;
386     last_bitmap_offset_ = current_ + 1;
387     data_block_hash_index_ = data_block_hash_index;
388   }
389 
value()390   Slice value() const override {
391     assert(Valid());
392     if (read_amp_bitmap_ && current_ < restarts_ &&
393         current_ != last_bitmap_offset_) {
394       read_amp_bitmap_->Mark(current_ /* current entry offset */,
395                              NextEntryOffset() - 1);
396       last_bitmap_offset_ = current_;
397     }
398     return value_;
399   }
400 
401   void Seek(const Slice& target) override;
402 
SeekForGet(const Slice & target)403   inline bool SeekForGet(const Slice& target) {
404     if (!data_block_hash_index_) {
405       Seek(target);
406       return true;
407     }
408 
409     return SeekForGetImpl(target);
410   }
411 
412   void SeekForPrev(const Slice& target) override;
413 
414   void Prev() override;
415 
416   void Next() final override;
417 
418   // Try to advance to the next entry in the block. If there is data corruption
419   // or error, report it to the caller instead of aborting the process. May
420   // incur higher CPU overhead because we need to perform check on every entry.
421   void NextOrReport();
422 
423   void SeekToFirst() override;
424 
425   // Try to seek to the first entry in the block. If there is data corruption
426   // or error, report it to caller instead of aborting the process. May incur
427   // higher CPU overhead because we need to perform check on every entry.
428   void SeekToFirstOrReport();
429 
430   void SeekToLast() override;
431 
Invalidate(Status s)432   void Invalidate(Status s) {
433     InvalidateBase(s);
434     // Clear prev entries cache.
435     prev_entries_keys_buff_.clear();
436     prev_entries_.clear();
437     prev_entries_idx_ = -1;
438   }
439 
440  private:
441   // read-amp bitmap
442   BlockReadAmpBitmap* read_amp_bitmap_;
443   // last `current_` value we report to read-amp bitmp
444   mutable uint32_t last_bitmap_offset_;
445   struct CachedPrevEntry {
CachedPrevEntryCachedPrevEntry446     explicit CachedPrevEntry(uint32_t _offset, const char* _key_ptr,
447                              size_t _key_offset, size_t _key_size, Slice _value)
448         : offset(_offset),
449           key_ptr(_key_ptr),
450           key_offset(_key_offset),
451           key_size(_key_size),
452           value(_value) {}
453 
454     // offset of entry in block
455     uint32_t offset;
456     // Pointer to key data in block (nullptr if key is delta-encoded)
457     const char* key_ptr;
458     // offset of key in prev_entries_keys_buff_ (0 if key_ptr is not nullptr)
459     size_t key_offset;
460     // size of key
461     size_t key_size;
462     // value slice pointing to data in block
463     Slice value;
464   };
465   std::string prev_entries_keys_buff_;
466   std::vector<CachedPrevEntry> prev_entries_;
467   int32_t prev_entries_idx_ = -1;
468 
469   DataBlockHashIndex* data_block_hash_index_;
470   const Comparator* user_comparator_;
471 
472   template <typename DecodeEntryFunc>
473   inline bool ParseNextDataKey(const char* limit = nullptr);
474 
Compare(const IterKey & ikey,const Slice & b)475   inline int Compare(const IterKey& ikey, const Slice& b) const {
476     return comparator_->Compare(ikey.GetInternalKey(), b);
477   }
478 
479   bool SeekForGetImpl(const Slice& target);
480 };
481 
482 class IndexBlockIter final : public BlockIter<IndexValue> {
483  public:
IndexBlockIter()484   IndexBlockIter() : BlockIter(), prefix_index_(nullptr) {}
485 
key()486   Slice key() const override {
487     assert(Valid());
488     return key_.GetKey();
489   }
490   // key_includes_seq, default true, means that the keys are in internal key
491   // format.
492   // value_is_full, default true, means that no delta encoding is
493   // applied to values.
Initialize(const Comparator * comparator,const Comparator * user_comparator,const char * data,uint32_t restarts,uint32_t num_restarts,SequenceNumber global_seqno,BlockPrefixIndex * prefix_index,bool have_first_key,bool key_includes_seq,bool value_is_full,bool block_contents_pinned)494   void Initialize(const Comparator* comparator,
495                   const Comparator* user_comparator, const char* data,
496                   uint32_t restarts, uint32_t num_restarts,
497                   SequenceNumber global_seqno, BlockPrefixIndex* prefix_index,
498                   bool have_first_key, bool key_includes_seq,
499                   bool value_is_full, bool block_contents_pinned) {
500     InitializeBase(key_includes_seq ? comparator : user_comparator, data,
501                    restarts, num_restarts, kDisableGlobalSequenceNumber,
502                    block_contents_pinned);
503     key_includes_seq_ = key_includes_seq;
504     key_.SetIsUserKey(!key_includes_seq_);
505     prefix_index_ = prefix_index;
506     value_delta_encoded_ = !value_is_full;
507     have_first_key_ = have_first_key;
508     if (have_first_key_ && global_seqno != kDisableGlobalSequenceNumber) {
509       global_seqno_state_.reset(new GlobalSeqnoState(global_seqno));
510     } else {
511       global_seqno_state_.reset();
512     }
513   }
514 
user_key()515   Slice user_key() const override {
516     if (key_includes_seq_) {
517       return ExtractUserKey(key());
518     }
519     return key();
520   }
521 
value()522   IndexValue value() const override {
523     assert(Valid());
524     if (value_delta_encoded_ || global_seqno_state_ != nullptr) {
525       return decoded_value_;
526     } else {
527       IndexValue entry;
528       Slice v = value_;
529       Status decode_s __attribute__((__unused__)) =
530           entry.DecodeFrom(&v, have_first_key_, nullptr);
531       assert(decode_s.ok());
532       return entry;
533     }
534   }
535 
536   // IndexBlockIter follows a different contract for prefix iterator
537   // from data iterators.
538   // If prefix of the seek key `target` exists in the file, it must
539   // return the same result as total order seek.
540   // If the prefix of `target` doesn't exist in the file, it can either
541   // return the result of total order seek, or set both of Valid() = false
542   // and status() = NotFound().
543   void Seek(const Slice& target) override;
544 
SeekForPrev(const Slice &)545   void SeekForPrev(const Slice&) override {
546     assert(false);
547     current_ = restarts_;
548     restart_index_ = num_restarts_;
549     status_ = Status::InvalidArgument(
550         "RocksDB internal error: should never call SeekForPrev() on index "
551         "blocks");
552     key_.Clear();
553     value_.clear();
554   }
555 
556   void Prev() override;
557 
558   void Next() override;
559 
560   void SeekToFirst() override;
561 
562   void SeekToLast() override;
563 
Invalidate(Status s)564   void Invalidate(Status s) { InvalidateBase(s); }
565 
IsValuePinned()566   bool IsValuePinned() const override {
567     return global_seqno_state_ != nullptr ? false : BlockIter::IsValuePinned();
568   }
569 
570  private:
571   // Key is in InternalKey format
572   bool key_includes_seq_;
573   bool value_delta_encoded_;
574   bool have_first_key_;  // value includes first_internal_key
575   BlockPrefixIndex* prefix_index_;
576   // Whether the value is delta encoded. In that case the value is assumed to be
577   // BlockHandle. The first value in each restart interval is the full encoded
578   // BlockHandle; the restart of encoded size part of the BlockHandle. The
579   // offset of delta encoded BlockHandles is computed by adding the size of
580   // previous delta encoded values in the same restart interval to the offset of
581   // the first value in that restart interval.
582   IndexValue decoded_value_;
583 
584   // When sequence number overwriting is enabled, this struct contains the seqno
585   // to overwrite with, and current first_internal_key with overwritten seqno.
586   // This is rarely used, so we put it behind a pointer and only allocate when
587   // needed.
588   struct GlobalSeqnoState {
589     // First internal key according to current index entry, but with sequence
590     // number overwritten to global_seqno.
591     IterKey first_internal_key;
592     SequenceNumber global_seqno;
593 
GlobalSeqnoStateGlobalSeqnoState594     explicit GlobalSeqnoState(SequenceNumber seqno) : global_seqno(seqno) {}
595   };
596 
597   std::unique_ptr<GlobalSeqnoState> global_seqno_state_;
598 
599   // Set *prefix_may_exist to false if no key possibly share the same prefix
600   // as `target`. If not set, the result position should be the same as total
601   // order Seek.
602   bool PrefixSeek(const Slice& target, uint32_t* index, bool* prefix_may_exist);
603   // Set *prefix_may_exist to false if no key can possibly share the same
604   // prefix as `target`. If not set, the result position should be the same
605   // as total order seek.
606   bool BinaryBlockIndexSeek(const Slice& target, uint32_t* block_ids,
607                             uint32_t left, uint32_t right, uint32_t* index,
608                             bool* prefix_may_exist);
609   inline int CompareBlockKey(uint32_t block_index, const Slice& target);
610 
Compare(const Slice & a,const Slice & b)611   inline int Compare(const Slice& a, const Slice& b) const {
612     return comparator_->Compare(a, b);
613   }
614 
Compare(const IterKey & ikey,const Slice & b)615   inline int Compare(const IterKey& ikey, const Slice& b) const {
616     return comparator_->Compare(ikey.GetKey(), b);
617   }
618 
619   inline bool ParseNextIndexKey();
620 
621   // When value_delta_encoded_ is enabled it decodes the value which is assumed
622   // to be BlockHandle and put it to decoded_value_
623   inline void DecodeCurrentValue(uint32_t shared);
624 };
625 
626 }  // namespace ROCKSDB_NAMESPACE
627