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