block.h 36 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961
  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/kv_checksum.h"
  15. #include "db/pinned_iterators_manager.h"
  16. #include "port/malloc.h"
  17. #include "rocksdb/advanced_cache.h"
  18. #include "rocksdb/iterator.h"
  19. #include "rocksdb/options.h"
  20. #include "rocksdb/statistics.h"
  21. #include "rocksdb/table.h"
  22. #include "table/block_based/block_prefix_index.h"
  23. #include "table/block_based/data_block_hash_index.h"
  24. #include "table/format.h"
  25. #include "table/internal_iterator.h"
  26. #include "test_util/sync_point.h"
  27. #include "util/random.h"
  28. namespace ROCKSDB_NAMESPACE {
  29. struct BlockContents;
  30. class Comparator;
  31. template <class TValue>
  32. class BlockIter;
  33. class DataBlockIter;
  34. class IndexBlockIter;
  35. class MetaBlockIter;
  36. class BlockPrefixIndex;
  37. // BlockReadAmpBitmap is a bitmap that map the ROCKSDB_NAMESPACE::Block data
  38. // bytes to a bitmap with ratio bytes_per_bit. Whenever we access a range of
  39. // bytes in the Block we update the bitmap and increment
  40. // READ_AMP_ESTIMATE_USEFUL_BYTES.
  41. class BlockReadAmpBitmap {
  42. public:
  43. explicit BlockReadAmpBitmap(size_t block_size, size_t bytes_per_bit,
  44. Statistics* statistics)
  45. : bitmap_(nullptr),
  46. bytes_per_bit_pow_(0),
  47. statistics_(statistics),
  48. rnd_(Random::GetTLSInstance()->Uniform(
  49. static_cast<int>(bytes_per_bit))) {
  50. TEST_SYNC_POINT_CALLBACK("BlockReadAmpBitmap:rnd", &rnd_);
  51. assert(block_size > 0 && bytes_per_bit > 0);
  52. // convert bytes_per_bit to be a power of 2
  53. while (bytes_per_bit >>= 1) {
  54. bytes_per_bit_pow_++;
  55. }
  56. // num_bits_needed = ceil(block_size / bytes_per_bit)
  57. size_t num_bits_needed = ((block_size - 1) >> bytes_per_bit_pow_) + 1;
  58. assert(num_bits_needed > 0);
  59. // bitmap_size = ceil(num_bits_needed / kBitsPerEntry)
  60. size_t bitmap_size = (num_bits_needed - 1) / kBitsPerEntry + 1;
  61. // Create bitmap and set all the bits to 0
  62. bitmap_ = new std::atomic<uint32_t>[bitmap_size]();
  63. RecordTick(GetStatistics(), READ_AMP_TOTAL_READ_BYTES, block_size);
  64. }
  65. ~BlockReadAmpBitmap() { delete[] bitmap_; }
  66. void Mark(uint32_t start_offset, uint32_t end_offset) {
  67. assert(end_offset >= start_offset);
  68. // Index of first bit in mask
  69. uint32_t start_bit =
  70. (start_offset + (1 << bytes_per_bit_pow_) - rnd_ - 1) >>
  71. bytes_per_bit_pow_;
  72. // Index of last bit in mask + 1
  73. uint32_t exclusive_end_bit =
  74. (end_offset + (1 << bytes_per_bit_pow_) - rnd_) >> bytes_per_bit_pow_;
  75. if (start_bit >= exclusive_end_bit) {
  76. return;
  77. }
  78. assert(exclusive_end_bit > 0);
  79. if (GetAndSet(start_bit) == 0) {
  80. uint32_t new_useful_bytes = (exclusive_end_bit - start_bit)
  81. << bytes_per_bit_pow_;
  82. RecordTick(GetStatistics(), READ_AMP_ESTIMATE_USEFUL_BYTES,
  83. new_useful_bytes);
  84. }
  85. }
  86. Statistics* GetStatistics() {
  87. return statistics_.load(std::memory_order_relaxed);
  88. }
  89. void SetStatistics(Statistics* stats) { statistics_.store(stats); }
  90. uint32_t GetBytesPerBit() { return 1 << bytes_per_bit_pow_; }
  91. size_t ApproximateMemoryUsage() const {
  92. #ifdef ROCKSDB_MALLOC_USABLE_SIZE
  93. return malloc_usable_size((void*)this);
  94. #endif // ROCKSDB_MALLOC_USABLE_SIZE
  95. return sizeof(*this);
  96. }
  97. private:
  98. // Get the current value of bit at `bit_idx` and set it to 1
  99. inline bool GetAndSet(uint32_t bit_idx) {
  100. const uint32_t byte_idx = bit_idx / kBitsPerEntry;
  101. const uint32_t bit_mask = 1 << (bit_idx % kBitsPerEntry);
  102. return bitmap_[byte_idx].fetch_or(bit_mask, std::memory_order_relaxed) &
  103. bit_mask;
  104. }
  105. const uint32_t kBytesPersEntry = sizeof(uint32_t); // 4 bytes
  106. const uint32_t kBitsPerEntry = kBytesPersEntry * 8; // 32 bits
  107. // Bitmap used to record the bytes that we read, use atomic to protect
  108. // against multiple threads updating the same bit
  109. std::atomic<uint32_t>* bitmap_;
  110. // (1 << bytes_per_bit_pow_) is bytes_per_bit. Use power of 2 to optimize
  111. // muliplication and division
  112. uint8_t bytes_per_bit_pow_;
  113. // Pointer to DB Statistics object, Since this bitmap may outlive the DB
  114. // this pointer maybe invalid, but the DB will update it to a valid pointer
  115. // by using SetStatistics() before calling Mark()
  116. std::atomic<Statistics*> statistics_;
  117. uint32_t rnd_;
  118. };
  119. // class Block is the uncompressed and "parsed" form for blocks containing
  120. // key-value pairs. (See BlockContents comments for more on terminology.)
  121. // This includes the in-memory representation of data blocks, index blocks
  122. // (including partitions), range deletion blocks, properties blocks, metaindex
  123. // blocks, as well as the top level of the partitioned filter structure (which
  124. // is actually an index of the filter partitions). It is NOT suitable for
  125. // compressed blocks in general, filter blocks/partitions, or compression
  126. // dictionaries.
  127. //
  128. // See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format
  129. // for details of the format and the various block types.
  130. //
  131. // TODO: Rename to ParsedKvBlock?
  132. class Block {
  133. public:
  134. // Initialize the block with the specified contents.
  135. explicit Block(BlockContents&& contents, size_t read_amp_bytes_per_bit = 0,
  136. Statistics* statistics = nullptr);
  137. // No copying allowed
  138. Block(const Block&) = delete;
  139. void operator=(const Block&) = delete;
  140. ~Block();
  141. size_t size() const { return contents_.data.size(); }
  142. const char* data() const { return contents_.data.data(); }
  143. // The additional memory space taken by the block data.
  144. size_t usable_size() const { return contents_.usable_size(); }
  145. uint32_t NumRestarts() const;
  146. bool own_bytes() const { return contents_.own_bytes(); }
  147. BlockBasedTableOptions::DataBlockIndexType IndexType() const;
  148. // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key
  149. // comparator.
  150. //
  151. // If iter is null, return new Iterator
  152. // If iter is not null, update this one and return it as Iterator*
  153. //
  154. // Updates read_amp_bitmap_ if it is not nullptr.
  155. //
  156. // If `block_contents_pinned` is true, the caller will guarantee that when
  157. // the cleanup functions are transferred from the iterator to other
  158. // classes, e.g. PinnableSlice, the pointer to the bytes will still be
  159. // valid. Either the iterator holds cache handle or ownership of some resource
  160. // and release them in a release function, or caller is sure that the data
  161. // will not go away (for example, it's from mmapped file which will not be
  162. // closed).
  163. //
  164. // `user_defined_timestamps_persisted` controls whether a min timestamp is
  165. // padded while key is being parsed from the block.
  166. //
  167. // NOTE: for the hash based lookup, if a key prefix doesn't match any key,
  168. // the iterator will simply be set as "invalid", rather than returning
  169. // the key that is just pass the target key.
  170. DataBlockIter* NewDataIterator(const Comparator* raw_ucmp,
  171. SequenceNumber global_seqno,
  172. DataBlockIter* iter = nullptr,
  173. Statistics* stats = nullptr,
  174. bool block_contents_pinned = false,
  175. bool user_defined_timestamps_persisted = true);
  176. // Returns an MetaBlockIter for iterating over blocks containing metadata
  177. // (like Properties blocks). Unlike data blocks, the keys for these blocks
  178. // do not contain sequence numbers, do not use a user-define comparator, and
  179. // do not track read amplification/statistics. Additionally, MetaBlocks will
  180. // not assert if the block is formatted improperly.
  181. //
  182. // If `block_contents_pinned` is true, the caller will guarantee that when
  183. // the cleanup functions are transferred from the iterator to other
  184. // classes, e.g. PinnableSlice, the pointer to the bytes will still be
  185. // valid. Either the iterator holds cache handle or ownership of some resource
  186. // and release them in a release function, or caller is sure that the data
  187. // will not go away (for example, it's from mmapped file which will not be
  188. // closed).
  189. MetaBlockIter* NewMetaIterator(bool block_contents_pinned = false);
  190. // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key
  191. // comparator.
  192. //
  193. // key_includes_seq, default true, means that the keys are in internal key
  194. // format.
  195. // value_is_full, default true, means that no delta encoding is
  196. // applied to values.
  197. //
  198. // If `prefix_index` is not nullptr this block will do hash lookup for the key
  199. // prefix. If total_order_seek is true, prefix_index_ is ignored.
  200. //
  201. // `have_first_key` controls whether IndexValue will contain
  202. // first_internal_key. It affects data serialization format, so the same value
  203. // have_first_key must be used when writing and reading index.
  204. // It is determined by IndexType property of the table.
  205. // `user_defined_timestamps_persisted` controls whether a min timestamp is
  206. // padded while key is being parsed from the block.
  207. IndexBlockIter* NewIndexIterator(
  208. const Comparator* raw_ucmp, SequenceNumber global_seqno,
  209. IndexBlockIter* iter, Statistics* stats, bool total_order_seek,
  210. bool have_first_key, bool key_includes_seq, bool value_is_full,
  211. bool block_contents_pinned = false,
  212. bool user_defined_timestamps_persisted = true,
  213. BlockPrefixIndex* prefix_index = nullptr);
  214. // Report an approximation of how much memory has been used.
  215. size_t ApproximateMemoryUsage() const;
  216. // For TypedCacheInterface
  217. const Slice& ContentSlice() const { return contents_.data; }
  218. // Initializes per key-value checksum protection.
  219. // After this method is called, each DataBlockIterator returned
  220. // by NewDataIterator will verify per key-value checksum for any key it read.
  221. void InitializeDataBlockProtectionInfo(uint8_t protection_bytes_per_key,
  222. const Comparator* raw_ucmp);
  223. // Initializes per key-value checksum protection.
  224. // After this method is called, each IndexBlockIterator returned
  225. // by NewIndexIterator will verify per key-value checksum for any key it read.
  226. // value_is_full and index_has_first_key are needed to be able to parse
  227. // the index block content and construct checksums.
  228. void InitializeIndexBlockProtectionInfo(uint8_t protection_bytes_per_key,
  229. const Comparator* raw_ucmp,
  230. bool value_is_full,
  231. bool index_has_first_key);
  232. // Initializes per key-value checksum protection.
  233. // After this method is called, each MetaBlockIter returned
  234. // by NewMetaIterator will verify per key-value checksum for any key it read.
  235. void InitializeMetaIndexBlockProtectionInfo(uint8_t protection_bytes_per_key);
  236. static void GenerateKVChecksum(char* checksum_ptr, uint8_t checksum_len,
  237. const Slice& key, const Slice& value) {
  238. ProtectionInfo64().ProtectKV(key, value).Encode(checksum_len, checksum_ptr);
  239. }
  240. const char* TEST_GetKVChecksum() const { return kv_checksum_; }
  241. private:
  242. BlockContents contents_;
  243. uint32_t restart_offset_; // Offset in data_ of restart array
  244. uint32_t num_restarts_;
  245. std::unique_ptr<BlockReadAmpBitmap> read_amp_bitmap_;
  246. char* kv_checksum_{nullptr};
  247. uint32_t checksum_size_{0};
  248. // Used by block iterators to calculate current key index within a block
  249. uint32_t block_restart_interval_{0};
  250. uint8_t protection_bytes_per_key_{0};
  251. DataBlockHashIndex data_block_hash_index_;
  252. };
  253. // A `BlockIter` iterates over the entries in a `Block`'s data buffer. The
  254. // format of this data buffer is an uncompressed, sorted sequence of key-value
  255. // pairs (see `Block` API for more details).
  256. //
  257. // Notably, the keys may either be in internal key format or user key format.
  258. // Subclasses are responsible for configuring the key format.
  259. //
  260. // `BlockIter` intends to provide final overrides for all of
  261. // `InternalIteratorBase` functions that can move the iterator. It does
  262. // this to guarantee `UpdateKey()` is called exactly once after each key
  263. // movement potentially visible to users. In this step, the key is prepared
  264. // (e.g., serialized if global seqno is in effect) so it can be returned
  265. // immediately when the user asks for it via calling `key() const`.
  266. //
  267. // For its subclasses, it provides protected variants of the above-mentioned
  268. // final-overridden methods. They are named with the "Impl" suffix, e.g.,
  269. // `Seek()` logic would be implemented by subclasses in `SeekImpl()`. These
  270. // "Impl" functions are responsible for positioning `raw_key_` but not
  271. // invoking `UpdateKey()`.
  272. //
  273. // Per key-value checksum is enabled if relevant states are passed in during
  274. // `InitializeBase()`. The checksum verification is done in each call to
  275. // UpdateKey() for the current key. Each subclass is responsible for keeping
  276. // track of cur_entry_idx_, the index of the current key within the block.
  277. // BlockIter uses this index to get the corresponding checksum for current key.
  278. // Additional checksum verification may be done in subclasses if they read keys
  279. // other than the key being processed in UpdateKey().
  280. template <class TValue>
  281. class BlockIter : public InternalIteratorBase<TValue> {
  282. public:
  283. // Makes Valid() return false, status() return `s`, and Seek()/Prev()/etc do
  284. // nothing. Calls cleanup functions.
  285. virtual void Invalidate(const Status& s) {
  286. // Assert that the BlockIter is never deleted while Pinning is Enabled.
  287. assert(!pinned_iters_mgr_ || !pinned_iters_mgr_->PinningEnabled());
  288. data_ = nullptr;
  289. current_ = restarts_;
  290. status_ = s;
  291. // Call cleanup callbacks.
  292. Cleanable::Reset();
  293. }
  294. bool Valid() const override {
  295. // When status_ is not ok, iter should be invalid.
  296. assert(status_.ok() || current_ >= restarts_);
  297. return current_ < restarts_;
  298. }
  299. void SeekToFirst() override final {
  300. #ifndef NDEBUG
  301. if (TEST_Corrupt_Callback("BlockIter::SeekToFirst")) return;
  302. #endif
  303. SeekToFirstImpl();
  304. UpdateKey();
  305. }
  306. void SeekToLast() override final {
  307. SeekToLastImpl();
  308. UpdateKey();
  309. }
  310. void Seek(const Slice& target) override final {
  311. SeekImpl(target);
  312. UpdateKey();
  313. }
  314. void SeekForPrev(const Slice& target) override final {
  315. SeekForPrevImpl(target);
  316. UpdateKey();
  317. }
  318. void Next() override final {
  319. NextImpl();
  320. UpdateKey();
  321. }
  322. bool NextAndGetResult(IterateResult* result) override final {
  323. // This does not need to call `UpdateKey()` as the parent class only has
  324. // access to the `UpdateKey()`-invoking functions.
  325. return InternalIteratorBase<TValue>::NextAndGetResult(result);
  326. }
  327. void Prev() override final {
  328. PrevImpl();
  329. UpdateKey();
  330. }
  331. Status status() const override { return status_; }
  332. Slice key() const override {
  333. assert(Valid());
  334. return key_;
  335. }
  336. #ifndef NDEBUG
  337. ~BlockIter() override {
  338. // Assert that the BlockIter is never deleted while Pinning is Enabled.
  339. assert(!pinned_iters_mgr_ ||
  340. (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled()));
  341. status_.PermitUncheckedError();
  342. }
  343. void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
  344. pinned_iters_mgr_ = pinned_iters_mgr;
  345. }
  346. PinnedIteratorsManager* pinned_iters_mgr_ = nullptr;
  347. bool TEST_Corrupt_Callback(const std::string& sync_point) {
  348. bool corrupt = false;
  349. TEST_SYNC_POINT_CALLBACK(sync_point, static_cast<void*>(&corrupt));
  350. if (corrupt) {
  351. CorruptionError();
  352. }
  353. return corrupt;
  354. }
  355. #endif
  356. bool IsKeyPinned() const override {
  357. return block_contents_pinned_ && key_pinned_;
  358. }
  359. bool IsValuePinned() const override { return block_contents_pinned_; }
  360. size_t TEST_CurrentEntrySize() { return NextEntryOffset() - current_; }
  361. uint32_t ValueOffset() const {
  362. return static_cast<uint32_t>(value_.data() - data_);
  363. }
  364. void SetCacheHandle(Cache::Handle* handle) { cache_handle_ = handle; }
  365. Cache::Handle* cache_handle() { return cache_handle_; }
  366. protected:
  367. std::unique_ptr<InternalKeyComparator> icmp_;
  368. const char* data_; // underlying block contents
  369. uint32_t num_restarts_; // Number of uint32_t entries in restart array
  370. // Index of restart block in which current_ or current_-1 falls
  371. uint32_t restart_index_;
  372. uint32_t restarts_; // Offset of restart array (list of fixed32)
  373. // current_ is offset in data_ of current entry. >= restarts_ if !Valid
  374. uint32_t current_;
  375. // Raw key from block.
  376. IterKey raw_key_;
  377. // Buffer for key data when global seqno assignment is enabled.
  378. IterKey key_buf_;
  379. Slice value_;
  380. Status status_;
  381. // Key to be exposed to users.
  382. Slice key_;
  383. SequenceNumber global_seqno_;
  384. // Size of the user-defined timestamp.
  385. size_t ts_sz_ = 0;
  386. // If user-defined timestamp is enabled but not persisted. A min timestamp
  387. // will be padded to the key during key parsing where it applies. Such as when
  388. // parsing keys from data block, index block, parsing the first internal
  389. // key from IndexValue entry. Min timestamp padding is different for when
  390. // `raw_key_` is a user key vs is an internal key.
  391. //
  392. // This only applies to data block and index blocks including index block for
  393. // data blocks, index block for partitioned filter blocks, index block for
  394. // partitioned index blocks. In summary, this only applies to block whose key
  395. // are real user keys or internal keys created from user keys.
  396. bool pad_min_timestamp_;
  397. // Per key-value checksum related states
  398. const char* kv_checksum_;
  399. int32_t cur_entry_idx_;
  400. uint32_t block_restart_interval_;
  401. uint8_t protection_bytes_per_key_;
  402. bool key_pinned_;
  403. // Whether the block data is guaranteed to outlive this iterator, and
  404. // as long as the cleanup functions are transferred to another class,
  405. // e.g. PinnableSlice, the pointer to the bytes will still be valid.
  406. bool block_contents_pinned_;
  407. virtual void SeekToFirstImpl() = 0;
  408. virtual void SeekToLastImpl() = 0;
  409. virtual void SeekImpl(const Slice& target) = 0;
  410. virtual void SeekForPrevImpl(const Slice& target) = 0;
  411. virtual void NextImpl() = 0;
  412. virtual void PrevImpl() = 0;
  413. // Returns the restart interval of this block.
  414. // Returns 0 if num_restarts_ <= 1 or if the BlockIter is not initialized.
  415. virtual uint32_t GetRestartInterval() {
  416. if (num_restarts_ <= 1 || data_ == nullptr) {
  417. return 0;
  418. }
  419. SeekToFirstImpl();
  420. uint32_t end_index = GetRestartPoint(1);
  421. uint32_t count = 1;
  422. while (NextEntryOffset() < end_index && status_.ok()) {
  423. assert(Valid());
  424. NextImpl();
  425. ++count;
  426. }
  427. return count;
  428. }
  429. // Returns the number of keys in this block.
  430. virtual uint32_t NumberOfKeys(uint32_t block_restart_interval) {
  431. if (num_restarts_ == 0 || data_ == nullptr) {
  432. return 0;
  433. }
  434. uint32_t count = (num_restarts_ - 1) * block_restart_interval;
  435. // Add number of keys from the last restart interval
  436. SeekToRestartPoint(num_restarts_ - 1);
  437. while (NextEntryOffset() < restarts_ && status_.ok()) {
  438. NextImpl();
  439. ++count;
  440. }
  441. return count;
  442. }
  443. // Stores whether the current key has a shared bytes with prev key in
  444. // *is_shared.
  445. // Sets raw_key_, value_ to the current parsed key and value.
  446. // Sets restart_index_ to point to the restart interval that contains
  447. // the current key.
  448. template <typename DecodeEntryFunc>
  449. inline bool ParseNextKey(bool* is_shared);
  450. // protection_bytes_per_key, kv_checksum, and block_restart_interval
  451. // are needed only for per kv checksum verification.
  452. void InitializeBase(const Comparator* raw_ucmp, const char* data,
  453. uint32_t restarts, uint32_t num_restarts,
  454. SequenceNumber global_seqno, bool block_contents_pinned,
  455. bool user_defined_timestamp_persisted,
  456. uint8_t protection_bytes_per_key, const char* kv_checksum,
  457. uint32_t block_restart_interval) {
  458. assert(data_ == nullptr); // Ensure it is called only once
  459. assert(num_restarts > 0); // Ensure the param is valid
  460. icmp_ = std::make_unique<InternalKeyComparator>(raw_ucmp);
  461. data_ = data;
  462. restarts_ = restarts;
  463. num_restarts_ = num_restarts;
  464. current_ = restarts_;
  465. restart_index_ = num_restarts_;
  466. global_seqno_ = global_seqno;
  467. if (raw_ucmp != nullptr) {
  468. ts_sz_ = raw_ucmp->timestamp_size();
  469. }
  470. pad_min_timestamp_ = ts_sz_ > 0 && !user_defined_timestamp_persisted;
  471. block_contents_pinned_ = block_contents_pinned;
  472. cache_handle_ = nullptr;
  473. cur_entry_idx_ = -1;
  474. protection_bytes_per_key_ = protection_bytes_per_key;
  475. kv_checksum_ = kv_checksum;
  476. block_restart_interval_ = block_restart_interval;
  477. // Checksum related states are either all 0/nullptr or all non-zero.
  478. // One exception is when num_restarts == 0, block_restart_interval can be 0
  479. // since we are not able to compute it.
  480. assert((protection_bytes_per_key == 0 && kv_checksum == nullptr) ||
  481. (protection_bytes_per_key > 0 && kv_checksum != nullptr &&
  482. (block_restart_interval > 0 || num_restarts == 1)));
  483. }
  484. void CorruptionError(const std::string& error_msg = "bad entry in block") {
  485. current_ = restarts_;
  486. restart_index_ = num_restarts_;
  487. status_ = Status::Corruption(error_msg);
  488. raw_key_.Clear();
  489. value_.clear();
  490. }
  491. void PerKVChecksumCorruptionError() {
  492. std::string error_msg{
  493. "Corrupted block entry: per key-value checksum verification "
  494. "failed."};
  495. error_msg.append(" Offset: " + std::to_string(current_) + ".");
  496. error_msg.append(" Entry index: " + std::to_string(cur_entry_idx_) + ".");
  497. CorruptionError(error_msg);
  498. }
  499. void UpdateRawKeyAndMaybePadMinTimestamp(const Slice& key) {
  500. if (pad_min_timestamp_) {
  501. raw_key_.SetKeyWithPaddedMinTimestamp(key, ts_sz_);
  502. } else {
  503. raw_key_.SetKey(key, false /* copy */);
  504. }
  505. }
  506. // Must be called every time a key is found that needs to be returned to user,
  507. // and may be called when no key is found (as a no-op). Updates `key_`,
  508. // `key_buf_`, and `key_pinned_` with info about the found key.
  509. // Per key-value checksum verification is done if available for the key to be
  510. // returned. Iterator is invalidated with corruption status if checksum
  511. // verification fails.
  512. void UpdateKey() {
  513. key_buf_.Clear();
  514. if (!Valid()) {
  515. return;
  516. }
  517. if (raw_key_.IsUserKey()) {
  518. assert(global_seqno_ == kDisableGlobalSequenceNumber);
  519. key_ = raw_key_.GetUserKey();
  520. key_pinned_ = raw_key_.IsKeyPinned();
  521. } else if (global_seqno_ == kDisableGlobalSequenceNumber) {
  522. key_ = raw_key_.GetInternalKey();
  523. key_pinned_ = raw_key_.IsKeyPinned();
  524. } else {
  525. key_buf_.SetInternalKey(raw_key_.GetUserKey(), global_seqno_,
  526. ExtractValueType(raw_key_.GetInternalKey()));
  527. key_ = key_buf_.GetInternalKey();
  528. key_pinned_ = false;
  529. }
  530. TEST_SYNC_POINT_CALLBACK("BlockIter::UpdateKey::value",
  531. (void*)value_.data());
  532. TEST_SYNC_POINT_CALLBACK("Block::VerifyChecksum::checksum_len",
  533. &protection_bytes_per_key_);
  534. if (protection_bytes_per_key_ > 0) {
  535. if (!ProtectionInfo64()
  536. .ProtectKV(raw_key_.GetKey(), value_)
  537. .Verify(
  538. protection_bytes_per_key_,
  539. kv_checksum_ + protection_bytes_per_key_ * cur_entry_idx_)) {
  540. PerKVChecksumCorruptionError();
  541. }
  542. }
  543. }
  544. // Returns the result of `Comparator::Compare()`, where the appropriate
  545. // comparator is used for the block contents, the LHS argument is the current
  546. // key with global seqno applied, and the RHS argument is `other`.
  547. int CompareCurrentKey(const Slice& other) {
  548. if (raw_key_.IsUserKey()) {
  549. assert(global_seqno_ == kDisableGlobalSequenceNumber);
  550. return icmp_->user_comparator()->Compare(raw_key_.GetUserKey(), other);
  551. } else if (global_seqno_ == kDisableGlobalSequenceNumber) {
  552. return icmp_->Compare(raw_key_.GetInternalKey(), other);
  553. }
  554. return icmp_->Compare(raw_key_.GetInternalKey(), global_seqno_, other,
  555. kDisableGlobalSequenceNumber);
  556. }
  557. private:
  558. // Store the cache handle, if the block is cached. We need this since the
  559. // only other place the handle is stored is as an argument to the Cleanable
  560. // function callback, which is hard to retrieve. When multiple value
  561. // PinnableSlices reference the block, they need the cache handle in order
  562. // to bump up the ref count
  563. Cache::Handle* cache_handle_;
  564. public:
  565. // Return the offset in data_ just past the end of the current entry.
  566. inline uint32_t NextEntryOffset() const {
  567. // NOTE: We don't support blocks bigger than 2GB
  568. return static_cast<uint32_t>((value_.data() + value_.size()) - data_);
  569. }
  570. uint32_t GetRestartPoint(uint32_t index) const {
  571. assert(index < num_restarts_);
  572. return DecodeFixed32(data_ + restarts_ + index * sizeof(uint32_t));
  573. }
  574. void SeekToRestartPoint(uint32_t index) {
  575. raw_key_.Clear();
  576. restart_index_ = index;
  577. // current_ will be fixed by ParseNextKey();
  578. // ParseNextKey() starts at the end of value_, so set value_ accordingly
  579. uint32_t offset = GetRestartPoint(index);
  580. value_ = Slice(data_ + offset, 0);
  581. }
  582. protected:
  583. template <typename DecodeKeyFunc>
  584. inline bool BinarySeek(const Slice& target, uint32_t* index,
  585. bool* is_index_key_result);
  586. // Find the first key in restart interval `index` that is >= `target`.
  587. // If there is no such key, iterator is positioned at the first key in
  588. // restart interval `index + 1`.
  589. // If is_index_key_result is true, it positions the iterator at the first key
  590. // in this restart interval.
  591. // Per key-value checksum verification is done for all keys scanned
  592. // up to but not including the last key (the key that current_ points to
  593. // when this function returns). This key's checksum is verified in
  594. // UpdateKey().
  595. void FindKeyAfterBinarySeek(const Slice& target, uint32_t index,
  596. bool is_index_key_result);
  597. };
  598. class DataBlockIter final : public BlockIter<Slice> {
  599. public:
  600. DataBlockIter()
  601. : BlockIter(), read_amp_bitmap_(nullptr), last_bitmap_offset_(0) {}
  602. void Initialize(const Comparator* raw_ucmp, const char* data,
  603. uint32_t restarts, uint32_t num_restarts,
  604. SequenceNumber global_seqno,
  605. BlockReadAmpBitmap* read_amp_bitmap,
  606. bool block_contents_pinned,
  607. bool user_defined_timestamps_persisted,
  608. DataBlockHashIndex* data_block_hash_index,
  609. uint8_t protection_bytes_per_key, const char* kv_checksum,
  610. uint32_t block_restart_interval) {
  611. InitializeBase(raw_ucmp, data, restarts, num_restarts, global_seqno,
  612. block_contents_pinned, user_defined_timestamps_persisted,
  613. protection_bytes_per_key, kv_checksum,
  614. block_restart_interval);
  615. raw_key_.SetIsUserKey(false);
  616. read_amp_bitmap_ = read_amp_bitmap;
  617. last_bitmap_offset_ = current_ + 1;
  618. data_block_hash_index_ = data_block_hash_index;
  619. }
  620. Slice value() const override {
  621. assert(Valid());
  622. if (read_amp_bitmap_ && current_ < restarts_ &&
  623. current_ != last_bitmap_offset_) {
  624. read_amp_bitmap_->Mark(current_ /* current entry offset */,
  625. NextEntryOffset() - 1);
  626. last_bitmap_offset_ = current_;
  627. }
  628. return value_;
  629. }
  630. // Returns if `target` may exist.
  631. inline bool SeekForGet(const Slice& target) {
  632. #ifndef NDEBUG
  633. if (TEST_Corrupt_Callback("DataBlockIter::SeekForGet")) return true;
  634. #endif
  635. if (!data_block_hash_index_) {
  636. SeekImpl(target);
  637. UpdateKey();
  638. return true;
  639. }
  640. bool res = SeekForGetImpl(target);
  641. UpdateKey();
  642. return res;
  643. }
  644. void Invalidate(const Status& s) override {
  645. BlockIter::Invalidate(s);
  646. // Clear prev entries cache.
  647. prev_entries_keys_buff_.clear();
  648. prev_entries_.clear();
  649. prev_entries_idx_ = -1;
  650. }
  651. protected:
  652. friend Block;
  653. inline bool ParseNextDataKey(bool* is_shared);
  654. void SeekToFirstImpl() override;
  655. void SeekToLastImpl() override;
  656. void SeekImpl(const Slice& target) override;
  657. void SeekForPrevImpl(const Slice& target) override;
  658. void NextImpl() override;
  659. void PrevImpl() override;
  660. private:
  661. // read-amp bitmap
  662. BlockReadAmpBitmap* read_amp_bitmap_;
  663. // last `current_` value we report to read-amp bitmp
  664. mutable uint32_t last_bitmap_offset_;
  665. struct CachedPrevEntry {
  666. explicit CachedPrevEntry(uint32_t _offset, const char* _key_ptr,
  667. size_t _key_offset, size_t _key_size, Slice _value)
  668. : offset(_offset),
  669. key_ptr(_key_ptr),
  670. key_offset(_key_offset),
  671. key_size(_key_size),
  672. value(_value) {}
  673. // offset of entry in block
  674. uint32_t offset;
  675. // Pointer to key data in block (nullptr if key is delta-encoded)
  676. const char* key_ptr;
  677. // offset of key in prev_entries_keys_buff_ (0 if key_ptr is not nullptr)
  678. size_t key_offset;
  679. // size of key
  680. size_t key_size;
  681. // value slice pointing to data in block
  682. Slice value;
  683. };
  684. std::string prev_entries_keys_buff_;
  685. std::vector<CachedPrevEntry> prev_entries_;
  686. int32_t prev_entries_idx_ = -1;
  687. DataBlockHashIndex* data_block_hash_index_;
  688. bool SeekForGetImpl(const Slice& target);
  689. };
  690. // Iterator over MetaBlocks. MetaBlocks are similar to Data Blocks and
  691. // are used to store Properties associated with table.
  692. // Meta blocks always store user keys (no sequence number) and always
  693. // use the BytewiseComparator. Additionally, MetaBlock accesses are
  694. // not recorded in the Statistics or for Read-Amplification.
  695. class MetaBlockIter final : public BlockIter<Slice> {
  696. public:
  697. MetaBlockIter() : BlockIter() { raw_key_.SetIsUserKey(true); }
  698. void Initialize(const char* data, uint32_t restarts, uint32_t num_restarts,
  699. bool block_contents_pinned, uint8_t protection_bytes_per_key,
  700. const char* kv_checksum, uint32_t block_restart_interval) {
  701. // Initializes the iterator with a BytewiseComparator and
  702. // the raw key being a user key.
  703. InitializeBase(BytewiseComparator(), data, restarts, num_restarts,
  704. kDisableGlobalSequenceNumber, block_contents_pinned,
  705. /* user_defined_timestamps_persisted */ true,
  706. protection_bytes_per_key, kv_checksum,
  707. block_restart_interval);
  708. raw_key_.SetIsUserKey(true);
  709. }
  710. Slice value() const override {
  711. assert(Valid());
  712. return value_;
  713. }
  714. protected:
  715. friend Block;
  716. void SeekToFirstImpl() override;
  717. void SeekToLastImpl() override;
  718. void SeekImpl(const Slice& target) override;
  719. void SeekForPrevImpl(const Slice& target) override;
  720. void NextImpl() override;
  721. void PrevImpl() override;
  722. // Meta index block's restart interval is always 1. See
  723. // MetaIndexBuilder::MetaIndexBuilder() for hard-coded restart interval.
  724. uint32_t GetRestartInterval() override { return 1; }
  725. uint32_t NumberOfKeys(uint32_t) override { return num_restarts_; }
  726. };
  727. class IndexBlockIter final : public BlockIter<IndexValue> {
  728. public:
  729. IndexBlockIter() : BlockIter(), prefix_index_(nullptr) {}
  730. // key_includes_seq, default true, means that the keys are in internal key
  731. // format.
  732. // value_is_full, default true, means that no delta encoding is
  733. // applied to values.
  734. void Initialize(const Comparator* raw_ucmp, const char* data,
  735. uint32_t restarts, uint32_t num_restarts,
  736. SequenceNumber global_seqno, BlockPrefixIndex* prefix_index,
  737. bool have_first_key, bool key_includes_seq,
  738. bool value_is_full, bool block_contents_pinned,
  739. bool user_defined_timestamps_persisted,
  740. uint8_t protection_bytes_per_key, const char* kv_checksum,
  741. uint32_t block_restart_interval) {
  742. InitializeBase(raw_ucmp, data, restarts, num_restarts,
  743. kDisableGlobalSequenceNumber, block_contents_pinned,
  744. user_defined_timestamps_persisted, protection_bytes_per_key,
  745. kv_checksum, block_restart_interval);
  746. raw_key_.SetIsUserKey(!key_includes_seq);
  747. prefix_index_ = prefix_index;
  748. value_delta_encoded_ = !value_is_full;
  749. have_first_key_ = have_first_key;
  750. if (have_first_key_ && global_seqno != kDisableGlobalSequenceNumber) {
  751. global_seqno_state_.reset(new GlobalSeqnoState(global_seqno));
  752. } else {
  753. global_seqno_state_.reset();
  754. }
  755. }
  756. Slice user_key() const override {
  757. assert(Valid());
  758. return raw_key_.GetUserKey();
  759. }
  760. IndexValue value() const override {
  761. assert(Valid());
  762. if (value_delta_encoded_ || global_seqno_state_ != nullptr ||
  763. pad_min_timestamp_) {
  764. return decoded_value_;
  765. } else {
  766. IndexValue entry;
  767. Slice v = value_;
  768. Status decode_s __attribute__((__unused__)) =
  769. entry.DecodeFrom(&v, have_first_key_, nullptr);
  770. assert(decode_s.ok());
  771. return entry;
  772. }
  773. }
  774. Slice raw_value() const {
  775. assert(Valid());
  776. return value_;
  777. }
  778. bool IsValuePinned() const override {
  779. return global_seqno_state_ != nullptr ? false : BlockIter::IsValuePinned();
  780. }
  781. protected:
  782. friend Block;
  783. // IndexBlockIter follows a different contract for prefix iterator
  784. // from data iterators.
  785. // If prefix of the seek key `target` exists in the file, it must
  786. // return the same result as total order seek.
  787. // If the prefix of `target` doesn't exist in the file, it can either
  788. // return the result of total order seek, or set both of Valid() = false
  789. // and status() = NotFound().
  790. void SeekImpl(const Slice& target) override;
  791. void SeekForPrevImpl(const Slice&) override {
  792. assert(false);
  793. current_ = restarts_;
  794. restart_index_ = num_restarts_;
  795. status_ = Status::InvalidArgument(
  796. "RocksDB internal error: should never call SeekForPrev() on index "
  797. "blocks");
  798. raw_key_.Clear();
  799. value_.clear();
  800. }
  801. void PrevImpl() override;
  802. void NextImpl() override;
  803. void SeekToFirstImpl() override;
  804. void SeekToLastImpl() override;
  805. private:
  806. bool value_delta_encoded_;
  807. bool have_first_key_; // value includes first_internal_key
  808. BlockPrefixIndex* prefix_index_;
  809. // Whether the value is delta encoded. In that case the value is assumed to be
  810. // BlockHandle. The first value in each restart interval is the full encoded
  811. // BlockHandle; the restart of encoded size part of the BlockHandle. The
  812. // offset of delta encoded BlockHandles is computed by adding the size of
  813. // previous delta encoded values in the same restart interval to the offset of
  814. // the first value in that restart interval.
  815. IndexValue decoded_value_;
  816. // When sequence number overwriting is enabled, this struct contains the seqno
  817. // to overwrite with, and current first_internal_key with overwritten seqno.
  818. // This is rarely used, so we put it behind a pointer and only allocate when
  819. // needed.
  820. struct GlobalSeqnoState {
  821. // First internal key according to current index entry, but with sequence
  822. // number overwritten to global_seqno.
  823. IterKey first_internal_key;
  824. SequenceNumber global_seqno;
  825. explicit GlobalSeqnoState(SequenceNumber seqno) : global_seqno(seqno) {}
  826. };
  827. std::unique_ptr<GlobalSeqnoState> global_seqno_state_;
  828. // Buffers the `first_internal_key` referred by `decoded_value_` when
  829. // `pad_min_timestamp_` is true.
  830. std::string first_internal_key_with_ts_;
  831. // Set *prefix_may_exist to false if no key possibly share the same prefix
  832. // as `target`. If not set, the result position should be the same as total
  833. // order Seek.
  834. bool PrefixSeek(const Slice& target, uint32_t* index, bool* prefix_may_exist);
  835. // Set *prefix_may_exist to false if no key can possibly share the same
  836. // prefix as `target`. If not set, the result position should be the same
  837. // as total order seek.
  838. bool BinaryBlockIndexSeek(const Slice& target, uint32_t* block_ids,
  839. uint32_t left, uint32_t right, uint32_t* index,
  840. bool* prefix_may_exist);
  841. inline int CompareBlockKey(uint32_t block_index, const Slice& target);
  842. inline bool ParseNextIndexKey();
  843. // When value_delta_encoded_ is enabled it decodes the value which is assumed
  844. // to be BlockHandle and put it to decoded_value_
  845. inline void DecodeCurrentValue(bool is_shared);
  846. };
  847. } // namespace ROCKSDB_NAMESPACE