merging_iterator.cc 73 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761
  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. #include "table/merging_iterator.h"
  10. #include "db/arena_wrapped_db_iter.h"
  11. namespace ROCKSDB_NAMESPACE {
  12. // MergingIterator uses a min/max heap to combine data from point iterators.
  13. // Range tombstones can be added and keys covered by range tombstones will be
  14. // skipped.
  15. //
  16. // The following are implementation details and can be ignored by user.
  17. // For merging iterator to process range tombstones, it treats the start and end
  18. // keys of a range tombstone as two keys and put them into minHeap_ or maxHeap_
  19. // together with regular point keys. Each range tombstone is active only within
  20. // its internal key range [start_key, end_key). An `active_` set is used to
  21. // track levels that have an active range tombstone. Take forward scanning
  22. // for example. Level j is in active_ if its current range tombstone has its
  23. // start_key popped from minHeap_ and its end_key in minHeap_. If the top of
  24. // minHeap_ is a point key from level L, we can determine if the point key is
  25. // covered by any range tombstone by checking if there is an l <= L in active_.
  26. // The case of l == L also involves checking range tombstone's sequence number.
  27. //
  28. // The following (non-exhaustive) list of invariants are maintained by
  29. // MergingIterator during forward scanning. After each InternalIterator API,
  30. // i.e., Seek*() and Next(), and FindNextVisibleKey(), if minHeap_ is not empty:
  31. // (1) minHeap_.top().type == ITERATOR
  32. // (2) minHeap_.top()->key() is not covered by any range tombstone.
  33. //
  34. // After each call to SeekImpl() in addition to the functions mentioned above:
  35. // (3) For all level i and j <= i, range_tombstone_iters_[j].prev.end_key() <
  36. // children_[i].iter.key(). That is, range_tombstone_iters_[j] is at or before
  37. // the first range tombstone from level j with end_key() >
  38. // children_[i].iter.key().
  39. // (4) For all level i and j <= i, if j in active_, then
  40. // range_tombstone_iters_[j]->start_key() < children_[i].iter.key().
  41. // - When range_tombstone_iters_[j] is !Valid(), we consider its `prev` to be
  42. // the last range tombstone from that range tombstone iterator.
  43. // - When referring to range tombstone start/end keys, assume it is the value of
  44. // HeapItem::tombstone_pik. This value has op_type = kMaxValid, which makes
  45. // range tombstone keys have distinct values from point keys.
  46. //
  47. // Applicable class variables have their own (forward scanning) invariants
  48. // listed in the comments above their definition.
  49. class MergingIterator : public InternalIterator {
  50. public:
  51. MergingIterator(const InternalKeyComparator* comparator,
  52. InternalIterator** children, int n, bool is_arena_mode,
  53. bool prefix_seek_mode,
  54. const Slice* iterate_upper_bound = nullptr)
  55. : is_arena_mode_(is_arena_mode),
  56. prefix_seek_mode_(prefix_seek_mode),
  57. direction_(kForward),
  58. comparator_(comparator),
  59. current_(nullptr),
  60. minHeap_(MinHeapItemComparator(comparator_)),
  61. pinned_iters_mgr_(nullptr),
  62. iterate_upper_bound_(iterate_upper_bound) {
  63. children_.resize(n);
  64. for (int i = 0; i < n; i++) {
  65. children_[i].level = i;
  66. children_[i].iter.Set(children[i]);
  67. }
  68. }
  69. void considerStatus(Status s) {
  70. if (!s.ok() && status_.ok()) {
  71. status_ = s;
  72. }
  73. }
  74. virtual void AddIterator(InternalIterator* iter) {
  75. children_.emplace_back(children_.size(), iter);
  76. if (pinned_iters_mgr_) {
  77. iter->SetPinnedItersMgr(pinned_iters_mgr_);
  78. }
  79. // Invalidate to ensure `Seek*()` is called to construct the heaps before
  80. // use.
  81. current_ = nullptr;
  82. }
  83. // There must be either no range tombstone iterator or the same number of
  84. // range tombstone iterators as point iterators after all iters are added.
  85. // The i-th added range tombstone iterator and the i-th point iterator
  86. // must point to the same LSM level.
  87. // Merging iterator takes ownership of `iter` and is responsible for freeing
  88. // it. One exception to this is when a LevelIterator moves to a different SST
  89. // file or when Iterator::Refresh() is called, the range tombstone iterator
  90. // could be updated. In that case, this merging iterator is only responsible
  91. // for freeing the new range tombstone iterator that it has pointers to in
  92. // range_tombstone_iters_.
  93. void AddRangeTombstoneIterator(
  94. std::unique_ptr<TruncatedRangeDelIterator>&& iter) {
  95. range_tombstone_iters_.emplace_back(std::move(iter));
  96. }
  97. // Called by MergingIteratorBuilder when all point iterators and range
  98. // tombstone iterators are added. Initializes HeapItems for range tombstone
  99. // iterators.
  100. void Finish() {
  101. if (!range_tombstone_iters_.empty()) {
  102. assert(range_tombstone_iters_.size() == children_.size());
  103. pinned_heap_item_.resize(range_tombstone_iters_.size());
  104. for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
  105. pinned_heap_item_[i].level = i;
  106. // Range tombstone end key is exclusive. If a point internal key has the
  107. // same user key and sequence number as the start or end key of a range
  108. // tombstone, the order will be start < end key < internal key with the
  109. // following op_type change. This is helpful to ensure keys popped from
  110. // heap are in expected order since range tombstone start/end keys will
  111. // be distinct from point internal keys. Strictly speaking, this is only
  112. // needed for tombstone end points that are truncated in
  113. // TruncatedRangeDelIterator since untruncated tombstone end points
  114. // always have kMaxSequenceNumber and kTypeRangeDeletion (see
  115. // TruncatedRangeDelIterator::start_key()/end_key()).
  116. pinned_heap_item_[i].tombstone_pik.type = kTypeMaxValid;
  117. }
  118. }
  119. }
  120. ~MergingIterator() override {
  121. range_tombstone_iters_.clear();
  122. for (auto& child : children_) {
  123. child.iter.DeleteIter(is_arena_mode_);
  124. }
  125. status_.PermitUncheckedError();
  126. }
  127. void SetRangeDelReadSeqno(SequenceNumber read_seqno) override {
  128. for (auto& child : children_) {
  129. // This should only be needed for LevelIterator (iterators from L1+).
  130. child.iter.SetRangeDelReadSeqno(read_seqno);
  131. }
  132. for (auto& child : range_tombstone_iters_) {
  133. if (child) {
  134. child->SetRangeDelReadSeqno(read_seqno);
  135. }
  136. }
  137. }
  138. bool Valid() const override { return current_ != nullptr && status_.ok(); }
  139. Status status() const override { return status_; }
  140. // Add range_tombstone_iters_[level] into min heap.
  141. // Updates active_ if the end key of a range tombstone is inserted.
  142. // pinned_heap_items_[level].type is updated based on `start_key`.
  143. //
  144. // If range_tombstone_iters_[level] is after iterate_upper_bound_,
  145. // it is removed from the heap.
  146. // @param start_key specifies which end point of the range tombstone to add.
  147. void InsertRangeTombstoneToMinHeap(size_t level, bool start_key = true,
  148. bool replace_top = false) {
  149. assert(!range_tombstone_iters_.empty() &&
  150. range_tombstone_iters_[level]->Valid());
  151. // Maintains Invariant(phi)
  152. if (start_key) {
  153. pinned_heap_item_[level].type = HeapItem::Type::DELETE_RANGE_START;
  154. ParsedInternalKey pik = range_tombstone_iters_[level]->start_key();
  155. // iterate_upper_bound does not have timestamp
  156. if (iterate_upper_bound_ &&
  157. comparator_->user_comparator()->CompareWithoutTimestamp(
  158. pik.user_key, true /* a_has_ts */, *iterate_upper_bound_,
  159. false /* b_has_ts */) >= 0) {
  160. if (replace_top) {
  161. // replace_top implies this range tombstone iterator is still in
  162. // minHeap_ and at the top.
  163. minHeap_.pop();
  164. }
  165. return;
  166. }
  167. pinned_heap_item_[level].SetTombstoneKey(std::move(pik));
  168. // Checks Invariant(active_)
  169. assert(active_.count(level) == 0);
  170. } else {
  171. // allow end key to go over upper bound (if present) since start key is
  172. // before upper bound and the range tombstone could still cover a
  173. // range before upper bound.
  174. // Maintains Invariant(active_)
  175. pinned_heap_item_[level].SetTombstoneKey(
  176. range_tombstone_iters_[level]->end_key());
  177. pinned_heap_item_[level].type = HeapItem::Type::DELETE_RANGE_END;
  178. active_.insert(level);
  179. }
  180. if (replace_top) {
  181. minHeap_.replace_top(&pinned_heap_item_[level]);
  182. } else {
  183. minHeap_.push(&pinned_heap_item_[level]);
  184. }
  185. }
  186. // Add range_tombstone_iters_[level] into max heap.
  187. // Updates active_ if the start key of a range tombstone is inserted.
  188. // @param end_key specifies which end point of the range tombstone to add.
  189. void InsertRangeTombstoneToMaxHeap(size_t level, bool end_key = true,
  190. bool replace_top = false) {
  191. assert(!range_tombstone_iters_.empty() &&
  192. range_tombstone_iters_[level]->Valid());
  193. if (end_key) {
  194. pinned_heap_item_[level].SetTombstoneKey(
  195. range_tombstone_iters_[level]->end_key());
  196. pinned_heap_item_[level].type = HeapItem::Type::DELETE_RANGE_END;
  197. assert(active_.count(level) == 0);
  198. } else {
  199. pinned_heap_item_[level].SetTombstoneKey(
  200. range_tombstone_iters_[level]->start_key());
  201. pinned_heap_item_[level].type = HeapItem::Type::DELETE_RANGE_START;
  202. active_.insert(level);
  203. }
  204. if (replace_top) {
  205. maxHeap_->replace_top(&pinned_heap_item_[level]);
  206. } else {
  207. maxHeap_->push(&pinned_heap_item_[level]);
  208. }
  209. }
  210. // Remove HeapItems from top of minHeap_ that are of type DELETE_RANGE_START
  211. // until minHeap_ is empty or the top of the minHeap_ is not of type
  212. // DELETE_RANGE_START. Each such item means a range tombstone becomes active,
  213. // so `active_` is updated accordingly.
  214. void PopDeleteRangeStart() {
  215. while (!minHeap_.empty() &&
  216. minHeap_.top()->type == HeapItem::Type::DELETE_RANGE_START) {
  217. TEST_SYNC_POINT_CALLBACK("MergeIterator::PopDeleteRangeStart", nullptr);
  218. // Invariant(rti) holds since
  219. // range_tombstone_iters_[minHeap_.top()->level] is still valid, and
  220. // parameter `replace_top` is set to true here to ensure only one such
  221. // HeapItem is in minHeap_.
  222. InsertRangeTombstoneToMinHeap(
  223. minHeap_.top()->level, false /* start_key */, true /* replace_top */);
  224. }
  225. }
  226. // Remove HeapItems from top of maxHeap_ that are of type DELETE_RANGE_END
  227. // until maxHeap_ is empty or the top of the maxHeap_ is not of type
  228. // DELETE_RANGE_END. Each such item means a range tombstone becomes active,
  229. // so `active_` is updated accordingly.
  230. void PopDeleteRangeEnd() {
  231. while (!maxHeap_->empty() &&
  232. maxHeap_->top()->type == HeapItem::Type::DELETE_RANGE_END) {
  233. // insert start key of this range tombstone and updates active_
  234. InsertRangeTombstoneToMaxHeap(maxHeap_->top()->level, false /* end_key */,
  235. true /* replace_top */);
  236. }
  237. }
  238. void SeekToFirst() override {
  239. ClearHeaps();
  240. status_ = Status::OK();
  241. for (auto& child : children_) {
  242. child.iter.SeekToFirst();
  243. AddToMinHeapOrCheckStatus(&child);
  244. }
  245. for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
  246. if (range_tombstone_iters_[i]) {
  247. range_tombstone_iters_[i]->SeekToFirst();
  248. if (range_tombstone_iters_[i]->Valid()) {
  249. // It is possible to be invalid due to snapshots.
  250. InsertRangeTombstoneToMinHeap(i);
  251. }
  252. }
  253. }
  254. FindNextVisibleKey();
  255. direction_ = kForward;
  256. current_ = CurrentForward();
  257. }
  258. void SeekToLast() override {
  259. ClearHeaps();
  260. InitMaxHeap();
  261. status_ = Status::OK();
  262. for (auto& child : children_) {
  263. child.iter.SeekToLast();
  264. AddToMaxHeapOrCheckStatus(&child);
  265. }
  266. for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
  267. if (range_tombstone_iters_[i]) {
  268. range_tombstone_iters_[i]->SeekToLast();
  269. if (range_tombstone_iters_[i]->Valid()) {
  270. // It is possible to be invalid due to snapshots.
  271. InsertRangeTombstoneToMaxHeap(i);
  272. }
  273. }
  274. }
  275. FindPrevVisibleKey();
  276. direction_ = kReverse;
  277. current_ = CurrentReverse();
  278. }
  279. // Position this merging iterator at the first key >= target (internal key).
  280. // If range tombstones are present, keys covered by range tombstones are
  281. // skipped, and this merging iter points to the first non-range-deleted key >=
  282. // target after Seek(). If !Valid() and status().ok() then this iterator
  283. // reaches the end.
  284. //
  285. // If range tombstones are present, cascading seeks may be called (an
  286. // optimization adapted from Pebble https://github.com/cockroachdb/pebble).
  287. // Roughly, if there is a range tombstone [start, end) that covers the
  288. // target user key at level L, then this range tombstone must cover the range
  289. // [target key, end) in all levels > L. So for all levels > L, we can pretend
  290. // the target key is `end`. This optimization is applied at each level and
  291. // hence the name "cascading seek".
  292. void Seek(const Slice& target) override {
  293. // Define LevelNextVisible(i, k) to be the first key >= k in level i that is
  294. // not covered by any range tombstone.
  295. // After SeekImpl(target, 0), invariants (3) and (4) hold.
  296. // For all level i, target <= children_[i].iter.key() <= LevelNextVisible(i,
  297. // target). By the contract of FindNextVisibleKey(), Invariants (1)-(4)
  298. // holds after this call, and minHeap_.top().iter points to the
  299. // first key >= target among children_ that is not covered by any range
  300. // tombstone.
  301. status_ = Status::OK();
  302. SeekImpl(target);
  303. FindNextVisibleKey();
  304. direction_ = kForward;
  305. {
  306. PERF_TIMER_GUARD(seek_min_heap_time);
  307. current_ = CurrentForward();
  308. }
  309. }
  310. void SeekForPrev(const Slice& target) override {
  311. assert(range_tombstone_iters_.empty() ||
  312. range_tombstone_iters_.size() == children_.size());
  313. status_ = Status::OK();
  314. SeekForPrevImpl(target);
  315. FindPrevVisibleKey();
  316. direction_ = kReverse;
  317. {
  318. PERF_TIMER_GUARD(seek_max_heap_time);
  319. current_ = CurrentReverse();
  320. }
  321. }
  322. void Next() override {
  323. assert(Valid());
  324. // Ensure that all children are positioned after key().
  325. // If we are moving in the forward direction, it is already
  326. // true for all the non-current children since current_ is
  327. // the smallest child and key() == current_->key().
  328. if (direction_ != kForward) {
  329. // The loop advanced all non-current children to be > key() so current_
  330. // should still be strictly the smallest key.
  331. SwitchToForward();
  332. }
  333. // For the heap modifications below to be correct, current_ must be the
  334. // current top of the heap.
  335. assert(current_ == CurrentForward());
  336. // as the current points to the current record. move the iterator forward.
  337. current_->Next();
  338. if (current_->Valid()) {
  339. // current is still valid after the Next() call above. Call
  340. // replace_top() to restore the heap property. When the same child
  341. // iterator yields a sequence of keys, this is cheap.
  342. assert(current_->status().ok());
  343. minHeap_.replace_top(minHeap_.top());
  344. } else {
  345. // current stopped being valid, remove it from the heap.
  346. considerStatus(current_->status());
  347. minHeap_.pop();
  348. }
  349. // Invariants (3) and (4) hold when after advancing current_.
  350. // Let k be the smallest key among children_[i].iter.key().
  351. // k <= children_[i].iter.key() <= LevelNextVisible(i, k) holds for all
  352. // level i. After FindNextVisible(), Invariants (1)-(4) hold and
  353. // minHeap_.top()->key() is the first key >= k from any children_ that is
  354. // not covered by any range tombstone.
  355. FindNextVisibleKey();
  356. current_ = CurrentForward();
  357. }
  358. bool NextAndGetResult(IterateResult* result) override {
  359. Next();
  360. bool is_valid = Valid();
  361. if (is_valid) {
  362. result->key = key();
  363. result->bound_check_result = UpperBoundCheckResult();
  364. result->value_prepared = current_->IsValuePrepared();
  365. }
  366. return is_valid;
  367. }
  368. void Prev() override {
  369. assert(Valid());
  370. // Ensure that all children are positioned before key().
  371. // If we are moving in the reverse direction, it is already
  372. // true for all the non-current children since current_ is
  373. // the largest child and key() == current_->key().
  374. if (direction_ != kReverse) {
  375. // Otherwise, retreat the non-current children. We retreat current_
  376. // just after the if-block.
  377. SwitchToBackward();
  378. }
  379. // For the heap modifications below to be correct, current_ must be the
  380. // current top of the heap.
  381. assert(current_ == CurrentReverse());
  382. current_->Prev();
  383. if (current_->Valid()) {
  384. // current is still valid after the Prev() call above. Call
  385. // replace_top() to restore the heap property. When the same child
  386. // iterator yields a sequence of keys, this is cheap.
  387. assert(current_->status().ok());
  388. maxHeap_->replace_top(maxHeap_->top());
  389. } else {
  390. // current stopped being valid, remove it from the heap.
  391. considerStatus(current_->status());
  392. maxHeap_->pop();
  393. }
  394. FindPrevVisibleKey();
  395. current_ = CurrentReverse();
  396. }
  397. Slice key() const override {
  398. assert(Valid());
  399. return current_->key();
  400. }
  401. uint64_t write_unix_time() const override {
  402. assert(Valid());
  403. return current_->write_unix_time();
  404. }
  405. Slice value() const override {
  406. assert(Valid());
  407. return current_->value();
  408. }
  409. bool PrepareValue() override {
  410. assert(Valid());
  411. if (current_->PrepareValue()) {
  412. return true;
  413. }
  414. considerStatus(current_->status());
  415. assert(!status_.ok());
  416. return false;
  417. }
  418. // Here we simply relay MayBeOutOfLowerBound/MayBeOutOfUpperBound result
  419. // from current child iterator. Potentially as long as one of child iterator
  420. // report out of bound is not possible, we know current key is within bound.
  421. bool MayBeOutOfLowerBound() override {
  422. assert(Valid());
  423. return current_->MayBeOutOfLowerBound();
  424. }
  425. IterBoundCheck UpperBoundCheckResult() override {
  426. assert(Valid());
  427. return current_->UpperBoundCheckResult();
  428. }
  429. void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
  430. pinned_iters_mgr_ = pinned_iters_mgr;
  431. for (auto& child : children_) {
  432. child.iter.SetPinnedItersMgr(pinned_iters_mgr);
  433. }
  434. }
  435. bool IsKeyPinned() const override {
  436. assert(Valid());
  437. return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
  438. current_->IsKeyPinned();
  439. }
  440. bool IsValuePinned() const override {
  441. assert(Valid());
  442. return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
  443. current_->IsValuePinned();
  444. }
  445. void Prepare(const MultiScanArgs* scan_opts) override {
  446. for (auto& child : children_) {
  447. child.iter.Prepare(scan_opts);
  448. }
  449. }
  450. private:
  451. // Represents an element in the min/max heap. Each HeapItem corresponds to a
  452. // point iterator or a range tombstone iterator, differentiated by
  453. // HeapItem::type.
  454. struct HeapItem {
  455. HeapItem() = default;
  456. // corresponding point iterator
  457. IteratorWrapper iter;
  458. size_t level = 0;
  459. // corresponding range tombstone iterator's start or end key value
  460. // depending on value of `type`.
  461. ParsedInternalKey tombstone_pik;
  462. // Will be overwritten before use, initialize here so compiler does not
  463. // complain.
  464. enum class Type { ITERATOR, DELETE_RANGE_START, DELETE_RANGE_END };
  465. Type type = Type::ITERATOR;
  466. explicit HeapItem(size_t _level, InternalIteratorBase<Slice>* _iter)
  467. : level(_level), type(Type::ITERATOR) {
  468. iter.Set(_iter);
  469. }
  470. void SetTombstoneKey(ParsedInternalKey&& pik) {
  471. // op_type is already initialized in MergingIterator::Finish().
  472. tombstone_pik.user_key = pik.user_key;
  473. tombstone_pik.sequence = pik.sequence;
  474. }
  475. };
  476. class MinHeapItemComparator {
  477. public:
  478. explicit MinHeapItemComparator(const InternalKeyComparator* comparator)
  479. : comparator_(comparator) {}
  480. bool operator()(HeapItem* a, HeapItem* b) const {
  481. if (LIKELY(a->type == HeapItem::Type::ITERATOR)) {
  482. if (LIKELY(b->type == HeapItem::Type::ITERATOR)) {
  483. return comparator_->Compare(a->iter.key(), b->iter.key()) > 0;
  484. } else {
  485. return comparator_->Compare(a->iter.key(), b->tombstone_pik) > 0;
  486. }
  487. } else {
  488. if (LIKELY(b->type == HeapItem::Type::ITERATOR)) {
  489. return comparator_->Compare(a->tombstone_pik, b->iter.key()) > 0;
  490. } else {
  491. return comparator_->Compare(a->tombstone_pik, b->tombstone_pik) > 0;
  492. }
  493. }
  494. }
  495. private:
  496. const InternalKeyComparator* comparator_;
  497. };
  498. class MaxHeapItemComparator {
  499. public:
  500. explicit MaxHeapItemComparator(const InternalKeyComparator* comparator)
  501. : comparator_(comparator) {}
  502. bool operator()(HeapItem* a, HeapItem* b) const {
  503. if (LIKELY(a->type == HeapItem::Type::ITERATOR)) {
  504. if (LIKELY(b->type == HeapItem::Type::ITERATOR)) {
  505. return comparator_->Compare(a->iter.key(), b->iter.key()) < 0;
  506. } else {
  507. return comparator_->Compare(a->iter.key(), b->tombstone_pik) < 0;
  508. }
  509. } else {
  510. if (LIKELY(b->type == HeapItem::Type::ITERATOR)) {
  511. return comparator_->Compare(a->tombstone_pik, b->iter.key()) < 0;
  512. } else {
  513. return comparator_->Compare(a->tombstone_pik, b->tombstone_pik) < 0;
  514. }
  515. }
  516. }
  517. private:
  518. const InternalKeyComparator* comparator_;
  519. };
  520. using MergerMinIterHeap = BinaryHeap<HeapItem*, MinHeapItemComparator>;
  521. using MergerMaxIterHeap = BinaryHeap<HeapItem*, MaxHeapItemComparator>;
  522. friend class MergeIteratorBuilder;
  523. // Clears heaps for both directions, used when changing direction or seeking
  524. void ClearHeaps(bool clear_active = true);
  525. // Ensures that maxHeap_ is initialized when starting to go in the reverse
  526. // direction
  527. void InitMaxHeap();
  528. // Advance this merging iterator until the current key (minHeap_.top()) is
  529. // from a point iterator and is not covered by any range tombstone,
  530. // or that there is no more keys (heap is empty). SeekImpl() may be called
  531. // to seek to the end of a range tombstone as an optimization.
  532. void FindNextVisibleKey();
  533. void FindPrevVisibleKey();
  534. // Advance this merging iterators to the first key >= `target` for all
  535. // components from levels >= starting_level. All iterators before
  536. // starting_level are untouched.
  537. //
  538. // @param range_tombstone_reseek Whether target is some range tombstone
  539. // end, i.e., whether this SeekImpl() call is a part of a "cascading seek".
  540. // This is used only for recoding relevant perf_context.
  541. void SeekImpl(const Slice& target, size_t starting_level = 0,
  542. bool range_tombstone_reseek = false);
  543. // Seek to fist key <= target key (internal key) for
  544. // children_[starting_level:].
  545. void SeekForPrevImpl(const Slice& target, size_t starting_level = 0,
  546. bool range_tombstone_reseek = false);
  547. bool is_arena_mode_;
  548. bool prefix_seek_mode_;
  549. // Which direction is the iterator moving?
  550. enum Direction : uint8_t { kForward, kReverse };
  551. Direction direction_;
  552. const InternalKeyComparator* comparator_;
  553. // HeapItem for all child point iterators.
  554. // Invariant(children_): children_[i] is in minHeap_ iff
  555. // children_[i].iter.Valid(), and at most one children_[i] is in minHeap_.
  556. // TODO: We could use an autovector with a larger reserved size.
  557. std::vector<HeapItem> children_;
  558. // HeapItem for range tombstone start and end keys.
  559. // pinned_heap_item_[i] corresponds to range_tombstone_iters_[i].
  560. // Invariant(phi): If range_tombstone_iters_[i]->Valid(),
  561. // pinned_heap_item_[i].tombstone_pik is equal to
  562. // range_tombstone_iters_[i]->start_key() when
  563. // pinned_heap_item_[i].type is DELETE_RANGE_START and
  564. // range_tombstone_iters_[i]->end_key() when
  565. // pinned_heap_item_[i].type is DELETE_RANGE_END (ignoring op_type which is
  566. // kMaxValid for all pinned_heap_item_.tombstone_pik).
  567. // pinned_heap_item_[i].type is either DELETE_RANGE_START or DELETE_RANGE_END.
  568. std::vector<HeapItem> pinned_heap_item_;
  569. // range_tombstone_iters_[i] contains range tombstones in the sorted run that
  570. // corresponds to children_[i]. range_tombstone_iters_.empty() means not
  571. // handling range tombstones in merging iterator. range_tombstone_iters_[i] ==
  572. // nullptr means the sorted run of children_[i] does not have range
  573. // tombstones.
  574. // Invariant(rti): pinned_heap_item_[i] is in minHeap_ iff
  575. // range_tombstone_iters_[i]->Valid() and at most one pinned_heap_item_[i] is
  576. // in minHeap_.
  577. std::vector<std::unique_ptr<TruncatedRangeDelIterator>>
  578. range_tombstone_iters_;
  579. // Levels (indices into range_tombstone_iters_/children_ ) that currently have
  580. // "active" range tombstones. See comments above MergingIterator for meaning
  581. // of "active".
  582. // Invariant(active_): i is in active_ iff range_tombstone_iters_[i]->Valid()
  583. // and pinned_heap_item_[i].type == DELETE_RANGE_END.
  584. std::set<size_t> active_;
  585. bool SkipNextDeleted();
  586. bool SkipPrevDeleted();
  587. // Invariant: at the end of each InternalIterator API,
  588. // current_ points to minHeap_.top().iter (maxHeap_ if backward scanning)
  589. // or nullptr if no child iterator is valid.
  590. // This follows from that current_ = CurrentForward()/CurrentReverse() is
  591. // called at the end of each InternalIterator API.
  592. IteratorWrapper* current_;
  593. // If any of the children have non-ok status, this is one of them.
  594. Status status_;
  595. // Invariant: min heap property is maintained (parent is always <= child).
  596. // This holds by using only BinaryHeap APIs to modify heap. One
  597. // exception is to modify heap top item directly (by caller iter->Next()), and
  598. // it should be followed by a call to replace_top() or pop().
  599. MergerMinIterHeap minHeap_;
  600. // Max heap is used for reverse iteration, which is way less common than
  601. // forward. Lazily initialize it to save memory.
  602. std::unique_ptr<MergerMaxIterHeap> maxHeap_;
  603. PinnedIteratorsManager* pinned_iters_mgr_;
  604. // Used to bound range tombstones. For point keys, DBIter and SSTable iterator
  605. // take care of boundary checking.
  606. const Slice* iterate_upper_bound_;
  607. // In forward direction, process a child that is not in the min heap.
  608. // If valid, add to the min heap. Otherwise, check status.
  609. void AddToMinHeapOrCheckStatus(HeapItem*);
  610. // In backward direction, process a child that is not in the max heap.
  611. // If valid, add to the min heap. Otherwise, check status.
  612. void AddToMaxHeapOrCheckStatus(HeapItem*);
  613. void SwitchToForward();
  614. // Switch the direction from forward to backward without changing the
  615. // position. Iterator should still be valid.
  616. void SwitchToBackward();
  617. IteratorWrapper* CurrentForward() const {
  618. assert(direction_ == kForward);
  619. assert(minHeap_.empty() ||
  620. minHeap_.top()->type == HeapItem::Type::ITERATOR);
  621. return !minHeap_.empty() ? &minHeap_.top()->iter : nullptr;
  622. }
  623. IteratorWrapper* CurrentReverse() const {
  624. assert(direction_ == kReverse);
  625. assert(maxHeap_);
  626. assert(maxHeap_->empty() ||
  627. maxHeap_->top()->type == HeapItem::Type::ITERATOR);
  628. return !maxHeap_->empty() ? &maxHeap_->top()->iter : nullptr;
  629. }
  630. };
  631. // Pre-condition:
  632. // - Invariants (3) and (4) hold for i < starting_level
  633. // - For i < starting_level, range_tombstone_iters_[i].prev.end_key() <
  634. // `target`.
  635. // - For i < starting_level, if i in active_, then
  636. // range_tombstone_iters_[i]->start_key() < `target`.
  637. //
  638. // Post-condition:
  639. // - Invariants (3) and (4) hold for all level i.
  640. // - (*) target <= children_[i].iter.key() <= LevelNextVisible(i, target)
  641. // for i >= starting_level
  642. // - (**) target < pinned_heap_item_[i].tombstone_pik if
  643. // range_tombstone_iters_[i].Valid() for i >= starting_level
  644. //
  645. // Proof sketch:
  646. // Invariant (3) holds for all level i.
  647. // For j <= i < starting_level, it follows from Pre-condition that (3) holds
  648. // and that SeekImpl(-, starting_level) does not update children_[i] or
  649. // range_tombstone_iters_[j].
  650. // For j < starting_level and i >= starting_level, it follows from
  651. // - Pre-condition that range_tombstone_iters_[j].prev.end_key() < `target`
  652. // - range_tombstone_iters_[j] is not updated in SeekImpl(), and
  653. // - children_[i].iter.Seek(current_search_key) is called with
  654. // current_search_key >= target (shown below).
  655. // When current_search_key is updated, it is updated to some
  656. // range_tombstone_iter->end_key() after
  657. // range_tombstone_iter->SeekInternalKey(current_search_key) was called. So
  658. // current_search_key increases if updated and >= target.
  659. // For starting_level <= j <= i:
  660. // children_[i].iter.Seek(k1) and range_tombstone_iters_[j]->SeekInternalKey(k2)
  661. // are called in SeekImpl(). Seek(k1) positions children_[i] at the first key >=
  662. // k1 from level i. SeekInternalKey(k2) positions range_tombstone_iters_[j] at
  663. // the first range tombstone from level j with end_key() > k2. It suffices to
  664. // show that k1 >= k2. Since k1 and k2 are values of current_search_key where
  665. // k1 = k2 or k1 is value of a later current_search_key than k2, so k1 >= k2.
  666. //
  667. // Invariant (4) holds for all level >= 0.
  668. // By Pre-condition Invariant (4) holds for i < starting_level.
  669. // Since children_[i], range_tombstone_iters_[i] and contents of active_ for
  670. // i < starting_level do not change (4) holds for j <= i < starting_level.
  671. // By Pre-condition: for all j < starting_level, if j in active_, then
  672. // range_tombstone_iters_[j]->start_key() < target. For i >= starting_level,
  673. // children_[i].iter.Seek(k) is called for k >= target. So
  674. // children_[i].iter.key() >= target > range_tombstone_iters_[j]->start_key()
  675. // for j < starting_level and i >= starting_level. So invariant (4) holds for
  676. // j < starting_level and i >= starting_level.
  677. // For starting_level <= j <= i, j is added to active_ only if
  678. // - range_tombstone_iters_[j]->SeekInternalKey(k1) was called
  679. // - range_tombstone_iters_[j]->start_key() <= k1
  680. // Since children_[i].iter.Seek(k2) is called for some k2 >= k1 and for all
  681. // starting_level <= j <= i, (4) also holds for all starting_level <= j <= i.
  682. //
  683. // Post-condition (*): target <= children_[i].iter.key() <= LevelNextVisible(i,
  684. // target) for i >= starting_level.
  685. // target <= children_[i].iter.key() follows from that Seek() is called on some
  686. // current_search_key >= target for children_[i].iter. If current_search_key
  687. // is updated from k1 to k2 when level = i, we show that the range [k1, k2) is
  688. // not visible for children_[j] for any j > i. When current_search_key is
  689. // updated from k1 to k2,
  690. // - range_tombstone_iters_[i]->SeekInternalKey(k1) was called
  691. // - range_tombstone_iters_[i]->Valid()
  692. // - range_tombstone_iters_[i]->start_key().user_key <= k1.user_key
  693. // - k2 = range_tombstone_iters_[i]->end_key()
  694. // We assume that range_tombstone_iters_[i]->start_key() has a higher sequence
  695. // number compared to any key from levels > i that has the same user key. So no
  696. // point key from levels > i in range [k1, k2) is visible. So
  697. // children_[i].iter.key() <= LevelNextVisible(i, target).
  698. //
  699. // Post-condition (**) target < pinned_heap_item_[i].tombstone_pik for i >=
  700. // starting_level if range_tombstone_iters_[i].Valid(). This follows from that
  701. // SeekInternalKey() being called for each range_tombstone_iters_ with some key
  702. // >= `target` and that we pick start/end key that is > `target` to insert to
  703. // minHeap_.
  704. void MergingIterator::SeekImpl(const Slice& target, size_t starting_level,
  705. bool range_tombstone_reseek) {
  706. // active range tombstones before `starting_level` remain active
  707. ClearHeaps(false /* clear_active */);
  708. ParsedInternalKey pik;
  709. if (!range_tombstone_iters_.empty()) {
  710. // pik is only used in InsertRangeTombstoneToMinHeap().
  711. ParseInternalKey(target, &pik, false).PermitUncheckedError();
  712. }
  713. // TODO: perhaps we could save some upheap cost by add all child iters first
  714. // and then do a single heapify.
  715. // Invariant(children_) for level < starting_level
  716. for (size_t level = 0; level < starting_level; ++level) {
  717. PERF_TIMER_GUARD(seek_min_heap_time);
  718. AddToMinHeapOrCheckStatus(&children_[level]);
  719. }
  720. if (!range_tombstone_iters_.empty()) {
  721. // Add range tombstones from levels < starting_level. We can insert from
  722. // pinned_heap_item_ for the following reasons:
  723. // - pinned_heap_item_[level] is in minHeap_ iff
  724. // range_tombstone_iters[level]->Valid().
  725. // - If `level` is in active_, then range_tombstone_iters_[level]->Valid()
  726. // and pinned_heap_item_[level] is of type RANGE_DELETION_END.
  727. for (size_t level = 0; level < starting_level; ++level) {
  728. // Restores Invariants(rti), (phi) and (active_) for level <
  729. // starting_level
  730. if (range_tombstone_iters_[level] &&
  731. range_tombstone_iters_[level]->Valid()) {
  732. // use an iterator on active_ if performance becomes an issue here
  733. if (active_.count(level) > 0) {
  734. assert(pinned_heap_item_[level].type ==
  735. HeapItem::Type::DELETE_RANGE_END);
  736. // if it was active, then start key must be within upper_bound,
  737. // so we can add to minHeap_ directly.
  738. minHeap_.push(&pinned_heap_item_[level]);
  739. } else {
  740. assert(pinned_heap_item_[level].type ==
  741. HeapItem::Type::DELETE_RANGE_START);
  742. // this takes care of checking iterate_upper_bound, but with an extra
  743. // key comparison if range_tombstone_iters_[level] was already out of
  744. // bound. Consider using a new HeapItem type or some flag to remember
  745. // boundary checking result.
  746. InsertRangeTombstoneToMinHeap(level);
  747. }
  748. } else {
  749. assert(!active_.count(level));
  750. }
  751. }
  752. // levels >= starting_level will be reseeked below, so clearing their active
  753. // state here.
  754. active_.erase(active_.lower_bound(starting_level), active_.end());
  755. }
  756. IterKey current_search_key;
  757. current_search_key.SetInternalKey(target, false /* copy */);
  758. // Seek target might change to some range tombstone end key, so
  759. // we need to remember them for async requests.
  760. // (level, target) pairs
  761. autovector<std::pair<size_t, std::string>> prefetched_target;
  762. for (auto level = starting_level; level < children_.size(); ++level) {
  763. {
  764. PERF_TIMER_GUARD(seek_child_seek_time);
  765. children_[level].iter.Seek(current_search_key.GetInternalKey());
  766. }
  767. PERF_COUNTER_ADD(seek_child_seek_count, 1);
  768. if (!range_tombstone_iters_.empty()) {
  769. if (range_tombstone_reseek) {
  770. // This seek is to some range tombstone end key.
  771. // Should only happen when there are range tombstones.
  772. PERF_COUNTER_ADD(internal_range_del_reseek_count, 1);
  773. }
  774. if (children_[level].iter.status().IsTryAgain()) {
  775. prefetched_target.emplace_back(
  776. level, current_search_key.GetInternalKey().ToString());
  777. }
  778. UnownedPtr<TruncatedRangeDelIterator> range_tombstone_iter =
  779. range_tombstone_iters_[level].get();
  780. if (range_tombstone_iter) {
  781. range_tombstone_iter->SeekInternalKey(
  782. current_search_key.GetInternalKey());
  783. // Invariants (rti) and (phi)
  784. if (range_tombstone_iter->Valid()) {
  785. // If range tombstone starts after `current_search_key`,
  786. // we should insert start key to heap as the range tombstone is not
  787. // active yet.
  788. InsertRangeTombstoneToMinHeap(
  789. level, comparator_->Compare(range_tombstone_iter->start_key(),
  790. pik) > 0 /* start_key */);
  791. // current_search_key < end_key guaranteed by the SeekInternalKey()
  792. // and Valid() calls above. Here we only need to compare user_key
  793. // since if target.user_key ==
  794. // range_tombstone_iter->start_key().user_key and target <
  795. // range_tombstone_iter->start_key(), no older level would have any
  796. // key in range [target, range_tombstone_iter->start_key()], so no
  797. // keys in range [target, range_tombstone_iter->end_key()) from older
  798. // level would be visible. So it is safe to seek to
  799. // range_tombstone_iter->end_key().
  800. //
  801. // TODO: range_tombstone_iter->Seek() finds the max covering
  802. // sequence number, can make it cheaper by not looking for max.
  803. if (comparator_->user_comparator()->Compare(
  804. range_tombstone_iter->start_key().user_key,
  805. current_search_key.GetUserKey()) <= 0) {
  806. range_tombstone_reseek = true;
  807. // Note that for prefix seek case, it is possible that the prefix
  808. // is not the same as the original target, it should not affect
  809. // correctness. Besides, in most cases, range tombstone start and
  810. // end key should have the same prefix?
  811. current_search_key.SetInternalKey(range_tombstone_iter->end_key());
  812. }
  813. }
  814. }
  815. }
  816. // child.iter.status() is set to Status::TryAgain indicating asynchronous
  817. // request for retrieval of data blocks has been submitted. So it should
  818. // return at this point and Seek should be called again to retrieve the
  819. // requested block and add the child to min heap.
  820. if (children_[level].iter.status().IsTryAgain()) {
  821. continue;
  822. }
  823. {
  824. // Strictly, we timed slightly more than min heap operation,
  825. // but these operations are very cheap.
  826. PERF_TIMER_GUARD(seek_min_heap_time);
  827. AddToMinHeapOrCheckStatus(&children_[level]);
  828. }
  829. }
  830. if (range_tombstone_iters_.empty()) {
  831. for (auto& child : children_) {
  832. if (child.iter.status().IsTryAgain()) {
  833. child.iter.Seek(target);
  834. {
  835. PERF_TIMER_GUARD(seek_min_heap_time);
  836. AddToMinHeapOrCheckStatus(&child);
  837. }
  838. PERF_COUNTER_ADD(number_async_seek, 1);
  839. }
  840. }
  841. } else {
  842. for (auto& prefetch : prefetched_target) {
  843. // (level, target) pairs
  844. children_[prefetch.first].iter.Seek(prefetch.second);
  845. {
  846. PERF_TIMER_GUARD(seek_min_heap_time);
  847. AddToMinHeapOrCheckStatus(&children_[prefetch.first]);
  848. }
  849. PERF_COUNTER_ADD(number_async_seek, 1);
  850. }
  851. }
  852. }
  853. // Returns true iff the current key (min heap top) should not be returned
  854. // to user (of the merging iterator). This can be because the current key
  855. // is deleted by some range tombstone, the current key is some fake file
  856. // boundary sentinel key, or the current key is an end point of a range
  857. // tombstone. Advance the iterator at heap top if needed. Heap order is restored
  858. // and `active_` is updated accordingly.
  859. // See FindNextVisibleKey() for more detail on internal implementation
  860. // of advancing child iters.
  861. // When false is returned, if minHeap is not empty, then minHeap_.top().type
  862. // == ITERATOR
  863. //
  864. // REQUIRES:
  865. // - min heap is currently not empty, and iter is in kForward direction.
  866. // - minHeap_ top is not DELETE_RANGE_START (so that `active_` is current).
  867. bool MergingIterator::SkipNextDeleted() {
  868. // 3 types of keys:
  869. // - point key
  870. // - file boundary sentinel keys
  871. // - range deletion end key
  872. auto current = minHeap_.top();
  873. if (current->type == HeapItem::Type::DELETE_RANGE_END) {
  874. // Invariant(active_): range_tombstone_iters_[current->level] is about to
  875. // become !Valid() or that its start key is going to be added to minHeap_.
  876. active_.erase(current->level);
  877. assert(range_tombstone_iters_[current->level] &&
  878. range_tombstone_iters_[current->level]->Valid());
  879. range_tombstone_iters_[current->level]->Next();
  880. // Maintain Invariants (rti) and (phi)
  881. if (range_tombstone_iters_[current->level]->Valid()) {
  882. InsertRangeTombstoneToMinHeap(current->level, true /* start_key */,
  883. true /* replace_top */);
  884. } else {
  885. // TruncatedRangeDelIterator does not have status
  886. minHeap_.pop();
  887. }
  888. return true /* current key deleted */;
  889. }
  890. if (current->iter.IsDeleteRangeSentinelKey()) {
  891. // If the file boundary is defined by a range deletion, the range
  892. // tombstone's end key must come before this sentinel key (see op_type in
  893. // SetTombstoneKey()).
  894. assert(ExtractValueType(current->iter.key()) != kTypeRangeDeletion ||
  895. active_.count(current->level) == 0);
  896. // When entering a new file, range tombstone iter from the old file is
  897. // freed, but the last key from that range tombstone iter may still be in
  898. // the heap. We need to ensure the data underlying its corresponding key
  899. // Slice is still alive. We do so by popping the range tombstone key from
  900. // heap before calling iter->Next(). Technically, this change is not needed:
  901. // if there is a range tombstone end key that is after file boundary
  902. // sentinel key in minHeap_, the range tombstone end key must have been
  903. // truncated at file boundary. The underlying data of the range tombstone
  904. // end key Slice is the SST file's largest internal key stored as file
  905. // metadata in Version. However, since there are too many implicit
  906. // assumptions made, it is safer to just ensure range tombstone iter is
  907. // still alive.
  908. minHeap_.pop();
  909. // Remove last SST file's range tombstone end key if there is one.
  910. // This means file boundary is before range tombstone end key,
  911. // which could happen when a range tombstone and a user key
  912. // straddle two SST files. Note that in TruncatedRangeDelIterator
  913. // constructor, parsed_largest.sequence is decremented 1 in this case.
  914. // Maintains Invariant(rti) that at most one
  915. // pinned_heap_item_[current->level] is in minHeap_.
  916. if (range_tombstone_iters_[current->level] &&
  917. range_tombstone_iters_[current->level]->Valid()) {
  918. if (!minHeap_.empty() && minHeap_.top()->level == current->level) {
  919. assert(minHeap_.top()->type == HeapItem::Type::DELETE_RANGE_END);
  920. minHeap_.pop();
  921. // Invariant(active_): we are about to enter a new SST file with new
  922. // range_tombstone_iters[current->level]. Either it is !Valid() or its
  923. // start key is going to be added to minHeap_.
  924. active_.erase(current->level);
  925. } else {
  926. // range tombstone is still valid, but it is not on heap.
  927. // This should only happen if the range tombstone is over iterator
  928. // upper bound.
  929. assert(iterate_upper_bound_ &&
  930. comparator_->user_comparator()->CompareWithoutTimestamp(
  931. range_tombstone_iters_[current->level]->start_key().user_key,
  932. true /* a_has_ts */, *iterate_upper_bound_,
  933. false /* b_has_ts */) >= 0);
  934. }
  935. }
  936. // LevelIterator enters a new SST file
  937. current->iter.Next();
  938. // Invariant(children_): current is popped from heap and added back only if
  939. // it is valid
  940. if (current->iter.Valid()) {
  941. assert(current->iter.status().ok());
  942. minHeap_.push(current);
  943. } else {
  944. // TODO(cbi): check status and early return if non-ok.
  945. considerStatus(current->iter.status());
  946. }
  947. // Invariants (rti) and (phi)
  948. if (range_tombstone_iters_[current->level] &&
  949. range_tombstone_iters_[current->level]->Valid()) {
  950. InsertRangeTombstoneToMinHeap(current->level);
  951. }
  952. return true /* current key deleted */;
  953. }
  954. assert(current->type == HeapItem::Type::ITERATOR);
  955. // Point key case: check active_ for range tombstone coverage.
  956. ParsedInternalKey pik;
  957. ParseInternalKey(current->iter.key(), &pik, false).PermitUncheckedError();
  958. if (!active_.empty()) {
  959. auto i = *active_.begin();
  960. if (i < current->level) {
  961. // range tombstone is from a newer level, definitely covers
  962. assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
  963. pik) <= 0);
  964. assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
  965. 0);
  966. std::string target;
  967. AppendInternalKey(&target, range_tombstone_iters_[i]->end_key());
  968. SeekImpl(target, current->level, true);
  969. return true /* current key deleted */;
  970. } else if (i == current->level) {
  971. // range tombstone is from the same level as current, check sequence
  972. // number. By `active_` we know current key is between start key and end
  973. // key.
  974. assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
  975. pik) <= 0);
  976. assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
  977. 0);
  978. if (pik.sequence < range_tombstone_iters_[current->level]->seq()) {
  979. // covered by range tombstone
  980. current->iter.Next();
  981. // Invariant (children_)
  982. if (current->iter.Valid()) {
  983. minHeap_.replace_top(current);
  984. } else {
  985. considerStatus(current->iter.status());
  986. minHeap_.pop();
  987. }
  988. return true /* current key deleted */;
  989. } else {
  990. return false /* current key not deleted */;
  991. }
  992. } else {
  993. return false /* current key not deleted */;
  994. // range tombstone from an older sorted run with current key < end key.
  995. // current key is not deleted and the older sorted run will have its range
  996. // tombstone updated when the range tombstone's end key are popped from
  997. // minHeap_.
  998. }
  999. }
  1000. // we can reach here only if active_ is empty
  1001. assert(active_.empty());
  1002. assert(minHeap_.top()->type == HeapItem::Type::ITERATOR);
  1003. return false /* current key not deleted */;
  1004. }
  1005. void MergingIterator::SeekForPrevImpl(const Slice& target,
  1006. size_t starting_level,
  1007. bool range_tombstone_reseek) {
  1008. // active range tombstones before `starting_level` remain active
  1009. ClearHeaps(false /* clear_active */);
  1010. InitMaxHeap();
  1011. ParsedInternalKey pik;
  1012. if (!range_tombstone_iters_.empty()) {
  1013. ParseInternalKey(target, &pik, false).PermitUncheckedError();
  1014. }
  1015. for (size_t level = 0; level < starting_level; ++level) {
  1016. PERF_TIMER_GUARD(seek_max_heap_time);
  1017. AddToMaxHeapOrCheckStatus(&children_[level]);
  1018. }
  1019. if (!range_tombstone_iters_.empty()) {
  1020. // Add range tombstones before starting_level.
  1021. for (size_t level = 0; level < starting_level; ++level) {
  1022. if (range_tombstone_iters_[level] &&
  1023. range_tombstone_iters_[level]->Valid()) {
  1024. assert(static_cast<bool>(active_.count(level)) ==
  1025. (pinned_heap_item_[level].type ==
  1026. HeapItem::Type::DELETE_RANGE_START));
  1027. maxHeap_->push(&pinned_heap_item_[level]);
  1028. } else {
  1029. assert(!active_.count(level));
  1030. }
  1031. }
  1032. // levels >= starting_level will be reseeked below,
  1033. active_.erase(active_.lower_bound(starting_level), active_.end());
  1034. }
  1035. IterKey current_search_key;
  1036. current_search_key.SetInternalKey(target, false /* copy */);
  1037. // Seek target might change to some range tombstone end key, so
  1038. // we need to remember them for async requests.
  1039. // (level, target) pairs
  1040. autovector<std::pair<size_t, std::string>> prefetched_target;
  1041. for (auto level = starting_level; level < children_.size(); ++level) {
  1042. {
  1043. PERF_TIMER_GUARD(seek_child_seek_time);
  1044. children_[level].iter.SeekForPrev(current_search_key.GetInternalKey());
  1045. }
  1046. PERF_COUNTER_ADD(seek_child_seek_count, 1);
  1047. if (!range_tombstone_iters_.empty()) {
  1048. if (range_tombstone_reseek) {
  1049. // This seek is to some range tombstone end key.
  1050. // Should only happen when there are range tombstones.
  1051. PERF_COUNTER_ADD(internal_range_del_reseek_count, 1);
  1052. }
  1053. if (children_[level].iter.status().IsTryAgain()) {
  1054. prefetched_target.emplace_back(
  1055. level, current_search_key.GetInternalKey().ToString());
  1056. }
  1057. UnownedPtr<TruncatedRangeDelIterator> range_tombstone_iter =
  1058. range_tombstone_iters_[level].get();
  1059. if (range_tombstone_iter) {
  1060. range_tombstone_iter->SeekForPrev(current_search_key.GetUserKey());
  1061. if (range_tombstone_iter->Valid()) {
  1062. InsertRangeTombstoneToMaxHeap(
  1063. level, comparator_->Compare(range_tombstone_iter->end_key(),
  1064. pik) <= 0 /* end_key */);
  1065. // start key <= current_search_key guaranteed by the Seek() call above
  1066. // Only interested in user key coverage since older sorted runs must
  1067. // have smaller sequence numbers than this tombstone.
  1068. if (comparator_->user_comparator()->Compare(
  1069. current_search_key.GetUserKey(),
  1070. range_tombstone_iter->end_key().user_key) < 0) {
  1071. range_tombstone_reseek = true;
  1072. current_search_key.SetInternalKey(
  1073. range_tombstone_iter->start_key().user_key, kMaxSequenceNumber,
  1074. kValueTypeForSeekForPrev);
  1075. }
  1076. }
  1077. }
  1078. }
  1079. // child.iter.status() is set to Status::TryAgain indicating asynchronous
  1080. // request for retrieval of data blocks has been submitted. So it should
  1081. // return at this point and Seek should be called again to retrieve the
  1082. // requested block and add the child to min heap.
  1083. if (children_[level].iter.status().IsTryAgain()) {
  1084. continue;
  1085. }
  1086. {
  1087. // Strictly, we timed slightly more than min heap operation,
  1088. // but these operations are very cheap.
  1089. PERF_TIMER_GUARD(seek_max_heap_time);
  1090. AddToMaxHeapOrCheckStatus(&children_[level]);
  1091. }
  1092. }
  1093. if (range_tombstone_iters_.empty()) {
  1094. for (auto& child : children_) {
  1095. if (child.iter.status().IsTryAgain()) {
  1096. child.iter.SeekForPrev(target);
  1097. {
  1098. PERF_TIMER_GUARD(seek_min_heap_time);
  1099. AddToMaxHeapOrCheckStatus(&child);
  1100. }
  1101. PERF_COUNTER_ADD(number_async_seek, 1);
  1102. }
  1103. }
  1104. } else {
  1105. for (auto& prefetch : prefetched_target) {
  1106. // (level, target) pairs
  1107. children_[prefetch.first].iter.SeekForPrev(prefetch.second);
  1108. {
  1109. PERF_TIMER_GUARD(seek_max_heap_time);
  1110. AddToMaxHeapOrCheckStatus(&children_[prefetch.first]);
  1111. }
  1112. PERF_COUNTER_ADD(number_async_seek, 1);
  1113. }
  1114. }
  1115. }
  1116. // See more in comments above SkipNextDeleted().
  1117. // REQUIRES:
  1118. // - max heap is currently not empty, and iter is in kReverse direction.
  1119. // - maxHeap_ top is not DELETE_RANGE_END (so that `active_` is current).
  1120. bool MergingIterator::SkipPrevDeleted() {
  1121. // 3 types of keys:
  1122. // - point key
  1123. // - file boundary sentinel keys
  1124. // - range deletion start key
  1125. auto current = maxHeap_->top();
  1126. if (current->type == HeapItem::Type::DELETE_RANGE_START) {
  1127. active_.erase(current->level);
  1128. assert(range_tombstone_iters_[current->level] &&
  1129. range_tombstone_iters_[current->level]->Valid());
  1130. range_tombstone_iters_[current->level]->Prev();
  1131. if (range_tombstone_iters_[current->level]->Valid()) {
  1132. InsertRangeTombstoneToMaxHeap(current->level, true /* end_key */,
  1133. true /* replace_top */);
  1134. } else {
  1135. maxHeap_->pop();
  1136. }
  1137. return true /* current key deleted */;
  1138. }
  1139. if (current->iter.IsDeleteRangeSentinelKey()) {
  1140. // LevelIterator enters a new SST file
  1141. maxHeap_->pop();
  1142. // Remove last SST file's range tombstone key if there is one.
  1143. if (!maxHeap_->empty() && maxHeap_->top()->level == current->level &&
  1144. maxHeap_->top()->type == HeapItem::Type::DELETE_RANGE_START) {
  1145. maxHeap_->pop();
  1146. active_.erase(current->level);
  1147. }
  1148. current->iter.Prev();
  1149. if (current->iter.Valid()) {
  1150. assert(current->iter.status().ok());
  1151. maxHeap_->push(current);
  1152. } else {
  1153. considerStatus(current->iter.status());
  1154. }
  1155. if (range_tombstone_iters_[current->level] &&
  1156. range_tombstone_iters_[current->level]->Valid()) {
  1157. InsertRangeTombstoneToMaxHeap(current->level);
  1158. }
  1159. return true /* current key deleted */;
  1160. }
  1161. assert(current->type == HeapItem::Type::ITERATOR);
  1162. // Point key case: check active_ for range tombstone coverage.
  1163. ParsedInternalKey pik;
  1164. ParseInternalKey(current->iter.key(), &pik, false).PermitUncheckedError();
  1165. if (!active_.empty()) {
  1166. auto i = *active_.begin();
  1167. if (i < current->level) {
  1168. // range tombstone is from a newer level, definitely covers
  1169. assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
  1170. pik) <= 0);
  1171. assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
  1172. 0);
  1173. std::string target;
  1174. AppendInternalKey(&target, range_tombstone_iters_[i]->start_key());
  1175. // This is different from SkipNextDeleted() which does reseek at sorted
  1176. // runs >= level (instead of i+1 here). With min heap, if level L is at
  1177. // top of the heap, then levels <L all have internal keys > level L's
  1178. // current internal key, which means levels <L are already at a different
  1179. // user key. With max heap, if level L is at top of the heap, then levels
  1180. // <L all have internal keys smaller than level L's current internal key,
  1181. // which might still be the same user key.
  1182. SeekForPrevImpl(target, i + 1, true);
  1183. return true /* current key deleted */;
  1184. } else if (i == current->level) {
  1185. // By `active_` we know current key is between start key and end key.
  1186. assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
  1187. pik) <= 0);
  1188. assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
  1189. 0);
  1190. if (pik.sequence < range_tombstone_iters_[current->level]->seq()) {
  1191. current->iter.Prev();
  1192. if (current->iter.Valid()) {
  1193. maxHeap_->replace_top(current);
  1194. } else {
  1195. considerStatus(current->iter.status());
  1196. maxHeap_->pop();
  1197. }
  1198. return true /* current key deleted */;
  1199. } else {
  1200. return false /* current key not deleted */;
  1201. }
  1202. } else {
  1203. return false /* current key not deleted */;
  1204. }
  1205. }
  1206. assert(active_.empty());
  1207. assert(maxHeap_->top()->type == HeapItem::Type::ITERATOR);
  1208. return false /* current key not deleted */;
  1209. }
  1210. void MergingIterator::AddToMinHeapOrCheckStatus(HeapItem* child) {
  1211. // Invariant(children_)
  1212. if (child->iter.Valid()) {
  1213. assert(child->iter.status().ok());
  1214. minHeap_.push(child);
  1215. } else {
  1216. considerStatus(child->iter.status());
  1217. }
  1218. }
  1219. void MergingIterator::AddToMaxHeapOrCheckStatus(HeapItem* child) {
  1220. if (child->iter.Valid()) {
  1221. assert(child->iter.status().ok());
  1222. maxHeap_->push(child);
  1223. } else {
  1224. considerStatus(child->iter.status());
  1225. }
  1226. }
  1227. // Advance all non current_ child to > current_.key().
  1228. // We advance current_ after the this function call as it does not require
  1229. // Seek().
  1230. // Advance all range tombstones iters, including the one corresponding to
  1231. // current_, to the first tombstone with end_key > current_.key().
  1232. // TODO: potentially do cascading seek here too
  1233. // TODO: show that invariants hold
  1234. void MergingIterator::SwitchToForward() {
  1235. ClearHeaps();
  1236. Slice target = key();
  1237. for (auto& child : children_) {
  1238. if (&child.iter != current_) {
  1239. child.iter.Seek(target);
  1240. // child.iter.status() is set to Status::TryAgain indicating asynchronous
  1241. // request for retrieval of data blocks has been submitted. So it should
  1242. // return at this point and Seek should be called again to retrieve the
  1243. // requested block and add the child to min heap.
  1244. if (child.iter.status() == Status::TryAgain()) {
  1245. continue;
  1246. }
  1247. if (child.iter.Valid() && comparator_->Equal(target, child.iter.key())) {
  1248. assert(child.iter.status().ok());
  1249. child.iter.Next();
  1250. }
  1251. }
  1252. AddToMinHeapOrCheckStatus(&child);
  1253. }
  1254. for (auto& child : children_) {
  1255. if (child.iter.status() == Status::TryAgain()) {
  1256. child.iter.Seek(target);
  1257. if (child.iter.Valid() && comparator_->Equal(target, child.iter.key())) {
  1258. assert(child.iter.status().ok());
  1259. child.iter.Next();
  1260. }
  1261. AddToMinHeapOrCheckStatus(&child);
  1262. }
  1263. }
  1264. // Current range tombstone iter also needs to seek for the following case:
  1265. // Previous direction is backward, so range tombstone iter may point to a
  1266. // tombstone before current_. If there is no such tombstone, then the range
  1267. // tombstone iter is !Valid(). Need to reseek here to make it valid again.
  1268. if (!range_tombstone_iters_.empty()) {
  1269. ParsedInternalKey pik;
  1270. ParseInternalKey(target, &pik, false /* log_err_key */)
  1271. .PermitUncheckedError();
  1272. for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
  1273. UnownedPtr<TruncatedRangeDelIterator> iter =
  1274. range_tombstone_iters_[i].get();
  1275. if (iter) {
  1276. iter->Seek(pik.user_key);
  1277. // The while loop is needed as the Seek() call above is only for user
  1278. // key. We could have a range tombstone with end_key covering user_key,
  1279. // but still is smaller than target. This happens when the range
  1280. // tombstone is truncated at iter.largest_.
  1281. while (iter->Valid() &&
  1282. comparator_->Compare(iter->end_key(), pik) <= 0) {
  1283. iter->Next();
  1284. }
  1285. if (range_tombstone_iters_[i]->Valid()) {
  1286. InsertRangeTombstoneToMinHeap(
  1287. i, comparator_->Compare(range_tombstone_iters_[i]->start_key(),
  1288. pik) > 0 /* start_key */);
  1289. }
  1290. }
  1291. }
  1292. }
  1293. direction_ = kForward;
  1294. assert(current_ == CurrentForward());
  1295. }
  1296. // Advance all range tombstones iters, including the one corresponding to
  1297. // current_, to the first tombstone with start_key <= current_.key().
  1298. void MergingIterator::SwitchToBackward() {
  1299. ClearHeaps();
  1300. InitMaxHeap();
  1301. Slice target = key();
  1302. for (auto& child : children_) {
  1303. if (&child.iter != current_) {
  1304. child.iter.SeekForPrev(target);
  1305. TEST_SYNC_POINT_CALLBACK("MergeIterator::Prev:BeforePrev", &child);
  1306. if (child.iter.Valid() && comparator_->Equal(target, child.iter.key())) {
  1307. assert(child.iter.status().ok());
  1308. child.iter.Prev();
  1309. }
  1310. }
  1311. AddToMaxHeapOrCheckStatus(&child);
  1312. }
  1313. ParsedInternalKey pik;
  1314. ParseInternalKey(target, &pik, false /* log_err_key */)
  1315. .PermitUncheckedError();
  1316. for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
  1317. UnownedPtr<TruncatedRangeDelIterator> iter =
  1318. range_tombstone_iters_[i].get();
  1319. if (iter) {
  1320. iter->SeekForPrev(pik.user_key);
  1321. // Since the SeekForPrev() call above is only for user key,
  1322. // we may end up with some range tombstone with start key having the
  1323. // same user key at current_, but with a smaller sequence number. This
  1324. // makes current_ not at maxHeap_ top for the CurrentReverse() call
  1325. // below. If there is a range tombstone start key with the same user
  1326. // key and the same sequence number as current_.key(), it will be fine as
  1327. // in InsertRangeTombstoneToMaxHeap() we change op_type to be the smallest
  1328. // op_type.
  1329. while (iter->Valid() &&
  1330. comparator_->Compare(iter->start_key(), pik) > 0) {
  1331. iter->Prev();
  1332. }
  1333. if (iter->Valid()) {
  1334. InsertRangeTombstoneToMaxHeap(
  1335. i, comparator_->Compare(range_tombstone_iters_[i]->end_key(),
  1336. pik) <= 0 /* end_key */);
  1337. }
  1338. }
  1339. }
  1340. direction_ = kReverse;
  1341. if (!prefix_seek_mode_) {
  1342. // Note that we don't do assert(current_ == CurrentReverse()) here
  1343. // because it is possible to have some keys larger than the seek-key
  1344. // inserted between Seek() and SeekToLast(), which makes current_ not
  1345. // equal to CurrentReverse().
  1346. current_ = CurrentReverse();
  1347. }
  1348. assert(current_ == CurrentReverse());
  1349. }
  1350. void MergingIterator::ClearHeaps(bool clear_active) {
  1351. minHeap_.clear();
  1352. if (maxHeap_) {
  1353. maxHeap_->clear();
  1354. }
  1355. if (clear_active) {
  1356. active_.clear();
  1357. }
  1358. }
  1359. void MergingIterator::InitMaxHeap() {
  1360. if (!maxHeap_) {
  1361. maxHeap_ =
  1362. std::make_unique<MergerMaxIterHeap>(MaxHeapItemComparator(comparator_));
  1363. }
  1364. }
  1365. // Assume there is a next key that is not covered by range tombstone.
  1366. // Pre-condition:
  1367. // - Invariants (3) and (4)
  1368. // - There is some k where k <= children_[i].iter.key() <= LevelNextVisible(i,
  1369. // k) for all levels i (LevelNextVisible() defined in Seek()).
  1370. //
  1371. // Define NextVisible(k) to be the first key >= k from among children_ that
  1372. // is not covered by any range tombstone.
  1373. // Post-condition:
  1374. // - Invariants (1)-(4) hold
  1375. // - (*): minHeap_->top()->key() == NextVisible(k)
  1376. //
  1377. // Loop invariants:
  1378. // - Invariants (3) and (4)
  1379. // - (*): k <= children_[i].iter.key() <= LevelNextVisible(i, k)
  1380. //
  1381. // Progress: minHeap_.top()->key() is non-decreasing and strictly increases in
  1382. // a finite number of iterations.
  1383. // TODO: it is possible to call SeekImpl(k2) after SeekImpl(k1) with
  1384. // k2 < k1 in the same FindNextVisibleKey(). For example, l1 has a range
  1385. // tombstone [2,3) and l2 has a range tombstone [1, 4). Point key 1 from l5
  1386. // triggers SeekImpl(4 /* target */, 5). Then point key 2 from l3 triggers
  1387. // SeekImpl(3 /* target */, 3).
  1388. // Ideally we should only move iterators forward in SeekImpl(), and the
  1389. // progress condition can be made simpler: iterator only moves forward.
  1390. //
  1391. // Proof sketch:
  1392. // Post-condition:
  1393. // Invariant (1) holds when this method returns:
  1394. // Ignoring the empty minHeap_ case, there are two cases:
  1395. // Case 1: active_ is empty and !minHeap_.top()->iter.IsDeleteRangeSentinelKey()
  1396. // By invariants (rti) and (active_), active_ being empty means if a
  1397. // pinned_heap_item_[i] is in minHeap_, it has type DELETE_RANGE_START. Note
  1398. // that PopDeleteRangeStart() was called right before the while loop condition,
  1399. // so minHeap_.top() is not of type DELETE_RANGE_START. So minHeap_.top() must
  1400. // be of type ITERATOR.
  1401. // Case 2: SkipNextDeleted() returns false. The method returns false only when
  1402. // minHeap_.top().type == ITERATOR.
  1403. //
  1404. // Invariant (2) holds when this method returns:
  1405. // From Invariant (1), minHeap_.top().type == ITERATOR. Suppose it is
  1406. // children_[i] for some i. Suppose that children_[i].iter.key() is covered by
  1407. // some range tombstone. This means there is a j <= i and a range tombstone from
  1408. // level j with start_key() < children_[i].iter.key() < end_key().
  1409. // - If range_tombstone_iters_[j]->Valid(), by Invariants (rti) and (phi),
  1410. // pinned_heap_item_[j] is in minHeap_, and pinned_heap_item_[j].tombstone_pik
  1411. // is either start or end key of this range tombstone. If
  1412. // pinned_heap_item_[j].tombstone_pik < children_[i].iter.key(), it would be at
  1413. // top of minHeap_ which would contradict Invariant (1). So
  1414. // pinned_heap_item_[j].tombstone_pik > children_[i].iter.key().
  1415. // By Invariant (3), range_tombstone_iters_[j].prev.end_key() <
  1416. // children_[i].iter.key(). We assume that in each level, range tombstones
  1417. // cover non-overlapping ranges. So range_tombstone_iters_[j] is at
  1418. // the range tombstone with start_key() < children_[i].iter.key() < end_key()
  1419. // and has its end_key() in minHeap_. By Invariants (phi) and (active_),
  1420. // j is in active_. From while loop condition, SkipNextDeleted() must have
  1421. // returned false for this method to return.
  1422. // - If j < i, then SeekImpl(range_tombstone_iters_[j']->end_key(), i)
  1423. // was called for some j' < i and j' in active_. Note that since j' is in
  1424. // active_, pinned_heap_item_[j'] is in minHeap_ and has tombstone_pik =
  1425. // range_tombstone_iters_[j']->end_key(). So
  1426. // range_tombstone_iters_[j']->end_key() must be larger than
  1427. // children_[i].iter.key() to not be at top of minHeap_. This means after
  1428. // SeekImpl(), children_[i] would be at a key > children_[i].iter.key()
  1429. // -- contradiction.
  1430. // - If j == i, children_[i]->Next() would have been called and children_[i]
  1431. // would be at a key > children_[i].iter.key() -- contradiction.
  1432. // - If !range_tombstone_iters_[j]->Valid(). Then range_tombstone_iters_[j]
  1433. // points to an SST file with all range tombstones from that file exhausted.
  1434. // The file must come before the file containing the first
  1435. // range tombstone with start_key() < children_[i].iter.key() < end_key().
  1436. // Assume files from same level have non-overlapping ranges, the current file's
  1437. // meta.largest is less than children_[i].iter.key(). So the file boundary key,
  1438. // which has value meta.largest must have been popped from minHeap_ before
  1439. // children_[i].iter.key(). So range_tombstone_iters_[j] would not point to
  1440. // this SST file -- contradiction.
  1441. // So it is impossible for children_[i].iter.key() to be covered by a range
  1442. // tombstone.
  1443. //
  1444. // Post-condition (*) holds when the function returns:
  1445. // From loop invariant (*) that k <= children_[i].iter.key() <=
  1446. // LevelNextVisible(i, k) and Invariant (2) above, when the function returns,
  1447. // minHeap_.top()->key() is the smallest LevelNextVisible(i, k) among all levels
  1448. // i. This is equal to NextVisible(k).
  1449. //
  1450. // Invariant (3) holds after each iteration:
  1451. // PopDeleteRangeStart() does not change range tombstone position.
  1452. // In SkipNextDeleted():
  1453. // - If DELETE_RANGE_END is popped from minHeap_, it means the range
  1454. // tombstone's end key is < all other point keys, so it is safe to advance to
  1455. // next range tombstone.
  1456. // - If file boundary is popped (current->iter.IsDeleteRangeSentinelKey()),
  1457. // we assume that file's last range tombstone's
  1458. // end_key <= file boundary key < all other point keys. So it is safe to
  1459. // move to the first range tombstone in the next SST file.
  1460. // - If children_[i]->Next() is called, then it is fine as it is advancing a
  1461. // point iterator.
  1462. // - If SeekImpl(target, l) is called, then (3) follows from SeekImpl()'s
  1463. // post-condition if its pre-condition holds. First pre-condition follows
  1464. // from loop invariant where Invariant (3) holds for all levels i.
  1465. // Now we should second pre-condition holds. Since Invariant (3) holds for
  1466. // all i, we have for all j <= l, range_tombstone_iters_[j].prev.end_key()
  1467. // < children_[l].iter.key(). `target` is the value of
  1468. // range_tombstone_iters_[j'].end_key() for some j' < l and j' in active_.
  1469. // By Invariant (active_) and (rti), pinned_heap_item_[j'] is in minHeap_ and
  1470. // pinned_heap_item_[j'].tombstone_pik = range_tombstone_iters_[j'].end_key().
  1471. // This end_key must be larger than children_[l].key() since it was not at top
  1472. // of minHeap_. So for all levels j <= l,
  1473. // range_tombstone_iters_[j].prev.end_key() < children_[l].iter.key() < target
  1474. //
  1475. // Invariant (4) holds after each iteration:
  1476. // A level i is inserted into active_ during calls to PopDeleteRangeStart().
  1477. // In that case, range_tombstone_iters_[i].start_key() < all point keys
  1478. // by heap property and the assumption that point keys and range tombstone keys
  1479. // are distinct.
  1480. // If SeekImpl(target, l) is called, then there is a range_tombstone_iters_[j]
  1481. // where target = range_tombstone_iters_[j]->end_key() and children_[l]->key()
  1482. // < target. By loop invariants, (3) and (4) holds for levels.
  1483. // Since target > children_[l]->key(), it also holds that for j < l,
  1484. // range_tombstone_iters_[j].prev.end_key() < target and that if j in active_,
  1485. // range_tombstone_iters_[i]->start_key() < target. So all pre-conditions of
  1486. // SeekImpl(target, l) holds, and (4) follow from its post-condition.
  1487. // All other places either in this function either advance point iterators
  1488. // or remove some level from active_, so (4) still holds.
  1489. //
  1490. // Look Invariant (*): for all level i, k <= children_[i] <= LevelNextVisible(i,
  1491. // k).
  1492. // k <= children_[i] follows from loop `progress` condition.
  1493. // Consider when children_[i] is changed for any i. It is through
  1494. // children_[i].iter.Next() or SeekImpl() in SkipNextDeleted().
  1495. // If children_[i].iter.Next() is called, there is a range tombstone from level
  1496. // i where tombstone seqno > children_[i].iter.key()'s seqno and i in active_.
  1497. // By Invariant (4), tombstone's start_key < children_[i].iter.key(). By
  1498. // invariants (active_), (phi), and (rti), tombstone's end_key is in minHeap_
  1499. // and that children_[i].iter.key() < end_key. So children_[i].iter.key() is
  1500. // not visible, and it is safe to call Next().
  1501. // If SeekImpl(target, l) is called, by its contract, when SeekImpl() returns,
  1502. // target <= children_[i]->key() <= LevelNextVisible(i, target) for i >= l,
  1503. // and children_[<l] is not touched. We know `target` is
  1504. // range_tombstone_iters_[j]->end_key() for some j < i and j is in active_.
  1505. // By Invariant (4), range_tombstone_iters_[j]->start_key() <
  1506. // children_[i].iter.key() for all i >= l. So for each level i >= l, the range
  1507. // [children_[i].iter.key(), target) is not visible. So after SeekImpl(),
  1508. // children_[i].iter.key() <= LevelNextVisible(i, target) <=
  1509. // LevelNextVisible(i, k).
  1510. //
  1511. // `Progress` holds for each iteration:
  1512. // Very sloppy intuition:
  1513. // - in PopDeleteRangeStart(): the value of a pinned_heap_item_.tombstone_pik_
  1514. // is updated from the start key to the end key of the same range tombstone.
  1515. // We assume that start key <= end key for the same range tombstone.
  1516. // - in SkipNextDeleted()
  1517. // - If the top of heap is DELETE_RANGE_END, the range tombstone is advanced
  1518. // and the relevant pinned_heap_item_.tombstone_pik is increased or popped
  1519. // from minHeap_.
  1520. // - If the top of heap is a file boundary key, then both point iter and
  1521. // range tombstone iter are advanced to the next file.
  1522. // - If the top of heap is ITERATOR and current->iter.Next() is called, it
  1523. // moves to a larger point key.
  1524. // - If the top of heap is ITERATOR and SeekImpl(k, l) is called, then all
  1525. // iterators from levels >= l are advanced to some key >= k by its contract.
  1526. // And top of minHeap_ before SeekImpl(k, l) was less than k.
  1527. // There are special cases where different heap items have the same key,
  1528. // e.g. when two range tombstone end keys share the same value). In
  1529. // these cases, iterators are being advanced, so the minimum key should increase
  1530. // in a finite number of steps.
  1531. inline void MergingIterator::FindNextVisibleKey() {
  1532. PopDeleteRangeStart();
  1533. // PopDeleteRangeStart() implies heap top is not DELETE_RANGE_START
  1534. // active_ being empty implies no DELETE_RANGE_END in heap.
  1535. // So minHeap_->top() must be of type ITERATOR.
  1536. while (
  1537. !minHeap_.empty() &&
  1538. (!active_.empty() || minHeap_.top()->iter.IsDeleteRangeSentinelKey()) &&
  1539. SkipNextDeleted()) {
  1540. PopDeleteRangeStart();
  1541. }
  1542. // Checks Invariant (1)
  1543. assert(minHeap_.empty() || minHeap_.top()->type == HeapItem::Type::ITERATOR);
  1544. }
  1545. inline void MergingIterator::FindPrevVisibleKey() {
  1546. PopDeleteRangeEnd();
  1547. // PopDeleteRangeEnd() implies heap top is not DELETE_RANGE_END
  1548. // active_ being empty implies no DELETE_RANGE_START in heap.
  1549. // So maxHeap_->top() must be of type ITERATOR.
  1550. while (
  1551. !maxHeap_->empty() &&
  1552. (!active_.empty() || maxHeap_->top()->iter.IsDeleteRangeSentinelKey()) &&
  1553. SkipPrevDeleted()) {
  1554. PopDeleteRangeEnd();
  1555. }
  1556. }
  1557. InternalIterator* NewMergingIterator(const InternalKeyComparator* cmp,
  1558. InternalIterator** list, int n,
  1559. Arena* arena, bool prefix_seek_mode) {
  1560. assert(n >= 0);
  1561. if (n == 0) {
  1562. return NewEmptyInternalIterator<Slice>(arena);
  1563. } else if (n == 1) {
  1564. return list[0];
  1565. } else {
  1566. if (arena == nullptr) {
  1567. return new MergingIterator(cmp, list, n, false, prefix_seek_mode);
  1568. } else {
  1569. auto mem = arena->AllocateAligned(sizeof(MergingIterator));
  1570. return new (mem) MergingIterator(cmp, list, n, true, prefix_seek_mode);
  1571. }
  1572. }
  1573. }
  1574. MergeIteratorBuilder::MergeIteratorBuilder(
  1575. const InternalKeyComparator* comparator, Arena* a, bool prefix_seek_mode,
  1576. const Slice* iterate_upper_bound)
  1577. : first_iter(nullptr), use_merging_iter(false), arena(a) {
  1578. auto mem = arena->AllocateAligned(sizeof(MergingIterator));
  1579. merge_iter = new (mem) MergingIterator(comparator, nullptr, 0, true,
  1580. prefix_seek_mode, iterate_upper_bound);
  1581. }
  1582. MergeIteratorBuilder::~MergeIteratorBuilder() {
  1583. if (first_iter != nullptr) {
  1584. first_iter->~InternalIterator();
  1585. }
  1586. if (merge_iter != nullptr) {
  1587. merge_iter->~MergingIterator();
  1588. }
  1589. }
  1590. void MergeIteratorBuilder::AddIterator(InternalIterator* iter) {
  1591. if (!use_merging_iter && first_iter != nullptr) {
  1592. merge_iter->AddIterator(first_iter);
  1593. use_merging_iter = true;
  1594. first_iter = nullptr;
  1595. }
  1596. if (use_merging_iter) {
  1597. merge_iter->AddIterator(iter);
  1598. } else {
  1599. first_iter = iter;
  1600. }
  1601. }
  1602. void MergeIteratorBuilder::AddPointAndTombstoneIterator(
  1603. InternalIterator* point_iter,
  1604. std::unique_ptr<TruncatedRangeDelIterator>&& tombstone_iter,
  1605. std::unique_ptr<TruncatedRangeDelIterator>** tombstone_iter_ptr) {
  1606. // tombstone_iter_ptr != nullptr means point_iter is a LevelIterator.
  1607. bool add_range_tombstone = tombstone_iter ||
  1608. !merge_iter->range_tombstone_iters_.empty() ||
  1609. tombstone_iter_ptr;
  1610. if (!use_merging_iter && (add_range_tombstone || first_iter)) {
  1611. use_merging_iter = true;
  1612. if (first_iter) {
  1613. merge_iter->AddIterator(first_iter);
  1614. first_iter = nullptr;
  1615. }
  1616. }
  1617. if (use_merging_iter) {
  1618. merge_iter->AddIterator(point_iter);
  1619. if (add_range_tombstone) {
  1620. // If there was a gap, fill in nullptr as empty range tombstone iterators.
  1621. while (merge_iter->range_tombstone_iters_.size() <
  1622. merge_iter->children_.size() - 1) {
  1623. merge_iter->AddRangeTombstoneIterator(nullptr);
  1624. }
  1625. merge_iter->AddRangeTombstoneIterator(std::move(tombstone_iter));
  1626. }
  1627. if (tombstone_iter_ptr) {
  1628. // This is needed instead of setting to &range_tombstone_iters_[i]
  1629. // directly here since the memory address of range_tombstone_iters_[i]
  1630. // might change during vector resizing.
  1631. range_del_iter_ptrs_.emplace_back(
  1632. merge_iter->range_tombstone_iters_.size() - 1, tombstone_iter_ptr);
  1633. }
  1634. } else {
  1635. first_iter = point_iter;
  1636. }
  1637. }
  1638. InternalIterator* MergeIteratorBuilder::Finish(ArenaWrappedDBIter* db_iter) {
  1639. InternalIterator* ret = nullptr;
  1640. if (!use_merging_iter) {
  1641. ret = first_iter;
  1642. first_iter = nullptr;
  1643. } else {
  1644. for (auto& p : range_del_iter_ptrs_) {
  1645. *(p.second) = &(merge_iter->range_tombstone_iters_[p.first]);
  1646. }
  1647. if (db_iter && !merge_iter->range_tombstone_iters_.empty()) {
  1648. // memtable is always the first level
  1649. db_iter->SetMemtableRangetombstoneIter(
  1650. &merge_iter->range_tombstone_iters_.front());
  1651. }
  1652. merge_iter->Finish();
  1653. ret = merge_iter;
  1654. merge_iter = nullptr;
  1655. }
  1656. return ret;
  1657. }
  1658. } // namespace ROCKSDB_NAMESPACE