range_tombstone_fragmenter.h 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360
  1. // Copyright (c) 2018-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. #pragma once
  6. #include <list>
  7. #include <memory>
  8. #include <set>
  9. #include <string>
  10. #include <vector>
  11. #include "db/dbformat.h"
  12. #include "db/pinned_iterators_manager.h"
  13. #include "rocksdb/status.h"
  14. #include "table/internal_iterator.h"
  15. namespace ROCKSDB_NAMESPACE {
  16. struct FragmentedRangeTombstoneList {
  17. public:
  18. // A compact representation of a "stack" of range tombstone fragments, which
  19. // start and end at the same user keys but have different sequence numbers.
  20. // The members seq_start_idx and seq_end_idx are intended to be parameters to
  21. // seq_iter().
  22. // If user-defined timestamp is enabled, `start` and `end` should be user keys
  23. // with timestamp, and the timestamps are set to max timestamp to be returned
  24. // by parsed_start_key()/parsed_end_key(). seq_start_idx and seq_end_idx will
  25. // also be used as parameters to ts_iter().
  26. struct RangeTombstoneStack {
  27. RangeTombstoneStack(const Slice& start, const Slice& end, size_t start_idx,
  28. size_t end_idx)
  29. : start_key(start),
  30. end_key(end),
  31. seq_start_idx(start_idx),
  32. seq_end_idx(end_idx) {}
  33. Slice start_key;
  34. Slice end_key;
  35. size_t seq_start_idx;
  36. size_t seq_end_idx;
  37. };
  38. // Assumes unfragmented_tombstones->key() and unfragmented_tombstones->value()
  39. // both contain timestamp if enabled.
  40. FragmentedRangeTombstoneList(
  41. std::unique_ptr<InternalIterator> unfragmented_tombstones,
  42. const InternalKeyComparator& icmp, bool for_compaction = false,
  43. const std::vector<SequenceNumber>& snapshots = {},
  44. const bool tombstone_end_include_ts = true);
  45. std::vector<RangeTombstoneStack>::const_iterator begin() const {
  46. return tombstones_.begin();
  47. }
  48. std::vector<RangeTombstoneStack>::const_iterator end() const {
  49. return tombstones_.end();
  50. }
  51. std::vector<SequenceNumber>::const_iterator seq_iter(size_t idx) const {
  52. return std::next(tombstone_seqs_.begin(), idx);
  53. }
  54. std::vector<Slice>::const_iterator ts_iter(size_t idx) const {
  55. return std::next(tombstone_timestamps_.begin(), idx);
  56. }
  57. std::vector<SequenceNumber>::const_iterator seq_begin() const {
  58. return tombstone_seqs_.begin();
  59. }
  60. std::vector<SequenceNumber>::const_iterator seq_end() const {
  61. return tombstone_seqs_.end();
  62. }
  63. bool empty() const { return tombstones_.empty(); }
  64. // Returns true if the stored tombstones contain with one with a sequence
  65. // number in [lower, upper].
  66. // This method is not const as it internally lazy initialize a set of
  67. // sequence numbers (`seq_set_`).
  68. bool ContainsRange(SequenceNumber lower, SequenceNumber upper);
  69. uint64_t num_unfragmented_tombstones() const {
  70. return num_unfragmented_tombstones_;
  71. }
  72. uint64_t total_tombstone_payload_bytes() const {
  73. return total_tombstone_payload_bytes_;
  74. }
  75. private:
  76. // Given an ordered range tombstone iterator unfragmented_tombstones,
  77. // "fragment" the tombstones into non-overlapping pieces. Each
  78. // "non-overlapping piece" is a RangeTombstoneStack in tombstones_, which
  79. // contains start_key, end_key, and indices that points to sequence numbers
  80. // (in tombstone_seqs_) and timestamps (in tombstone_timestamps_). If
  81. // for_compaction is true, then `snapshots` should be provided. Range
  82. // tombstone fragments are dropped if they are not visible in any snapshot and
  83. // user-defined timestamp is not enabled. That is, for each snapshot stripe
  84. // [lower, upper], the range tombstone fragment with largest seqno in [lower,
  85. // upper] is preserved, and all the other range tombstones are dropped.
  86. void FragmentTombstones(
  87. std::unique_ptr<InternalIterator> unfragmented_tombstones,
  88. const InternalKeyComparator& icmp, bool for_compaction,
  89. const std::vector<SequenceNumber>& snapshots);
  90. std::vector<RangeTombstoneStack> tombstones_;
  91. std::vector<SequenceNumber> tombstone_seqs_;
  92. std::vector<Slice> tombstone_timestamps_;
  93. std::once_flag seq_set_init_once_flag_;
  94. std::set<SequenceNumber> seq_set_;
  95. std::list<std::string> pinned_slices_;
  96. PinnedIteratorsManager pinned_iters_mgr_;
  97. uint64_t num_unfragmented_tombstones_;
  98. uint64_t total_tombstone_payload_bytes_;
  99. };
  100. struct FragmentedRangeTombstoneListCache {
  101. // ensure only the first reader needs to initialize l
  102. std::mutex reader_mutex;
  103. std::unique_ptr<FragmentedRangeTombstoneList> tombstones = nullptr;
  104. // readers will first check this bool to avoid
  105. std::atomic<bool> initialized = false;
  106. };
  107. // FragmentedRangeTombstoneIterator converts an InternalIterator of a range-del
  108. // meta block into an iterator over non-overlapping tombstone fragments. The
  109. // tombstone fragmentation process should be more efficient than the range
  110. // tombstone collapsing algorithm in RangeDelAggregator because this leverages
  111. // the internal key ordering already provided by the input iterator, if
  112. // applicable (when the iterator is unsorted, a new sorted iterator is created
  113. // before proceeding). If there are few overlaps, creating a
  114. // FragmentedRangeTombstoneIterator should be O(n), while the RangeDelAggregator
  115. // tombstone collapsing is always O(n log n).
  116. class FragmentedRangeTombstoneIterator : public InternalIterator {
  117. public:
  118. FragmentedRangeTombstoneIterator(FragmentedRangeTombstoneList* tombstones,
  119. const InternalKeyComparator& icmp,
  120. SequenceNumber upper_bound,
  121. const Slice* ts_upper_bound = nullptr,
  122. SequenceNumber lower_bound = 0);
  123. FragmentedRangeTombstoneIterator(
  124. const std::shared_ptr<FragmentedRangeTombstoneList>& tombstones,
  125. const InternalKeyComparator& icmp, SequenceNumber upper_bound,
  126. const Slice* ts_upper_bound = nullptr, SequenceNumber lower_bound = 0);
  127. FragmentedRangeTombstoneIterator(
  128. const std::shared_ptr<FragmentedRangeTombstoneListCache>& tombstones,
  129. const InternalKeyComparator& icmp, SequenceNumber upper_bound,
  130. const Slice* ts_upper_bound = nullptr, SequenceNumber lower_bound = 0);
  131. void SetRangeDelReadSeqno(SequenceNumber read_seqno) override {
  132. upper_bound_ = read_seqno;
  133. }
  134. void SeekToFirst() override;
  135. void SeekToLast() override;
  136. void SeekToTopFirst();
  137. void SeekToTopLast();
  138. // NOTE: Seek and SeekForPrev do not behave in the way InternalIterator
  139. // seeking should behave. This is OK because they are not currently used, but
  140. // eventually FragmentedRangeTombstoneIterator should no longer implement
  141. // InternalIterator.
  142. //
  143. // Seeks to the range tombstone that covers target at a seqnum in the
  144. // snapshot. If no such tombstone exists, seek to the earliest tombstone in
  145. // the snapshot that ends after target.
  146. void Seek(const Slice& target) override;
  147. // Seeks to the range tombstone that covers target at a seqnum in the
  148. // snapshot. If no such tombstone exists, seek to the latest tombstone in the
  149. // snapshot that starts before target.
  150. void SeekForPrev(const Slice& target) override;
  151. void Next() override;
  152. void Prev() override;
  153. void TopNext();
  154. void TopPrev();
  155. bool Valid() const override;
  156. // Note that key() and value() do not return correct timestamp.
  157. // Caller should call timestamp() to get the current timestamp.
  158. Slice key() const override {
  159. MaybePinKey();
  160. return current_start_key_.Encode();
  161. }
  162. Slice value() const override { return pos_->end_key; }
  163. bool IsKeyPinned() const override { return false; }
  164. bool IsValuePinned() const override { return true; }
  165. Status status() const override { return Status::OK(); }
  166. bool empty() const { return tombstones_->empty(); }
  167. void Invalidate() {
  168. pos_ = tombstones_->end();
  169. seq_pos_ = tombstones_->seq_end();
  170. pinned_pos_ = tombstones_->end();
  171. pinned_seq_pos_ = tombstones_->seq_end();
  172. }
  173. RangeTombstone Tombstone() const {
  174. assert(Valid());
  175. if (icmp_->user_comparator()->timestamp_size()) {
  176. return RangeTombstone(start_key(), end_key(), seq(), timestamp());
  177. }
  178. return RangeTombstone(start_key(), end_key(), seq());
  179. }
  180. // Note that start_key() and end_key() are not guaranteed to have the
  181. // correct timestamp. User can call timestamp() to get the correct
  182. // timestamp().
  183. Slice start_key() const { return pos_->start_key; }
  184. Slice end_key() const { return pos_->end_key; }
  185. SequenceNumber seq() const { return *seq_pos_; }
  186. Slice timestamp() const {
  187. // seqno and timestamp are stored in the same order.
  188. return *tombstones_->ts_iter(seq_pos_ - tombstones_->seq_begin());
  189. }
  190. // Current use case is by CompactionRangeDelAggregator to set
  191. // full_history_ts_low_.
  192. void SetTimestampUpperBound(const Slice* ts_upper_bound) {
  193. ts_upper_bound_ = ts_upper_bound;
  194. }
  195. ParsedInternalKey parsed_start_key() const {
  196. return ParsedInternalKey(pos_->start_key, seq(), kTypeRangeDeletion);
  197. }
  198. ParsedInternalKey parsed_end_key() const {
  199. return ParsedInternalKey(pos_->end_key, kMaxSequenceNumber,
  200. kTypeRangeDeletion);
  201. }
  202. // Return the max sequence number of a range tombstone that covers
  203. // the given user key.
  204. // If there is no covering tombstone, then 0 is returned.
  205. SequenceNumber MaxCoveringTombstoneSeqnum(const Slice& user_key);
  206. // Splits the iterator into n+1 iterators (where n is the number of
  207. // snapshots), each providing a view over a "stripe" of sequence numbers. The
  208. // iterators are keyed by the upper bound of their ranges (the provided
  209. // snapshots + kMaxSequenceNumber).
  210. //
  211. // NOTE: the iterators in the returned map are no longer valid if their
  212. // parent iterator is deleted, since they do not modify the refcount of the
  213. // underlying tombstone list. Therefore, this map should be deleted before
  214. // the parent iterator.
  215. std::map<SequenceNumber, std::unique_ptr<FragmentedRangeTombstoneIterator>>
  216. SplitBySnapshot(const std::vector<SequenceNumber>& snapshots);
  217. SequenceNumber upper_bound() const { return upper_bound_; }
  218. SequenceNumber lower_bound() const { return lower_bound_; }
  219. uint64_t num_unfragmented_tombstones() const {
  220. return tombstones_->num_unfragmented_tombstones();
  221. }
  222. uint64_t total_tombstone_payload_bytes() const {
  223. return tombstones_->total_tombstone_payload_bytes();
  224. }
  225. private:
  226. using RangeTombstoneStack = FragmentedRangeTombstoneList::RangeTombstoneStack;
  227. struct RangeTombstoneStackStartComparator {
  228. explicit RangeTombstoneStackStartComparator(const Comparator* c) : cmp(c) {}
  229. bool operator()(const RangeTombstoneStack& a,
  230. const RangeTombstoneStack& b) const {
  231. return cmp->CompareWithoutTimestamp(a.start_key, b.start_key) < 0;
  232. }
  233. bool operator()(const RangeTombstoneStack& a, const Slice& b) const {
  234. return cmp->CompareWithoutTimestamp(a.start_key, b) < 0;
  235. }
  236. bool operator()(const Slice& a, const RangeTombstoneStack& b) const {
  237. return cmp->CompareWithoutTimestamp(a, b.start_key) < 0;
  238. }
  239. const Comparator* cmp;
  240. };
  241. struct RangeTombstoneStackEndComparator {
  242. explicit RangeTombstoneStackEndComparator(const Comparator* c) : cmp(c) {}
  243. bool operator()(const RangeTombstoneStack& a,
  244. const RangeTombstoneStack& b) const {
  245. return cmp->CompareWithoutTimestamp(a.end_key, b.end_key) < 0;
  246. }
  247. bool operator()(const RangeTombstoneStack& a, const Slice& b) const {
  248. return cmp->CompareWithoutTimestamp(a.end_key, b) < 0;
  249. }
  250. bool operator()(const Slice& a, const RangeTombstoneStack& b) const {
  251. return cmp->CompareWithoutTimestamp(a, b.end_key) < 0;
  252. }
  253. const Comparator* cmp;
  254. };
  255. void MaybePinKey() const {
  256. if (pos_ != tombstones_->end() && seq_pos_ != tombstones_->seq_end() &&
  257. (pinned_pos_ != pos_ || pinned_seq_pos_ != seq_pos_)) {
  258. current_start_key_.Set(pos_->start_key, *seq_pos_, kTypeRangeDeletion);
  259. pinned_pos_ = pos_;
  260. pinned_seq_pos_ = seq_pos_;
  261. }
  262. }
  263. void SeekToCoveringTombstone(const Slice& key);
  264. void SeekForPrevToCoveringTombstone(const Slice& key);
  265. void ScanForwardToVisibleTombstone();
  266. void ScanBackwardToVisibleTombstone();
  267. bool ValidPos() const {
  268. return Valid() && seq_pos_ != tombstones_->seq_iter(pos_->seq_end_idx);
  269. }
  270. const RangeTombstoneStackStartComparator tombstone_start_cmp_;
  271. const RangeTombstoneStackEndComparator tombstone_end_cmp_;
  272. const InternalKeyComparator* icmp_;
  273. const Comparator* ucmp_;
  274. std::shared_ptr<FragmentedRangeTombstoneList> tombstones_ref_;
  275. std::shared_ptr<FragmentedRangeTombstoneListCache> tombstones_cache_ref_;
  276. FragmentedRangeTombstoneList* tombstones_;
  277. SequenceNumber upper_bound_;
  278. SequenceNumber lower_bound_;
  279. // Only consider timestamps <= ts_upper_bound_.
  280. const Slice* ts_upper_bound_;
  281. std::vector<RangeTombstoneStack>::const_iterator pos_;
  282. std::vector<SequenceNumber>::const_iterator seq_pos_;
  283. mutable std::vector<RangeTombstoneStack>::const_iterator pinned_pos_;
  284. mutable std::vector<SequenceNumber>::const_iterator pinned_seq_pos_;
  285. mutable InternalKey current_start_key_;
  286. // Check the current RangeTombstoneStack `pos_` against timestamp
  287. // upper bound `ts_upper_bound_` and sequence number upper bound
  288. // `upper_bound_`. Update the sequence number (and timestamp) pointer
  289. // `seq_pos_` to the first valid position satisfying both bounds.
  290. void SetMaxVisibleSeqAndTimestamp() {
  291. seq_pos_ = std::lower_bound(tombstones_->seq_iter(pos_->seq_start_idx),
  292. tombstones_->seq_iter(pos_->seq_end_idx),
  293. upper_bound_, std::greater<SequenceNumber>());
  294. if (ts_upper_bound_ && !ts_upper_bound_->empty()) {
  295. auto ts_pos = std::lower_bound(
  296. tombstones_->ts_iter(pos_->seq_start_idx),
  297. tombstones_->ts_iter(pos_->seq_end_idx), *ts_upper_bound_,
  298. [this](const Slice& s1, const Slice& s2) {
  299. return ucmp_->CompareTimestamp(s1, s2) > 0;
  300. });
  301. auto ts_idx = ts_pos - tombstones_->ts_iter(pos_->seq_start_idx);
  302. auto seq_idx = seq_pos_ - tombstones_->seq_iter(pos_->seq_start_idx);
  303. if (seq_idx < ts_idx) {
  304. // seq and ts are ordered in non-increasing order. Only updates seq_pos_
  305. // to a larger index for smaller sequence number and timestamp.
  306. seq_pos_ = tombstones_->seq_iter(pos_->seq_start_idx + ts_idx);
  307. }
  308. }
  309. }
  310. };
  311. } // namespace ROCKSDB_NAMESPACE