block.h 23 KB

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