compaction_iterator_test.cc 72 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756
  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. #include "db/compaction/compaction_iterator.h"
  6. #include <string>
  7. #include <vector>
  8. #include "db/dbformat.h"
  9. #include "port/port.h"
  10. #include "test_util/testharness.h"
  11. #include "test_util/testutil.h"
  12. #include "util/string_util.h"
  13. #include "util/vector_iterator.h"
  14. #include "utilities/merge_operators.h"
  15. namespace ROCKSDB_NAMESPACE {
  16. namespace {
  17. std::string ValueWithPreferredSeqno(std::string val,
  18. SequenceNumber preferred_seqno = 0) {
  19. std::string result = val;
  20. PutFixed64(&result, preferred_seqno);
  21. return result;
  22. }
  23. } // namespace
  24. // Expects no merging attempts.
  25. class NoMergingMergeOp : public MergeOperator {
  26. public:
  27. bool FullMergeV2(const MergeOperationInput& /*merge_in*/,
  28. MergeOperationOutput* /*merge_out*/) const override {
  29. ADD_FAILURE();
  30. return false;
  31. }
  32. bool PartialMergeMulti(const Slice& /*key*/,
  33. const std::deque<Slice>& /*operand_list*/,
  34. std::string* /*new_value*/,
  35. Logger* /*logger*/) const override {
  36. ADD_FAILURE();
  37. return false;
  38. }
  39. const char* Name() const override {
  40. return "CompactionIteratorTest NoMergingMergeOp";
  41. }
  42. };
  43. // Compaction filter that gets stuck when it sees a particular key,
  44. // then gets unstuck when told to.
  45. // Always returns Decision::kRemove.
  46. class StallingFilter : public CompactionFilter {
  47. public:
  48. Decision FilterV2(int /*level*/, const Slice& key, ValueType /*type*/,
  49. const Slice& /*existing_value*/, std::string* /*new_value*/,
  50. std::string* /*skip_until*/) const override {
  51. int k = std::atoi(key.ToString().c_str());
  52. last_seen.store(k);
  53. while (k >= stall_at.load()) {
  54. std::this_thread::yield();
  55. }
  56. return Decision::kRemove;
  57. }
  58. const char* Name() const override {
  59. return "CompactionIteratorTest StallingFilter";
  60. }
  61. // Wait until the filter sees a key >= k and stalls at that key.
  62. // If `exact`, asserts that the seen key is equal to k.
  63. void WaitForStall(int k, bool exact = true) {
  64. stall_at.store(k);
  65. while (last_seen.load() < k) {
  66. std::this_thread::yield();
  67. }
  68. if (exact) {
  69. EXPECT_EQ(k, last_seen.load());
  70. }
  71. }
  72. // Filter will stall on key >= stall_at. Advance stall_at to unstall.
  73. mutable std::atomic<int> stall_at{0};
  74. // Last key the filter was called with.
  75. mutable std::atomic<int> last_seen{0};
  76. };
  77. // Compaction filter that filter out all keys.
  78. class FilterAllKeysCompactionFilter : public CompactionFilter {
  79. public:
  80. Decision FilterV2(int /*level*/, const Slice& /*key*/, ValueType /*type*/,
  81. const Slice& /*existing_value*/, std::string* /*new_value*/,
  82. std::string* /*skip_until*/) const override {
  83. return Decision::kRemove;
  84. }
  85. const char* Name() const override { return "AllKeysCompactionFilter"; }
  86. };
  87. class LoggingForwardVectorIterator : public VectorIterator {
  88. public:
  89. struct Action {
  90. enum class Type {
  91. SEEK_TO_FIRST,
  92. SEEK,
  93. NEXT,
  94. };
  95. Type type;
  96. std::string arg;
  97. explicit Action(Type _type, std::string _arg = "")
  98. : type(_type), arg(_arg) {}
  99. bool operator==(const Action& rhs) const {
  100. return std::tie(type, arg) == std::tie(rhs.type, rhs.arg);
  101. }
  102. };
  103. LoggingForwardVectorIterator(const std::vector<std::string>& keys,
  104. const std::vector<std::string>& values)
  105. : VectorIterator(keys, values) {
  106. current_ = keys_.size();
  107. }
  108. void SeekToFirst() override {
  109. log.emplace_back(Action::Type::SEEK_TO_FIRST);
  110. VectorIterator::SeekToFirst();
  111. }
  112. void SeekToLast() override { assert(false); }
  113. void Seek(const Slice& target) override {
  114. log.emplace_back(Action::Type::SEEK, target.ToString());
  115. VectorIterator::Seek(target);
  116. }
  117. void SeekForPrev(const Slice& /*target*/) override { assert(false); }
  118. void Next() override {
  119. assert(Valid());
  120. log.emplace_back(Action::Type::NEXT);
  121. VectorIterator::Next();
  122. }
  123. void Prev() override { assert(false); }
  124. Slice key() const override {
  125. assert(Valid());
  126. return VectorIterator::key();
  127. }
  128. Slice value() const override {
  129. assert(Valid());
  130. return VectorIterator::value();
  131. }
  132. std::vector<Action> log;
  133. };
  134. class FakeCompaction : public CompactionIterator::CompactionProxy {
  135. public:
  136. int level() const override { return 0; }
  137. bool KeyNotExistsBeyondOutputLevel(
  138. const Slice& /*user_key*/,
  139. std::vector<size_t>* /*level_ptrs*/) const override {
  140. return is_bottommost_level || key_not_exists_beyond_output_level;
  141. }
  142. bool bottommost_level() const override { return is_bottommost_level; }
  143. int number_levels() const override { return 1; }
  144. Slice GetLargestUserKey() const override {
  145. return "\xff\xff\xff\xff\xff\xff\xff\xff\xff";
  146. }
  147. bool allow_ingest_behind() const override { return is_allow_ingest_behind; }
  148. bool allow_mmap_reads() const override { return false; }
  149. bool enable_blob_garbage_collection() const override { return false; }
  150. double blob_garbage_collection_age_cutoff() const override { return 0.0; }
  151. uint64_t blob_compaction_readahead_size() const override { return 0; }
  152. const Version* input_version() const override { return nullptr; }
  153. bool DoesInputReferenceBlobFiles() const override { return false; }
  154. const Compaction* real_compaction() const override { return nullptr; }
  155. bool SupportsPerKeyPlacement() const override {
  156. return supports_per_key_placement;
  157. }
  158. bool key_not_exists_beyond_output_level = false;
  159. bool is_bottommost_level = false;
  160. bool is_allow_ingest_behind = false;
  161. bool supports_per_key_placement = false;
  162. };
  163. // A simplified snapshot checker which assumes each snapshot has a global
  164. // last visible sequence.
  165. class TestSnapshotChecker : public SnapshotChecker {
  166. public:
  167. explicit TestSnapshotChecker(
  168. SequenceNumber last_committed_sequence,
  169. const std::unordered_map<SequenceNumber, SequenceNumber>& snapshots =
  170. {{}})
  171. : last_committed_sequence_(last_committed_sequence),
  172. snapshots_(snapshots) {}
  173. SnapshotCheckerResult CheckInSnapshot(
  174. SequenceNumber seq, SequenceNumber snapshot_seq) const override {
  175. if (snapshot_seq == kMaxSequenceNumber) {
  176. return seq <= last_committed_sequence_
  177. ? SnapshotCheckerResult::kInSnapshot
  178. : SnapshotCheckerResult::kNotInSnapshot;
  179. }
  180. assert(snapshots_.count(snapshot_seq) > 0);
  181. return seq <= snapshots_.at(snapshot_seq)
  182. ? SnapshotCheckerResult::kInSnapshot
  183. : SnapshotCheckerResult::kNotInSnapshot;
  184. }
  185. private:
  186. SequenceNumber last_committed_sequence_;
  187. // A map of valid snapshot to last visible sequence to the snapshot.
  188. std::unordered_map<SequenceNumber, SequenceNumber> snapshots_;
  189. };
  190. // Test param:
  191. // bool: whether to pass snapshot_checker to compaction iterator.
  192. class CompactionIteratorTest : public testing::TestWithParam<bool> {
  193. public:
  194. CompactionIteratorTest()
  195. : cmp_(BytewiseComparator()), icmp_(cmp_), snapshots_({}) {}
  196. explicit CompactionIteratorTest(const Comparator* ucmp)
  197. : cmp_(ucmp), icmp_(cmp_), snapshots_({}) {}
  198. void InitIterators(
  199. const std::vector<std::string>& ks, const std::vector<std::string>& vs,
  200. const std::vector<std::string>& range_del_ks,
  201. const std::vector<std::string>& range_del_vs,
  202. SequenceNumber last_sequence,
  203. SequenceNumber last_committed_sequence = kMaxSequenceNumber,
  204. MergeOperator* merge_op = nullptr, CompactionFilter* filter = nullptr,
  205. bool bottommost_level = false,
  206. SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber,
  207. bool key_not_exists_beyond_output_level = false,
  208. const std::string* full_history_ts_low = nullptr) {
  209. std::unique_ptr<InternalIterator> unfragmented_range_del_iter(
  210. new VectorIterator(range_del_ks, range_del_vs, &icmp_));
  211. auto tombstone_list = std::make_shared<FragmentedRangeTombstoneList>(
  212. std::move(unfragmented_range_del_iter), icmp_);
  213. std::unique_ptr<FragmentedRangeTombstoneIterator> range_del_iter(
  214. new FragmentedRangeTombstoneIterator(tombstone_list, icmp_,
  215. kMaxSequenceNumber));
  216. range_del_agg_.reset(new CompactionRangeDelAggregator(&icmp_, snapshots_));
  217. range_del_agg_->AddTombstones(std::move(range_del_iter));
  218. std::unique_ptr<CompactionIterator::CompactionProxy> compaction;
  219. if (filter || bottommost_level || key_not_exists_beyond_output_level) {
  220. compaction_proxy_ = new FakeCompaction();
  221. compaction_proxy_->is_bottommost_level = bottommost_level;
  222. compaction_proxy_->is_allow_ingest_behind = AllowIngestBehind();
  223. compaction_proxy_->key_not_exists_beyond_output_level =
  224. key_not_exists_beyond_output_level;
  225. compaction_proxy_->supports_per_key_placement = SupportsPerKeyPlacement();
  226. compaction.reset(compaction_proxy_);
  227. }
  228. bool use_snapshot_checker = UseSnapshotChecker() || GetParam();
  229. if (use_snapshot_checker || last_committed_sequence < kMaxSequenceNumber) {
  230. snapshot_checker_.reset(
  231. new TestSnapshotChecker(last_committed_sequence, snapshot_map_));
  232. }
  233. merge_helper_.reset(
  234. new MergeHelper(Env::Default(), cmp_, merge_op, filter, nullptr, false,
  235. 0 /*latest_snapshot*/, snapshot_checker_.get(),
  236. 0 /*level*/, nullptr /*statistics*/, &shutting_down_));
  237. if (c_iter_) {
  238. // Since iter_ is still used in ~CompactionIterator(), we call
  239. // ~CompactionIterator() first.
  240. c_iter_.reset();
  241. }
  242. iter_.reset(new LoggingForwardVectorIterator(ks, vs));
  243. iter_->SeekToFirst();
  244. c_iter_.reset(new CompactionIterator(
  245. iter_.get(), cmp_, merge_helper_.get(), last_sequence, &snapshots_,
  246. snapshots_.empty() ? kMaxSequenceNumber : snapshots_.at(0),
  247. earliest_write_conflict_snapshot, kMaxSequenceNumber,
  248. snapshot_checker_.get(), Env::Default(),
  249. false /* report_detailed_time */, range_del_agg_.get(),
  250. nullptr /* blob_file_builder */, true /*allow_data_in_errors*/,
  251. true /*enforce_single_del_contracts*/,
  252. /*manual_compaction_canceled=*/kManualCompactionCanceledFalse_,
  253. std::move(compaction), /*must_count_input_entries=*/false, filter,
  254. &shutting_down_, /*info_log=*/nullptr, full_history_ts_low));
  255. }
  256. void AddSnapshot(SequenceNumber snapshot,
  257. SequenceNumber last_visible_seq = kMaxSequenceNumber) {
  258. snapshots_.push_back(snapshot);
  259. snapshot_map_[snapshot] = last_visible_seq;
  260. }
  261. virtual bool UseSnapshotChecker() const { return false; }
  262. virtual bool AllowIngestBehind() const { return false; }
  263. virtual bool SupportsPerKeyPlacement() const { return false; }
  264. void RunTest(
  265. const std::vector<std::string>& input_keys,
  266. const std::vector<std::string>& input_values,
  267. const std::vector<std::string>& expected_keys,
  268. const std::vector<std::string>& expected_values,
  269. SequenceNumber last_committed_seq = kMaxSequenceNumber,
  270. MergeOperator* merge_operator = nullptr,
  271. CompactionFilter* compaction_filter = nullptr,
  272. bool bottommost_level = false,
  273. SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber,
  274. bool key_not_exists_beyond_output_level = false,
  275. const std::string* full_history_ts_low = nullptr) {
  276. InitIterators(input_keys, input_values, {}, {}, kMaxSequenceNumber,
  277. last_committed_seq, merge_operator, compaction_filter,
  278. bottommost_level, earliest_write_conflict_snapshot,
  279. key_not_exists_beyond_output_level, full_history_ts_low);
  280. c_iter_->SeekToFirst();
  281. for (size_t i = 0; i < expected_keys.size(); i++) {
  282. std::string info = "i = " + std::to_string(i);
  283. ASSERT_TRUE(c_iter_->Valid()) << info;
  284. ASSERT_OK(c_iter_->status()) << info;
  285. ASSERT_EQ(expected_keys[i], c_iter_->key().ToString()) << info;
  286. ASSERT_EQ(expected_values[i], c_iter_->value().ToString()) << info;
  287. c_iter_->Next();
  288. }
  289. ASSERT_OK(c_iter_->status());
  290. ASSERT_FALSE(c_iter_->Valid());
  291. }
  292. void ClearSnapshots() {
  293. snapshots_.clear();
  294. snapshot_map_.clear();
  295. }
  296. const Comparator* cmp_;
  297. const InternalKeyComparator icmp_;
  298. std::vector<SequenceNumber> snapshots_;
  299. // A map of valid snapshot to last visible sequence to the snapshot.
  300. std::unordered_map<SequenceNumber, SequenceNumber> snapshot_map_;
  301. std::unique_ptr<MergeHelper> merge_helper_;
  302. std::unique_ptr<LoggingForwardVectorIterator> iter_;
  303. std::unique_ptr<CompactionIterator> c_iter_;
  304. std::unique_ptr<CompactionRangeDelAggregator> range_del_agg_;
  305. std::unique_ptr<SnapshotChecker> snapshot_checker_;
  306. std::atomic<bool> shutting_down_{false};
  307. const std::atomic<bool> kManualCompactionCanceledFalse_{false};
  308. FakeCompaction* compaction_proxy_;
  309. };
  310. // It is possible that the output of the compaction iterator is empty even if
  311. // the input is not.
  312. TEST_P(CompactionIteratorTest, EmptyResult) {
  313. InitIterators({test::KeyStr("a", 5, kTypeSingleDeletion),
  314. test::KeyStr("a", 3, kTypeValue)},
  315. {"", "val"}, {}, {}, 5);
  316. c_iter_->SeekToFirst();
  317. ASSERT_OK(c_iter_->status());
  318. ASSERT_FALSE(c_iter_->Valid());
  319. }
  320. // If there is a corruption after a single deletion, the compaction should fail.
  321. TEST_P(CompactionIteratorTest, CorruptionAfterSingleDeletion) {
  322. InitIterators({test::KeyStr("a", 5, kTypeSingleDeletion),
  323. test::KeyStr("a", 3, kTypeValue, true),
  324. test::KeyStr("b", 10, kTypeValue)},
  325. {"", "val", "val2"}, {}, {}, 10);
  326. c_iter_->SeekToFirst();
  327. ASSERT_TRUE(c_iter_->Valid());
  328. ASSERT_EQ(test::KeyStr("a", 5, kTypeSingleDeletion),
  329. c_iter_->key().ToString());
  330. c_iter_->Next();
  331. // The iterator should now fail when encountering the corrupted key
  332. ASSERT_FALSE(c_iter_->Valid());
  333. ASSERT_FALSE(c_iter_->status().ok());
  334. ASSERT_TRUE(c_iter_->status().IsCorruption());
  335. }
  336. // Tests compatibility of TimedPut and SingleDelete. TimedPut should act as if
  337. // it's a Put.
  338. TEST_P(CompactionIteratorTest, TimedPutAndSingleDelete) {
  339. InitIterators({test::KeyStr("a", 5, kTypeSingleDeletion),
  340. test::KeyStr("a", 3, kTypeValuePreferredSeqno)},
  341. {"", "val"}, {}, {}, 5);
  342. c_iter_->SeekToFirst();
  343. ASSERT_OK(c_iter_->status());
  344. ASSERT_FALSE(c_iter_->Valid());
  345. }
  346. TEST_P(CompactionIteratorTest, SimpleRangeDeletion) {
  347. InitIterators({test::KeyStr("morning", 5, kTypeValue),
  348. test::KeyStr("morning", 2, kTypeValue),
  349. test::KeyStr("night", 3, kTypeValue)},
  350. {"zao", "zao", "wan"},
  351. {test::KeyStr("ma", 4, kTypeRangeDeletion)}, {"mz"}, 5);
  352. c_iter_->SeekToFirst();
  353. ASSERT_TRUE(c_iter_->Valid());
  354. ASSERT_EQ(test::KeyStr("morning", 5, kTypeValue), c_iter_->key().ToString());
  355. c_iter_->Next();
  356. ASSERT_TRUE(c_iter_->Valid());
  357. ASSERT_EQ(test::KeyStr("night", 3, kTypeValue), c_iter_->key().ToString());
  358. c_iter_->Next();
  359. ASSERT_OK(c_iter_->status());
  360. ASSERT_FALSE(c_iter_->Valid());
  361. }
  362. TEST_P(CompactionIteratorTest, RangeDeletionWithSnapshots) {
  363. AddSnapshot(10);
  364. std::vector<std::string> ks1;
  365. ks1.push_back(test::KeyStr("ma", 28, kTypeRangeDeletion));
  366. std::vector<std::string> vs1{"mz"};
  367. std::vector<std::string> ks2{test::KeyStr("morning", 15, kTypeValue),
  368. test::KeyStr("morning", 5, kTypeValue),
  369. test::KeyStr("night", 40, kTypeValue),
  370. test::KeyStr("night", 20, kTypeValue)};
  371. std::vector<std::string> vs2{"zao 15", "zao 5", "wan 40", "wan 20"};
  372. InitIterators(ks2, vs2, ks1, vs1, 40);
  373. c_iter_->SeekToFirst();
  374. ASSERT_TRUE(c_iter_->Valid());
  375. ASSERT_EQ(test::KeyStr("morning", 5, kTypeValue), c_iter_->key().ToString());
  376. c_iter_->Next();
  377. ASSERT_TRUE(c_iter_->Valid());
  378. ASSERT_EQ(test::KeyStr("night", 40, kTypeValue), c_iter_->key().ToString());
  379. c_iter_->Next();
  380. ASSERT_OK(c_iter_->status());
  381. ASSERT_FALSE(c_iter_->Valid());
  382. }
  383. // Tests compatibility of TimedPut and Range delete. TimedPut should act as if
  384. // it's a Put.
  385. TEST_P(CompactionIteratorTest, TimedPutAndRangeDeletion) {
  386. InitIterators(
  387. {test::KeyStr("morning", 5, kTypeValuePreferredSeqno),
  388. test::KeyStr("morning", 2, kTypeValuePreferredSeqno),
  389. test::KeyStr("night", 3, kTypeValuePreferredSeqno)},
  390. {ValueWithPreferredSeqno("zao5"), ValueWithPreferredSeqno("zao2"),
  391. ValueWithPreferredSeqno("wan")},
  392. {test::KeyStr("ma", 4, kTypeRangeDeletion)}, {"mz"}, 5);
  393. c_iter_->SeekToFirst();
  394. ASSERT_TRUE(c_iter_->Valid());
  395. ASSERT_EQ(test::KeyStr("morning", 5, kTypeValuePreferredSeqno),
  396. c_iter_->key().ToString());
  397. ASSERT_EQ(ValueWithPreferredSeqno("zao5"), c_iter_->value().ToString());
  398. c_iter_->Next();
  399. ASSERT_TRUE(c_iter_->Valid());
  400. ASSERT_EQ(test::KeyStr("night", 3, kTypeValuePreferredSeqno),
  401. c_iter_->key().ToString());
  402. ASSERT_EQ(ValueWithPreferredSeqno("wan"), c_iter_->value().ToString());
  403. c_iter_->Next();
  404. ASSERT_OK(c_iter_->status());
  405. ASSERT_FALSE(c_iter_->Valid());
  406. }
  407. TEST_P(CompactionIteratorTest, CompactionFilterSkipUntil) {
  408. class Filter : public CompactionFilter {
  409. Decision FilterV2(int /*level*/, const Slice& key, ValueType t,
  410. const Slice& existing_value, std::string* /*new_value*/,
  411. std::string* skip_until) const override {
  412. std::string k = key.ToString();
  413. std::string v = existing_value.ToString();
  414. // See InitIterators() call below for the sequence of keys and their
  415. // filtering decisions. Here we closely assert that compaction filter is
  416. // called with the expected keys and only them, and with the right values.
  417. if (k == "a") {
  418. EXPECT_EQ(ValueType::kValue, t);
  419. EXPECT_EQ("av50", v);
  420. return Decision::kKeep;
  421. }
  422. if (k == "b") {
  423. EXPECT_EQ(ValueType::kValue, t);
  424. EXPECT_EQ("bv60", v);
  425. *skip_until = "d+";
  426. return Decision::kRemoveAndSkipUntil;
  427. }
  428. if (k == "e") {
  429. EXPECT_EQ(ValueType::kMergeOperand, t);
  430. EXPECT_EQ("em71", v);
  431. return Decision::kKeep;
  432. }
  433. if (k == "f") {
  434. if (v == "fm65") {
  435. EXPECT_EQ(ValueType::kMergeOperand, t);
  436. *skip_until = "f";
  437. } else {
  438. EXPECT_EQ("fm30", v);
  439. EXPECT_EQ(ValueType::kMergeOperand, t);
  440. *skip_until = "g+";
  441. }
  442. return Decision::kRemoveAndSkipUntil;
  443. }
  444. if (k == "h") {
  445. EXPECT_EQ(ValueType::kValue, t);
  446. EXPECT_EQ("hv91", v);
  447. return Decision::kKeep;
  448. }
  449. if (k == "i") {
  450. EXPECT_EQ(ValueType::kMergeOperand, t);
  451. EXPECT_EQ("im95", v);
  452. *skip_until = "z";
  453. return Decision::kRemoveAndSkipUntil;
  454. }
  455. ADD_FAILURE();
  456. return Decision::kKeep;
  457. }
  458. const char* Name() const override {
  459. return "CompactionIteratorTest.CompactionFilterSkipUntil::Filter";
  460. }
  461. };
  462. NoMergingMergeOp merge_op;
  463. Filter filter;
  464. InitIterators(
  465. {test::KeyStr("a", 50, kTypeValue), // keep
  466. test::KeyStr("a", 45, kTypeMerge),
  467. test::KeyStr("b", 60, kTypeValue), // skip to "d+"
  468. test::KeyStr("b", 40, kTypeValue), test::KeyStr("c", 35, kTypeValue),
  469. test::KeyStr("d", 70, kTypeMerge),
  470. test::KeyStr("e", 71, kTypeMerge), // keep
  471. test::KeyStr("f", 65, kTypeMerge), // skip to "f", aka keep
  472. test::KeyStr("f", 30, kTypeMerge), // skip to "g+"
  473. test::KeyStr("f", 25, kTypeValue), test::KeyStr("g", 90, kTypeValue),
  474. test::KeyStr("h", 91, kTypeValue), // keep
  475. test::KeyStr("i", 95, kTypeMerge), // skip to "z"
  476. test::KeyStr("j", 99, kTypeValue),
  477. test::KeyStr("k", 100, kTypeValuePreferredSeqno)},
  478. {"av50", "am45", "bv60", "bv40", "cv35", "dm70", "em71", "fm65", "fm30",
  479. "fv25", "gv90", "hv91", "im95", "jv99",
  480. ValueWithPreferredSeqno("kv100")},
  481. {}, {}, kMaxSequenceNumber, kMaxSequenceNumber, &merge_op, &filter);
  482. // Compaction should output just "a", "e" and "h" keys.
  483. c_iter_->SeekToFirst();
  484. ASSERT_TRUE(c_iter_->Valid());
  485. ASSERT_EQ(test::KeyStr("a", 50, kTypeValue), c_iter_->key().ToString());
  486. ASSERT_EQ("av50", c_iter_->value().ToString());
  487. c_iter_->Next();
  488. ASSERT_TRUE(c_iter_->Valid());
  489. ASSERT_EQ(test::KeyStr("e", 71, kTypeMerge), c_iter_->key().ToString());
  490. ASSERT_EQ("em71", c_iter_->value().ToString());
  491. c_iter_->Next();
  492. ASSERT_TRUE(c_iter_->Valid());
  493. ASSERT_EQ(test::KeyStr("h", 91, kTypeValue), c_iter_->key().ToString());
  494. ASSERT_EQ("hv91", c_iter_->value().ToString());
  495. c_iter_->Next();
  496. ASSERT_OK(c_iter_->status());
  497. ASSERT_FALSE(c_iter_->Valid());
  498. // Check that the compaction iterator did the correct sequence of calls on
  499. // the underlying iterator.
  500. using A = LoggingForwardVectorIterator::Action;
  501. using T = A::Type;
  502. std::vector<A> expected_actions = {
  503. A(T::SEEK_TO_FIRST),
  504. A(T::NEXT),
  505. A(T::NEXT),
  506. A(T::SEEK, test::KeyStr("d+", kMaxSequenceNumber, kValueTypeForSeek)),
  507. A(T::NEXT),
  508. A(T::NEXT),
  509. A(T::SEEK, test::KeyStr("g+", kMaxSequenceNumber, kValueTypeForSeek)),
  510. A(T::NEXT),
  511. A(T::SEEK, test::KeyStr("z", kMaxSequenceNumber, kValueTypeForSeek))};
  512. ASSERT_EQ(expected_actions, iter_->log);
  513. }
  514. TEST_P(CompactionIteratorTest, ShuttingDownInFilter) {
  515. NoMergingMergeOp merge_op;
  516. StallingFilter filter;
  517. InitIterators(
  518. {test::KeyStr("1", 1, kTypeValue), test::KeyStr("2", 2, kTypeValue),
  519. test::KeyStr("3", 3, kTypeValue), test::KeyStr("4", 4, kTypeValue)},
  520. {"v1", "v2", "v3", "v4"}, {}, {}, kMaxSequenceNumber, kMaxSequenceNumber,
  521. &merge_op, &filter);
  522. // Don't leave tombstones (kTypeDeletion) for filtered keys.
  523. compaction_proxy_->key_not_exists_beyond_output_level = true;
  524. std::atomic<bool> seek_done{false};
  525. ROCKSDB_NAMESPACE::port::Thread compaction_thread([&] {
  526. c_iter_->SeekToFirst();
  527. EXPECT_FALSE(c_iter_->Valid());
  528. EXPECT_TRUE(c_iter_->status().IsShutdownInProgress());
  529. seek_done.store(true);
  530. });
  531. // Let key 1 through.
  532. filter.WaitForStall(1);
  533. // Shutdown during compaction filter call for key 2.
  534. filter.WaitForStall(2);
  535. shutting_down_.store(true);
  536. EXPECT_FALSE(seek_done.load());
  537. // Unstall filter and wait for SeekToFirst() to return.
  538. filter.stall_at.store(3);
  539. compaction_thread.join();
  540. assert(seek_done.load());
  541. // Check that filter was never called again.
  542. EXPECT_EQ(2, filter.last_seen.load());
  543. }
  544. // Same as ShuttingDownInFilter, but shutdown happens during filter call for
  545. // a merge operand, not for a value.
  546. TEST_P(CompactionIteratorTest, ShuttingDownInMerge) {
  547. NoMergingMergeOp merge_op;
  548. StallingFilter filter;
  549. InitIterators(
  550. {test::KeyStr("1", 1, kTypeValue), test::KeyStr("2", 2, kTypeMerge),
  551. test::KeyStr("3", 3, kTypeMerge), test::KeyStr("4", 4, kTypeValue)},
  552. {"v1", "v2", "v3", "v4"}, {}, {}, kMaxSequenceNumber, kMaxSequenceNumber,
  553. &merge_op, &filter);
  554. compaction_proxy_->key_not_exists_beyond_output_level = true;
  555. std::atomic<bool> seek_done{false};
  556. ROCKSDB_NAMESPACE::port::Thread compaction_thread([&] {
  557. c_iter_->SeekToFirst();
  558. ASSERT_FALSE(c_iter_->Valid());
  559. ASSERT_TRUE(c_iter_->status().IsShutdownInProgress());
  560. seek_done.store(true);
  561. });
  562. // Let key 1 through.
  563. filter.WaitForStall(1);
  564. // Shutdown during compaction filter call for key 2.
  565. filter.WaitForStall(2);
  566. shutting_down_.store(true);
  567. EXPECT_FALSE(seek_done.load());
  568. // Unstall filter and wait for SeekToFirst() to return.
  569. filter.stall_at.store(3);
  570. compaction_thread.join();
  571. assert(seek_done.load());
  572. // Check that filter was never called again.
  573. EXPECT_EQ(2, filter.last_seen.load());
  574. }
  575. class Filter : public CompactionFilter {
  576. Decision FilterV2(int /*level*/, const Slice& key, ValueType t,
  577. const Slice& existing_value, std::string* /*new_value*/,
  578. std::string* /*skip_until*/) const override {
  579. std::string k = key.ToString();
  580. std::string v = existing_value.ToString();
  581. // See InitIterators() call below for the sequence of keys and their
  582. // filtering decisions. Here we closely assert that compaction filter is
  583. // called with the expected keys and only them, and with the right values.
  584. if (k == "a") {
  585. EXPECT_EQ(ValueType::kMergeOperand, t);
  586. EXPECT_EQ("av1", v);
  587. return Decision::kKeep;
  588. } else if (k == "b") {
  589. EXPECT_EQ(ValueType::kMergeOperand, t);
  590. return Decision::kKeep;
  591. } else if (k == "c") {
  592. return Decision::kKeep;
  593. }
  594. ADD_FAILURE();
  595. return Decision::kKeep;
  596. }
  597. const char* Name() const override {
  598. return "CompactionIteratorTest.SingleMergeOperand::Filter";
  599. }
  600. };
  601. class SingleMergeOp : public MergeOperator {
  602. public:
  603. bool FullMergeV2(const MergeOperationInput& merge_in,
  604. MergeOperationOutput* merge_out) const override {
  605. // See InitIterators() call below for why "c" is the only key for which
  606. // FullMergeV2 should be called.
  607. EXPECT_EQ("c", merge_in.key.ToString());
  608. std::string temp_value;
  609. if (merge_in.existing_value != nullptr) {
  610. temp_value = merge_in.existing_value->ToString();
  611. }
  612. for (auto& operand : merge_in.operand_list) {
  613. temp_value.append(operand.ToString());
  614. }
  615. merge_out->new_value = temp_value;
  616. return true;
  617. }
  618. bool PartialMergeMulti(const Slice& key,
  619. const std::deque<Slice>& operand_list,
  620. std::string* new_value,
  621. Logger* /*logger*/) const override {
  622. std::string string_key = key.ToString();
  623. EXPECT_TRUE(string_key == "a" || string_key == "b");
  624. if (string_key == "a") {
  625. EXPECT_EQ(1, operand_list.size());
  626. } else if (string_key == "b") {
  627. EXPECT_EQ(2, operand_list.size());
  628. }
  629. std::string temp_value;
  630. for (auto& operand : operand_list) {
  631. temp_value.append(operand.ToString());
  632. }
  633. swap(temp_value, *new_value);
  634. return true;
  635. }
  636. const char* Name() const override {
  637. return "CompactionIteratorTest SingleMergeOp";
  638. }
  639. bool AllowSingleOperand() const override { return true; }
  640. };
  641. TEST_P(CompactionIteratorTest, SingleMergeOperand) {
  642. SingleMergeOp merge_op;
  643. Filter filter;
  644. InitIterators(
  645. // a should invoke PartialMergeMulti with a single merge operand.
  646. {test::KeyStr("a", 50, kTypeMerge),
  647. // b should invoke PartialMergeMulti with two operands.
  648. test::KeyStr("b", 70, kTypeMerge), test::KeyStr("b", 60, kTypeMerge),
  649. // c should invoke FullMerge due to kTypeValue at the beginning.
  650. test::KeyStr("c", 90, kTypeMerge), test::KeyStr("c", 80, kTypeValue)},
  651. {"av1", "bv2", "bv1", "cv2", "cv1"}, {}, {}, kMaxSequenceNumber,
  652. kMaxSequenceNumber, &merge_op, &filter);
  653. c_iter_->SeekToFirst();
  654. ASSERT_TRUE(c_iter_->Valid());
  655. ASSERT_EQ(test::KeyStr("a", 50, kTypeMerge), c_iter_->key().ToString());
  656. ASSERT_EQ("av1", c_iter_->value().ToString());
  657. c_iter_->Next();
  658. ASSERT_TRUE(c_iter_->Valid());
  659. ASSERT_EQ("bv1bv2", c_iter_->value().ToString());
  660. c_iter_->Next();
  661. ASSERT_OK(c_iter_->status());
  662. ASSERT_EQ("cv1cv2", c_iter_->value().ToString());
  663. }
  664. // Tests compatibility of TimedPut and Merge operation. When a TimedPut is
  665. // merged with some merge operand in compaction, it will become a regular Put
  666. // and lose its preferred sequence number.
  667. TEST_P(CompactionIteratorTest, TimedPutAndMerge) {
  668. SingleMergeOp merge_op;
  669. Filter filter;
  670. InitIterators({test::KeyStr("c", 90, kTypeMerge),
  671. test::KeyStr("c", 80, kTypeValuePreferredSeqno)},
  672. {"cv2", ValueWithPreferredSeqno("cv1")}, {}, {},
  673. kMaxSequenceNumber, kMaxSequenceNumber, &merge_op, &filter);
  674. c_iter_->SeekToFirst();
  675. ASSERT_TRUE(c_iter_->Valid());
  676. ASSERT_EQ(test::KeyStr("c", 90, kTypeValue), c_iter_->key().ToString());
  677. ASSERT_OK(c_iter_->status());
  678. ASSERT_EQ("cv1cv2", c_iter_->value().ToString());
  679. }
  680. // In bottommost level, values earlier than earliest snapshot can be output
  681. // with sequence = 0.
  682. TEST_P(CompactionIteratorTest, ZeroOutSequenceAtBottomLevel) {
  683. AddSnapshot(1);
  684. RunTest({test::KeyStr("a", 1, kTypeValue), test::KeyStr("b", 2, kTypeValue)},
  685. {"v1", "v2"},
  686. {test::KeyStr("a", 0, kTypeValue), test::KeyStr("b", 2, kTypeValue)},
  687. {"v1", "v2"}, kMaxSequenceNumber /*last_committed_seq*/,
  688. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  689. true /*bottommost_level*/);
  690. }
  691. // In bottommost level, deletions earlier than earliest snapshot can be removed
  692. // permanently.
  693. TEST_P(CompactionIteratorTest, RemoveDeletionAtBottomLevel) {
  694. AddSnapshot(1);
  695. RunTest(
  696. {test::KeyStr("a", 1, kTypeDeletion), test::KeyStr("b", 3, kTypeDeletion),
  697. test::KeyStr("b", 1, kTypeValue)},
  698. {"", "", ""},
  699. {test::KeyStr("b", 3, kTypeDeletion), test::KeyStr("b", 0, kTypeValue)},
  700. {"", ""}, kMaxSequenceNumber /*last_committed_seq*/,
  701. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  702. true /*bottommost_level*/);
  703. }
  704. // In bottommost level, single deletions earlier than earliest snapshot can be
  705. // removed permanently.
  706. TEST_P(CompactionIteratorTest, RemoveSingleDeletionAtBottomLevel) {
  707. AddSnapshot(1);
  708. RunTest({test::KeyStr("a", 1, kTypeSingleDeletion),
  709. test::KeyStr("b", 2, kTypeSingleDeletion)},
  710. {"", ""}, {test::KeyStr("b", 2, kTypeSingleDeletion)}, {""},
  711. kMaxSequenceNumber /*last_committed_seq*/, nullptr /*merge_operator*/,
  712. nullptr /*compaction_filter*/, true /*bottommost_level*/);
  713. }
  714. TEST_P(CompactionIteratorTest, ConvertToPutAtBottom) {
  715. std::shared_ptr<MergeOperator> merge_op =
  716. MergeOperators::CreateStringAppendOperator();
  717. RunTest({test::KeyStr("a", 4, kTypeMerge), test::KeyStr("a", 3, kTypeMerge),
  718. test::KeyStr("a", 2, kTypeMerge), test::KeyStr("b", 1, kTypeValue)},
  719. {"a4", "a3", "a2", "b1"},
  720. {test::KeyStr("a", 0, kTypeValue), test::KeyStr("b", 0, kTypeValue)},
  721. {"a2,a3,a4", "b1"}, kMaxSequenceNumber /*last_committed_seq*/,
  722. merge_op.get(), nullptr /*compaction_filter*/,
  723. true /*bottomost_level*/);
  724. }
  725. TEST_P(CompactionIteratorTest, ZeroSeqOfKeyAndSnapshot) {
  726. AddSnapshot(0);
  727. const std::vector<std::string> input_keys = {
  728. test::KeyStr("a", 0, kTypeValue), test::KeyStr("b", 0, kTypeValue)};
  729. const std::vector<std::string> input_values = {"a1", "b1"};
  730. RunTest(input_keys, input_values, input_keys, input_values);
  731. }
  732. INSTANTIATE_TEST_CASE_P(CompactionIteratorTestInstance, CompactionIteratorTest,
  733. testing::Values(true, false));
  734. // Tests how CompactionIterator work together with SnapshotChecker.
  735. class CompactionIteratorWithSnapshotCheckerTest
  736. : public CompactionIteratorTest {
  737. public:
  738. bool UseSnapshotChecker() const override { return true; }
  739. };
  740. // Uncommitted keys (keys with seq > last_committed_seq) should be output as-is
  741. // while committed version of these keys should get compacted as usual.
  742. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  743. PreserveUncommittedKeys_Value) {
  744. RunTest(
  745. {test::KeyStr("foo", 3, kTypeValue), test::KeyStr("foo", 2, kTypeValue),
  746. test::KeyStr("foo", 1, kTypeValue)},
  747. {"v3", "v2", "v1"},
  748. {test::KeyStr("foo", 3, kTypeValue), test::KeyStr("foo", 2, kTypeValue)},
  749. {"v3", "v2"}, 2 /*last_committed_seq*/);
  750. }
  751. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  752. PreserveUncommittedKeys_Deletion) {
  753. RunTest({test::KeyStr("foo", 2, kTypeDeletion),
  754. test::KeyStr("foo", 1, kTypeValue)},
  755. {"", "v1"},
  756. {test::KeyStr("foo", 2, kTypeDeletion),
  757. test::KeyStr("foo", 1, kTypeValue)},
  758. {"", "v1"}, 1 /*last_committed_seq*/);
  759. }
  760. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  761. PreserveUncommittedKeys_Merge) {
  762. auto merge_op = MergeOperators::CreateStringAppendOperator();
  763. RunTest(
  764. {test::KeyStr("foo", 3, kTypeMerge), test::KeyStr("foo", 2, kTypeMerge),
  765. test::KeyStr("foo", 1, kTypeValue)},
  766. {"v3", "v2", "v1"},
  767. {test::KeyStr("foo", 3, kTypeMerge), test::KeyStr("foo", 2, kTypeValue)},
  768. {"v3", "v1,v2"}, 2 /*last_committed_seq*/, merge_op.get());
  769. }
  770. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  771. PreserveUncommittedKeys_SingleDelete) {
  772. RunTest({test::KeyStr("foo", 2, kTypeSingleDeletion),
  773. test::KeyStr("foo", 1, kTypeValue)},
  774. {"", "v1"},
  775. {test::KeyStr("foo", 2, kTypeSingleDeletion),
  776. test::KeyStr("foo", 1, kTypeValue)},
  777. {"", "v1"}, 1 /*last_committed_seq*/);
  778. }
  779. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  780. PreserveUncommittedKeys_BlobIndex) {
  781. RunTest({test::KeyStr("foo", 3, kTypeBlobIndex),
  782. test::KeyStr("foo", 2, kTypeBlobIndex),
  783. test::KeyStr("foo", 1, kTypeBlobIndex)},
  784. {"v3", "v2", "v1"},
  785. {test::KeyStr("foo", 3, kTypeBlobIndex),
  786. test::KeyStr("foo", 2, kTypeBlobIndex)},
  787. {"v3", "v2"}, 2 /*last_committed_seq*/);
  788. }
  789. // Test compaction iterator dedup keys visible to the same snapshot.
  790. TEST_F(CompactionIteratorWithSnapshotCheckerTest, DedupSameSnapshot_Value) {
  791. AddSnapshot(2, 1);
  792. RunTest(
  793. {test::KeyStr("foo", 4, kTypeValue), test::KeyStr("foo", 3, kTypeValue),
  794. test::KeyStr("foo", 2, kTypeValue), test::KeyStr("foo", 1, kTypeValue)},
  795. {"v4", "v3", "v2", "v1"},
  796. {test::KeyStr("foo", 4, kTypeValue), test::KeyStr("foo", 3, kTypeValue),
  797. test::KeyStr("foo", 1, kTypeValue)},
  798. {"v4", "v3", "v1"}, 3 /*last_committed_seq*/);
  799. }
  800. TEST_F(CompactionIteratorWithSnapshotCheckerTest, DedupSameSnapshot_TimedPut) {
  801. AddSnapshot(2, 1);
  802. RunTest({test::KeyStr("foo", 4, kTypeValuePreferredSeqno),
  803. test::KeyStr("foo", 3, kTypeValuePreferredSeqno),
  804. test::KeyStr("foo", 2, kTypeValuePreferredSeqno),
  805. test::KeyStr("foo", 1, kTypeValuePreferredSeqno)},
  806. {ValueWithPreferredSeqno("v4"), ValueWithPreferredSeqno("v3"),
  807. ValueWithPreferredSeqno("v2"), ValueWithPreferredSeqno("v1")},
  808. {test::KeyStr("foo", 4, kTypeValuePreferredSeqno),
  809. test::KeyStr("foo", 3, kTypeValuePreferredSeqno),
  810. test::KeyStr("foo", 1, kTypeValuePreferredSeqno)},
  811. {ValueWithPreferredSeqno("v4"), ValueWithPreferredSeqno("v3"),
  812. ValueWithPreferredSeqno("v1")},
  813. 3 /*last_committed_seq*/);
  814. }
  815. TEST_F(CompactionIteratorWithSnapshotCheckerTest, DedupSameSnapshot_Deletion) {
  816. AddSnapshot(2, 1);
  817. RunTest(
  818. {test::KeyStr("foo", 4, kTypeValue),
  819. test::KeyStr("foo", 3, kTypeDeletion),
  820. test::KeyStr("foo", 2, kTypeValue), test::KeyStr("foo", 1, kTypeValue)},
  821. {"v4", "", "v2", "v1"},
  822. {test::KeyStr("foo", 4, kTypeValue),
  823. test::KeyStr("foo", 3, kTypeDeletion),
  824. test::KeyStr("foo", 1, kTypeValue)},
  825. {"v4", "", "v1"}, 3 /*last_committed_seq*/);
  826. }
  827. TEST_F(CompactionIteratorWithSnapshotCheckerTest, DedupSameSnapshot_Merge) {
  828. AddSnapshot(2, 1);
  829. AddSnapshot(4, 3);
  830. auto merge_op = MergeOperators::CreateStringAppendOperator();
  831. RunTest(
  832. {test::KeyStr("foo", 5, kTypeMerge), test::KeyStr("foo", 4, kTypeMerge),
  833. test::KeyStr("foo", 3, kTypeMerge), test::KeyStr("foo", 2, kTypeMerge),
  834. test::KeyStr("foo", 1, kTypeValue)},
  835. {"v5", "v4", "v3", "v2", "v1"},
  836. {test::KeyStr("foo", 5, kTypeMerge), test::KeyStr("foo", 4, kTypeMerge),
  837. test::KeyStr("foo", 3, kTypeMerge), test::KeyStr("foo", 1, kTypeValue)},
  838. {"v5", "v4", "v2,v3", "v1"}, 4 /*last_committed_seq*/, merge_op.get());
  839. }
  840. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  841. DedupSameSnapshot_SingleDeletion) {
  842. AddSnapshot(2, 1);
  843. RunTest(
  844. {test::KeyStr("foo", 4, kTypeValue),
  845. test::KeyStr("foo", 3, kTypeSingleDeletion),
  846. test::KeyStr("foo", 2, kTypeValue), test::KeyStr("foo", 1, kTypeValue)},
  847. {"v4", "", "v2", "v1"},
  848. {test::KeyStr("foo", 4, kTypeValue), test::KeyStr("foo", 1, kTypeValue)},
  849. {"v4", "v1"}, 3 /*last_committed_seq*/);
  850. }
  851. TEST_F(CompactionIteratorWithSnapshotCheckerTest, DedupSameSnapshot_BlobIndex) {
  852. AddSnapshot(2, 1);
  853. RunTest({test::KeyStr("foo", 4, kTypeBlobIndex),
  854. test::KeyStr("foo", 3, kTypeBlobIndex),
  855. test::KeyStr("foo", 2, kTypeBlobIndex),
  856. test::KeyStr("foo", 1, kTypeBlobIndex)},
  857. {"v4", "v3", "v2", "v1"},
  858. {test::KeyStr("foo", 4, kTypeBlobIndex),
  859. test::KeyStr("foo", 3, kTypeBlobIndex),
  860. test::KeyStr("foo", 1, kTypeBlobIndex)},
  861. {"v4", "v3", "v1"}, 3 /*last_committed_seq*/);
  862. }
  863. // At bottom level, sequence numbers can be zero out, and deletions can be
  864. // removed, but only when they are visible to earliest snapshot.
  865. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  866. NotZeroOutSequenceIfNotVisibleToEarliestSnapshot) {
  867. AddSnapshot(2, 1);
  868. RunTest({test::KeyStr("a", 1, kTypeValue), test::KeyStr("b", 2, kTypeValue),
  869. test::KeyStr("c", 3, kTypeValue)},
  870. {"v1", "v2", "v3"},
  871. {test::KeyStr("a", 0, kTypeValue), test::KeyStr("b", 2, kTypeValue),
  872. test::KeyStr("c", 3, kTypeValue)},
  873. {"v1", "v2", "v3"}, kMaxSequenceNumber /*last_committed_seq*/,
  874. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  875. true /*bottommost_level*/);
  876. }
  877. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  878. NotRemoveDeletionIfNotVisibleToEarliestSnapshot) {
  879. AddSnapshot(2, 1);
  880. RunTest(
  881. {test::KeyStr("a", 1, kTypeDeletion), test::KeyStr("b", 2, kTypeDeletion),
  882. test::KeyStr("c", 3, kTypeDeletion)},
  883. {"", "", ""}, {}, {"", ""}, kMaxSequenceNumber /*last_committed_seq*/,
  884. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  885. true /*bottommost_level*/);
  886. }
  887. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  888. NotRemoveDeletionIfValuePresentToEarlierSnapshot) {
  889. AddSnapshot(2, 1);
  890. RunTest({test::KeyStr("a", 4, kTypeDeletion),
  891. test::KeyStr("a", 1, kTypeValue), test::KeyStr("b", 3, kTypeValue)},
  892. {"", "", ""},
  893. {test::KeyStr("a", 4, kTypeDeletion),
  894. test::KeyStr("a", 0, kTypeValue), test::KeyStr("b", 3, kTypeValue)},
  895. {"", "", ""}, kMaxSequenceNumber /*last_committed_seq*/,
  896. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  897. true /*bottommost_level*/);
  898. }
  899. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  900. NotRemoveSingleDeletionIfNotVisibleToEarliestSnapshot) {
  901. AddSnapshot(2, 1);
  902. RunTest({test::KeyStr("a", 1, kTypeSingleDeletion),
  903. test::KeyStr("b", 2, kTypeSingleDeletion),
  904. test::KeyStr("c", 3, kTypeSingleDeletion)},
  905. {"", "", ""},
  906. {test::KeyStr("b", 2, kTypeSingleDeletion),
  907. test::KeyStr("c", 3, kTypeSingleDeletion)},
  908. {"", ""}, kMaxSequenceNumber /*last_committed_seq*/,
  909. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  910. true /*bottommost_level*/);
  911. }
  912. // Single delete should not cancel out values that not visible to the
  913. // same set of snapshots
  914. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  915. SingleDeleteAcrossSnapshotBoundary) {
  916. AddSnapshot(2, 1);
  917. RunTest({test::KeyStr("a", 2, kTypeSingleDeletion),
  918. test::KeyStr("a", 1, kTypeValue)},
  919. {"", "v1"},
  920. {test::KeyStr("a", 2, kTypeSingleDeletion),
  921. test::KeyStr("a", 1, kTypeValue)},
  922. {"", "v1"}, 2 /*last_committed_seq*/);
  923. }
  924. // Single delete should be kept in case it is not visible to the
  925. // earliest write conflict snapshot. If a single delete is kept for this reason,
  926. // corresponding value can be trimmed to save space.
  927. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  928. KeepSingleDeletionForWriteConflictChecking) {
  929. AddSnapshot(2, 0);
  930. RunTest({test::KeyStr("a", 2, kTypeSingleDeletion),
  931. test::KeyStr("a", 1, kTypeValue)},
  932. {"", "v1"},
  933. {test::KeyStr("a", 2, kTypeSingleDeletion),
  934. test::KeyStr("a", 1, kTypeValue)},
  935. {"", ""}, 2 /*last_committed_seq*/, nullptr /*merge_operator*/,
  936. nullptr /*compaction_filter*/, false /*bottommost_level*/,
  937. 2 /*earliest_write_conflict_snapshot*/);
  938. }
  939. // Same as above but with a blob index. In addition to the value getting
  940. // trimmed, the type of the KV is changed to kTypeValue.
  941. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  942. KeepSingleDeletionForWriteConflictChecking_BlobIndex) {
  943. AddSnapshot(2, 0);
  944. RunTest({test::KeyStr("a", 2, kTypeSingleDeletion),
  945. test::KeyStr("a", 1, kTypeBlobIndex)},
  946. {"", "fake_blob_index"},
  947. {test::KeyStr("a", 2, kTypeSingleDeletion),
  948. test::KeyStr("a", 1, kTypeValue)},
  949. {"", ""}, 2 /*last_committed_seq*/, nullptr /*merge_operator*/,
  950. nullptr /*compaction_filter*/, false /*bottommost_level*/,
  951. 2 /*earliest_write_conflict_snapshot*/);
  952. }
  953. // Same as above but with a wide-column entity. In addition to the value getting
  954. // trimmed, the type of the KV is changed to kTypeValue.
  955. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  956. KeepSingleDeletionForWriteConflictChecking_WideColumnEntity) {
  957. AddSnapshot(2, 0);
  958. RunTest({test::KeyStr("a", 2, kTypeSingleDeletion),
  959. test::KeyStr("a", 1, kTypeWideColumnEntity)},
  960. {"", "fake_entity"},
  961. {test::KeyStr("a", 2, kTypeSingleDeletion),
  962. test::KeyStr("a", 1, kTypeValue)},
  963. {"", ""}, 2 /* last_committed_seq */, nullptr /* merge_operator */,
  964. nullptr /* compaction_filter */, false /* bottommost_level */,
  965. 2 /* earliest_write_conflict_snapshot */);
  966. }
  967. // Same as above but with a value with preferred seqno entry. In addition to the
  968. // value getting trimmed, the type of the KV is changed to kTypeValue.
  969. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  970. KeepSingleDeletionForWriteConflictChecking_TimedPut) {
  971. AddSnapshot(2, 0);
  972. RunTest({test::KeyStr("a", 2, kTypeSingleDeletion),
  973. test::KeyStr("a", 1, kTypeValuePreferredSeqno)},
  974. {"", ValueWithPreferredSeqno("v1")},
  975. {test::KeyStr("a", 2, kTypeSingleDeletion),
  976. test::KeyStr("a", 1, kTypeValue)},
  977. {"", ""}, 2 /* last_committed_seq */, nullptr /* merge_operator */,
  978. nullptr /* compaction_filter */, false /* bottommost_level */,
  979. 2 /* earliest_write_conflict_snapshot */);
  980. }
  981. // Tests when a kTypeValuePreferredSeqno entry can have its preferred sequence
  982. // number swapped in. The required and sufficient conditions for an entry's
  983. // preferred sequence number to get swapped in are:
  984. // 1) The entry is visible to the earliest snapshot, AND
  985. // 2) No more entries with the same user key on lower levels, AND
  986. // This is either because:
  987. // 2a) This is a compaction to the bottommost level, OR
  988. // 2b) Keys do not exist beyond output level
  989. // 3) The entry will not resurface a range deletion entry after swapping in the
  990. // preferred sequence number.
  991. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  992. TimedPut_NotVisibleToEarliestSnapshot_NoSwapPreferredSeqno) {
  993. AddSnapshot(3);
  994. RunTest({test::KeyStr("bar", 5, kTypeValuePreferredSeqno)},
  995. {ValueWithPreferredSeqno("bv2", 2)},
  996. {test::KeyStr("bar", 5, kTypeValuePreferredSeqno)},
  997. {ValueWithPreferredSeqno("bv2", 2), "bv1"}, 5 /*last_committed_seq*/,
  998. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  999. true /*bottommost_level*/,
  1000. kMaxSequenceNumber /*earliest_write_conflict_snapshot*/,
  1001. true /*key_not_exists_beyond_output_level*/);
  1002. }
  1003. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1004. TimedPut_MoreEntriesInLowerLevels_NoSwapPreferredSeqno) {
  1005. // This tests mimics more entries in lower levels with `bottommost_level` and
  1006. // `key_not_exists_beyond_output_level` set to false.
  1007. RunTest({test::KeyStr("bar", 5, kTypeValuePreferredSeqno)},
  1008. {ValueWithPreferredSeqno("bv2", 2)},
  1009. {test::KeyStr("bar", 5, kTypeValuePreferredSeqno)},
  1010. {ValueWithPreferredSeqno("bv2", 2)}, 5 /*last_committed_seq*/,
  1011. nullptr /*merge_operator*/, nullptr /*compaction_filter*/,
  1012. false /*bottommost_level*/,
  1013. kMaxSequenceNumber /*earliest_write_conflict_snapshot*/,
  1014. false /*key_not_exists_beyond_output_level*/);
  1015. }
  1016. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1017. TimedPut_ShouldBeCoverredByRangeDeletionBeforeSwap_NoOutput) {
  1018. InitIterators({test::KeyStr("morning", 5, kTypeValuePreferredSeqno),
  1019. test::KeyStr("morning", 2, kTypeValuePreferredSeqno),
  1020. test::KeyStr("night", 6, kTypeValue)},
  1021. {ValueWithPreferredSeqno("zao", 3),
  1022. ValueWithPreferredSeqno("zao", 1), "wan"},
  1023. {test::KeyStr("ma", 6, kTypeRangeDeletion)}, {"mz"}, 6,
  1024. kMaxSequenceNumber /*last_committed_sequence*/,
  1025. nullptr /*merge_op*/, nullptr /*filter*/,
  1026. false /*bottommost_level*/,
  1027. kMaxSequenceNumber /*earliest_write_conflict_snapshot*/,
  1028. true /*key_not_exists_beyond_output_level*/);
  1029. c_iter_->SeekToFirst();
  1030. ASSERT_TRUE(c_iter_->Valid());
  1031. ASSERT_EQ(test::KeyStr("night", 6, kTypeValue), c_iter_->key().ToString());
  1032. ASSERT_EQ("wan", c_iter_->value().ToString());
  1033. c_iter_->Next();
  1034. ASSERT_FALSE(c_iter_->Valid());
  1035. ASSERT_OK(c_iter_->status());
  1036. }
  1037. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1038. TimedPut_WillBeHiddenByRangeDeletionAfterSwap_NoSwap) {
  1039. InitIterators({test::KeyStr("morning", 5, kTypeValuePreferredSeqno),
  1040. test::KeyStr("night", 6, kTypeValue)},
  1041. {ValueWithPreferredSeqno("zao", 3), "wan"},
  1042. {test::KeyStr("ma", 4, kTypeRangeDeletion)}, {"mz"}, 6,
  1043. kMaxSequenceNumber /*last_committed_sequence*/,
  1044. nullptr /*merge_op*/, nullptr /*filter*/,
  1045. false /*bottommost_level*/,
  1046. kMaxSequenceNumber /*earliest_write_conflict_snapshot*/,
  1047. true /*key_not_exists_beyond_output_level*/);
  1048. c_iter_->SeekToFirst();
  1049. ASSERT_TRUE(c_iter_->Valid());
  1050. ASSERT_EQ(test::KeyStr("morning", 5, kTypeValuePreferredSeqno),
  1051. c_iter_->key().ToString());
  1052. ASSERT_EQ(ValueWithPreferredSeqno("zao", 3), c_iter_->value().ToString());
  1053. c_iter_->Next();
  1054. ASSERT_TRUE(c_iter_->Valid());
  1055. ASSERT_EQ(test::KeyStr("night", 6, kTypeValue), c_iter_->key().ToString());
  1056. ASSERT_EQ("wan", c_iter_->value().ToString());
  1057. c_iter_->Next();
  1058. ASSERT_FALSE(c_iter_->Valid());
  1059. ASSERT_OK(c_iter_->status());
  1060. }
  1061. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1062. TimedPut_BottomMostLevelVisibleToEarliestSnapshot_SwapPreferredSeqno) {
  1063. // Preferred seqno got swapped in and also zeroed out as a bottommost level
  1064. // optimization.
  1065. RunTest(
  1066. {test::KeyStr("bar", 5, kTypeValuePreferredSeqno),
  1067. test::KeyStr("bar", 4, kTypeValuePreferredSeqno),
  1068. test::KeyStr("foo", 6, kTypeValue)},
  1069. {ValueWithPreferredSeqno("bv2", 2), ValueWithPreferredSeqno("bv1", 1),
  1070. "fv1"},
  1071. {test::KeyStr("bar", 0, kTypeValue), test::KeyStr("foo", 0, kTypeValue)},
  1072. {"bv2", "fv1"}, 6 /*last_committed_seq*/, nullptr /*merge_operator*/,
  1073. nullptr /*compaction_filter*/, true /*bottommost_level*/);
  1074. }
  1075. TEST_F(
  1076. CompactionIteratorWithSnapshotCheckerTest,
  1077. TimedPut_NonBottomMostLevelVisibleToEarliestSnapshot_SwapPreferredSeqno) {
  1078. RunTest(
  1079. {test::KeyStr("bar", 5, kTypeValuePreferredSeqno),
  1080. test::KeyStr("bar", 4, kTypeValuePreferredSeqno),
  1081. test::KeyStr("foo", 6, kTypeValue)},
  1082. {ValueWithPreferredSeqno("bv2", 2), ValueWithPreferredSeqno("bv1", 1),
  1083. "fv1"},
  1084. {test::KeyStr("bar", 2, kTypeValue), test::KeyStr("foo", 6, kTypeValue)},
  1085. {"bv2", "fv1"}, 6 /*last_committed_seq*/, nullptr /*merge_operator*/,
  1086. nullptr /*compaction_filter*/, false /*bottommost_level*/,
  1087. kMaxSequenceNumber /*earliest_write_conflict_snapshot*/,
  1088. true /*key_not_exists_beyond_output_level*/);
  1089. }
  1090. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1091. TimedPut_SequenceNumberAlreadyZeroedOut_ChangeType) {
  1092. RunTest(
  1093. {test::KeyStr("bar", 0, kTypeValuePreferredSeqno),
  1094. test::KeyStr("bar", 0, kTypeValuePreferredSeqno),
  1095. test::KeyStr("foo", 0, kTypeValue)},
  1096. {ValueWithPreferredSeqno("bv2", 2), ValueWithPreferredSeqno("bv1", 1),
  1097. "fv1"},
  1098. {test::KeyStr("bar", 0, kTypeValue), test::KeyStr("foo", 0, kTypeValue)},
  1099. {"bv2", "fv1"}, 6 /*last_committed_seq*/, nullptr /*merge_operator*/,
  1100. nullptr /*compaction_filter*/, true /*bottommost_level*/);
  1101. }
  1102. // Compaction filter should keep uncommitted key as-is, and
  1103. // * Convert the latest value to deletion, and/or
  1104. // * if latest value is a merge, apply filter to all subsequent merges.
  1105. TEST_F(CompactionIteratorWithSnapshotCheckerTest, CompactionFilter_Value) {
  1106. std::unique_ptr<CompactionFilter> compaction_filter(
  1107. new FilterAllKeysCompactionFilter());
  1108. RunTest(
  1109. {test::KeyStr("a", 2, kTypeValue), test::KeyStr("a", 1, kTypeValue),
  1110. test::KeyStr("b", 3, kTypeValue), test::KeyStr("c", 1, kTypeValue)},
  1111. {"v2", "v1", "v3", "v4"},
  1112. {test::KeyStr("a", 2, kTypeValue), test::KeyStr("a", 1, kTypeDeletion),
  1113. test::KeyStr("b", 3, kTypeValue), test::KeyStr("c", 1, kTypeDeletion)},
  1114. {"v2", "", "v3", ""}, 1 /*last_committed_seq*/,
  1115. nullptr /*merge_operator*/, compaction_filter.get());
  1116. }
  1117. TEST_F(CompactionIteratorWithSnapshotCheckerTest, CompactionFilter_TimedPut) {
  1118. // TODO(yuzhangyu): Add support for this type in compaction filter.
  1119. // Type kTypeValuePreferredSeqno is not explicitly exposed in the compaction
  1120. // filter API, so users can not operate on it through compaction filter API
  1121. // to remove/purge/change value etc. But this type of entry can be impacted by
  1122. // other entries' filter result, currently only kRemoveAndSkip type of result
  1123. // can affect it.
  1124. std::unique_ptr<CompactionFilter> compaction_filter(
  1125. new FilterAllKeysCompactionFilter());
  1126. RunTest({test::KeyStr("a", 2, kTypeValuePreferredSeqno)},
  1127. {ValueWithPreferredSeqno("v1")},
  1128. {test::KeyStr("a", 2, kTypeValuePreferredSeqno)},
  1129. {ValueWithPreferredSeqno("v1")}, 2 /*last_committed_seq*/,
  1130. nullptr /*merge_operator*/, compaction_filter.get());
  1131. }
  1132. TEST_F(CompactionIteratorWithSnapshotCheckerTest, CompactionFilter_Deletion) {
  1133. std::unique_ptr<CompactionFilter> compaction_filter(
  1134. new FilterAllKeysCompactionFilter());
  1135. RunTest(
  1136. {test::KeyStr("a", 2, kTypeDeletion), test::KeyStr("a", 1, kTypeValue)},
  1137. {"", "v1"},
  1138. {test::KeyStr("a", 2, kTypeDeletion),
  1139. test::KeyStr("a", 1, kTypeDeletion)},
  1140. {"", ""}, 1 /*last_committed_seq*/, nullptr /*merge_operator*/,
  1141. compaction_filter.get());
  1142. }
  1143. TEST_F(CompactionIteratorWithSnapshotCheckerTest,
  1144. CompactionFilter_PartialMerge) {
  1145. std::shared_ptr<MergeOperator> merge_op =
  1146. MergeOperators::CreateStringAppendOperator();
  1147. std::unique_ptr<CompactionFilter> compaction_filter(
  1148. new FilterAllKeysCompactionFilter());
  1149. RunTest({test::KeyStr("a", 3, kTypeMerge), test::KeyStr("a", 2, kTypeMerge),
  1150. test::KeyStr("a", 1, kTypeMerge)},
  1151. {"v3", "v2", "v1"}, {test::KeyStr("a", 3, kTypeMerge)}, {"v3"},
  1152. 2 /*last_committed_seq*/, merge_op.get(), compaction_filter.get());
  1153. }
  1154. TEST_F(CompactionIteratorWithSnapshotCheckerTest, CompactionFilter_FullMerge) {
  1155. std::shared_ptr<MergeOperator> merge_op =
  1156. MergeOperators::CreateStringAppendOperator();
  1157. std::unique_ptr<CompactionFilter> compaction_filter(
  1158. new FilterAllKeysCompactionFilter());
  1159. RunTest(
  1160. {test::KeyStr("a", 3, kTypeMerge), test::KeyStr("a", 2, kTypeMerge),
  1161. test::KeyStr("a", 1, kTypeValue)},
  1162. {"v3", "v2", "v1"},
  1163. {test::KeyStr("a", 3, kTypeMerge), test::KeyStr("a", 1, kTypeDeletion)},
  1164. {"v3", ""}, 2 /*last_committed_seq*/, merge_op.get(),
  1165. compaction_filter.get());
  1166. }
  1167. // Tests how CompactionIterator work together with AllowIngestBehind.
  1168. class CompactionIteratorWithAllowIngestBehindTest
  1169. : public CompactionIteratorTest {
  1170. public:
  1171. bool AllowIngestBehind() const override { return true; }
  1172. };
  1173. // When allow_ingest_behind is set, compaction iterator is not targeting
  1174. // the bottommost level since there is no guarantee there won't be further
  1175. // data ingested under the compaction output in future.
  1176. TEST_P(CompactionIteratorWithAllowIngestBehindTest, NoConvertToPutAtBottom) {
  1177. std::shared_ptr<MergeOperator> merge_op =
  1178. MergeOperators::CreateStringAppendOperator();
  1179. RunTest({test::KeyStr("a", 4, kTypeMerge), test::KeyStr("a", 3, kTypeMerge),
  1180. test::KeyStr("a", 2, kTypeMerge), test::KeyStr("b", 1, kTypeValue)},
  1181. {"a4", "a3", "a2", "b1"},
  1182. {test::KeyStr("a", 4, kTypeMerge), test::KeyStr("b", 1, kTypeValue)},
  1183. {"a2,a3,a4", "b1"}, kMaxSequenceNumber /*last_committed_seq*/,
  1184. merge_op.get(), nullptr /*compaction_filter*/,
  1185. true /*bottomost_level*/);
  1186. }
  1187. TEST_P(CompactionIteratorWithAllowIngestBehindTest,
  1188. MergeToPutIfEncounteredPutAtBottom) {
  1189. std::shared_ptr<MergeOperator> merge_op =
  1190. MergeOperators::CreateStringAppendOperator();
  1191. RunTest({test::KeyStr("a", 4, kTypeMerge), test::KeyStr("a", 3, kTypeMerge),
  1192. test::KeyStr("a", 2, kTypeValue), test::KeyStr("b", 1, kTypeValue)},
  1193. {"a4", "a3", "a2", "b1"},
  1194. {test::KeyStr("a", 4, kTypeValue), test::KeyStr("b", 1, kTypeValue)},
  1195. {"a2,a3,a4", "b1"}, kMaxSequenceNumber /*last_committed_seq*/,
  1196. merge_op.get(), nullptr /*compaction_filter*/,
  1197. true /*bottomost_level*/);
  1198. }
  1199. INSTANTIATE_TEST_CASE_P(CompactionIteratorWithAllowIngestBehindTestInstance,
  1200. CompactionIteratorWithAllowIngestBehindTest,
  1201. testing::Values(true, false));
  1202. class CompactionIteratorTsGcTest : public CompactionIteratorTest {
  1203. public:
  1204. CompactionIteratorTsGcTest()
  1205. : CompactionIteratorTest(test::BytewiseComparatorWithU64TsWrapper()) {}
  1206. };
  1207. TEST_P(CompactionIteratorTsGcTest, NoKeyEligibleForGC) {
  1208. constexpr char user_key[][2] = {{'a', '\0'}, {'b', '\0'}};
  1209. const std::vector<std::string> input_keys = {
  1210. test::KeyStr(/*ts=*/103, user_key[0], /*seq=*/4, kTypeValue),
  1211. test::KeyStr(/*ts=*/102, user_key[0], /*seq=*/3,
  1212. kTypeDeletionWithTimestamp),
  1213. test::KeyStr(/*ts=*/104, user_key[1], /*seq=*/5, kTypeValue)};
  1214. const std::vector<std::string> input_values = {"a3", "", "b2"};
  1215. std::string full_history_ts_low;
  1216. // All keys' timestamps are newer than or equal to 102, thus none of them
  1217. // will be eligible for GC.
  1218. PutFixed64(&full_history_ts_low, 102);
  1219. const std::vector<std::string>& expected_keys = input_keys;
  1220. const std::vector<std::string>& expected_values = input_values;
  1221. const std::vector<std::pair<bool, bool>> params = {
  1222. {false, false}, {false, true}, {true, true}};
  1223. for (const std::pair<bool, bool>& param : params) {
  1224. const bool bottommost_level = param.first;
  1225. const bool key_not_exists_beyond_output_level = param.second;
  1226. RunTest(input_keys, input_values, expected_keys, expected_values,
  1227. /*last_committed_seq=*/kMaxSequenceNumber,
  1228. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1229. bottommost_level,
  1230. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1231. key_not_exists_beyond_output_level, &full_history_ts_low);
  1232. }
  1233. }
  1234. TEST_P(CompactionIteratorTsGcTest, NoMergeEligibleForGc) {
  1235. constexpr char user_key[] = "a";
  1236. const std::vector<std::string> input_keys = {
  1237. test::KeyStr(10002, user_key, 102, kTypeMerge),
  1238. test::KeyStr(10001, user_key, 101, kTypeMerge),
  1239. test::KeyStr(10000, user_key, 100, kTypeValue)};
  1240. const std::vector<std::string> input_values = {"2", "1", "a0"};
  1241. std::shared_ptr<MergeOperator> merge_op =
  1242. MergeOperators::CreateStringAppendTESTOperator();
  1243. const std::vector<std::string>& expected_keys = input_keys;
  1244. const std::vector<std::string>& expected_values = input_values;
  1245. const std::vector<std::pair<bool, bool>> params = {
  1246. {false, false}, {false, true}, {true, true}};
  1247. for (const auto& param : params) {
  1248. const bool bottommost_level = param.first;
  1249. const bool key_not_exists_beyond_output_level = param.second;
  1250. RunTest(input_keys, input_values, expected_keys, expected_values,
  1251. /*last_committed_seq=*/kMaxSequenceNumber, merge_op.get(),
  1252. /*compaction_filter=*/nullptr, bottommost_level,
  1253. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1254. key_not_exists_beyond_output_level,
  1255. /*full_history_ts_low=*/nullptr);
  1256. }
  1257. }
  1258. TEST_P(CompactionIteratorTsGcTest, AllKeysOlderThanThreshold) {
  1259. constexpr char user_key[][2] = {{'a', '\0'}, {'b', '\0'}};
  1260. const std::vector<std::string> input_keys = {
  1261. test::KeyStr(/*ts=*/103, user_key[0], /*seq=*/4,
  1262. kTypeDeletionWithTimestamp),
  1263. test::KeyStr(/*ts=*/102, user_key[0], /*seq=*/3, kTypeValue),
  1264. test::KeyStr(/*ts=*/101, user_key[0], /*seq=*/2, kTypeValue),
  1265. test::KeyStr(/*ts=*/104, user_key[1], /*seq=*/5, kTypeValue)};
  1266. const std::vector<std::string> input_values = {"", "a2", "a1", "b5"};
  1267. std::string full_history_ts_low;
  1268. PutFixed64(&full_history_ts_low, std::numeric_limits<uint64_t>::max());
  1269. {
  1270. // With a snapshot at seq 3, both the deletion marker and the key at 3 must
  1271. // be preserved.
  1272. AddSnapshot(3);
  1273. const std::vector<std::string> expected_keys = {
  1274. input_keys[0], input_keys[1], input_keys[3]};
  1275. const std::vector<std::string> expected_values = {"", "a2", "b5"};
  1276. RunTest(input_keys, input_values, expected_keys, expected_values,
  1277. /*last_committed_seq=*/kMaxSequenceNumber,
  1278. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1279. /*bottommost_level=*/false,
  1280. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1281. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1282. ClearSnapshots();
  1283. }
  1284. {
  1285. // No snapshot, the deletion marker should be preserved because the user
  1286. // key may appear beyond output level.
  1287. const std::vector<std::string> expected_keys = {input_keys[0],
  1288. input_keys[3]};
  1289. const std::vector<std::string> expected_values = {"", "b5"};
  1290. RunTest(input_keys, input_values, expected_keys, expected_values,
  1291. /*last_committed_seq=*/kMaxSequenceNumber,
  1292. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1293. /*bottommost_level=*/false,
  1294. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1295. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1296. }
  1297. {
  1298. // No snapshot, the deletion marker can be dropped because the user key
  1299. // does not appear in higher levels.
  1300. const std::vector<std::string> expected_keys = {input_keys[3]};
  1301. const std::vector<std::string> expected_values = {"b5"};
  1302. RunTest(input_keys, input_values, expected_keys, expected_values,
  1303. /*last_committed_seq=*/kMaxSequenceNumber,
  1304. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1305. /*bottommost_level=*/false,
  1306. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1307. /*key_not_exists_beyond_output_level=*/true, &full_history_ts_low);
  1308. }
  1309. }
  1310. TEST_P(CompactionIteratorTsGcTest, SomeMergesOlderThanThreshold) {
  1311. constexpr char user_key[][2] = {"a", "f"};
  1312. const std::vector<std::string> input_keys = {
  1313. test::KeyStr(/*ts=*/25000, user_key[0], /*seq=*/2500, kTypeMerge),
  1314. test::KeyStr(/*ts=*/19000, user_key[0], /*seq=*/2300, kTypeMerge),
  1315. test::KeyStr(/*ts=*/18000, user_key[0], /*seq=*/1800, kTypeMerge),
  1316. test::KeyStr(/*ts=*/16000, user_key[0], /*seq=*/1600, kTypeValue),
  1317. test::KeyStr(/*ts=*/19000, user_key[1], /*seq=*/2000, kTypeMerge),
  1318. test::KeyStr(/*ts=*/17000, user_key[1], /*seq=*/1700, kTypeMerge),
  1319. test::KeyStr(/*ts=*/15000, user_key[1], /*seq=*/1600,
  1320. kTypeDeletionWithTimestamp)};
  1321. const std::vector<std::string> input_values = {"25", "19", "18", "16",
  1322. "19", "17", ""};
  1323. std::shared_ptr<MergeOperator> merge_op =
  1324. MergeOperators::CreateStringAppendTESTOperator();
  1325. std::string full_history_ts_low;
  1326. PutFixed64(&full_history_ts_low, 20000);
  1327. const std::vector<std::pair<bool, bool>> params = {
  1328. {false, false}, {false, true}, {true, true}};
  1329. {
  1330. AddSnapshot(1600);
  1331. AddSnapshot(1900);
  1332. const std::vector<std::string> expected_keys = {
  1333. test::KeyStr(/*ts=*/25000, user_key[0], /*seq=*/2500, kTypeMerge),
  1334. test::KeyStr(/*ts=*/19000, user_key[0], /*seq=*/2300, kTypeMerge),
  1335. test::KeyStr(/*ts=*/18000, user_key[0], /*seq=*/1800, kTypeMerge),
  1336. test::KeyStr(/*ts=*/16000, user_key[0], /*seq=*/1600, kTypeValue),
  1337. test::KeyStr(/*ts=*/19000, user_key[1], /*seq=*/2000, kTypeMerge),
  1338. test::KeyStr(/*ts=*/17000, user_key[1], /*seq=*/1700, kTypeMerge),
  1339. test::KeyStr(/*ts=*/15000, user_key[1], /*seq=*/1600,
  1340. kTypeDeletionWithTimestamp)};
  1341. const std::vector<std::string> expected_values = {"25", "19", "18", "16",
  1342. "19", "17", ""};
  1343. for (const auto& param : params) {
  1344. const bool bottommost_level = param.first;
  1345. const bool key_not_exists_beyond_output_level = param.second;
  1346. auto expected_keys_copy = expected_keys;
  1347. auto expected_values_copy = expected_values;
  1348. if (bottommost_level || key_not_exists_beyond_output_level) {
  1349. // the kTypeDeletionWithTimestamp will be dropped
  1350. expected_keys_copy.pop_back();
  1351. expected_values_copy.pop_back();
  1352. if (bottommost_level) {
  1353. // seq zero
  1354. expected_keys_copy[3] =
  1355. test::KeyStr(/*ts=*/0, user_key[0], /*seq=*/0, kTypeValue);
  1356. }
  1357. }
  1358. RunTest(input_keys, input_values, expected_keys_copy,
  1359. expected_values_copy,
  1360. /*last_committed_seq=*/kMaxSequenceNumber, merge_op.get(),
  1361. /*compaction_filter=*/nullptr, bottommost_level,
  1362. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1363. key_not_exists_beyond_output_level, &full_history_ts_low);
  1364. }
  1365. ClearSnapshots();
  1366. }
  1367. // No snapshots
  1368. {
  1369. const std::vector<std::string> expected_keys = {
  1370. test::KeyStr(/*ts=*/25000, user_key[0], /*seq=*/2500, kTypeValue),
  1371. test::KeyStr(/*ts=*/19000, user_key[1], /*seq=*/2000, kTypeValue)};
  1372. const std::vector<std::string> expected_values = {"16,18,19,25", "17,19"};
  1373. for (const auto& param : params) {
  1374. const bool bottommost_level = param.first;
  1375. const bool key_not_exists_beyond_output_level = param.second;
  1376. auto expected_keys_copy = expected_keys;
  1377. auto expected_values_copy = expected_values;
  1378. if (bottommost_level) {
  1379. expected_keys_copy[1] =
  1380. test::KeyStr(/*ts=*/0, user_key[1], /*seq=*/0, kTypeValue);
  1381. }
  1382. RunTest(input_keys, input_values, expected_keys_copy,
  1383. expected_values_copy,
  1384. /*last_committed_seq=*/kMaxSequenceNumber, merge_op.get(),
  1385. /*compaction_filter=*/nullptr, bottommost_level,
  1386. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1387. key_not_exists_beyond_output_level, &full_history_ts_low);
  1388. }
  1389. }
  1390. }
  1391. TEST_P(CompactionIteratorTsGcTest, NewHidesOldSameSnapshot) {
  1392. constexpr char user_key[] = "a";
  1393. const std::vector<std::string> input_keys = {
  1394. test::KeyStr(/*ts=*/103, user_key, /*seq=*/4, kTypeDeletionWithTimestamp),
  1395. test::KeyStr(/*ts=*/102, user_key, /*seq=*/3, kTypeValue),
  1396. test::KeyStr(/*ts=*/101, user_key, /*seq=*/2, kTypeValue),
  1397. test::KeyStr(/*ts=*/100, user_key, /*seq=*/1, kTypeValue)};
  1398. const std::vector<std::string> input_values = {"", "a2", "a1", "a0"};
  1399. {
  1400. std::string full_history_ts_low;
  1401. // Keys whose timestamps larger than or equal to 102 will be preserved.
  1402. PutFixed64(&full_history_ts_low, 102);
  1403. const std::vector<std::string> expected_keys = {
  1404. input_keys[0], input_keys[1], input_keys[2]};
  1405. const std::vector<std::string> expected_values = {"", input_values[1],
  1406. input_values[2]};
  1407. RunTest(input_keys, input_values, expected_keys, expected_values,
  1408. /*last_committed_seq=*/kMaxSequenceNumber,
  1409. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1410. /*bottommost_level=*/false,
  1411. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1412. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1413. }
  1414. }
  1415. TEST_P(CompactionIteratorTsGcTest, DropTombstones) {
  1416. constexpr char user_key[] = "a";
  1417. const std::vector<std::string> input_keys = {
  1418. test::KeyStr(/*ts=*/103, user_key, /*seq=*/4, kTypeDeletionWithTimestamp),
  1419. test::KeyStr(/*ts=*/102, user_key, /*seq=*/3, kTypeValue),
  1420. test::KeyStr(/*ts=*/101, user_key, /*seq=*/2, kTypeDeletionWithTimestamp),
  1421. test::KeyStr(/*ts=*/100, user_key, /*seq=*/1, kTypeValue)};
  1422. const std::vector<std::string> input_values = {"", "a2", "", "a0"};
  1423. const std::vector<std::string> expected_keys = {input_keys[0], input_keys[1]};
  1424. const std::vector<std::string> expected_values = {"", "a2"};
  1425. // Take a snapshot at seq 2.
  1426. AddSnapshot(2);
  1427. {
  1428. // Non-bottommost level, but key does not exist beyond output level.
  1429. std::string full_history_ts_low;
  1430. PutFixed64(&full_history_ts_low, 102);
  1431. RunTest(input_keys, input_values, expected_keys, expected_values,
  1432. /*last_committed_sequence=*/kMaxSequenceNumber,
  1433. /*merge_op=*/nullptr, /*compaction_filter=*/nullptr,
  1434. /*bottommost_level=*/false,
  1435. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1436. /*key_not_exists_beyond_output_level=*/true, &full_history_ts_low);
  1437. }
  1438. {
  1439. // Bottommost level
  1440. std::string full_history_ts_low;
  1441. PutFixed64(&full_history_ts_low, 102);
  1442. RunTest(input_keys, input_values, expected_keys, expected_values,
  1443. /*last_committed_seq=*/kMaxSequenceNumber,
  1444. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1445. /*bottommost_level=*/true,
  1446. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1447. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1448. }
  1449. }
  1450. TEST_P(CompactionIteratorTsGcTest, RewriteTs) {
  1451. constexpr char user_key[] = "a";
  1452. const std::vector<std::string> input_keys = {
  1453. test::KeyStr(/*ts=*/103, user_key, /*seq=*/4, kTypeDeletionWithTimestamp),
  1454. test::KeyStr(/*ts=*/102, user_key, /*seq=*/3, kTypeValue),
  1455. test::KeyStr(/*ts=*/101, user_key, /*seq=*/2, kTypeDeletionWithTimestamp),
  1456. test::KeyStr(/*ts=*/100, user_key, /*seq=*/1, kTypeValue)};
  1457. const std::vector<std::string> input_values = {"", "a2", "", "a0"};
  1458. const std::vector<std::string> expected_keys = {
  1459. input_keys[0], input_keys[1], input_keys[2],
  1460. test::KeyStr(/*ts=*/0, user_key, /*seq=*/0, kTypeValue)};
  1461. const std::vector<std::string> expected_values = {"", "a2", "", "a0"};
  1462. AddSnapshot(1);
  1463. AddSnapshot(2);
  1464. {
  1465. // Bottommost level and need to rewrite both ts and seq.
  1466. std::string full_history_ts_low;
  1467. PutFixed64(&full_history_ts_low, 102);
  1468. RunTest(input_keys, input_values, expected_keys, expected_values,
  1469. /*last_committed_seq=*/kMaxSequenceNumber,
  1470. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1471. /*bottommost_level=*/true,
  1472. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1473. /*key_not_exists_beyond_output_level=*/true, &full_history_ts_low);
  1474. }
  1475. }
  1476. TEST_P(CompactionIteratorTsGcTest, SingleDeleteNoKeyEligibleForGC) {
  1477. constexpr char user_key[][2] = {{'a', '\0'}, {'b', '\0'}};
  1478. const std::vector<std::string> input_keys = {
  1479. test::KeyStr(/*ts=*/104, user_key[0], /*seq=*/4, kTypeSingleDeletion),
  1480. test::KeyStr(/*ts=*/103, user_key[0], /*seq=*/3, kTypeValue),
  1481. test::KeyStr(/*ts=*/102, user_key[1], /*seq=*/2, kTypeValue)};
  1482. const std::vector<std::string> input_values = {"", "a3", "b2"};
  1483. std::string full_history_ts_low;
  1484. // All keys' timestamps are newer than or equal to 102, thus none of them
  1485. // will be eligible for GC.
  1486. PutFixed64(&full_history_ts_low, 102);
  1487. const std::vector<std::string>& expected_keys = input_keys;
  1488. const std::vector<std::string>& expected_values = input_values;
  1489. const std::vector<std::pair<bool, bool>> params = {
  1490. {false, false}, {false, true}, {true, true}};
  1491. for (const std::pair<bool, bool>& param : params) {
  1492. const bool bottommost_level = param.first;
  1493. const bool key_not_exists_beyond_output_level = param.second;
  1494. RunTest(input_keys, input_values, expected_keys, expected_values,
  1495. /*last_committed_seq=*/kMaxSequenceNumber,
  1496. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1497. bottommost_level,
  1498. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1499. key_not_exists_beyond_output_level, &full_history_ts_low);
  1500. }
  1501. }
  1502. TEST_P(CompactionIteratorTsGcTest, SingleDeleteDropTombstones) {
  1503. constexpr char user_key[] = "a";
  1504. const std::vector<std::string> input_keys = {
  1505. test::KeyStr(/*ts=*/103, user_key, /*seq=*/4, kTypeSingleDeletion),
  1506. test::KeyStr(/*ts=*/102, user_key, /*seq=*/3, kTypeValue),
  1507. test::KeyStr(/*ts=*/101, user_key, /*seq=*/2, kTypeSingleDeletion),
  1508. test::KeyStr(/*ts=*/100, user_key, /*seq=*/1, kTypeValue)};
  1509. const std::vector<std::string> input_values = {"", "a2", "", "a0"};
  1510. const std::vector<std::string> expected_keys = {input_keys[0], input_keys[1]};
  1511. const std::vector<std::string> expected_values = {"", "a2"};
  1512. // Take a snapshot at seq 2.
  1513. AddSnapshot(2);
  1514. {
  1515. const std::vector<std::pair<bool, bool>> params = {
  1516. {false, false}, {false, true}, {true, true}};
  1517. for (const std::pair<bool, bool>& param : params) {
  1518. const bool bottommost_level = param.first;
  1519. const bool key_not_exists_beyond_output_level = param.second;
  1520. std::string full_history_ts_low;
  1521. PutFixed64(&full_history_ts_low, 102);
  1522. RunTest(input_keys, input_values, expected_keys, expected_values,
  1523. /*last_committed_seq=*/kMaxSequenceNumber,
  1524. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1525. bottommost_level,
  1526. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1527. key_not_exists_beyond_output_level, &full_history_ts_low);
  1528. }
  1529. }
  1530. }
  1531. TEST_P(CompactionIteratorTsGcTest, SingleDeleteAllKeysOlderThanThreshold) {
  1532. constexpr char user_key[][2] = {{'a', '\0'}, {'b', '\0'}};
  1533. const std::vector<std::string> input_keys = {
  1534. test::KeyStr(/*ts=*/103, user_key[0], /*seq=*/4, kTypeSingleDeletion),
  1535. test::KeyStr(/*ts=*/102, user_key[0], /*seq=*/3, kTypeValue),
  1536. test::KeyStr(/*ts=*/104, user_key[1], /*seq=*/5, kTypeValue)};
  1537. const std::vector<std::string> input_values = {"", "a2", "b5"};
  1538. std::string full_history_ts_low;
  1539. PutFixed64(&full_history_ts_low, std::numeric_limits<uint64_t>::max());
  1540. {
  1541. // With a snapshot at seq 3, both the deletion marker and the key at 3 must
  1542. // be preserved.
  1543. AddSnapshot(3);
  1544. const std::vector<std::string> expected_keys = {
  1545. input_keys[0], input_keys[1], input_keys[2]};
  1546. const std::vector<std::string> expected_values = {"", "a2", "b5"};
  1547. RunTest(input_keys, input_values, expected_keys, expected_values,
  1548. /*last_committed_seq=*/kMaxSequenceNumber,
  1549. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1550. /*bottommost_level=*/false,
  1551. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1552. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1553. ClearSnapshots();
  1554. }
  1555. {
  1556. // No snapshot.
  1557. const std::vector<std::string> expected_keys = {input_keys[2]};
  1558. const std::vector<std::string> expected_values = {"b5"};
  1559. RunTest(input_keys, input_values, expected_keys, expected_values,
  1560. /*last_committed_seq=*/kMaxSequenceNumber,
  1561. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1562. /*bottommost_level=*/false,
  1563. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1564. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1565. }
  1566. }
  1567. TEST_P(CompactionIteratorTsGcTest, ZeroSeqOfKeyAndSnapshot) {
  1568. AddSnapshot(0);
  1569. std::string full_history_ts_low;
  1570. PutFixed64(&full_history_ts_low, std::numeric_limits<uint64_t>::max());
  1571. const std::vector<std::string> input_keys = {
  1572. test::KeyStr(101, "a", 0, kTypeValue),
  1573. test::KeyStr(102, "b", 0, kTypeValue)};
  1574. const std::vector<std::string> input_values = {"a1", "b1"};
  1575. RunTest(input_keys, input_values, input_keys, input_values,
  1576. /*last_committed_seq=*/kMaxSequenceNumber,
  1577. /*merge_operator=*/nullptr, /*compaction_filter=*/nullptr,
  1578. /*bottommost_level=*/false,
  1579. /*earliest_write_conflict_snapshot=*/kMaxSequenceNumber,
  1580. /*key_not_exists_beyond_output_level=*/false, &full_history_ts_low);
  1581. }
  1582. INSTANTIATE_TEST_CASE_P(CompactionIteratorTsGcTestInstance,
  1583. CompactionIteratorTsGcTest,
  1584. testing::Values(true, false));
  1585. } // namespace ROCKSDB_NAMESPACE
  1586. int main(int argc, char** argv) {
  1587. ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
  1588. ::testing::InitGoogleTest(&argc, argv);
  1589. return RUN_ALL_TESTS();
  1590. }