db_range_del_test.cc 142 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510351135123513351435153516351735183519352035213522352335243525352635273528352935303531353235333534353535363537353835393540354135423543354435453546354735483549355035513552355335543555355635573558355935603561356235633564356535663567356835693570357135723573357435753576357735783579358035813582358335843585358635873588358935903591359235933594359535963597359835993600360136023603360436053606360736083609361036113612361336143615361636173618361936203621362236233624362536263627362836293630363136323633363436353636363736383639364036413642364336443645364636473648364936503651365236533654365536563657365836593660366136623663366436653666366736683669367036713672367336743675367636773678367936803681368236833684368536863687368836893690369136923693369436953696369736983699370037013702370337043705370637073708370937103711371237133714371537163717371837193720372137223723372437253726372737283729373037313732373337343735373637373738373937403741374237433744374537463747374837493750375137523753375437553756375737583759376037613762376337643765376637673768376937703771377237733774377537763777377837793780378137823783378437853786378737883789379037913792379337943795379637973798379938003801380238033804380538063807380838093810381138123813381438153816381738183819382038213822382338243825382638273828382938303831383238333834
  1. // Copyright (c) 2016-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/db_test_util.h"
  6. #include "db/version_set.h"
  7. #include "port/stack_trace.h"
  8. #include "rocksdb/utilities/write_batch_with_index.h"
  9. #include "test_util/testutil.h"
  10. #include "util/random.h"
  11. #include "utilities/merge_operators.h"
  12. namespace ROCKSDB_NAMESPACE {
  13. // TODO(cbi): parameterize the test to cover user-defined timestamp cases
  14. class DBRangeDelTest : public DBTestBase {
  15. public:
  16. DBRangeDelTest() : DBTestBase("db_range_del_test", /*env_do_fsync=*/false) {}
  17. std::string GetNumericStr(int key) {
  18. uint64_t uint64_key = static_cast<uint64_t>(key);
  19. std::string str;
  20. str.resize(8);
  21. memcpy(str.data(), static_cast<void*>(&uint64_key), 8);
  22. return str;
  23. }
  24. };
  25. TEST_F(DBRangeDelTest, NonBlockBasedTableNotSupported) {
  26. // TODO: figure out why MmapReads trips the iterator pinning assertion in
  27. // RangeDelAggregator. Ideally it would be supported; otherwise it should at
  28. // least be explicitly unsupported.
  29. for (auto config : {kPlainTableAllBytesPrefix, /* kWalDirAndMmapReads */}) {
  30. option_config_ = config;
  31. DestroyAndReopen(CurrentOptions());
  32. ASSERT_TRUE(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  33. "dr1", "dr1")
  34. .IsNotSupported());
  35. }
  36. }
  37. TEST_F(DBRangeDelTest, WriteBatchWithIndexNotSupported) {
  38. WriteBatchWithIndex indexedBatch{};
  39. ASSERT_TRUE(indexedBatch.DeleteRange(db_->DefaultColumnFamily(), "dr1", "dr1")
  40. .IsNotSupported());
  41. ASSERT_TRUE(indexedBatch.DeleteRange("dr1", "dr1").IsNotSupported());
  42. }
  43. TEST_F(DBRangeDelTest, EndSameAsStartCoversNothing) {
  44. ASSERT_OK(db_->Put(WriteOptions(), "b", "val"));
  45. ASSERT_OK(
  46. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "b", "b"));
  47. ASSERT_EQ("val", Get("b"));
  48. }
  49. TEST_F(DBRangeDelTest, EndComesBeforeStartInvalidArgument) {
  50. ASSERT_OK(db_->Put(WriteOptions(), "b", "val"));
  51. ASSERT_TRUE(
  52. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "b", "a")
  53. .IsInvalidArgument());
  54. ASSERT_EQ("val", Get("b"));
  55. }
  56. TEST_F(DBRangeDelTest, FlushOutputHasOnlyRangeTombstones) {
  57. do {
  58. DestroyAndReopen(CurrentOptions());
  59. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  60. "dr1", "dr2"));
  61. ASSERT_OK(db_->Flush(FlushOptions()));
  62. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  63. } while (ChangeOptions(kRangeDelSkipConfigs));
  64. }
  65. TEST_F(DBRangeDelTest, DictionaryCompressionWithOnlyRangeTombstones) {
  66. Options opts = CurrentOptions();
  67. opts.compression_opts.max_dict_bytes = 16384;
  68. Reopen(opts);
  69. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "dr1",
  70. "dr2"));
  71. ASSERT_OK(db_->Flush(FlushOptions()));
  72. }
  73. TEST_F(DBRangeDelTest, CompactionOutputHasOnlyRangeTombstone) {
  74. do {
  75. Options opts = CurrentOptions();
  76. opts.disable_auto_compactions = true;
  77. opts.statistics = CreateDBStatistics();
  78. DestroyAndReopen(opts);
  79. // snapshot protects range tombstone from dropping due to becoming obsolete.
  80. const Snapshot* snapshot = db_->GetSnapshot();
  81. ASSERT_OK(
  82. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  83. ASSERT_OK(db_->Flush(FlushOptions()));
  84. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  85. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  86. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  87. true /* disallow_trivial_move */));
  88. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  89. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  90. ASSERT_EQ(0, TestGetTickerCount(opts, COMPACTION_RANGE_DEL_DROP_OBSOLETE));
  91. db_->ReleaseSnapshot(snapshot);
  92. // Skip cuckoo memtables, which do not support snapshots. Skip non-leveled
  93. // compactions as the above assertions about the number of files in a level
  94. // do not hold true.
  95. } while (ChangeOptions(kRangeDelSkipConfigs | kSkipUniversalCompaction |
  96. kSkipFIFOCompaction));
  97. }
  98. TEST_F(DBRangeDelTest, CompactionOutputFilesExactlyFilled) {
  99. // regression test for exactly filled compaction output files. Previously
  100. // another file would be generated containing all range deletions, which
  101. // could invalidate the non-overlapping file boundary invariant.
  102. const int kNumPerFile = 4, kNumFiles = 2, kFileBytes = 9 << 10;
  103. Options options = CurrentOptions();
  104. options.disable_auto_compactions = true;
  105. options.level0_file_num_compaction_trigger = kNumFiles;
  106. options.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  107. options.num_levels = 2;
  108. options.target_file_size_base = kFileBytes;
  109. BlockBasedTableOptions table_options;
  110. table_options.block_size_deviation = 50; // each block holds two keys
  111. options.table_factory.reset(NewBlockBasedTableFactory(table_options));
  112. Reopen(options);
  113. // snapshot protects range tombstone from dropping due to becoming obsolete.
  114. const Snapshot* snapshot = db_->GetSnapshot();
  115. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  116. Key(1)));
  117. Random rnd(301);
  118. for (int i = 0; i < kNumFiles; ++i) {
  119. std::vector<std::string> values;
  120. // Write 12K (4 values, each 3K)
  121. for (int j = 0; j < kNumPerFile; j++) {
  122. values.push_back(rnd.RandomString(3 << 10));
  123. ASSERT_OK(Put(Key(i * kNumPerFile + j), values[j]));
  124. if (j == 0 && i > 0) {
  125. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  126. }
  127. }
  128. }
  129. // put extra key to trigger final flush
  130. ASSERT_OK(Put("", ""));
  131. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  132. ASSERT_EQ(kNumFiles, NumTableFilesAtLevel(0));
  133. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  134. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  135. true /* disallow_trivial_move */));
  136. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  137. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  138. db_->ReleaseSnapshot(snapshot);
  139. }
  140. TEST_F(DBRangeDelTest, MaxCompactionBytesCutsOutputFiles) {
  141. // Ensures range deletion spanning multiple compaction output files that are
  142. // cut by max_compaction_bytes will have non-overlapping key-ranges.
  143. // https://github.com/facebook/rocksdb/issues/1778
  144. const int kNumFiles = 2, kNumPerFile = 1 << 8, kBytesPerVal = 1 << 12;
  145. Options opts = CurrentOptions();
  146. opts.comparator = test::Uint64Comparator();
  147. opts.disable_auto_compactions = true;
  148. opts.level0_file_num_compaction_trigger = kNumFiles;
  149. opts.max_compaction_bytes = kNumPerFile * kBytesPerVal;
  150. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  151. // Want max_compaction_bytes to trigger the end of compaction output file, not
  152. // target_file_size_base, so make the latter much bigger
  153. // opts.target_file_size_base = 100 * opts.max_compaction_bytes;
  154. opts.target_file_size_base = 1;
  155. DestroyAndReopen(opts);
  156. // snapshot protects range tombstone from dropping due to becoming obsolete.
  157. const Snapshot* snapshot = db_->GetSnapshot();
  158. Random rnd(301);
  159. ASSERT_OK(Put(GetNumericStr(0), rnd.RandomString(kBytesPerVal)));
  160. ASSERT_OK(
  161. Put(GetNumericStr(kNumPerFile - 1), rnd.RandomString(kBytesPerVal)));
  162. ASSERT_OK(Flush());
  163. ASSERT_OK(Put(GetNumericStr(kNumPerFile), rnd.RandomString(kBytesPerVal)));
  164. ASSERT_OK(
  165. Put(GetNumericStr(kNumPerFile * 2 - 1), rnd.RandomString(kBytesPerVal)));
  166. ASSERT_OK(Flush());
  167. MoveFilesToLevel(2);
  168. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  169. ASSERT_EQ(NumTableFilesAtLevel(2), 2);
  170. ASSERT_OK(
  171. db_->SetOptions(db_->DefaultColumnFamily(),
  172. {{"target_file_size_base",
  173. std::to_string(100 * opts.max_compaction_bytes)}}));
  174. // It spans the whole key-range, thus will be included in all output files
  175. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  176. GetNumericStr(0),
  177. GetNumericStr(kNumFiles * kNumPerFile - 1)));
  178. for (int i = 0; i < kNumFiles; ++i) {
  179. std::vector<std::string> values;
  180. // Write 1MB (256 values, each 4K)
  181. for (int j = 0; j < kNumPerFile; j++) {
  182. values.push_back(rnd.RandomString(kBytesPerVal));
  183. ASSERT_OK(Put(GetNumericStr(kNumPerFile * i + j), values[j]));
  184. }
  185. // extra entry to trigger SpecialSkipListFactory's flush
  186. ASSERT_OK(Put(GetNumericStr(kNumPerFile), ""));
  187. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  188. ASSERT_EQ(i + 1, NumTableFilesAtLevel(0));
  189. }
  190. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr,
  191. /*column_family=*/nullptr,
  192. /*disallow_trivial_move=*/true));
  193. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  194. ASSERT_GE(NumTableFilesAtLevel(1), 2);
  195. std::vector<std::vector<FileMetaData>> files;
  196. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  197. for (size_t i = 0; i + 1 < files[1].size(); ++i) {
  198. ASSERT_TRUE(InternalKeyComparator(opts.comparator)
  199. .Compare(files[1][i].largest, files[1][i + 1].smallest) <
  200. 0);
  201. }
  202. db_->ReleaseSnapshot(snapshot);
  203. }
  204. TEST_F(DBRangeDelTest, SentinelsOmittedFromOutputFile) {
  205. // Regression test for bug where sentinel range deletions (i.e., ones with
  206. // sequence number of zero) were included in output files.
  207. // snapshot protects range tombstone from dropping due to becoming obsolete.
  208. const Snapshot* snapshot = db_->GetSnapshot();
  209. // gaps between ranges creates sentinels in our internal representation
  210. std::vector<std::pair<std::string, std::string>> range_dels = {
  211. {"a", "b"}, {"c", "d"}, {"e", "f"}};
  212. for (const auto& range_del : range_dels) {
  213. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  214. range_del.first, range_del.second));
  215. }
  216. ASSERT_OK(db_->Flush(FlushOptions()));
  217. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  218. std::vector<std::vector<FileMetaData>> files;
  219. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  220. ASSERT_GT(files[0][0].fd.smallest_seqno, 0);
  221. db_->ReleaseSnapshot(snapshot);
  222. }
  223. TEST_F(DBRangeDelTest, FlushRangeDelsSameStartKey) {
  224. ASSERT_OK(db_->Put(WriteOptions(), "b1", "val"));
  225. ASSERT_OK(
  226. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "c"));
  227. ASSERT_OK(db_->Put(WriteOptions(), "b2", "val"));
  228. ASSERT_OK(
  229. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "b"));
  230. // first iteration verifies query correctness in memtable, second verifies
  231. // query correctness for a single SST file
  232. for (int i = 0; i < 2; ++i) {
  233. if (i > 0) {
  234. ASSERT_OK(db_->Flush(FlushOptions()));
  235. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  236. }
  237. std::string value;
  238. ASSERT_TRUE(db_->Get(ReadOptions(), "b1", &value).IsNotFound());
  239. ASSERT_OK(db_->Get(ReadOptions(), "b2", &value));
  240. }
  241. }
  242. TEST_F(DBRangeDelTest, CompactRangeDelsSameStartKey) {
  243. ASSERT_OK(db_->Put(WriteOptions(), "unused",
  244. "val")); // prevents empty after compaction
  245. ASSERT_OK(db_->Put(WriteOptions(), "b1", "val"));
  246. ASSERT_OK(db_->Flush(FlushOptions()));
  247. ASSERT_OK(
  248. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "c"));
  249. ASSERT_OK(db_->Flush(FlushOptions()));
  250. ASSERT_OK(
  251. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "b"));
  252. ASSERT_OK(db_->Flush(FlushOptions()));
  253. ASSERT_EQ(3, NumTableFilesAtLevel(0));
  254. for (int i = 0; i < 2; ++i) {
  255. if (i > 0) {
  256. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  257. true /* disallow_trivial_move */));
  258. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  259. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  260. }
  261. std::string value;
  262. ASSERT_TRUE(db_->Get(ReadOptions(), "b1", &value).IsNotFound());
  263. }
  264. }
  265. TEST_F(DBRangeDelTest, FlushRemovesCoveredKeys) {
  266. const int kNum = 300, kRangeBegin = 50, kRangeEnd = 250;
  267. Options opts = CurrentOptions();
  268. opts.comparator = test::Uint64Comparator();
  269. DestroyAndReopen(opts);
  270. // Write a third before snapshot, a third between snapshot and tombstone, and
  271. // a third after the tombstone. Keys older than snapshot or newer than the
  272. // tombstone should be preserved.
  273. const Snapshot* snapshot = nullptr;
  274. for (int i = 0; i < kNum; ++i) {
  275. if (i == kNum / 3) {
  276. snapshot = db_->GetSnapshot();
  277. } else if (i == 2 * kNum / 3) {
  278. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  279. GetNumericStr(kRangeBegin),
  280. GetNumericStr(kRangeEnd)));
  281. }
  282. ASSERT_OK(db_->Put(WriteOptions(), GetNumericStr(i), "val"));
  283. }
  284. ASSERT_OK(db_->Flush(FlushOptions()));
  285. for (int i = 0; i < kNum; ++i) {
  286. ReadOptions read_opts;
  287. read_opts.ignore_range_deletions = true;
  288. std::string value;
  289. if (i < kRangeBegin || i > kRangeEnd || i < kNum / 3 || i >= 2 * kNum / 3) {
  290. ASSERT_OK(db_->Get(read_opts, GetNumericStr(i), &value));
  291. } else {
  292. ASSERT_TRUE(db_->Get(read_opts, GetNumericStr(i), &value).IsNotFound());
  293. }
  294. }
  295. db_->ReleaseSnapshot(snapshot);
  296. }
  297. TEST_F(DBRangeDelTest, CompactionRemovesCoveredKeys) {
  298. const int kNumPerFile = 100, kNumFiles = 4;
  299. Options opts = CurrentOptions();
  300. opts.comparator = test::Uint64Comparator();
  301. opts.disable_auto_compactions = true;
  302. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  303. opts.num_levels = 2;
  304. opts.statistics = CreateDBStatistics();
  305. DestroyAndReopen(opts);
  306. for (int i = 0; i < kNumFiles; ++i) {
  307. if (i > 0) {
  308. // range tombstone covers first half of the previous file
  309. ASSERT_OK(db_->DeleteRange(
  310. WriteOptions(), db_->DefaultColumnFamily(),
  311. GetNumericStr((i - 1) * kNumPerFile),
  312. GetNumericStr((i - 1) * kNumPerFile + kNumPerFile / 2)));
  313. }
  314. // Make sure a given key appears in each file so compaction won't be able to
  315. // use trivial move, which would happen if the ranges were non-overlapping.
  316. // Also, we need an extra element since flush is only triggered when the
  317. // number of keys is one greater than SpecialSkipListFactory's limit.
  318. // We choose a key outside the key-range used by the test to avoid conflict.
  319. ASSERT_OK(db_->Put(WriteOptions(), GetNumericStr(kNumPerFile * kNumFiles),
  320. "val"));
  321. for (int j = 0; j < kNumPerFile; ++j) {
  322. ASSERT_OK(
  323. db_->Put(WriteOptions(), GetNumericStr(i * kNumPerFile + j), "val"));
  324. }
  325. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  326. ASSERT_EQ(i + 1, NumTableFilesAtLevel(0));
  327. }
  328. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  329. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  330. ASSERT_GT(NumTableFilesAtLevel(1), 0);
  331. ASSERT_EQ((kNumFiles - 1) * kNumPerFile / 2,
  332. TestGetTickerCount(opts, COMPACTION_KEY_DROP_RANGE_DEL));
  333. for (int i = 0; i < kNumFiles; ++i) {
  334. for (int j = 0; j < kNumPerFile; ++j) {
  335. ReadOptions read_opts;
  336. read_opts.ignore_range_deletions = true;
  337. std::string value;
  338. if (i == kNumFiles - 1 || j >= kNumPerFile / 2) {
  339. ASSERT_OK(
  340. db_->Get(read_opts, GetNumericStr(i * kNumPerFile + j), &value));
  341. } else {
  342. ASSERT_TRUE(
  343. db_->Get(read_opts, GetNumericStr(i * kNumPerFile + j), &value)
  344. .IsNotFound());
  345. }
  346. }
  347. }
  348. }
  349. TEST_F(DBRangeDelTest, ValidLevelSubcompactionBoundaries) {
  350. const int kNumPerFile = 100, kNumFiles = 4, kFileBytes = 100 << 10;
  351. Options options = CurrentOptions();
  352. options.disable_auto_compactions = true;
  353. options.level0_file_num_compaction_trigger = kNumFiles;
  354. options.max_bytes_for_level_base = 2 * kFileBytes;
  355. options.max_subcompactions = 4;
  356. options.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  357. options.num_levels = 3;
  358. options.target_file_size_base = kFileBytes;
  359. options.target_file_size_multiplier = 1;
  360. options.max_compaction_bytes = 1500;
  361. Reopen(options);
  362. Random rnd(301);
  363. for (int i = 0; i < 2; ++i) {
  364. for (int j = 0; j < kNumFiles; ++j) {
  365. if (i > 0) {
  366. // delete [95,105) in two files, [295,305) in next two
  367. int mid = (j + (1 - j % 2)) * kNumPerFile;
  368. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  369. Key(mid - 5), Key(mid + 5)));
  370. }
  371. std::vector<std::string> values;
  372. // Write 100KB (100 values, each 1K)
  373. for (int k = 0; k < kNumPerFile; k++) {
  374. values.push_back(rnd.RandomString(990));
  375. ASSERT_OK(Put(Key(j * kNumPerFile + k), values[k]));
  376. }
  377. // put extra key to trigger flush
  378. ASSERT_OK(Put("", ""));
  379. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  380. if (j < kNumFiles - 1) {
  381. // background compaction may happen early for kNumFiles'th file
  382. ASSERT_EQ(NumTableFilesAtLevel(0), j + 1);
  383. }
  384. if (j == options.level0_file_num_compaction_trigger - 1) {
  385. // When i == 1, compaction will output some files to L1, at which point
  386. // L1 is not bottommost so range deletions cannot be compacted away. The
  387. // new L1 files must be generated with non-overlapping key ranges even
  388. // though multiple subcompactions see the same ranges deleted, else an
  389. // assertion will fail.
  390. //
  391. // Only enable auto-compactions when we're ready; otherwise, the
  392. // oversized L0 (relative to base_level) causes the compaction to run
  393. // earlier.
  394. ASSERT_OK(db_->EnableAutoCompaction({db_->DefaultColumnFamily()}));
  395. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  396. ASSERT_OK(db_->SetOptions(db_->DefaultColumnFamily(),
  397. {{"disable_auto_compactions", "true"}}));
  398. ASSERT_EQ(NumTableFilesAtLevel(0), 0);
  399. ASSERT_GT(NumTableFilesAtLevel(1), 0);
  400. ASSERT_GT(NumTableFilesAtLevel(2), 0);
  401. }
  402. }
  403. }
  404. }
  405. TEST_F(DBRangeDelTest, ValidUniversalSubcompactionBoundaries) {
  406. const int kNumPerFile = 100, kFilesPerLevel = 4, kNumLevels = 4;
  407. Options options = CurrentOptions();
  408. options.compaction_options_universal.min_merge_width = kFilesPerLevel;
  409. options.compaction_options_universal.max_merge_width = kFilesPerLevel;
  410. options.compaction_options_universal.size_ratio = 10;
  411. options.compaction_style = kCompactionStyleUniversal;
  412. options.level0_file_num_compaction_trigger = kFilesPerLevel;
  413. options.max_subcompactions = 4;
  414. options.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  415. options.num_levels = kNumLevels;
  416. options.target_file_size_base = kNumPerFile << 10;
  417. options.target_file_size_multiplier = 1;
  418. Reopen(options);
  419. Random rnd(301);
  420. for (int i = 0; i < kNumLevels - 1; ++i) {
  421. for (int j = 0; j < kFilesPerLevel; ++j) {
  422. if (i == kNumLevels - 2) {
  423. // insert range deletions [95,105) in two files, [295,305) in next two
  424. // to prepare L1 for later manual compaction.
  425. int mid = (j + (1 - j % 2)) * kNumPerFile;
  426. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  427. Key(mid - 5), Key(mid + 5)));
  428. }
  429. std::vector<std::string> values;
  430. // Write 100KB (100 values, each 1K)
  431. for (int k = 0; k < kNumPerFile; k++) {
  432. // For the highest level, use smaller value size such that it does not
  433. // prematurely cause auto compaction due to range tombstone adding
  434. // additional compensated file size
  435. values.push_back(rnd.RandomString((i == kNumLevels - 2) ? 600 : 990));
  436. ASSERT_OK(Put(Key(j * kNumPerFile + k), values[k]));
  437. }
  438. // put extra key to trigger flush
  439. ASSERT_OK(Put("", ""));
  440. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  441. if (j < kFilesPerLevel - 1) {
  442. // background compaction may happen early for kFilesPerLevel'th file
  443. ASSERT_EQ(NumTableFilesAtLevel(0), j + 1);
  444. }
  445. }
  446. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  447. ASSERT_EQ(NumTableFilesAtLevel(0), 0);
  448. if (i == kNumLevels - 2) {
  449. // For the highest level, value size is smaller (see Put() above),
  450. // so output file number is smaller.
  451. ASSERT_GT(NumTableFilesAtLevel(kNumLevels - 1 - i), kFilesPerLevel - 2);
  452. } else {
  453. ASSERT_GT(NumTableFilesAtLevel(kNumLevels - 1 - i), kFilesPerLevel - 1);
  454. }
  455. }
  456. // Now L1-L3 are full, when we compact L1->L2 we should see (1) subcompactions
  457. // happen since input level > 0; (2) range deletions are not dropped since
  458. // output level is not bottommost. If no file boundary assertion fails, that
  459. // probably means universal compaction + subcompaction + range deletion are
  460. // compatible.
  461. ASSERT_OK(dbfull()->RunManualCompaction(
  462. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  463. ->cfd(),
  464. 1 /* input_level */, 2 /* output_level */, CompactRangeOptions(),
  465. nullptr /* begin */, nullptr /* end */, true /* exclusive */,
  466. true /* disallow_trivial_move */,
  467. std::numeric_limits<uint64_t>::max() /* max_file_num_to_ignore */,
  468. "" /*trim_ts*/));
  469. }
  470. TEST_F(DBRangeDelTest, CompactionRemovesCoveredMergeOperands) {
  471. const int kNumPerFile = 3, kNumFiles = 3;
  472. Options opts = CurrentOptions();
  473. opts.disable_auto_compactions = true;
  474. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(2 * kNumPerFile));
  475. opts.merge_operator = MergeOperators::CreateUInt64AddOperator();
  476. opts.num_levels = 2;
  477. Reopen(opts);
  478. // Iterates kNumFiles * kNumPerFile + 1 times since flushing the last file
  479. // requires an extra entry.
  480. for (int i = 0; i <= kNumFiles * kNumPerFile; ++i) {
  481. if (i % kNumPerFile == 0 && i / kNumPerFile == kNumFiles - 1) {
  482. // Delete merge operands from all but the last file
  483. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  484. "key", "key_"));
  485. }
  486. std::string val;
  487. PutFixed64(&val, i);
  488. ASSERT_OK(db_->Merge(WriteOptions(), "key", val));
  489. // we need to prevent trivial move using Puts so compaction will actually
  490. // process the merge operands.
  491. ASSERT_OK(db_->Put(WriteOptions(), "prevent_trivial_move", ""));
  492. if (i > 0 && i % kNumPerFile == 0) {
  493. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  494. }
  495. }
  496. ReadOptions read_opts;
  497. read_opts.ignore_range_deletions = true;
  498. std::string expected, actual;
  499. ASSERT_OK(db_->Get(read_opts, "key", &actual));
  500. PutFixed64(&expected, 45); // 1+2+...+9
  501. ASSERT_EQ(expected, actual);
  502. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  503. expected.clear();
  504. ASSERT_OK(db_->Get(read_opts, "key", &actual));
  505. uint64_t tmp;
  506. Slice tmp2(actual);
  507. GetFixed64(&tmp2, &tmp);
  508. PutFixed64(&expected, 30); // 6+7+8+9 (earlier operands covered by tombstone)
  509. ASSERT_EQ(expected, actual);
  510. }
  511. TEST_F(DBRangeDelTest, PutDeleteRangeMergeFlush) {
  512. // Test the sequence of operations: (1) Put, (2) DeleteRange, (3) Merge, (4)
  513. // Flush. The `CompactionIterator` previously had a bug where we forgot to
  514. // check for covering range tombstones when processing the (1) Put, causing
  515. // it to reappear after the flush.
  516. Options opts = CurrentOptions();
  517. opts.merge_operator = MergeOperators::CreateUInt64AddOperator();
  518. Reopen(opts);
  519. std::string val;
  520. PutFixed64(&val, 1);
  521. ASSERT_OK(db_->Put(WriteOptions(), "key", val));
  522. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "key",
  523. "key_"));
  524. ASSERT_OK(db_->Merge(WriteOptions(), "key", val));
  525. ASSERT_OK(db_->Flush(FlushOptions()));
  526. ReadOptions read_opts;
  527. std::string expected, actual;
  528. ASSERT_OK(db_->Get(read_opts, "key", &actual));
  529. PutFixed64(&expected, 1);
  530. ASSERT_EQ(expected, actual);
  531. }
  532. TEST_F(DBRangeDelTest, ObsoleteTombstoneCleanup) {
  533. // During compaction to bottommost level, verify range tombstones older than
  534. // the oldest snapshot are removed, while others are preserved.
  535. Options opts = CurrentOptions();
  536. opts.disable_auto_compactions = true;
  537. opts.num_levels = 2;
  538. opts.statistics = CreateDBStatistics();
  539. Reopen(opts);
  540. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "dr1",
  541. "dr10")); // obsolete after compaction
  542. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  543. ASSERT_OK(db_->Flush(FlushOptions()));
  544. const Snapshot* snapshot = db_->GetSnapshot();
  545. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "dr2",
  546. "dr20")); // protected by snapshot
  547. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  548. ASSERT_OK(db_->Flush(FlushOptions()));
  549. ASSERT_EQ(2, NumTableFilesAtLevel(0));
  550. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  551. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  552. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  553. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  554. ASSERT_EQ(1, TestGetTickerCount(opts, COMPACTION_RANGE_DEL_DROP_OBSOLETE));
  555. db_->ReleaseSnapshot(snapshot);
  556. }
  557. TEST_F(DBRangeDelTest, TableEvictedDuringScan) {
  558. // The RangeDelAggregator holds pointers into range deletion blocks created by
  559. // table readers. This test ensures the aggregator can still access those
  560. // blocks even if it outlives the table readers that created them.
  561. //
  562. // DBIter always keeps readers open for L0 files. So, in order to test
  563. // aggregator outliving reader, we need to have deletions in L1 files, which
  564. // are opened/closed on-demand during the scan. This is accomplished by
  565. // setting kNumRanges > level0_stop_writes_trigger, which prevents deletions
  566. // from all lingering in L0 (there is at most one range deletion per L0 file).
  567. //
  568. // The first L1 file will contain a range deletion since its begin key is 0.
  569. // SeekToFirst() references that table's reader and adds its range tombstone
  570. // to the aggregator. Upon advancing beyond that table's key-range via Next(),
  571. // the table reader will be unreferenced by the iterator. Since we manually
  572. // call Evict() on all readers before the full scan, this unreference causes
  573. // the reader's refcount to drop to zero and thus be destroyed.
  574. //
  575. // When it is destroyed, we do not remove its range deletions from the
  576. // aggregator. So, subsequent calls to Next() must be able to use these
  577. // deletions to decide whether a key is covered. This will work as long as
  578. // the aggregator properly references the range deletion block.
  579. const int kNum = 25, kRangeBegin = 0, kRangeEnd = 7, kNumRanges = 5;
  580. Options opts = CurrentOptions();
  581. opts.comparator = test::Uint64Comparator();
  582. opts.level0_file_num_compaction_trigger = 4;
  583. opts.level0_stop_writes_trigger = 4;
  584. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
  585. opts.num_levels = 2;
  586. BlockBasedTableOptions bbto;
  587. bbto.cache_index_and_filter_blocks = true;
  588. bbto.block_cache = NewLRUCache(8 << 20);
  589. opts.table_factory.reset(NewBlockBasedTableFactory(bbto));
  590. DestroyAndReopen(opts);
  591. // Hold a snapshot so range deletions can't become obsolete during compaction
  592. // to bottommost level (i.e., L1).
  593. const Snapshot* snapshot = db_->GetSnapshot();
  594. for (int i = 0; i < kNum; ++i) {
  595. ASSERT_OK(db_->Put(WriteOptions(), GetNumericStr(i), "val"));
  596. if (i > 0) {
  597. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  598. }
  599. if (i >= kNum / 2 && i < kNum / 2 + kNumRanges) {
  600. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  601. GetNumericStr(kRangeBegin),
  602. GetNumericStr(kRangeEnd)));
  603. }
  604. }
  605. // Must be > 1 so the first L1 file can be closed before scan finishes
  606. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  607. ASSERT_GT(NumTableFilesAtLevel(1), 1);
  608. std::vector<uint64_t> file_numbers = ListTableFiles(env_, dbname_);
  609. ReadOptions read_opts;
  610. auto* iter = db_->NewIterator(read_opts);
  611. ASSERT_OK(iter->status());
  612. int expected = kRangeEnd;
  613. iter->SeekToFirst();
  614. for (auto file_number : file_numbers) {
  615. // This puts table caches in the state of being externally referenced only
  616. // so they are destroyed immediately upon iterator unreferencing.
  617. TableCache::Evict(dbfull()->TEST_table_cache(), file_number);
  618. }
  619. for (; iter->Valid(); iter->Next()) {
  620. ASSERT_EQ(GetNumericStr(expected), iter->key());
  621. ++expected;
  622. // Keep clearing block cache's LRU so range deletion block can be freed as
  623. // soon as its refcount drops to zero.
  624. bbto.block_cache->EraseUnRefEntries();
  625. }
  626. ASSERT_OK(iter->status());
  627. ASSERT_EQ(kNum, expected);
  628. delete iter;
  629. db_->ReleaseSnapshot(snapshot);
  630. // Also test proper cache handling in GetRangeTombstoneIterator,
  631. // via TablesRangeTombstoneSummary. (This once triggered memory leak
  632. // report with ASAN.)
  633. opts.max_open_files = 1;
  634. Reopen(opts);
  635. std::string str;
  636. ASSERT_OK(dbfull()->TablesRangeTombstoneSummary(db_->DefaultColumnFamily(),
  637. 100, &str));
  638. }
  639. TEST_F(DBRangeDelTest, GetCoveredKeyFromMutableMemtable) {
  640. do {
  641. DestroyAndReopen(CurrentOptions());
  642. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  643. ASSERT_OK(
  644. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  645. ReadOptions read_opts;
  646. std::string value;
  647. ASSERT_TRUE(db_->Get(read_opts, "key", &value).IsNotFound());
  648. } while (ChangeOptions(kRangeDelSkipConfigs));
  649. }
  650. TEST_F(DBRangeDelTest, GetCoveredKeyFromImmutableMemtable) {
  651. do {
  652. Options opts = CurrentOptions();
  653. opts.max_write_buffer_number = 3;
  654. opts.min_write_buffer_number_to_merge = 2;
  655. // SpecialSkipListFactory lets us specify maximum number of elements the
  656. // memtable can hold. It switches the active memtable to immutable (flush is
  657. // prevented by the above options) upon inserting an element that would
  658. // overflow the memtable.
  659. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
  660. DestroyAndReopen(opts);
  661. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  662. ASSERT_OK(
  663. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  664. ASSERT_OK(db_->Put(WriteOptions(), "blah", "val"));
  665. ReadOptions read_opts;
  666. std::string value;
  667. ASSERT_TRUE(db_->Get(read_opts, "key", &value).IsNotFound());
  668. } while (ChangeOptions(kRangeDelSkipConfigs));
  669. }
  670. TEST_F(DBRangeDelTest, GetCoveredKeyFromSst) {
  671. do {
  672. DestroyAndReopen(CurrentOptions());
  673. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  674. // snapshot prevents key from being deleted during flush
  675. const Snapshot* snapshot = db_->GetSnapshot();
  676. ASSERT_OK(
  677. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  678. ASSERT_OK(db_->Flush(FlushOptions()));
  679. ReadOptions read_opts;
  680. std::string value;
  681. ASSERT_TRUE(db_->Get(read_opts, "key", &value).IsNotFound());
  682. db_->ReleaseSnapshot(snapshot);
  683. } while (ChangeOptions(kRangeDelSkipConfigs));
  684. }
  685. TEST_F(DBRangeDelTest, GetCoveredMergeOperandFromMemtable) {
  686. const int kNumMergeOps = 10;
  687. Options opts = CurrentOptions();
  688. opts.merge_operator = MergeOperators::CreateUInt64AddOperator();
  689. Reopen(opts);
  690. for (int i = 0; i < kNumMergeOps; ++i) {
  691. std::string val;
  692. PutFixed64(&val, i);
  693. ASSERT_OK(db_->Merge(WriteOptions(), "key", val));
  694. if (i == kNumMergeOps / 2) {
  695. // deletes [0, 5]
  696. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  697. "key", "key_"));
  698. }
  699. }
  700. ReadOptions read_opts;
  701. std::string expected, actual;
  702. ASSERT_OK(db_->Get(read_opts, "key", &actual));
  703. PutFixed64(&expected, 30); // 6+7+8+9
  704. ASSERT_EQ(expected, actual);
  705. expected.clear();
  706. read_opts.ignore_range_deletions = true;
  707. ASSERT_OK(db_->Get(read_opts, "key", &actual));
  708. PutFixed64(&expected, 45); // 0+1+2+...+9
  709. ASSERT_EQ(expected, actual);
  710. }
  711. TEST_F(DBRangeDelTest, GetIgnoresRangeDeletions) {
  712. Options opts = CurrentOptions();
  713. opts.max_write_buffer_number = 4;
  714. opts.min_write_buffer_number_to_merge = 3;
  715. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
  716. Reopen(opts);
  717. ASSERT_OK(db_->Put(WriteOptions(), "sst_key", "val"));
  718. // snapshot prevents key from being deleted during flush
  719. const Snapshot* snapshot = db_->GetSnapshot();
  720. ASSERT_OK(
  721. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  722. ASSERT_OK(db_->Flush(FlushOptions()));
  723. ASSERT_OK(db_->Put(WriteOptions(), "imm_key", "val"));
  724. ASSERT_OK(
  725. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  726. ASSERT_OK(db_->Put(WriteOptions(), "mem_key", "val"));
  727. ASSERT_OK(
  728. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  729. ReadOptions read_opts;
  730. read_opts.ignore_range_deletions = true;
  731. for (std::string key : {"sst_key", "imm_key", "mem_key"}) {
  732. std::string value;
  733. ASSERT_OK(db_->Get(read_opts, key, &value));
  734. }
  735. db_->ReleaseSnapshot(snapshot);
  736. }
  737. TEST_F(DBRangeDelTest, IteratorRemovesCoveredKeys) {
  738. const int kNum = 200, kRangeBegin = 50, kRangeEnd = 150, kNumPerFile = 25;
  739. Options opts = CurrentOptions();
  740. opts.comparator = test::Uint64Comparator();
  741. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  742. DestroyAndReopen(opts);
  743. // Write half of the keys before the tombstone and half after the tombstone.
  744. // Only covered keys (i.e., within the range and older than the tombstone)
  745. // should be deleted.
  746. for (int i = 0; i < kNum; ++i) {
  747. if (i == kNum / 2) {
  748. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  749. GetNumericStr(kRangeBegin),
  750. GetNumericStr(kRangeEnd)));
  751. }
  752. ASSERT_OK(db_->Put(WriteOptions(), GetNumericStr(i), "val"));
  753. }
  754. ReadOptions read_opts;
  755. auto* iter = db_->NewIterator(read_opts);
  756. ASSERT_OK(iter->status());
  757. int expected = 0;
  758. for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
  759. ASSERT_EQ(GetNumericStr(expected), iter->key());
  760. if (expected == kRangeBegin - 1) {
  761. expected = kNum / 2;
  762. } else {
  763. ++expected;
  764. }
  765. }
  766. ASSERT_OK(iter->status());
  767. ASSERT_EQ(kNum, expected);
  768. delete iter;
  769. }
  770. TEST_F(DBRangeDelTest, IteratorOverUserSnapshot) {
  771. const int kNum = 200, kRangeBegin = 50, kRangeEnd = 150, kNumPerFile = 25;
  772. Options opts = CurrentOptions();
  773. opts.comparator = test::Uint64Comparator();
  774. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(kNumPerFile));
  775. DestroyAndReopen(opts);
  776. const Snapshot* snapshot = nullptr;
  777. // Put a snapshot before the range tombstone, verify an iterator using that
  778. // snapshot sees all inserted keys.
  779. for (int i = 0; i < kNum; ++i) {
  780. if (i == kNum / 2) {
  781. snapshot = db_->GetSnapshot();
  782. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  783. GetNumericStr(kRangeBegin),
  784. GetNumericStr(kRangeEnd)));
  785. }
  786. ASSERT_OK(db_->Put(WriteOptions(), GetNumericStr(i), "val"));
  787. }
  788. ReadOptions read_opts;
  789. read_opts.snapshot = snapshot;
  790. auto* iter = db_->NewIterator(read_opts);
  791. ASSERT_OK(iter->status());
  792. int expected = 0;
  793. for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
  794. ASSERT_EQ(GetNumericStr(expected), iter->key());
  795. ++expected;
  796. }
  797. ASSERT_EQ(kNum / 2, expected);
  798. delete iter;
  799. db_->ReleaseSnapshot(snapshot);
  800. }
  801. TEST_F(DBRangeDelTest, IteratorIgnoresRangeDeletions) {
  802. Options opts = CurrentOptions();
  803. opts.max_write_buffer_number = 4;
  804. opts.min_write_buffer_number_to_merge = 3;
  805. opts.memtable_factory.reset(test::NewSpecialSkipListFactory(1));
  806. Reopen(opts);
  807. ASSERT_OK(db_->Put(WriteOptions(), "sst_key", "val"));
  808. // snapshot prevents key from being deleted during flush
  809. const Snapshot* snapshot = db_->GetSnapshot();
  810. ASSERT_OK(
  811. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  812. ASSERT_OK(db_->Flush(FlushOptions()));
  813. ASSERT_OK(db_->Put(WriteOptions(), "imm_key", "val"));
  814. ASSERT_OK(
  815. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  816. ASSERT_OK(db_->Put(WriteOptions(), "mem_key", "val"));
  817. ASSERT_OK(
  818. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  819. ReadOptions read_opts;
  820. read_opts.ignore_range_deletions = true;
  821. auto* iter = db_->NewIterator(read_opts);
  822. ASSERT_OK(iter->status());
  823. int i = 0;
  824. std::string expected[] = {"imm_key", "mem_key", "sst_key"};
  825. for (iter->SeekToFirst(); iter->Valid(); iter->Next(), ++i) {
  826. std::string key;
  827. ASSERT_EQ(expected[i], iter->key());
  828. }
  829. ASSERT_OK(iter->status());
  830. ASSERT_EQ(3, i);
  831. delete iter;
  832. db_->ReleaseSnapshot(snapshot);
  833. }
  834. #ifndef ROCKSDB_UBSAN_RUN
  835. TEST_F(DBRangeDelTest, TailingIteratorRangeTombstoneUnsupported) {
  836. ASSERT_OK(db_->Put(WriteOptions(), "key", "val"));
  837. // snapshot prevents key from being deleted during flush
  838. const Snapshot* snapshot = db_->GetSnapshot();
  839. ASSERT_OK(
  840. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  841. // iterations check unsupported in memtable, l0, and then l1
  842. for (int i = 0; i < 3; ++i) {
  843. ReadOptions read_opts;
  844. read_opts.tailing = true;
  845. auto* iter = db_->NewIterator(read_opts);
  846. if (i == 2) {
  847. // For L1+, iterators over files are created on-demand, so need seek
  848. iter->SeekToFirst();
  849. }
  850. ASSERT_TRUE(iter->status().IsNotSupported());
  851. delete iter;
  852. if (i == 0) {
  853. ASSERT_OK(db_->Flush(FlushOptions()));
  854. } else if (i == 1) {
  855. MoveFilesToLevel(1);
  856. }
  857. }
  858. db_->ReleaseSnapshot(snapshot);
  859. }
  860. #endif // !ROCKSDB_UBSAN_RUN
  861. TEST_F(DBRangeDelTest, SubcompactionHasEmptyDedicatedRangeDelFile) {
  862. const int kNumFiles = 2, kNumKeysPerFile = 4;
  863. Options options = CurrentOptions();
  864. options.compression = kNoCompression;
  865. options.disable_auto_compactions = true;
  866. options.level0_file_num_compaction_trigger = kNumFiles;
  867. options.max_subcompactions = 2;
  868. options.num_levels = 2;
  869. options.target_file_size_base = 4096;
  870. Reopen(options);
  871. // need a L1 file for subcompaction to be triggered
  872. ASSERT_OK(
  873. db_->Put(WriteOptions(), db_->DefaultColumnFamily(), Key(0), "val"));
  874. ASSERT_OK(db_->Flush(FlushOptions()));
  875. MoveFilesToLevel(1);
  876. // put enough keys to fill up the first subcompaction, and later range-delete
  877. // them so that the first subcompaction outputs no key-values. In that case
  878. // it'll consider making an SST file dedicated to range deletions.
  879. for (int i = 0; i < kNumKeysPerFile; ++i) {
  880. ASSERT_OK(db_->Put(WriteOptions(), db_->DefaultColumnFamily(), Key(i),
  881. std::string(1024, 'a')));
  882. }
  883. ASSERT_OK(db_->Flush(FlushOptions()));
  884. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  885. Key(kNumKeysPerFile)));
  886. // the above range tombstone can be dropped, so that one alone won't cause a
  887. // dedicated file to be opened. We can make one protected by snapshot that
  888. // must be considered. Make its range outside the first subcompaction's range
  889. // to exercise the tricky part of the code.
  890. const Snapshot* snapshot = db_->GetSnapshot();
  891. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  892. Key(kNumKeysPerFile + 1),
  893. Key(kNumKeysPerFile + 2)));
  894. ASSERT_OK(db_->Flush(FlushOptions()));
  895. ASSERT_EQ(kNumFiles, NumTableFilesAtLevel(0));
  896. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  897. ASSERT_OK(db_->EnableAutoCompaction({db_->DefaultColumnFamily()}));
  898. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  899. db_->ReleaseSnapshot(snapshot);
  900. }
  901. TEST_F(DBRangeDelTest, MemtableBloomFilter) {
  902. // regression test for #2743. the range delete tombstones in memtable should
  903. // be added even when Get() skips searching due to its prefix bloom filter
  904. const int kMemtableSize = 1 << 20; // 1MB
  905. const int kMemtablePrefixFilterSize = 1 << 13; // 8KB
  906. const int kNumKeys = 1000;
  907. const int kPrefixLen = 8;
  908. Options options = CurrentOptions();
  909. options.memtable_prefix_bloom_size_ratio =
  910. static_cast<double>(kMemtablePrefixFilterSize) / kMemtableSize;
  911. options.prefix_extractor.reset(
  912. ROCKSDB_NAMESPACE::NewFixedPrefixTransform(kPrefixLen));
  913. options.write_buffer_size = kMemtableSize;
  914. Reopen(options);
  915. for (int i = 0; i < kNumKeys; ++i) {
  916. ASSERT_OK(Put(Key(i), "val"));
  917. }
  918. ASSERT_OK(Flush());
  919. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  920. Key(kNumKeys)));
  921. for (int i = 0; i < kNumKeys; ++i) {
  922. std::string value;
  923. ASSERT_TRUE(db_->Get(ReadOptions(), Key(i), &value).IsNotFound());
  924. }
  925. }
  926. TEST_F(DBRangeDelTest, CompactionTreatsSplitInputLevelDeletionAtomically) {
  927. // This test originally verified that compaction treated files containing a
  928. // split range deletion in the input level as an atomic unit. I.e.,
  929. // compacting any input-level file(s) containing a portion of the range
  930. // deletion causes all other input-level files containing portions of that
  931. // same range deletion to be included in the compaction. Range deletion
  932. // tombstones are now truncated to sstable boundaries which removed the need
  933. // for that behavior (which could lead to excessively large
  934. // compactions).
  935. const int kNumFilesPerLevel = 4, kValueBytes = 4 << 10;
  936. Options options = CurrentOptions();
  937. options.compression = kNoCompression;
  938. options.level0_file_num_compaction_trigger = kNumFilesPerLevel;
  939. options.memtable_factory.reset(
  940. test::NewSpecialSkipListFactory(2 /* num_entries_flush */));
  941. // max file size could be 2x of target file size, so set it to half of that
  942. options.target_file_size_base = kValueBytes / 2;
  943. options.max_compaction_bytes = 1500;
  944. // i == 0: CompactFiles
  945. // i == 1: CompactRange
  946. // i == 2: automatic compaction
  947. for (int i = 0; i < 3; ++i) {
  948. DestroyAndReopen(options);
  949. ASSERT_OK(Put(Key(0), ""));
  950. ASSERT_OK(db_->Flush(FlushOptions()));
  951. MoveFilesToLevel(2);
  952. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  953. // snapshot protects range tombstone from dropping due to becoming obsolete.
  954. const Snapshot* snapshot = db_->GetSnapshot();
  955. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  956. Key(0), Key(2 * kNumFilesPerLevel)));
  957. Random rnd(301);
  958. std::string value = rnd.RandomString(kValueBytes);
  959. for (int j = 0; j < kNumFilesPerLevel; ++j) {
  960. // give files overlapping key-ranges to prevent trivial move
  961. ASSERT_OK(Put(Key(j), value));
  962. ASSERT_OK(Put(Key(2 * kNumFilesPerLevel - 1 - j), value));
  963. if (j > 0) {
  964. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  965. ASSERT_EQ(j, NumTableFilesAtLevel(0));
  966. }
  967. }
  968. // put extra key to trigger final flush
  969. ASSERT_OK(Put("", ""));
  970. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  971. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  972. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  973. ASSERT_EQ(kNumFilesPerLevel, NumTableFilesAtLevel(1));
  974. ColumnFamilyMetaData meta;
  975. db_->GetColumnFamilyMetaData(&meta);
  976. if (i == 0) {
  977. ASSERT_OK(db_->CompactFiles(
  978. CompactionOptions(), {meta.levels[1].files[0].name}, 2 /* level */));
  979. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  980. } else if (i == 1) {
  981. auto begin_str = Key(0), end_str = Key(1);
  982. Slice begin = begin_str, end = end_str;
  983. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &begin, &end));
  984. ASSERT_EQ(3, NumTableFilesAtLevel(1));
  985. } else if (i == 2) {
  986. ASSERT_OK(db_->SetOptions(db_->DefaultColumnFamily(),
  987. {{"max_bytes_for_level_base", "10000"}}));
  988. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  989. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  990. }
  991. ASSERT_GT(NumTableFilesAtLevel(2), 0);
  992. db_->ReleaseSnapshot(snapshot);
  993. }
  994. }
  995. TEST_F(DBRangeDelTest, RangeTombstoneEndKeyAsSstableUpperBound) {
  996. // Test the handling of the range-tombstone end-key as the
  997. // upper-bound for an sstable.
  998. const int kNumFilesPerLevel = 2, kValueBytes = 4 << 10;
  999. Options options = CurrentOptions();
  1000. options.compression = kNoCompression;
  1001. options.level0_file_num_compaction_trigger = kNumFilesPerLevel;
  1002. options.memtable_factory.reset(
  1003. test::NewSpecialSkipListFactory(2 /* num_entries_flush */));
  1004. // Compaction can generate files of size at most 2 * target_file_size_base.
  1005. options.target_file_size_base = kValueBytes / 2;
  1006. options.disable_auto_compactions = true;
  1007. DestroyAndReopen(options);
  1008. // Create an initial sstable at L2:
  1009. // [key000000#1,1, key000000#1,1]
  1010. ASSERT_OK(Put(Key(0), ""));
  1011. ASSERT_OK(db_->Flush(FlushOptions()));
  1012. MoveFilesToLevel(2);
  1013. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  1014. // A snapshot protects the range tombstone from dropping due to
  1015. // becoming obsolete.
  1016. const Snapshot* snapshot = db_->GetSnapshot();
  1017. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1018. Key(2 * kNumFilesPerLevel)));
  1019. // Create 2 additional sstables in L0. Note that the first sstable
  1020. // contains the range tombstone.
  1021. // [key000000#3,1, key000004#72057594037927935,15]
  1022. // [key000001#5,1, key000002#6,1]
  1023. Random rnd(301);
  1024. std::string value = rnd.RandomString(kValueBytes);
  1025. for (int j = 0; j < kNumFilesPerLevel; ++j) {
  1026. // Give files overlapping key-ranges to prevent a trivial move when we
  1027. // compact from L0 to L1.
  1028. ASSERT_OK(Put(Key(j), value));
  1029. ASSERT_OK(Put(Key(2 * kNumFilesPerLevel - 1 - j), value));
  1030. ASSERT_OK(db_->Flush(FlushOptions()));
  1031. ASSERT_EQ(j + 1, NumTableFilesAtLevel(0));
  1032. }
  1033. // Compact the 2 L0 sstables to L1, resulting in the following LSM. There
  1034. // are 2 sstables generated in L1 due to the target_file_size_base setting.
  1035. // L1:
  1036. // [key000000#3,1, key000002#72057594037927935,15]
  1037. // [key000002#6,1, key000004#72057594037927935,15]
  1038. // L2:
  1039. // [key000000#1,1, key000000#1,1]
  1040. MoveFilesToLevel(1);
  1041. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  1042. std::vector<std::vector<FileMetaData>> files;
  1043. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  1044. ASSERT_EQ(
  1045. files[1][0].largest.Encode(),
  1046. InternalKey(Key(2), kMaxSequenceNumber, kTypeRangeDeletion).Encode());
  1047. ASSERT_EQ(files[1][1].smallest.Encode(),
  1048. InternalKey(Key(2), 6, kTypeValue).Encode());
  1049. {
  1050. // Compact the second sstable in L1:
  1051. // L1:
  1052. // [key000000#3,1, key000002#72057594037927935,15]
  1053. // L2:
  1054. // [key000000#1,1, key000000#1,1]
  1055. // [key000002#6,1, key000004#72057594037927935,15]
  1056. //
  1057. // At the same time, verify the compaction does not cause the key at the
  1058. // endpoint (key000002#6,1) to disappear.
  1059. ASSERT_EQ(value, Get(Key(2)));
  1060. auto begin_str = Key(3);
  1061. const ROCKSDB_NAMESPACE::Slice begin = begin_str;
  1062. ASSERT_OK(dbfull()->TEST_CompactRange(1, &begin, nullptr));
  1063. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1064. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1065. ASSERT_EQ(value, Get(Key(2)));
  1066. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  1067. ASSERT_EQ(files[2][1].smallest.Encode(),
  1068. InternalKey(Key(2), 6, kTypeValue).Encode());
  1069. ASSERT_EQ(
  1070. files[2][1].largest.Encode(),
  1071. InternalKey(Key(4), kMaxSequenceNumber, kTypeRangeDeletion).Encode());
  1072. }
  1073. {
  1074. // Compact the first sstable in L1. This should be copacetic, but
  1075. // was previously resulting in overlapping sstables in L2 due to
  1076. // mishandling of the range tombstone end-key when used as the
  1077. // largest key for an sstable. The resulting LSM structure should
  1078. // be:
  1079. //
  1080. // L2:
  1081. // [key000000#1,1, key000001#72057594037927935,15]
  1082. // [key000001#5,1, key000002#72057594037927935,15]
  1083. // [key000002#6,1, key000004#72057594037927935,15]
  1084. auto begin_str = Key(0);
  1085. const ROCKSDB_NAMESPACE::Slice begin = begin_str;
  1086. ASSERT_OK(dbfull()->TEST_CompactRange(1, &begin, &begin));
  1087. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  1088. ASSERT_EQ(3, NumTableFilesAtLevel(2));
  1089. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  1090. ASSERT_EQ(
  1091. files[2][0].largest.Encode(),
  1092. InternalKey(Key(1), kMaxSequenceNumber, kTypeRangeDeletion).Encode());
  1093. ASSERT_EQ(files[2][1].smallest.Encode(),
  1094. InternalKey(Key(1), 5, kTypeValue).Encode());
  1095. ASSERT_EQ(
  1096. files[2][1].largest.Encode(),
  1097. InternalKey(Key(2), kMaxSequenceNumber, kTypeRangeDeletion).Encode());
  1098. ASSERT_EQ(files[2][2].smallest.Encode(),
  1099. InternalKey(Key(2), 6, kTypeValue).Encode());
  1100. ASSERT_EQ(
  1101. files[2][2].largest.Encode(),
  1102. InternalKey(Key(4), kMaxSequenceNumber, kTypeRangeDeletion).Encode());
  1103. }
  1104. db_->ReleaseSnapshot(snapshot);
  1105. }
  1106. TEST_F(DBRangeDelTest, UnorderedTombstones) {
  1107. // Regression test for #2752. Range delete tombstones between
  1108. // different snapshot stripes are not stored in order, so the first
  1109. // tombstone of each snapshot stripe should be checked as a smallest
  1110. // candidate.
  1111. Options options = CurrentOptions();
  1112. DestroyAndReopen(options);
  1113. auto cf = db_->DefaultColumnFamily();
  1114. ASSERT_OK(db_->Put(WriteOptions(), cf, "a", "a"));
  1115. ASSERT_OK(db_->Flush(FlushOptions(), cf));
  1116. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1117. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr));
  1118. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1119. ASSERT_OK(db_->DeleteRange(WriteOptions(), cf, "b", "c"));
  1120. // Hold a snapshot to separate these two delete ranges.
  1121. auto snapshot = db_->GetSnapshot();
  1122. ASSERT_OK(db_->DeleteRange(WriteOptions(), cf, "a", "b"));
  1123. ASSERT_OK(db_->Flush(FlushOptions(), cf));
  1124. db_->ReleaseSnapshot(snapshot);
  1125. std::vector<std::vector<FileMetaData>> files;
  1126. dbfull()->TEST_GetFilesMetaData(cf, &files);
  1127. ASSERT_EQ(1, files[0].size());
  1128. ASSERT_EQ("a", files[0][0].smallest.user_key());
  1129. ASSERT_EQ("c", files[0][0].largest.user_key());
  1130. std::string v;
  1131. auto s = db_->Get(ReadOptions(), "a", &v);
  1132. ASSERT_TRUE(s.IsNotFound());
  1133. }
  1134. class MockMergeOperator : public MergeOperator {
  1135. // Mock non-associative operator. Non-associativity is expressed by lack of
  1136. // implementation for any `PartialMerge*` functions.
  1137. public:
  1138. bool FullMergeV2(const MergeOperationInput& merge_in,
  1139. MergeOperationOutput* merge_out) const override {
  1140. assert(merge_out != nullptr);
  1141. merge_out->new_value = merge_in.operand_list.back().ToString();
  1142. return true;
  1143. }
  1144. const char* Name() const override { return "MockMergeOperator"; }
  1145. };
  1146. TEST_F(DBRangeDelTest, KeyAtOverlappingEndpointReappears) {
  1147. // This test uses a non-associative merge operator since that is a convenient
  1148. // way to get compaction to write out files with overlapping user-keys at the
  1149. // endpoints. Note, however, overlapping endpoints can also occur with other
  1150. // value types (Put, etc.), assuming the right snapshots are present.
  1151. const int kFileBytes = 1 << 20;
  1152. const int kValueBytes = 1 << 10;
  1153. const int kNumFiles = 4;
  1154. Options options = CurrentOptions();
  1155. options.compression = kNoCompression;
  1156. options.disable_auto_compactions = true;
  1157. options.merge_operator.reset(new MockMergeOperator());
  1158. options.target_file_size_base = kFileBytes;
  1159. Reopen(options);
  1160. // Push dummy data to L3 so that our actual test files on L0-L2
  1161. // will not be considered "bottommost" level, otherwise compaction
  1162. // may prevent us from creating overlapping user keys
  1163. // as on the bottommost layer MergeHelper
  1164. ASSERT_OK(db_->Merge(WriteOptions(), "key", "dummy"));
  1165. ASSERT_OK(db_->Flush(FlushOptions()));
  1166. MoveFilesToLevel(3);
  1167. Random rnd(301);
  1168. const Snapshot* snapshot = nullptr;
  1169. for (int i = 0; i < kNumFiles; ++i) {
  1170. for (int j = 0; j < kFileBytes / kValueBytes; ++j) {
  1171. auto value = rnd.RandomString(kValueBytes);
  1172. ASSERT_OK(db_->Merge(WriteOptions(), "key", value));
  1173. }
  1174. if (i == kNumFiles - 1) {
  1175. // Take snapshot to prevent covered merge operands from being dropped by
  1176. // compaction.
  1177. snapshot = db_->GetSnapshot();
  1178. // The DeleteRange is the last write so all merge operands are covered.
  1179. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1180. "key", "key_"));
  1181. }
  1182. ASSERT_OK(db_->Flush(FlushOptions()));
  1183. }
  1184. ASSERT_EQ(kNumFiles, NumTableFilesAtLevel(0));
  1185. std::string value;
  1186. ASSERT_TRUE(db_->Get(ReadOptions(), "key", &value).IsNotFound());
  1187. ASSERT_OK(dbfull()->TEST_CompactRange(
  1188. 0 /* level */, nullptr /* begin */, nullptr /* end */,
  1189. nullptr /* column_family */, true /* disallow_trivial_move */));
  1190. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  1191. // Now we have multiple files at L1 all containing a single user key, thus
  1192. // guaranteeing overlap in the file endpoints.
  1193. ASSERT_GT(NumTableFilesAtLevel(1), 1);
  1194. // Verify no merge operands reappeared after the compaction.
  1195. ASSERT_TRUE(db_->Get(ReadOptions(), "key", &value).IsNotFound());
  1196. // Compact and verify again. It's worthwhile because now the files have
  1197. // tighter endpoints, so we can verify that doesn't mess anything up.
  1198. ASSERT_OK(dbfull()->TEST_CompactRange(
  1199. 1 /* level */, nullptr /* begin */, nullptr /* end */,
  1200. nullptr /* column_family */, true /* disallow_trivial_move */));
  1201. ASSERT_GT(NumTableFilesAtLevel(2), 1);
  1202. ASSERT_TRUE(db_->Get(ReadOptions(), "key", &value).IsNotFound());
  1203. db_->ReleaseSnapshot(snapshot);
  1204. }
  1205. TEST_F(DBRangeDelTest, UntruncatedTombstoneDoesNotDeleteNewerKey) {
  1206. // Verify a key newer than a range tombstone cannot be deleted by being
  1207. // compacted to the bottom level (and thus having its seqnum zeroed) before
  1208. // the range tombstone. This used to happen when range tombstones were
  1209. // untruncated on reads such that they extended past their file boundaries.
  1210. //
  1211. // Test summary:
  1212. //
  1213. // - L1 is bottommost.
  1214. // - A couple snapshots are strategically taken to prevent seqnums from being
  1215. // zeroed, range tombstone from being dropped, merge operands from being
  1216. // dropped, and merge operands from being combined.
  1217. // - Left half of files in L1 all have same user key, ensuring their file
  1218. // boundaries overlap. In the past this would cause range tombstones to be
  1219. // untruncated.
  1220. // - Right half of L1 files all have different keys, ensuring no overlap.
  1221. // - A range tombstone spans all L1 keys, so it is stored in every L1 file.
  1222. // - Keys in the right side of the key-range are overwritten. These are
  1223. // compacted down to L1 after releasing snapshots such that their seqnums
  1224. // will be zeroed.
  1225. // - A full range scan is performed. If the tombstone in the left L1 files
  1226. // were untruncated, it would now cover keys newer than it (but with zeroed
  1227. // seqnums) in the right L1 files.
  1228. const int kFileBytes = 1 << 20;
  1229. const int kValueBytes = 1 << 10;
  1230. const int kNumFiles = 4;
  1231. const int kMaxKey = kNumFiles * kFileBytes / kValueBytes;
  1232. const int kKeysOverwritten = 10;
  1233. Options options = CurrentOptions();
  1234. options.compression = kNoCompression;
  1235. options.disable_auto_compactions = true;
  1236. options.merge_operator.reset(new MockMergeOperator());
  1237. options.num_levels = 2;
  1238. options.target_file_size_base = kFileBytes;
  1239. Reopen(options);
  1240. Random rnd(301);
  1241. // - snapshots[0] prevents merge operands from being combined during
  1242. // compaction.
  1243. // - snapshots[1] prevents merge operands from being dropped due to the
  1244. // covering range tombstone.
  1245. const Snapshot* snapshots[] = {nullptr, nullptr};
  1246. for (int i = 0; i < kNumFiles; ++i) {
  1247. for (int j = 0; j < kFileBytes / kValueBytes; ++j) {
  1248. auto value = rnd.RandomString(kValueBytes);
  1249. std::string key;
  1250. if (i < kNumFiles / 2) {
  1251. key = Key(0);
  1252. } else {
  1253. key = Key(1 + i * kFileBytes / kValueBytes + j);
  1254. }
  1255. ASSERT_OK(db_->Merge(WriteOptions(), key, value));
  1256. }
  1257. if (i == 0) {
  1258. snapshots[0] = db_->GetSnapshot();
  1259. }
  1260. if (i == kNumFiles - 1) {
  1261. snapshots[1] = db_->GetSnapshot();
  1262. // The DeleteRange is the last write so all merge operands are covered.
  1263. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1264. Key(0), Key(kMaxKey + 1)));
  1265. }
  1266. ASSERT_OK(db_->Flush(FlushOptions()));
  1267. }
  1268. ASSERT_EQ(kNumFiles, NumTableFilesAtLevel(0));
  1269. auto get_key_count = [this]() -> int {
  1270. auto* iter = db_->NewIterator(ReadOptions());
  1271. assert(iter->status().ok());
  1272. iter->SeekToFirst();
  1273. int keys_found = 0;
  1274. for (; iter->Valid(); iter->Next()) {
  1275. ++keys_found;
  1276. }
  1277. EXPECT_OK(iter->status());
  1278. delete iter;
  1279. return keys_found;
  1280. };
  1281. // All keys should be covered
  1282. ASSERT_EQ(0, get_key_count());
  1283. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr /* begin_key */,
  1284. nullptr /* end_key */));
  1285. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  1286. // Roughly the left half of L1 files should have overlapping boundary keys,
  1287. // while the right half should not.
  1288. ASSERT_GE(NumTableFilesAtLevel(1), kNumFiles);
  1289. // Now overwrite a few keys that are in L1 files that definitely don't have
  1290. // overlapping boundary keys.
  1291. for (int i = kMaxKey; i > kMaxKey - kKeysOverwritten; --i) {
  1292. auto value = rnd.RandomString(kValueBytes);
  1293. ASSERT_OK(db_->Merge(WriteOptions(), Key(i), value));
  1294. }
  1295. ASSERT_OK(db_->Flush(FlushOptions()));
  1296. // The overwritten keys are in L0 now, so clearly aren't covered by the range
  1297. // tombstone in L1.
  1298. ASSERT_EQ(kKeysOverwritten, get_key_count());
  1299. // Release snapshots so seqnums can be zeroed when L0->L1 happens.
  1300. db_->ReleaseSnapshot(snapshots[0]);
  1301. db_->ReleaseSnapshot(snapshots[1]);
  1302. auto begin_key_storage = Key(kMaxKey - kKeysOverwritten + 1);
  1303. auto end_key_storage = Key(kMaxKey);
  1304. Slice begin_key(begin_key_storage);
  1305. Slice end_key(end_key_storage);
  1306. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), &begin_key, &end_key));
  1307. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  1308. ASSERT_GE(NumTableFilesAtLevel(1), kNumFiles);
  1309. ASSERT_EQ(kKeysOverwritten, get_key_count());
  1310. }
  1311. TEST_F(DBRangeDelTest, DeletedMergeOperandReappearsIterPrev) {
  1312. // Exposes a bug where we were using
  1313. // `RangeDelPositioningMode::kBackwardTraversal` while scanning merge operands
  1314. // in the forward direction. Confusingly, this case happened during
  1315. // `DBIter::Prev`. It could cause assertion failure, or reappearing keys.
  1316. const int kFileBytes = 1 << 20;
  1317. const int kValueBytes = 1 << 10;
  1318. // Need multiple keys so we can get results when calling `Prev()` after
  1319. // `SeekToLast()`.
  1320. const int kNumKeys = 3;
  1321. const int kNumFiles = 4;
  1322. Options options = CurrentOptions();
  1323. options.compression = kNoCompression;
  1324. options.disable_auto_compactions = true;
  1325. options.merge_operator.reset(new MockMergeOperator());
  1326. options.target_file_size_base = kFileBytes;
  1327. Reopen(options);
  1328. Random rnd(301);
  1329. const Snapshot* snapshot = nullptr;
  1330. for (int i = 0; i < kNumFiles; ++i) {
  1331. for (int j = 0; j < kFileBytes / kValueBytes; ++j) {
  1332. auto value = rnd.RandomString(kValueBytes);
  1333. ASSERT_OK(db_->Merge(WriteOptions(), Key(j % kNumKeys), value));
  1334. if (i == 0 && j == kNumKeys) {
  1335. // Take snapshot to prevent covered merge operands from being dropped or
  1336. // merged by compaction.
  1337. snapshot = db_->GetSnapshot();
  1338. // Do a DeleteRange near the beginning so only the oldest merge operand
  1339. // for each key is covered. This ensures the sequence of events:
  1340. //
  1341. // - `DBIter::Prev()` is called
  1342. // - After several same versions of the same user key are encountered,
  1343. // it decides to seek using `DBIter::FindValueForCurrentKeyUsingSeek`.
  1344. // - Binary searches to the newest version of the key, which is in the
  1345. // leftmost file containing the user key.
  1346. // - Scans forwards to collect all merge operands. Eventually reaches
  1347. // the rightmost file containing the oldest merge operand, which
  1348. // should be covered by the `DeleteRange`. If `RangeDelAggregator`
  1349. // were not properly using `kForwardTraversal` here, that operand
  1350. // would reappear.
  1351. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1352. Key(0), Key(kNumKeys + 1)));
  1353. }
  1354. }
  1355. ASSERT_OK(db_->Flush(FlushOptions()));
  1356. }
  1357. ASSERT_EQ(kNumFiles, NumTableFilesAtLevel(0));
  1358. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr /* begin_key */,
  1359. nullptr /* end_key */));
  1360. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  1361. ASSERT_GT(NumTableFilesAtLevel(1), 1);
  1362. auto* iter = db_->NewIterator(ReadOptions());
  1363. ASSERT_OK(iter->status());
  1364. iter->SeekToLast();
  1365. int keys_found = 0;
  1366. for (; iter->Valid(); iter->Prev()) {
  1367. ++keys_found;
  1368. }
  1369. ASSERT_OK(iter->status());
  1370. delete iter;
  1371. ASSERT_EQ(kNumKeys, keys_found);
  1372. db_->ReleaseSnapshot(snapshot);
  1373. }
  1374. TEST_F(DBRangeDelTest, SnapshotPreventsDroppedKeys) {
  1375. const int kFileBytes = 1 << 20;
  1376. Options options = CurrentOptions();
  1377. options.compression = kNoCompression;
  1378. options.disable_auto_compactions = true;
  1379. options.target_file_size_base = kFileBytes;
  1380. Reopen(options);
  1381. ASSERT_OK(Put(Key(0), "a"));
  1382. const Snapshot* snapshot = db_->GetSnapshot();
  1383. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1384. Key(10)));
  1385. ASSERT_OK(db_->Flush(FlushOptions()));
  1386. ReadOptions read_opts;
  1387. read_opts.snapshot = snapshot;
  1388. auto* iter = db_->NewIterator(read_opts);
  1389. ASSERT_OK(iter->status());
  1390. iter->SeekToFirst();
  1391. ASSERT_TRUE(iter->Valid());
  1392. ASSERT_EQ(Key(0), iter->key());
  1393. iter->Next();
  1394. ASSERT_FALSE(iter->Valid());
  1395. ASSERT_OK(iter->status());
  1396. delete iter;
  1397. db_->ReleaseSnapshot(snapshot);
  1398. }
  1399. TEST_F(DBRangeDelTest, SnapshotPreventsDroppedKeysInImmMemTables) {
  1400. const int kFileBytes = 1 << 20;
  1401. Options options = CurrentOptions();
  1402. options.compression = kNoCompression;
  1403. options.disable_auto_compactions = true;
  1404. options.target_file_size_base = kFileBytes;
  1405. Reopen(options);
  1406. // block flush thread -> pin immtables in memory
  1407. SyncPoint::GetInstance()->DisableProcessing();
  1408. SyncPoint::GetInstance()->LoadDependency({
  1409. {"SnapshotPreventsDroppedKeysInImmMemTables:AfterNewIterator",
  1410. "DBImpl::BGWorkFlush"},
  1411. });
  1412. SyncPoint::GetInstance()->EnableProcessing();
  1413. ASSERT_OK(Put(Key(0), "a"));
  1414. std::unique_ptr<const Snapshot, std::function<void(const Snapshot*)>>
  1415. snapshot(db_->GetSnapshot(),
  1416. [this](const Snapshot* s) { db_->ReleaseSnapshot(s); });
  1417. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1418. Key(10)));
  1419. ASSERT_OK(dbfull()->TEST_SwitchMemtable());
  1420. ReadOptions read_opts;
  1421. read_opts.snapshot = snapshot.get();
  1422. std::unique_ptr<Iterator> iter(db_->NewIterator(read_opts));
  1423. ASSERT_OK(iter->status());
  1424. TEST_SYNC_POINT("SnapshotPreventsDroppedKeysInImmMemTables:AfterNewIterator");
  1425. iter->SeekToFirst();
  1426. ASSERT_TRUE(iter->Valid());
  1427. ASSERT_EQ(Key(0), iter->key());
  1428. iter->Next();
  1429. ASSERT_FALSE(iter->Valid());
  1430. ASSERT_OK(iter->status());
  1431. }
  1432. TEST_F(DBRangeDelTest, RangeTombstoneWrittenToMinimalSsts) {
  1433. // Adapted from
  1434. // https://github.com/cockroachdb/cockroach/blob/de8b3ea603dd1592d9dc26443c2cc92c356fbc2f/pkg/storage/engine/rocksdb_test.go#L1267-L1398.
  1435. // Regression test for issue where range tombstone was written to more files
  1436. // than necessary when it began exactly at the begin key in the next
  1437. // compaction output file.
  1438. const int kFileBytes = 1 << 20;
  1439. const int kValueBytes = 4 << 10;
  1440. Options options = CurrentOptions();
  1441. options.compression = kNoCompression;
  1442. options.disable_auto_compactions = true;
  1443. // Have a bit of slack in the size limits but we enforce them more strictly
  1444. // when manually flushing/compacting.
  1445. options.max_compaction_bytes = 2 * kFileBytes;
  1446. options.target_file_size_base = 2 * kFileBytes;
  1447. options.write_buffer_size = 2 * kFileBytes;
  1448. Reopen(options);
  1449. Random rnd(301);
  1450. for (char first_char : {'a', 'b', 'c'}) {
  1451. for (int i = 0; i < kFileBytes / kValueBytes; ++i) {
  1452. std::string key(1, first_char);
  1453. key.append(Key(i));
  1454. std::string value = rnd.RandomString(kValueBytes);
  1455. ASSERT_OK(Put(key, value));
  1456. }
  1457. ASSERT_OK(db_->Flush(FlushOptions()));
  1458. MoveFilesToLevel(2);
  1459. }
  1460. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  1461. ASSERT_EQ(3, NumTableFilesAtLevel(2));
  1462. // Populate the memtable lightly while spanning the whole key-space. The
  1463. // setting of `max_compaction_bytes` will cause the L0->L1 to output multiple
  1464. // files to prevent a large L1->L2 compaction later.
  1465. ASSERT_OK(Put("a", "val"));
  1466. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1467. "c" + Key(1), "d"));
  1468. // Our compaction output file cutting logic currently only considers point
  1469. // keys. So, in order for the range tombstone to have a chance at landing at
  1470. // the start of a new file, we need a point key at the range tombstone's
  1471. // start.
  1472. // TODO(ajkr): remove this `Put` after file cutting accounts for range
  1473. // tombstones (#3977).
  1474. ASSERT_OK(Put("c" + Key(1), "value"));
  1475. ASSERT_OK(db_->Flush(FlushOptions()));
  1476. // Ensure manual L0->L1 compaction cuts the outputs before the range tombstone
  1477. // and the range tombstone is only placed in the second SST.
  1478. std::string begin_key_storage("c" + Key(1));
  1479. Slice begin_key(begin_key_storage);
  1480. std::string end_key_storage("d");
  1481. Slice end_key(end_key_storage);
  1482. ASSERT_OK(dbfull()->TEST_CompactRange(
  1483. 0 /* level */, &begin_key /* begin */, &end_key /* end */,
  1484. nullptr /* column_family */, true /* disallow_trivial_move */));
  1485. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  1486. std::vector<LiveFileMetaData> all_metadata;
  1487. std::vector<LiveFileMetaData> l1_metadata;
  1488. db_->GetLiveFilesMetaData(&all_metadata);
  1489. for (const auto& metadata : all_metadata) {
  1490. if (metadata.level == 1) {
  1491. l1_metadata.push_back(metadata);
  1492. }
  1493. }
  1494. std::sort(l1_metadata.begin(), l1_metadata.end(),
  1495. [&](const LiveFileMetaData& a, const LiveFileMetaData& b) {
  1496. return options.comparator->Compare(a.smallestkey, b.smallestkey) <
  1497. 0;
  1498. });
  1499. ASSERT_EQ("a", l1_metadata[0].smallestkey);
  1500. ASSERT_EQ("a", l1_metadata[0].largestkey);
  1501. ASSERT_EQ("c" + Key(1), l1_metadata[1].smallestkey);
  1502. ASSERT_EQ("d", l1_metadata[1].largestkey);
  1503. TablePropertiesCollection all_table_props;
  1504. ASSERT_OK(db_->GetPropertiesOfAllTables(&all_table_props));
  1505. int64_t num_range_deletions = 0;
  1506. for (const auto& name_and_table_props : all_table_props) {
  1507. const auto& name = name_and_table_props.first;
  1508. const auto& table_props = name_and_table_props.second;
  1509. // The range tombstone should only be output to the second L1 SST.
  1510. if (name.size() >= l1_metadata[1].name.size() &&
  1511. name.substr(name.size() - l1_metadata[1].name.size())
  1512. .compare(l1_metadata[1].name) == 0) {
  1513. ASSERT_EQ(1, table_props->num_range_deletions);
  1514. ++num_range_deletions;
  1515. } else {
  1516. ASSERT_EQ(0, table_props->num_range_deletions);
  1517. }
  1518. }
  1519. ASSERT_EQ(1, num_range_deletions);
  1520. }
  1521. TEST_F(DBRangeDelTest, LevelCompactOutputCutAtRangeTombstoneForTtlFiles) {
  1522. Options options = CurrentOptions();
  1523. options.compression = kNoCompression;
  1524. options.compaction_pri = kMinOverlappingRatio;
  1525. options.disable_auto_compactions = true;
  1526. options.ttl = 24 * 60 * 60; // 24 hours
  1527. options.target_file_size_base = 8 << 10;
  1528. env_->SetMockSleep();
  1529. options.env = env_;
  1530. DestroyAndReopen(options);
  1531. Random rnd(301);
  1532. // Fill some data so that future compactions are not bottommost level
  1533. // compaction, and hence they would try cut around files for ttl
  1534. for (int i = 5; i < 10; ++i) {
  1535. ASSERT_OK(Put(Key(i), rnd.RandomString(1 << 10)));
  1536. }
  1537. ASSERT_OK(Flush());
  1538. MoveFilesToLevel(3);
  1539. ASSERT_EQ("0,0,0,1", FilesPerLevel());
  1540. for (int i = 5; i < 10; ++i) {
  1541. ASSERT_OK(Put(Key(i), rnd.RandomString(1 << 10)));
  1542. }
  1543. ASSERT_OK(Flush());
  1544. MoveFilesToLevel(1);
  1545. ASSERT_EQ("0,1,0,1", FilesPerLevel());
  1546. env_->MockSleepForSeconds(20 * 60 * 60);
  1547. // Prevent range tombstone from being dropped during compaction.
  1548. const Snapshot* snapshot = db_->GetSnapshot();
  1549. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1550. Key(11), Key(12)));
  1551. ASSERT_OK(Put(Key(0), rnd.RandomString(1 << 10)));
  1552. ASSERT_OK(Flush());
  1553. ASSERT_EQ("1,1,0,1", FilesPerLevel());
  1554. // L0 file is new, L1 and L3 file are old and qualified for TTL
  1555. env_->MockSleepForSeconds(10 * 60 * 60);
  1556. MoveFilesToLevel(1);
  1557. // L1 output should be cut into 3 files:
  1558. // File 0: Key(0)
  1559. // File 1: (qualified for TTL): Key(5) - Key(10)
  1560. // File 1: DeleteRange [11, 12)
  1561. ASSERT_EQ("0,3,0,1", FilesPerLevel());
  1562. db_->ReleaseSnapshot(snapshot);
  1563. }
  1564. // Test SST partitioner cut after every single key
  1565. class SingleKeySstPartitioner : public SstPartitioner {
  1566. public:
  1567. const char* Name() const override { return "SingleKeySstPartitioner"; }
  1568. PartitionerResult ShouldPartition(
  1569. const PartitionerRequest& /*request*/) override {
  1570. return kRequired;
  1571. }
  1572. bool CanDoTrivialMove(const Slice& /*smallest_user_key*/,
  1573. const Slice& /*largest_user_key*/) override {
  1574. return false;
  1575. }
  1576. };
  1577. class SingleKeySstPartitionerFactory : public SstPartitionerFactory {
  1578. public:
  1579. static const char* kClassName() { return "SingleKeySstPartitionerFactory"; }
  1580. const char* Name() const override { return kClassName(); }
  1581. std::unique_ptr<SstPartitioner> CreatePartitioner(
  1582. const SstPartitioner::Context& /* context */) const override {
  1583. return std::unique_ptr<SstPartitioner>(new SingleKeySstPartitioner());
  1584. }
  1585. };
  1586. TEST_F(DBRangeDelTest, CompactionEmitRangeTombstoneToSSTPartitioner) {
  1587. Options options = CurrentOptions();
  1588. auto factory = std::make_shared<SingleKeySstPartitionerFactory>();
  1589. options.sst_partitioner_factory = factory;
  1590. options.disable_auto_compactions = true;
  1591. DestroyAndReopen(options);
  1592. Random rnd(301);
  1593. // range deletion keys are not processed when compacting to bottommost level,
  1594. // so creating a file at older level to make the next compaction not
  1595. // bottommost level
  1596. ASSERT_OK(db_->Put(WriteOptions(), Key(4), rnd.RandomString(10)));
  1597. ASSERT_OK(Flush());
  1598. MoveFilesToLevel(5);
  1599. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(10)));
  1600. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  1601. Key(5)));
  1602. ASSERT_OK(Flush());
  1603. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1604. MoveFilesToLevel(1);
  1605. // SSTPartitioner decides to cut when range tombstone start key is passed to
  1606. // it. Note that the range tombstone [2, 5) itself span multiple keys, but we
  1607. // are not able to partition within its range yet.
  1608. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  1609. }
  1610. TEST_F(DBRangeDelTest, OversizeCompactionGapBetweenPointKeyAndTombstone) {
  1611. // L2 has 2 files
  1612. // L2_0: 0, 1, 2, 3, 4
  1613. // L2_1: 5, 6, 7
  1614. // L0 has 1 file
  1615. // L0: 0, [5, 6), 8
  1616. // max_compaction_bytes is less than the size of L2_0 and L2_1.
  1617. // When compacting L0 into L1, it should split into 3 files:
  1618. // compaction output should cut before key 5 and key 8 to
  1619. // limit future compaction size.
  1620. const int kNumPerFile = 4, kNumFiles = 2;
  1621. Options options = CurrentOptions();
  1622. options.disable_auto_compactions = true;
  1623. options.target_file_size_base = 9 * 1024;
  1624. options.max_compaction_bytes = 9 * 1024;
  1625. DestroyAndReopen(options);
  1626. Random rnd(301);
  1627. for (int i = 0; i < kNumFiles; ++i) {
  1628. std::vector<std::string> values;
  1629. for (int j = 0; j < kNumPerFile; j++) {
  1630. values.push_back(rnd.RandomString(3 << 10));
  1631. ASSERT_OK(Put(Key(i * kNumPerFile + j), values[j]));
  1632. }
  1633. }
  1634. ASSERT_OK(db_->Flush(FlushOptions()));
  1635. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1636. MoveFilesToLevel(2);
  1637. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1638. ASSERT_OK(Put(Key(0), rnd.RandomString(1 << 10)));
  1639. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(5),
  1640. Key(6)));
  1641. ASSERT_OK(Put(Key(8), rnd.RandomString(1 << 10)));
  1642. ASSERT_OK(db_->Flush(FlushOptions()));
  1643. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1644. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  1645. true /* disallow_trivial_move */));
  1646. ASSERT_EQ(3, NumTableFilesAtLevel(1));
  1647. }
  1648. TEST_F(DBRangeDelTest, OversizeCompactionGapBetweenTombstone) {
  1649. // L2 has two files
  1650. // L2_0: 0, 1, 2, 3, 4. L2_1: 5, 6, 7
  1651. // L0 has two range tombstones [0, 1), [7, 8).
  1652. // max_compaction_bytes is less than the size of L2_0.
  1653. // When compacting L0 into L1, the two range tombstones should be
  1654. // split into two files.
  1655. const int kNumPerFile = 4, kNumFiles = 2;
  1656. Options options = CurrentOptions();
  1657. options.disable_auto_compactions = true;
  1658. options.target_file_size_base = 9 * 1024;
  1659. options.max_compaction_bytes = 9 * 1024;
  1660. DestroyAndReopen(options);
  1661. Random rnd(301);
  1662. for (int i = 0; i < kNumFiles; ++i) {
  1663. std::vector<std::string> values;
  1664. // Write 12K (4 values, each 3K)
  1665. for (int j = 0; j < kNumPerFile; j++) {
  1666. values.push_back(rnd.RandomString(3 << 10));
  1667. ASSERT_OK(Put(Key(i * kNumPerFile + j), values[j]));
  1668. }
  1669. }
  1670. ASSERT_OK(db_->Flush(FlushOptions()));
  1671. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1672. MoveFilesToLevel(2);
  1673. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1674. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1675. Key(1)));
  1676. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(7),
  1677. Key(8)));
  1678. ASSERT_OK(db_->Flush(FlushOptions()));
  1679. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1680. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  1681. true /* disallow_trivial_move */));
  1682. // This is L0 -> L1 compaction
  1683. // The two range tombstones are broken up into two output files
  1684. // to limit compaction size.
  1685. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  1686. }
  1687. TEST_F(DBRangeDelTest, OversizeCompactionPointKeyWithinRangetombstone) {
  1688. // L2 has two files
  1689. // L2_0: 0, 1, 2, 3, 4. L2_1: 6, 7, 8
  1690. // L0 has [0, 9) and point key 5
  1691. // max_compaction_bytes is less than the size of L2_0.
  1692. // When compacting L0 into L1, the compaction should cut at point key 5.
  1693. Options options = CurrentOptions();
  1694. options.disable_auto_compactions = true;
  1695. options.target_file_size_base = 9 * 1024;
  1696. options.max_compaction_bytes = 9 * 1024;
  1697. DestroyAndReopen(options);
  1698. Random rnd(301);
  1699. for (int i = 0; i < 9; ++i) {
  1700. if (i == 5) {
  1701. ++i;
  1702. }
  1703. ASSERT_OK(Put(Key(i), rnd.RandomString(3 << 10)));
  1704. }
  1705. ASSERT_OK(db_->Flush(FlushOptions()));
  1706. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1707. MoveFilesToLevel(2);
  1708. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1709. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1710. Key(9)));
  1711. ASSERT_OK(Put(Key(5), rnd.RandomString(1 << 10)));
  1712. ASSERT_OK(db_->Flush(FlushOptions()));
  1713. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1714. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  1715. true /* disallow_trivial_move */));
  1716. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  1717. }
  1718. TEST_F(DBRangeDelTest, OverlappedTombstones) {
  1719. const int kNumPerFile = 4, kNumFiles = 2;
  1720. Options options = CurrentOptions();
  1721. options.disable_auto_compactions = true;
  1722. options.target_file_size_base = 9 * 1024;
  1723. options.max_compaction_bytes = 9 * 1024;
  1724. DestroyAndReopen(options);
  1725. Random rnd(301);
  1726. for (int i = 0; i < kNumFiles; ++i) {
  1727. std::vector<std::string> values;
  1728. // Write 12K (4 values, each 3K)
  1729. for (int j = 0; j < kNumPerFile; j++) {
  1730. values.push_back(rnd.RandomString(3 << 10));
  1731. ASSERT_OK(Put(Key(i * kNumPerFile + j), values[j]));
  1732. }
  1733. }
  1734. ASSERT_OK(db_->Flush(FlushOptions()));
  1735. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1736. MoveFilesToLevel(2);
  1737. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1738. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(1),
  1739. Key((kNumFiles)*kNumPerFile + 1)));
  1740. ASSERT_OK(db_->Flush(FlushOptions()));
  1741. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1742. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  1743. true /* disallow_trivial_move */));
  1744. // The tombstone range is not broken up into multiple SSTs which may incur a
  1745. // large compaction with L2.
  1746. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1747. std::vector<std::vector<FileMetaData>> files;
  1748. ASSERT_OK(dbfull()->TEST_CompactRange(1, nullptr, nullptr, nullptr,
  1749. true /* disallow_trivial_move */));
  1750. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  1751. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  1752. }
  1753. TEST_F(DBRangeDelTest, OverlappedKeys) {
  1754. const int kNumPerFile = 4, kNumFiles = 2;
  1755. Options options = CurrentOptions();
  1756. options.disable_auto_compactions = true;
  1757. options.target_file_size_base = 9 * 1024;
  1758. options.max_compaction_bytes = 9 * 1024;
  1759. DestroyAndReopen(options);
  1760. Random rnd(301);
  1761. for (int i = 0; i < kNumFiles; ++i) {
  1762. std::vector<std::string> values;
  1763. // Write 12K (4 values, each 3K)
  1764. for (int j = 0; j < kNumPerFile; j++) {
  1765. values.push_back(rnd.RandomString(3 << 10));
  1766. ASSERT_OK(Put(Key(i * kNumPerFile + j), values[j]));
  1767. }
  1768. }
  1769. ASSERT_OK(db_->Flush(FlushOptions()));
  1770. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1771. MoveFilesToLevel(2);
  1772. ASSERT_EQ(2, NumTableFilesAtLevel(2));
  1773. for (int i = 1; i < kNumFiles * kNumPerFile + 1; i++) {
  1774. ASSERT_OK(Put(Key(i), "0x123"));
  1775. }
  1776. ASSERT_OK(db_->Flush(FlushOptions()));
  1777. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1778. // The key range is broken up into three SSTs to avoid a future big compaction
  1779. // with the grandparent
  1780. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  1781. true /* disallow_trivial_move */));
  1782. ASSERT_EQ(3, NumTableFilesAtLevel(1));
  1783. ASSERT_OK(dbfull()->TEST_CompactRange(1, nullptr, nullptr, nullptr,
  1784. true /* disallow_trivial_move */));
  1785. // L1->L2 compaction size is limited to max_compaction_bytes
  1786. ASSERT_EQ(3, NumTableFilesAtLevel(2));
  1787. ASSERT_EQ(0, NumTableFilesAtLevel(1));
  1788. }
  1789. TEST_F(DBRangeDelTest, IteratorRefresh) {
  1790. // Refreshing an iterator after a range tombstone is added should cause the
  1791. // deleted range of keys to disappear.
  1792. for (bool sv_changed : {false, true}) {
  1793. ASSERT_OK(db_->Put(WriteOptions(), "key1", "value1"));
  1794. ASSERT_OK(db_->Put(WriteOptions(), "key2", "value2"));
  1795. auto* iter = db_->NewIterator(ReadOptions());
  1796. ASSERT_OK(iter->status());
  1797. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  1798. "key2", "key3"));
  1799. if (sv_changed) {
  1800. ASSERT_OK(db_->Flush(FlushOptions()));
  1801. }
  1802. ASSERT_OK(iter->Refresh());
  1803. ASSERT_OK(iter->status());
  1804. iter->SeekToFirst();
  1805. ASSERT_EQ("key1", iter->key());
  1806. iter->Next();
  1807. ASSERT_FALSE(iter->Valid());
  1808. ASSERT_OK(iter->status());
  1809. delete iter;
  1810. }
  1811. }
  1812. void VerifyIteratorReachesEnd(InternalIterator* iter) {
  1813. ASSERT_TRUE(!iter->Valid() && iter->status().ok());
  1814. }
  1815. void VerifyIteratorReachesEnd(Iterator* iter) {
  1816. ASSERT_TRUE(!iter->Valid() && iter->status().ok());
  1817. }
  1818. TEST_F(DBRangeDelTest, IteratorReseek) {
  1819. // Range tombstone triggers reseek (seeking to a range tombstone end key) in
  1820. // merging iterator. Test set up:
  1821. // one memtable: range tombstone [0, 1)
  1822. // one immutable memtable: range tombstone [1, 2)
  1823. // one L0 file with range tombstone [2, 3)
  1824. // one L1 file with range tombstone [3, 4)
  1825. // Seek(0) should trigger cascading reseeks at all levels below memtable.
  1826. // Seek(1) should trigger cascading reseeks at all levels below immutable
  1827. // memtable. SeekToFirst and SeekToLast trigger no reseek.
  1828. Options options = CurrentOptions();
  1829. options.compression = kNoCompression;
  1830. options.disable_auto_compactions = true;
  1831. DestroyAndReopen(options);
  1832. // L1
  1833. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  1834. Key(4)));
  1835. ASSERT_OK(db_->Flush(FlushOptions()));
  1836. MoveFilesToLevel(1);
  1837. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1838. // L0
  1839. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  1840. Key(3)));
  1841. ASSERT_OK(db_->Flush(FlushOptions()));
  1842. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1843. // Immutable memtable
  1844. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(1),
  1845. Key(2)));
  1846. ASSERT_OK(static_cast_with_check<DBImpl>(db_)->TEST_SwitchMemtable());
  1847. std::string value;
  1848. ASSERT_TRUE(dbfull()->GetProperty(db_->DefaultColumnFamily(),
  1849. "rocksdb.num-immutable-mem-table", &value));
  1850. ASSERT_EQ(1, std::stoi(value));
  1851. // live memtable
  1852. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1853. Key(1)));
  1854. // this memtable is still active
  1855. ASSERT_TRUE(dbfull()->GetProperty(db_->DefaultColumnFamily(),
  1856. "rocksdb.num-immutable-mem-table", &value));
  1857. ASSERT_EQ(1, std::stoi(value));
  1858. auto iter = db_->NewIterator(ReadOptions());
  1859. get_perf_context()->Reset();
  1860. iter->Seek(Key(0));
  1861. // Reseeked immutable memtable, L0 and L1
  1862. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 3);
  1863. VerifyIteratorReachesEnd(iter);
  1864. get_perf_context()->Reset();
  1865. iter->SeekForPrev(Key(1));
  1866. // Reseeked L0 and L1
  1867. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1868. VerifyIteratorReachesEnd(iter);
  1869. get_perf_context()->Reset();
  1870. iter->SeekToFirst();
  1871. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 0);
  1872. VerifyIteratorReachesEnd(iter);
  1873. iter->SeekToLast();
  1874. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 0);
  1875. VerifyIteratorReachesEnd(iter);
  1876. delete iter;
  1877. }
  1878. TEST_F(DBRangeDelTest, ReseekDuringNextAndPrev) {
  1879. // Range tombstone triggers reseek during Next()/Prev() in merging iterator.
  1880. // Test set up:
  1881. // memtable has: [0, 1) [2, 3)
  1882. // L0 has: 2
  1883. // L1 has: 1, 2, 3
  1884. // Seek(0) will reseek to 1 for L0 and L1. Seek(1) will not trigger any
  1885. // reseek. Then Next() determines 2 is covered by [2, 3), it will try to
  1886. // reseek to 3 for L0 and L1. Similar story for Prev() and SeekForPrev() is
  1887. // tested.
  1888. Options options = CurrentOptions();
  1889. options.compression = kNoCompression;
  1890. options.disable_auto_compactions = true;
  1891. DestroyAndReopen(options);
  1892. // L1
  1893. ASSERT_OK(db_->Put(WriteOptions(), Key(1), "foo"));
  1894. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  1895. ASSERT_OK(db_->Put(WriteOptions(), Key(3), "foo"));
  1896. ASSERT_OK(db_->Flush(FlushOptions()));
  1897. MoveFilesToLevel(1);
  1898. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1899. // L0
  1900. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  1901. ASSERT_OK(db_->Flush(FlushOptions()));
  1902. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1903. // Memtable
  1904. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1905. Key(1)));
  1906. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  1907. Key(3)));
  1908. auto iter = db_->NewIterator(ReadOptions());
  1909. auto iter_test_forward = [&] {
  1910. ASSERT_TRUE(iter->Valid());
  1911. ASSERT_EQ(iter->key().ToString(), Key(1));
  1912. get_perf_context()->Reset();
  1913. iter->Next();
  1914. ASSERT_TRUE(iter->Valid());
  1915. ASSERT_EQ(iter->key().ToString(), Key(3));
  1916. // Reseeked L0 and L1
  1917. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1918. // Next to Prev
  1919. get_perf_context()->Reset();
  1920. iter->Prev();
  1921. ASSERT_TRUE(iter->Valid());
  1922. ASSERT_EQ(iter->key().ToString(), Key(1));
  1923. // Reseeked L0 and L1
  1924. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1925. // Prev to Next
  1926. get_perf_context()->Reset();
  1927. iter->Next();
  1928. ASSERT_TRUE(iter->Valid());
  1929. ASSERT_EQ(iter->key().ToString(), Key(3));
  1930. // Reseeked L0 and L1
  1931. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1932. iter->Next();
  1933. VerifyIteratorReachesEnd(iter);
  1934. };
  1935. get_perf_context()->Reset();
  1936. iter->Seek(Key(0));
  1937. // Reseeked L0 and L1
  1938. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1939. iter_test_forward();
  1940. get_perf_context()->Reset();
  1941. iter->Seek(Key(1));
  1942. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 0);
  1943. iter_test_forward();
  1944. get_perf_context()->Reset();
  1945. iter->SeekForPrev(Key(2));
  1946. // Reseeked L0 and L1
  1947. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1948. iter_test_forward();
  1949. get_perf_context()->Reset();
  1950. iter->SeekForPrev(Key(1));
  1951. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 0);
  1952. iter_test_forward();
  1953. get_perf_context()->Reset();
  1954. iter->SeekToFirst();
  1955. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 0);
  1956. iter_test_forward();
  1957. iter->SeekToLast();
  1958. iter->Prev();
  1959. iter_test_forward();
  1960. delete iter;
  1961. }
  1962. TEST_F(DBRangeDelTest, TombstoneFromCurrentLevel) {
  1963. // Range tombstone triggers reseek when covering key from the same level.
  1964. // in merging iterator. Test set up:
  1965. // memtable has: [0, 1)
  1966. // L0 has: [2, 3), 2
  1967. // L1 has: 1, 2, 3
  1968. // Seek(0) will reseek to 1 for L0 and L1.
  1969. // Then Next() will reseek to 3 for L1 since 2 in L0 is covered by [2, 3) in
  1970. // L0.
  1971. Options options = CurrentOptions();
  1972. options.compression = kNoCompression;
  1973. options.disable_auto_compactions = true;
  1974. DestroyAndReopen(options);
  1975. // L1
  1976. ASSERT_OK(db_->Put(WriteOptions(), Key(1), "foo"));
  1977. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  1978. ASSERT_OK(db_->Put(WriteOptions(), Key(3), "foo"));
  1979. ASSERT_OK(db_->Flush(FlushOptions()));
  1980. MoveFilesToLevel(1);
  1981. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  1982. // L0
  1983. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  1984. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  1985. Key(3)));
  1986. ASSERT_OK(db_->Flush(FlushOptions()));
  1987. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  1988. // Memtable
  1989. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(0),
  1990. Key(1)));
  1991. auto iter = db_->NewIterator(ReadOptions());
  1992. get_perf_context()->Reset();
  1993. iter->Seek(Key(0));
  1994. ASSERT_TRUE(iter->Valid());
  1995. ASSERT_EQ(iter->key().ToString(), Key(1));
  1996. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 2);
  1997. get_perf_context()->Reset();
  1998. iter->Next();
  1999. ASSERT_TRUE(iter->Valid());
  2000. ASSERT_EQ(iter->key().ToString(), Key(3));
  2001. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 1);
  2002. delete iter;
  2003. }
  2004. class TombstoneTestSstPartitioner : public SstPartitioner {
  2005. public:
  2006. const char* Name() const override { return "SingleKeySstPartitioner"; }
  2007. PartitionerResult ShouldPartition(
  2008. const PartitionerRequest& request) override {
  2009. if (cmp->Compare(*request.current_user_key, DBTestBase::Key(5)) == 0) {
  2010. return kRequired;
  2011. } else {
  2012. return kNotRequired;
  2013. }
  2014. }
  2015. bool CanDoTrivialMove(const Slice& /*smallest_user_key*/,
  2016. const Slice& /*largest_user_key*/) override {
  2017. return false;
  2018. }
  2019. const Comparator* cmp = BytewiseComparator();
  2020. };
  2021. class TombstoneTestSstPartitionerFactory : public SstPartitionerFactory {
  2022. public:
  2023. static const char* kClassName() {
  2024. return "TombstoneTestSstPartitionerFactory";
  2025. }
  2026. const char* Name() const override { return kClassName(); }
  2027. std::unique_ptr<SstPartitioner> CreatePartitioner(
  2028. const SstPartitioner::Context& /* context */) const override {
  2029. return std::unique_ptr<SstPartitioner>(new TombstoneTestSstPartitioner());
  2030. }
  2031. };
  2032. TEST_F(DBRangeDelTest, TombstoneAcrossFileBoundary) {
  2033. // Verify that a range tombstone across file boundary covers keys from older
  2034. // levels. Test set up:
  2035. // L1_0: 1, 3, [2, 6) L1_1: 5, 7, [2, 6) ([2, 6) is from compaction with
  2036. // L1_0) L2 has: 5
  2037. // Seek(1) and then Next() should move the L1 level iterator to
  2038. // L1_1. Check if 5 is returned after Next().
  2039. Options options = CurrentOptions();
  2040. options.compression = kNoCompression;
  2041. options.disable_auto_compactions = true;
  2042. options.target_file_size_base = 2 * 1024;
  2043. options.max_compaction_bytes = 2 * 1024;
  2044. // Make sure L1 files are split before "5"
  2045. auto factory = std::make_shared<TombstoneTestSstPartitionerFactory>();
  2046. options.sst_partitioner_factory = factory;
  2047. DestroyAndReopen(options);
  2048. Random rnd(301);
  2049. // L2
  2050. // the file should be smaller than max_compaction_bytes, otherwise the file
  2051. // will be cut before 7.
  2052. ASSERT_OK(db_->Put(WriteOptions(), Key(5), rnd.RandomString(1 << 9)));
  2053. ASSERT_OK(db_->Flush(FlushOptions()));
  2054. MoveFilesToLevel(2);
  2055. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2056. // L1_1
  2057. ASSERT_OK(db_->Put(WriteOptions(), Key(5), rnd.RandomString(1 << 10)));
  2058. ASSERT_OK(db_->Put(WriteOptions(), Key(7), rnd.RandomString(1 << 10)));
  2059. ASSERT_OK(db_->Flush(FlushOptions()));
  2060. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  2061. // L1_0
  2062. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(1 << 10)));
  2063. ASSERT_OK(db_->Put(WriteOptions(), Key(3), rnd.RandomString(1 << 10)));
  2064. // Prevent keys being compacted away
  2065. const Snapshot* snapshot = db_->GetSnapshot();
  2066. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2067. Key(6)));
  2068. ASSERT_OK(db_->Flush(FlushOptions()));
  2069. ASSERT_EQ(2, NumTableFilesAtLevel(0));
  2070. MoveFilesToLevel(1);
  2071. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2072. auto iter = db_->NewIterator(ReadOptions());
  2073. get_perf_context()->Reset();
  2074. iter->Seek(Key(1));
  2075. ASSERT_TRUE(iter->Valid());
  2076. ASSERT_EQ(iter->key().ToString(), Key(1));
  2077. iter->Next();
  2078. ASSERT_TRUE(iter->Valid());
  2079. ASSERT_EQ(iter->key().ToString(), Key(7));
  2080. // 1 reseek into L2 when key 5 in L2 is covered by [2, 6) from L1
  2081. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 1);
  2082. delete iter;
  2083. db_->ReleaseSnapshot(snapshot);
  2084. }
  2085. TEST_F(DBRangeDelTest, NonOverlappingTombstonAtBoundary) {
  2086. // Verify that a range tombstone across file boundary covers keys from older
  2087. // levels.
  2088. // Test set up:
  2089. // L1_0: 1, 3, [4, 7) L1_1: 6, 8, [4, 7)
  2090. // L2: 5
  2091. // L1_0's largest key: Key(6)@kMaxSequenceNumber with type kTypeRangeDeletion
  2092. // Note that [4, 7) is at end of L1_0 and not overlapping with any point key
  2093. // in L1_0. [4, 7) from L1_0 should cover 5 if sentinel in LevelIterator works
  2094. Options options = CurrentOptions();
  2095. options.compression = kNoCompression;
  2096. options.disable_auto_compactions = true;
  2097. options.target_file_size_base = 4 * 1024;
  2098. DestroyAndReopen(options);
  2099. Random rnd(301);
  2100. // L2
  2101. ASSERT_OK(db_->Put(WriteOptions(), Key(5), rnd.RandomString(4 << 10)));
  2102. ASSERT_OK(db_->Flush(FlushOptions()));
  2103. MoveFilesToLevel(2);
  2104. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2105. // L1_1
  2106. ASSERT_OK(db_->Put(WriteOptions(), Key(6), rnd.RandomString(4 << 10)));
  2107. ASSERT_OK(db_->Put(WriteOptions(), Key(8), rnd.RandomString(4 << 10)));
  2108. ASSERT_OK(db_->Flush(FlushOptions()));
  2109. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  2110. // L1_0
  2111. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2112. ASSERT_OK(db_->Put(WriteOptions(), Key(3), rnd.RandomString(4 << 10)));
  2113. // Prevent keys being compacted away
  2114. const Snapshot* snapshot = db_->GetSnapshot();
  2115. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(4),
  2116. Key(7)));
  2117. ASSERT_OK(db_->Flush(FlushOptions()));
  2118. ASSERT_EQ(2, NumTableFilesAtLevel(0));
  2119. MoveFilesToLevel(1);
  2120. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2121. std::vector<std::vector<FileMetaData>> files;
  2122. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  2123. InternalKey ik = InternalKey(Key(6), kMaxSequenceNumber, kTypeRangeDeletion);
  2124. ASSERT_EQ(files[1][0].largest.Encode(), ik.Encode());
  2125. auto iter = db_->NewIterator(ReadOptions());
  2126. iter->Seek(Key(3));
  2127. ASSERT_TRUE(iter->Valid());
  2128. ASSERT_EQ(iter->key(), Key(3));
  2129. get_perf_context()->Reset();
  2130. iter->Next();
  2131. ASSERT_TRUE(iter->Valid());
  2132. ASSERT_EQ(iter->key().ToString(), Key(8));
  2133. // 1 reseek into L1 since 5 from L2 is covered by [4, 7) from L1
  2134. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 1);
  2135. for (auto& k : {4, 5, 6}) {
  2136. get_perf_context()->Reset();
  2137. iter->Seek(Key(k));
  2138. ASSERT_TRUE(iter->Valid());
  2139. ASSERT_EQ(iter->key().ToString(), Key(8));
  2140. // 1 reseek into L1
  2141. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count, 1);
  2142. }
  2143. delete iter;
  2144. db_->ReleaseSnapshot(snapshot);
  2145. }
  2146. TEST_F(DBRangeDelTest, OlderLevelHasNewerData) {
  2147. // L1_0: 1, 3, [2, 7) L1_1: 5, 6 at a newer sequence number than [2, 7)
  2148. // Compact L1_1 to L2. Seek(3) should not skip 5 or 6.
  2149. Options options = CurrentOptions();
  2150. options.compression = kNoCompression;
  2151. options.disable_auto_compactions = true;
  2152. options.target_file_size_base = 3 * 1024;
  2153. DestroyAndReopen(options);
  2154. Random rnd(301);
  2155. // L1_0
  2156. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2157. ASSERT_OK(db_->Put(WriteOptions(), Key(3), rnd.RandomString(4 << 10)));
  2158. const Snapshot* snapshot = db_->GetSnapshot();
  2159. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2160. Key(7)));
  2161. ASSERT_OK(db_->Flush(FlushOptions()));
  2162. MoveFilesToLevel(1);
  2163. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2164. // L1_1
  2165. ASSERT_OK(db_->Put(WriteOptions(), Key(5), rnd.RandomString(4 << 10)));
  2166. ASSERT_OK(db_->Put(WriteOptions(), Key(6), rnd.RandomString(4 << 10)));
  2167. ASSERT_OK(db_->Flush(FlushOptions()));
  2168. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  2169. MoveFilesToLevel(1);
  2170. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2171. auto key = Key(6);
  2172. Slice begin(key);
  2173. EXPECT_OK(dbfull()->TEST_CompactRange(1, &begin, nullptr));
  2174. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2175. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2176. auto iter = db_->NewIterator(ReadOptions());
  2177. iter->Seek(Key(3));
  2178. ASSERT_TRUE(iter->Valid());
  2179. ASSERT_EQ(iter->key().ToString(), Key(5));
  2180. iter->Next();
  2181. ASSERT_TRUE(iter->Valid());
  2182. ASSERT_EQ(iter->key().ToString(), Key(6));
  2183. delete iter;
  2184. db_->ReleaseSnapshot(snapshot);
  2185. }
  2186. TEST_F(DBRangeDelTest, LevelBoundaryDefinedByTombstone) {
  2187. // L1 has: 1, 2, [4, 5)
  2188. // L2 has: 4
  2189. // Seek(3), which is over all points keys in L1, check whether
  2190. // sentinel key from L1 works in this case.
  2191. Options options = CurrentOptions();
  2192. options.compression = kNoCompression;
  2193. options.disable_auto_compactions = true;
  2194. options.target_file_size_base = 3 * 1024;
  2195. DestroyAndReopen(options);
  2196. Random rnd(301);
  2197. // L2
  2198. ASSERT_OK(db_->Put(WriteOptions(), Key(4), "foo"));
  2199. ASSERT_OK(db_->Flush(FlushOptions()));
  2200. const Snapshot* snapshot = db_->GetSnapshot();
  2201. MoveFilesToLevel(2);
  2202. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2203. // L1_0
  2204. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2205. ASSERT_OK(db_->Put(WriteOptions(), Key(2), rnd.RandomString(4 << 10)));
  2206. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(4),
  2207. Key(5)));
  2208. ASSERT_OK(db_->Flush(FlushOptions()));
  2209. MoveFilesToLevel(1);
  2210. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2211. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2212. auto iter = db_->NewIterator(ReadOptions());
  2213. iter->Seek(Key(3));
  2214. ASSERT_TRUE(!iter->Valid());
  2215. ASSERT_OK(iter->status());
  2216. get_perf_context()->Reset();
  2217. iter->SeekForPrev(Key(5));
  2218. ASSERT_TRUE(iter->Valid());
  2219. ASSERT_EQ(iter->key(), Key(2));
  2220. db_->ReleaseSnapshot(snapshot);
  2221. delete iter;
  2222. }
  2223. TEST_F(DBRangeDelTest, TombstoneOnlyFile) {
  2224. // L1_0: 1, 2, L1_1: [3, 5)
  2225. // L2: 3
  2226. // Seek(2) then Next() should advance L1 iterator into L1_1.
  2227. // If sentinel works with tombstone only file, it should cover the key in L2.
  2228. // Similar story for SeekForPrev(4).
  2229. Options options = CurrentOptions();
  2230. options.compression = kNoCompression;
  2231. options.disable_auto_compactions = true;
  2232. options.target_file_size_base = 3 * 1024;
  2233. DestroyAndReopen(options);
  2234. Random rnd(301);
  2235. // L2
  2236. ASSERT_OK(db_->Put(WriteOptions(), Key(3), "foo"));
  2237. ASSERT_OK(db_->Flush(FlushOptions()));
  2238. MoveFilesToLevel(2);
  2239. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2240. // L1_0
  2241. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2242. ASSERT_OK(db_->Put(WriteOptions(), Key(2), rnd.RandomString(4 << 10)));
  2243. ASSERT_OK(db_->Flush(FlushOptions()));
  2244. MoveFilesToLevel(1);
  2245. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2246. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2247. // L1_1
  2248. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  2249. Key(5)));
  2250. ASSERT_OK(db_->Flush(FlushOptions()));
  2251. MoveFilesToLevel(1);
  2252. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2253. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2254. auto iter = db_->NewIterator(ReadOptions());
  2255. iter->Seek(Key(2));
  2256. ASSERT_TRUE(iter->Valid());
  2257. ASSERT_EQ(iter->key(), Key(2));
  2258. iter->Next();
  2259. VerifyIteratorReachesEnd(iter);
  2260. iter->SeekForPrev(Key(4));
  2261. ASSERT_TRUE(iter->Valid());
  2262. ASSERT_EQ(iter->key(), Key(2));
  2263. iter->Next();
  2264. VerifyIteratorReachesEnd(iter);
  2265. delete iter;
  2266. }
  2267. void VerifyIteratorKey(InternalIterator* iter,
  2268. const std::vector<std::string>& expected_keys,
  2269. bool forward = true) {
  2270. for (auto& key : expected_keys) {
  2271. ASSERT_TRUE(iter->Valid());
  2272. ASSERT_EQ(iter->user_key(), key);
  2273. if (forward) {
  2274. iter->Next();
  2275. } else {
  2276. iter->Prev();
  2277. }
  2278. }
  2279. }
  2280. TEST_F(DBRangeDelTest, TombstoneOnlyLevel) {
  2281. // L1 [3, 5)
  2282. // L2 has: 3, 4
  2283. // Any kind of iterator seek should skip 3 and 4 in L2.
  2284. // L1 level iterator should produce sentinel key.
  2285. Options options = CurrentOptions();
  2286. options.compression = kNoCompression;
  2287. options.disable_auto_compactions = true;
  2288. options.target_file_size_base = 3 * 1024;
  2289. DestroyAndReopen(options);
  2290. // L2
  2291. ASSERT_OK(db_->Put(WriteOptions(), Key(3), "foo"));
  2292. ASSERT_OK(db_->Put(WriteOptions(), Key(4), "bar"));
  2293. ASSERT_OK(db_->Flush(FlushOptions()));
  2294. MoveFilesToLevel(2);
  2295. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2296. // L1
  2297. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  2298. Key(5)));
  2299. ASSERT_OK(db_->Flush(FlushOptions()));
  2300. MoveFilesToLevel(1);
  2301. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2302. auto iter = db_->NewIterator(ReadOptions());
  2303. get_perf_context()->Reset();
  2304. uint64_t expected_reseek = 0;
  2305. for (auto i = 0; i < 7; ++i) {
  2306. iter->Seek(Key(i));
  2307. VerifyIteratorReachesEnd(iter);
  2308. if (i < 5) {
  2309. ++expected_reseek;
  2310. }
  2311. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count,
  2312. expected_reseek);
  2313. iter->SeekForPrev(Key(i));
  2314. VerifyIteratorReachesEnd(iter);
  2315. if (i > 2) {
  2316. ++expected_reseek;
  2317. }
  2318. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count,
  2319. expected_reseek);
  2320. iter->SeekToFirst();
  2321. VerifyIteratorReachesEnd(iter);
  2322. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count,
  2323. ++expected_reseek);
  2324. iter->SeekToLast();
  2325. VerifyIteratorReachesEnd(iter);
  2326. ASSERT_EQ(get_perf_context()->internal_range_del_reseek_count,
  2327. ++expected_reseek);
  2328. }
  2329. delete iter;
  2330. // Check L1 LevelIterator behavior
  2331. ColumnFamilyData* cfd =
  2332. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  2333. ->cfd();
  2334. SuperVersion* sv = cfd->GetSuperVersion();
  2335. Arena arena;
  2336. ReadOptions read_options;
  2337. MergeIteratorBuilder merge_iter_builder(&cfd->internal_comparator(), &arena,
  2338. false /* prefix seek */);
  2339. InternalIterator* level_iter = sv->current->TEST_GetLevelIterator(
  2340. read_options, &merge_iter_builder, 1 /* level */, true);
  2341. // This is needed to make LevelIterator range tombstone aware
  2342. auto miter = merge_iter_builder.Finish();
  2343. auto k = Key(3);
  2344. IterKey target;
  2345. target.SetInternalKey(k, kMaxSequenceNumber, kValueTypeForSeek);
  2346. level_iter->Seek(target.GetInternalKey());
  2347. // sentinel key (file boundary as a fake key)
  2348. VerifyIteratorKey(level_iter, {Key(5)});
  2349. VerifyIteratorReachesEnd(level_iter);
  2350. k = Key(5);
  2351. target.SetInternalKey(k, 0, kValueTypeForSeekForPrev);
  2352. level_iter->SeekForPrev(target.GetInternalKey());
  2353. VerifyIteratorKey(level_iter, {Key(3)}, false);
  2354. VerifyIteratorReachesEnd(level_iter);
  2355. level_iter->SeekToFirst();
  2356. VerifyIteratorKey(level_iter, {Key(5)});
  2357. VerifyIteratorReachesEnd(level_iter);
  2358. level_iter->SeekToLast();
  2359. VerifyIteratorKey(level_iter, {Key(3)}, false);
  2360. VerifyIteratorReachesEnd(level_iter);
  2361. miter->~InternalIterator();
  2362. }
  2363. TEST_F(DBRangeDelTest, TombstoneOnlyWithOlderVisibleKey) {
  2364. // L1: [3, 5)
  2365. // L2: 2, 4, 5
  2366. // 2 and 5 should be visible
  2367. Options options = CurrentOptions();
  2368. options.compression = kNoCompression;
  2369. options.disable_auto_compactions = true;
  2370. options.target_file_size_base = 3 * 1024;
  2371. DestroyAndReopen(options);
  2372. // L2
  2373. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  2374. ASSERT_OK(db_->Put(WriteOptions(), Key(4), "bar"));
  2375. ASSERT_OK(db_->Put(WriteOptions(), Key(5), "foobar"));
  2376. ASSERT_OK(db_->Flush(FlushOptions()));
  2377. MoveFilesToLevel(2);
  2378. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2379. // l1
  2380. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  2381. Key(5)));
  2382. ASSERT_OK(db_->Flush(FlushOptions()));
  2383. MoveFilesToLevel(1);
  2384. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2385. auto iter = db_->NewIterator(ReadOptions());
  2386. auto iter_test_backward = [&] {
  2387. ASSERT_TRUE(iter->Valid());
  2388. ASSERT_EQ(iter->key(), Key(5));
  2389. iter->Prev();
  2390. ASSERT_TRUE(iter->Valid());
  2391. ASSERT_EQ(iter->key(), Key(2));
  2392. iter->Prev();
  2393. VerifyIteratorReachesEnd(iter);
  2394. };
  2395. auto iter_test_forward = [&] {
  2396. ASSERT_TRUE(iter->Valid());
  2397. ASSERT_EQ(iter->key(), Key(2));
  2398. iter->Next();
  2399. ASSERT_TRUE(iter->Valid());
  2400. ASSERT_EQ(iter->key(), Key(5));
  2401. iter->Next();
  2402. VerifyIteratorReachesEnd(iter);
  2403. };
  2404. iter->Seek(Key(4));
  2405. iter_test_backward();
  2406. iter->SeekForPrev(Key(4));
  2407. iter->Next();
  2408. iter_test_backward();
  2409. iter->Seek(Key(4));
  2410. iter->Prev();
  2411. iter_test_forward();
  2412. iter->SeekForPrev(Key(4));
  2413. iter_test_forward();
  2414. iter->SeekToFirst();
  2415. iter_test_forward();
  2416. iter->SeekToLast();
  2417. iter_test_backward();
  2418. delete iter;
  2419. }
  2420. TEST_F(DBRangeDelTest, TombstoneSentinelDirectionChange) {
  2421. // L1: 7
  2422. // L2: [4, 6)
  2423. // L3: 4
  2424. // Seek(5) will have sentinel key 6 at the top of minHeap in merging iterator.
  2425. // then do a prev, how would sentinel work?
  2426. // Redo the test after Put(5) into L1 so that there is a visible key in range
  2427. // [4, 6).
  2428. Options options = CurrentOptions();
  2429. options.compression = kNoCompression;
  2430. options.disable_auto_compactions = true;
  2431. options.target_file_size_base = 3 * 1024;
  2432. DestroyAndReopen(options);
  2433. // L3
  2434. ASSERT_OK(db_->Put(WriteOptions(), Key(4), "bar"));
  2435. ASSERT_OK(db_->Flush(FlushOptions()));
  2436. MoveFilesToLevel(3);
  2437. ASSERT_EQ(1, NumTableFilesAtLevel(3));
  2438. // L2
  2439. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(4),
  2440. Key(6)));
  2441. ASSERT_OK(db_->Flush(FlushOptions()));
  2442. MoveFilesToLevel(2);
  2443. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2444. // L1
  2445. ASSERT_OK(db_->Put(WriteOptions(), Key(7), "foobar"));
  2446. ASSERT_OK(db_->Flush(FlushOptions()));
  2447. MoveFilesToLevel(1);
  2448. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2449. auto iter = db_->NewIterator(ReadOptions());
  2450. iter->Seek(Key(5));
  2451. ASSERT_TRUE(iter->Valid());
  2452. ASSERT_EQ(iter->key(), Key(7));
  2453. iter->Prev();
  2454. ASSERT_TRUE(!iter->Valid() && iter->status().ok());
  2455. delete iter;
  2456. ASSERT_OK(db_->Put(WriteOptions(), Key(5), "foobar"));
  2457. ASSERT_OK(db_->Flush(FlushOptions()));
  2458. MoveFilesToLevel(1);
  2459. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2460. iter = db_->NewIterator(ReadOptions());
  2461. iter->Seek(Key(5));
  2462. ASSERT_TRUE(iter->Valid());
  2463. ASSERT_EQ(iter->key(), Key(5));
  2464. iter->Prev();
  2465. ASSERT_TRUE(!iter->Valid() && iter->status().ok());
  2466. delete iter;
  2467. }
  2468. // Right sentinel tested in many test cases above
  2469. TEST_F(DBRangeDelTest, LeftSentinelKeyTest) {
  2470. // L1_0: 0, 1 L1_1: [2, 3), 5
  2471. // L2: 2
  2472. // SeekForPrev(4) should give 1 due to sentinel key keeping [2, 3) alive.
  2473. Options options = CurrentOptions();
  2474. options.compression = kNoCompression;
  2475. options.disable_auto_compactions = true;
  2476. options.target_file_size_base = 3 * 1024;
  2477. options.max_compaction_bytes = 2048;
  2478. DestroyAndReopen(options);
  2479. // L2
  2480. ASSERT_OK(db_->Put(WriteOptions(), Key(2), "foo"));
  2481. ASSERT_OK(db_->Flush(FlushOptions()));
  2482. MoveFilesToLevel(2);
  2483. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2484. // L1_0
  2485. Random rnd(301);
  2486. ASSERT_OK(db_->Put(WriteOptions(), Key(0), rnd.RandomString(4 << 10)));
  2487. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2488. ASSERT_OK(db_->Flush(FlushOptions()));
  2489. MoveFilesToLevel(1);
  2490. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2491. // L1_1
  2492. ASSERT_OK(db_->Put(WriteOptions(), Key(5), "bar"));
  2493. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2494. Key(3)));
  2495. ASSERT_OK(db_->Flush(FlushOptions()));
  2496. MoveFilesToLevel(1);
  2497. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2498. auto iter = db_->NewIterator(ReadOptions());
  2499. iter->SeekForPrev(Key(4));
  2500. ASSERT_TRUE(iter->Valid());
  2501. ASSERT_EQ(iter->key(), Key(1));
  2502. iter->Prev();
  2503. ASSERT_TRUE(iter->Valid());
  2504. ASSERT_EQ(iter->key(), Key(0));
  2505. iter->Prev();
  2506. ASSERT_TRUE(!iter->Valid());
  2507. ASSERT_OK(iter->status());
  2508. delete iter;
  2509. }
  2510. TEST_F(DBRangeDelTest, LeftSentinelKeyTestWithNewerKey) {
  2511. // L1_0: 1, 2 newer than L1_1, L1_1: [2, 4), 5
  2512. // L2: 3
  2513. // SeekForPrev(4) then Prev() should give 2 and then 1.
  2514. Options options = CurrentOptions();
  2515. options.compression = kNoCompression;
  2516. options.disable_auto_compactions = true;
  2517. options.target_file_size_base = 3 * 1024;
  2518. options.max_compaction_bytes = 3 * 1024;
  2519. DestroyAndReopen(options);
  2520. // L2
  2521. ASSERT_OK(db_->Put(WriteOptions(), Key(3), "foo"));
  2522. ASSERT_OK(db_->Flush(FlushOptions()));
  2523. MoveFilesToLevel(2);
  2524. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2525. // L1_1
  2526. ASSERT_OK(db_->Put(WriteOptions(), Key(5), "bar"));
  2527. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2528. Key(4)));
  2529. ASSERT_OK(db_->Flush(FlushOptions()));
  2530. MoveFilesToLevel(1);
  2531. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2532. // L1_0
  2533. Random rnd(301);
  2534. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2535. ASSERT_OK(db_->Put(WriteOptions(), Key(2), rnd.RandomString(4 << 10)));
  2536. // Used to verify sequence number of iterator key later.
  2537. auto seq = dbfull()->TEST_GetLastVisibleSequence();
  2538. ASSERT_OK(db_->Flush(FlushOptions()));
  2539. MoveFilesToLevel(1);
  2540. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2541. Arena arena;
  2542. InternalKeyComparator icmp(options.comparator);
  2543. ReadOptions read_options;
  2544. ScopedArenaPtr<InternalIterator> iter;
  2545. iter.reset(
  2546. dbfull()->NewInternalIterator(read_options, &arena, kMaxSequenceNumber));
  2547. auto k = Key(4);
  2548. IterKey target;
  2549. target.SetInternalKey(k, 0 /* sequence_number */, kValueTypeForSeekForPrev);
  2550. iter->SeekForPrev(target.GetInternalKey());
  2551. ASSERT_TRUE(iter->Valid());
  2552. ASSERT_EQ(iter->user_key(), Key(2));
  2553. SequenceNumber actual_seq;
  2554. ValueType type;
  2555. UnPackSequenceAndType(ExtractInternalKeyFooter(iter->key()), &actual_seq,
  2556. &type);
  2557. ASSERT_EQ(seq, actual_seq);
  2558. // might as well check type
  2559. ASSERT_EQ(type, kTypeValue);
  2560. iter->Prev();
  2561. ASSERT_TRUE(iter->Valid());
  2562. ASSERT_EQ(iter->user_key(), Key(1));
  2563. iter->Prev();
  2564. ASSERT_TRUE(!iter->Valid());
  2565. ASSERT_OK(iter->status());
  2566. }
  2567. TEST_F(DBRangeDelTest, SentinelKeyCommonCaseTest) {
  2568. // L1 has 3 files
  2569. // L1_0: 1, 2 L1_1: [3, 4) 5, 6, [7, 8) L1_2: 9
  2570. // Check iterator operations on LevelIterator.
  2571. Options options = CurrentOptions();
  2572. options.compression = kNoCompression;
  2573. options.disable_auto_compactions = true;
  2574. options.target_file_size_base = 3 * 1024;
  2575. DestroyAndReopen(options);
  2576. Random rnd(301);
  2577. // L1_0
  2578. ASSERT_OK(db_->Put(WriteOptions(), Key(1), rnd.RandomString(4 << 10)));
  2579. ASSERT_OK(db_->Put(WriteOptions(), Key(2), rnd.RandomString(4 << 10)));
  2580. ASSERT_OK(db_->Flush(FlushOptions()));
  2581. MoveFilesToLevel(1);
  2582. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2583. // L1_1
  2584. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  2585. Key(4)));
  2586. ASSERT_OK(db_->Put(WriteOptions(), Key(5), rnd.RandomString(4 << 10)));
  2587. ASSERT_OK(db_->Put(WriteOptions(), Key(6), rnd.RandomString(4 << 10)));
  2588. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(7),
  2589. Key(8)));
  2590. ASSERT_OK(db_->Flush(FlushOptions()));
  2591. MoveFilesToLevel(1);
  2592. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2593. // L1_2
  2594. ASSERT_OK(db_->Put(WriteOptions(), Key(9), rnd.RandomString(4 << 10)));
  2595. ASSERT_OK(db_->Flush(FlushOptions()));
  2596. MoveFilesToLevel(1);
  2597. ASSERT_EQ(3, NumTableFilesAtLevel(1));
  2598. ColumnFamilyData* cfd =
  2599. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  2600. ->cfd();
  2601. SuperVersion* sv = cfd->GetSuperVersion();
  2602. Arena arena;
  2603. ReadOptions read_options;
  2604. MergeIteratorBuilder merge_iter_builder(&cfd->internal_comparator(), &arena,
  2605. false /* prefix seek */);
  2606. InternalIterator* level_iter = sv->current->TEST_GetLevelIterator(
  2607. read_options, &merge_iter_builder, 1 /* level */, true);
  2608. // This is needed to make LevelIterator range tombstone aware
  2609. auto miter = merge_iter_builder.Finish();
  2610. auto k = Key(7);
  2611. IterKey target;
  2612. target.SetInternalKey(k, kMaxSequenceNumber, kValueTypeForSeek);
  2613. level_iter->Seek(target.GetInternalKey());
  2614. // The last Key(9) is a sentinel key.
  2615. VerifyIteratorKey(level_iter, {Key(8), Key(9), Key(9)});
  2616. ASSERT_TRUE(!level_iter->Valid() && level_iter->status().ok());
  2617. k = Key(6);
  2618. target.SetInternalKey(k, kMaxSequenceNumber, kValueTypeForSeek);
  2619. level_iter->Seek(target.GetInternalKey());
  2620. VerifyIteratorKey(level_iter, {Key(6), Key(8), Key(9), Key(9)});
  2621. ASSERT_TRUE(!level_iter->Valid() && level_iter->status().ok());
  2622. k = Key(4);
  2623. target.SetInternalKey(k, 0, kValueTypeForSeekForPrev);
  2624. level_iter->SeekForPrev(target.GetInternalKey());
  2625. VerifyIteratorKey(level_iter, {Key(3), Key(2), Key(1), Key(1)}, false);
  2626. ASSERT_TRUE(!level_iter->Valid() && level_iter->status().ok());
  2627. k = Key(5);
  2628. target.SetInternalKey(k, 0, kValueTypeForSeekForPrev);
  2629. level_iter->SeekForPrev(target.GetInternalKey());
  2630. VerifyIteratorKey(level_iter, {Key(5), Key(3), Key(2), Key(1), Key(1)},
  2631. false);
  2632. level_iter->SeekToFirst();
  2633. VerifyIteratorKey(level_iter, {Key(1), Key(2), Key(2), Key(5), Key(6), Key(8),
  2634. Key(9), Key(9)});
  2635. ASSERT_TRUE(!level_iter->Valid() && level_iter->status().ok());
  2636. level_iter->SeekToLast();
  2637. VerifyIteratorKey(
  2638. level_iter,
  2639. {Key(9), Key(9), Key(6), Key(5), Key(3), Key(2), Key(1), Key(1)}, false);
  2640. ASSERT_TRUE(!level_iter->Valid() && level_iter->status().ok());
  2641. miter->~InternalIterator();
  2642. }
  2643. TEST_F(DBRangeDelTest, PrefixSentinelKey) {
  2644. // L1: ['aaaa', 'aaad'), 'bbbb'
  2645. // L2: 'aaac', 'aaae'
  2646. // Prefix extracts first 3 chars
  2647. // Seek('aaab') should give 'aaae' as first key.
  2648. // This is to test a previous bug where prefix seek sees there is no prefix in
  2649. // the SST file, and will just set file iter to null in LevelIterator and may
  2650. // just skip to the next SST file. But in this case, we should keep the file's
  2651. // tombstone alive.
  2652. Options options = CurrentOptions();
  2653. options.compression = kNoCompression;
  2654. options.disable_auto_compactions = true;
  2655. options.prefix_extractor.reset(NewFixedPrefixTransform(3));
  2656. BlockBasedTableOptions table_options;
  2657. table_options.filter_policy.reset(NewBloomFilterPolicy(10, false));
  2658. table_options.whole_key_filtering = false;
  2659. options.table_factory.reset(NewBlockBasedTableFactory(table_options));
  2660. DestroyAndReopen(options);
  2661. Random rnd(301);
  2662. // L2:
  2663. ASSERT_OK(db_->Put(WriteOptions(), "aaac", rnd.RandomString(10)));
  2664. ASSERT_OK(db_->Put(WriteOptions(), "aaae", rnd.RandomString(10)));
  2665. ASSERT_OK(db_->Flush(FlushOptions()));
  2666. MoveFilesToLevel(2);
  2667. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  2668. // L1
  2669. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "aaaa",
  2670. "aaad"));
  2671. ASSERT_OK(db_->Put(WriteOptions(), "bbbb", rnd.RandomString(10)));
  2672. ASSERT_OK(db_->Flush(FlushOptions()));
  2673. MoveFilesToLevel(1);
  2674. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  2675. auto iter = db_->NewIterator(ReadOptions());
  2676. iter->Seek("aaab");
  2677. ASSERT_TRUE(iter->Valid());
  2678. ASSERT_EQ(iter->key(), "aaae");
  2679. delete iter;
  2680. }
  2681. TEST_F(DBRangeDelTest, RefreshMemtableIter) {
  2682. Options options = CurrentOptions();
  2683. options.disable_auto_compactions = true;
  2684. DestroyAndReopen(options);
  2685. ASSERT_OK(
  2686. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  2687. ReadOptions ro;
  2688. ro.read_tier = kMemtableTier;
  2689. std::unique_ptr<Iterator> iter{db_->NewIterator(ro)};
  2690. ASSERT_OK(Flush());
  2691. // First refresh reinits iter, which had a bug where
  2692. // iter.memtable_range_tombstone_iter_ was not set to nullptr, and caused
  2693. // subsequent refresh to double free.
  2694. ASSERT_OK(iter->Refresh());
  2695. ASSERT_OK(iter->Refresh());
  2696. }
  2697. TEST_F(DBRangeDelTest, RangeTombstoneRespectIterateUpperBound) {
  2698. // Memtable: a, [b, bz)
  2699. // Do a Seek on `a` with iterate_upper_bound being az
  2700. // range tombstone [b, bz) should not be processed (added to and
  2701. // popped from the min_heap in MergingIterator).
  2702. Options options = CurrentOptions();
  2703. options.disable_auto_compactions = true;
  2704. DestroyAndReopen(options);
  2705. ASSERT_OK(Put("a", "bar"));
  2706. ASSERT_OK(
  2707. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "b", "bz"));
  2708. // I could not find a cleaner way to test this without relying on
  2709. // implementation detail. Tried to test the value of
  2710. // `internal_range_del_reseek_count` but that did not work
  2711. // since BlockBasedTable iterator becomes !Valid() when point key
  2712. // is out of bound and that reseek only happens when a point key
  2713. // is covered by some range tombstone.
  2714. SyncPoint::GetInstance()->SetCallBack("MergeIterator::PopDeleteRangeStart",
  2715. [](void*) {
  2716. // there should not be any range
  2717. // tombstone in the heap.
  2718. FAIL();
  2719. });
  2720. SyncPoint::GetInstance()->EnableProcessing();
  2721. ReadOptions read_opts;
  2722. std::string upper_bound = "az";
  2723. Slice upper_bound_slice = upper_bound;
  2724. read_opts.iterate_upper_bound = &upper_bound_slice;
  2725. std::unique_ptr<Iterator> iter{db_->NewIterator(read_opts)};
  2726. iter->Seek("a");
  2727. ASSERT_TRUE(iter->Valid());
  2728. ASSERT_EQ(iter->key(), "a");
  2729. iter->Next();
  2730. ASSERT_FALSE(iter->Valid());
  2731. ASSERT_OK(iter->status());
  2732. }
  2733. TEST_F(DBRangeDelTest, RangetombesoneCompensateFilesize) {
  2734. Options opts = CurrentOptions();
  2735. opts.disable_auto_compactions = true;
  2736. DestroyAndReopen(opts);
  2737. std::vector<std::string> values;
  2738. Random rnd(301);
  2739. // file in L2
  2740. values.push_back(rnd.RandomString(1 << 10));
  2741. ASSERT_OK(Put("a", values.back()));
  2742. values.push_back(rnd.RandomString(1 << 10));
  2743. ASSERT_OK(Put("b", values.back()));
  2744. ASSERT_OK(Flush());
  2745. MoveFilesToLevel(2);
  2746. uint64_t l2_size = 0;
  2747. ASSERT_OK(Size("a", "c", 0 /* cf */, &l2_size));
  2748. ASSERT_GT(l2_size, 0);
  2749. // file in L1
  2750. values.push_back(rnd.RandomString(1 << 10));
  2751. ASSERT_OK(Put("d", values.back()));
  2752. values.push_back(rnd.RandomString(1 << 10));
  2753. ASSERT_OK(Put("e", values.back()));
  2754. ASSERT_OK(Flush());
  2755. MoveFilesToLevel(1);
  2756. uint64_t l1_size = 0;
  2757. ASSERT_OK(Size("d", "f", 0 /* cf */, &l1_size));
  2758. ASSERT_GT(l1_size, 0);
  2759. ASSERT_OK(
  2760. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "f"));
  2761. ASSERT_OK(Flush());
  2762. // Range deletion compensated size computed during flush time
  2763. std::vector<std::vector<FileMetaData>> level_to_files;
  2764. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2765. &level_to_files);
  2766. ASSERT_EQ(level_to_files[0].size(), 1);
  2767. ASSERT_EQ(level_to_files[0][0].compensated_range_deletion_size,
  2768. l1_size + l2_size);
  2769. ASSERT_EQ(level_to_files[1].size(), 1);
  2770. ASSERT_EQ(level_to_files[1][0].compensated_range_deletion_size, 0);
  2771. ASSERT_EQ(level_to_files[2].size(), 1);
  2772. ASSERT_EQ(level_to_files[2][0].compensated_range_deletion_size, 0);
  2773. // Range deletion compensated size computed during compaction time
  2774. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  2775. true /* disallow_trivial_move */));
  2776. ASSERT_EQ(NumTableFilesAtLevel(0), 0);
  2777. ASSERT_EQ(NumTableFilesAtLevel(1), 1);
  2778. ASSERT_EQ(NumTableFilesAtLevel(2), 1);
  2779. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2780. &level_to_files);
  2781. ASSERT_EQ(level_to_files[1].size(), 1);
  2782. ASSERT_EQ(level_to_files[1][0].compensated_range_deletion_size, l2_size);
  2783. ASSERT_EQ(level_to_files[2].size(), 1);
  2784. ASSERT_EQ(level_to_files[2][0].compensated_range_deletion_size, 0);
  2785. }
  2786. TEST_F(DBRangeDelTest, RangetombesoneCompensateFilesizePersistDuringReopen) {
  2787. Options opts = CurrentOptions();
  2788. opts.disable_auto_compactions = true;
  2789. DestroyAndReopen(opts);
  2790. std::vector<std::string> values;
  2791. Random rnd(301);
  2792. values.push_back(rnd.RandomString(1 << 10));
  2793. ASSERT_OK(Put("a", values.back()));
  2794. values.push_back(rnd.RandomString(1 << 10));
  2795. ASSERT_OK(Put("b", values.back()));
  2796. ASSERT_OK(Flush());
  2797. MoveFilesToLevel(2);
  2798. ASSERT_OK(
  2799. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "c"));
  2800. ASSERT_OK(Flush());
  2801. MoveFilesToLevel(1);
  2802. ASSERT_OK(
  2803. db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "z"));
  2804. ASSERT_OK(Flush());
  2805. std::vector<std::vector<FileMetaData>> level_to_files;
  2806. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2807. &level_to_files);
  2808. ASSERT_EQ(level_to_files[0].size(), 1);
  2809. ASSERT_EQ(level_to_files[1].size(), 1);
  2810. ASSERT_EQ(level_to_files[2].size(), 1);
  2811. uint64_t l2_size = level_to_files[2][0].fd.GetFileSize();
  2812. uint64_t l1_size = level_to_files[1][0].fd.GetFileSize();
  2813. ASSERT_GT(l2_size, 0);
  2814. ASSERT_GT(l1_size, 0);
  2815. ASSERT_EQ(level_to_files[0][0].compensated_range_deletion_size,
  2816. l1_size + l2_size);
  2817. ASSERT_EQ(level_to_files[1][0].compensated_range_deletion_size, l2_size);
  2818. Reopen(opts);
  2819. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2820. &level_to_files);
  2821. ASSERT_EQ(level_to_files[0].size(), 1);
  2822. ASSERT_EQ(level_to_files[0][0].compensated_range_deletion_size,
  2823. l1_size + l2_size);
  2824. ASSERT_EQ(level_to_files[1].size(), 1);
  2825. ASSERT_EQ(level_to_files[1][0].compensated_range_deletion_size, l2_size);
  2826. }
  2827. TEST_F(DBRangeDelTest, SingleKeyFile) {
  2828. // Test for a bug fix where a range tombstone could be added
  2829. // to an SST file while is not within the file's key range.
  2830. // Create 3 files in L0 and then compact them to L1 where all keys have the
  2831. // same user key `Key(2)`.
  2832. // L0_0: Key(2)@5
  2833. // L0_1: Key(2)@4
  2834. // L0_2: Key(2)@3, range tombstone [Key(2), Key(5))@2
  2835. //
  2836. // After compaction, the first output file contains Key(2)@5 and Key(2)@4.
  2837. // Before fix, the range tombstone [Key(2), Key(5))@2 would be added to this
  2838. // file during compaction, but it is not in this file's key range.
  2839. Options opts = CurrentOptions();
  2840. opts.disable_auto_compactions = true;
  2841. opts.target_file_size_base = 1 << 10;
  2842. DestroyAndReopen(opts);
  2843. // prevent range tombstone drop
  2844. std::vector<const Snapshot*> snapshots;
  2845. snapshots.push_back(db_->GetSnapshot());
  2846. // write a key to bottommost file so the compactions below
  2847. // are not bottommost compactions and will calculate
  2848. // compensated range tombstone size. Before bug fix, an assert would fail
  2849. // during this process.
  2850. Random rnd(301);
  2851. ASSERT_OK(Put(Key(2), rnd.RandomString(8 << 10)));
  2852. ASSERT_OK(Flush());
  2853. MoveFilesToLevel(6);
  2854. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2855. Key(5)));
  2856. snapshots.push_back(db_->GetSnapshot());
  2857. std::vector<std::string> values;
  2858. values.push_back(rnd.RandomString(8 << 10));
  2859. ASSERT_OK(Put(Key(2), rnd.RandomString(8 << 10)));
  2860. snapshots.push_back(db_->GetSnapshot());
  2861. ASSERT_OK(Flush());
  2862. ASSERT_OK(Put(Key(2), rnd.RandomString(8 << 10)));
  2863. snapshots.push_back(db_->GetSnapshot());
  2864. ASSERT_OK(Flush());
  2865. ASSERT_OK(Put(Key(2), rnd.RandomString(8 << 10)));
  2866. snapshots.push_back(db_->GetSnapshot());
  2867. ASSERT_OK(Flush());
  2868. ASSERT_EQ(NumTableFilesAtLevel(0), 3);
  2869. CompactRangeOptions co;
  2870. co.bottommost_level_compaction = BottommostLevelCompaction::kForce;
  2871. ASSERT_OK(dbfull()->RunManualCompaction(
  2872. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  2873. ->cfd(),
  2874. 0, 1, co, nullptr, nullptr, true, true,
  2875. std::numeric_limits<uint64_t>::max() /*max_file_num_to_ignore*/,
  2876. "" /*trim_ts*/));
  2877. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  2878. std::vector<std::vector<FileMetaData>> files;
  2879. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  2880. ASSERT_EQ(files[1][0].largest.Encode(),
  2881. InternalKey(Key(2), 4, kTypeValue).Encode());
  2882. for (const auto s : snapshots) {
  2883. db_->ReleaseSnapshot(s);
  2884. }
  2885. }
  2886. TEST_F(DBRangeDelTest, DoubleCountRangeTombstoneCompensatedSize) {
  2887. // Test for a bug fix if a file has multiple range tombstones
  2888. // with same start and end key but with different sequence numbers,
  2889. // we should only calculate compensated range tombstone size
  2890. // for one of them.
  2891. Options opts = CurrentOptions();
  2892. opts.disable_auto_compactions = true;
  2893. DestroyAndReopen(opts);
  2894. std::vector<std::string> values;
  2895. Random rnd(301);
  2896. // file in L2
  2897. ASSERT_OK(Put(Key(1), rnd.RandomString(1 << 10)));
  2898. ASSERT_OK(Put(Key(2), rnd.RandomString(1 << 10)));
  2899. ASSERT_OK(Flush());
  2900. MoveFilesToLevel(2);
  2901. uint64_t l2_size = 0;
  2902. ASSERT_OK(Size(Key(1), Key(3), 0 /* cf */, &l2_size));
  2903. ASSERT_GT(l2_size, 0);
  2904. // file in L1
  2905. ASSERT_OK(Put(Key(3), rnd.RandomString(1 << 10)));
  2906. ASSERT_OK(Put(Key(4), rnd.RandomString(1 << 10)));
  2907. ASSERT_OK(Flush());
  2908. MoveFilesToLevel(1);
  2909. uint64_t l1_size = 0;
  2910. ASSERT_OK(Size(Key(3), Key(5), 0 /* cf */, &l1_size));
  2911. ASSERT_GT(l1_size, 0);
  2912. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(1),
  2913. Key(5)));
  2914. // so that the range tombstone above is not dropped
  2915. const Snapshot* snapshot = db_->GetSnapshot();
  2916. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(1),
  2917. Key(5)));
  2918. ASSERT_OK(Flush());
  2919. // Range deletion compensated size computed during flush time
  2920. std::vector<std::vector<FileMetaData>> level_to_files;
  2921. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2922. &level_to_files);
  2923. ASSERT_EQ(level_to_files[0].size(), 1);
  2924. // instead of 2 * (l1_size + l2_size)
  2925. ASSERT_EQ(level_to_files[0][0].compensated_range_deletion_size,
  2926. l1_size + l2_size);
  2927. // Range deletion compensated size computed during compaction time
  2928. ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr,
  2929. true /* disallow_trivial_move */));
  2930. dbfull()->TEST_GetFilesMetaData(dbfull()->DefaultColumnFamily(),
  2931. &level_to_files);
  2932. ASSERT_EQ(level_to_files[1].size(), 1);
  2933. ASSERT_EQ(level_to_files[1][0].compensated_range_deletion_size, l2_size);
  2934. db_->ReleaseSnapshot(snapshot);
  2935. }
  2936. TEST_F(DBRangeDelTest, AddRangeDelsSameLowerAndUpperBound) {
  2937. // Test for an edge case where CompactionOutputs::AddRangeDels()
  2938. // is called with an empty range: `range_tombstone_lower_bound_` is not empty
  2939. // and have the same user_key and sequence number as `next_table_min_key.
  2940. // This used to cause file's smallest and largest key to be incorrectly set
  2941. // such that smallest > largest, and fail some assertions in iterator and/or
  2942. // assertion in VersionSet::ApproximateSize().
  2943. Options opts = CurrentOptions();
  2944. opts.disable_auto_compactions = true;
  2945. opts.target_file_size_base = 1 << 10;
  2946. DestroyAndReopen(opts);
  2947. Random rnd(301);
  2948. // Create file at bottommost level so the manual compaction below is
  2949. // non-bottommost level and goes through code path in
  2950. // versions->ApproximateSize() to calculate compensated range tombstone size
  2951. ASSERT_OK(Put(Key(1), "v1"));
  2952. ASSERT_OK(Put(Key(4), "v2"));
  2953. ASSERT_OK(Flush());
  2954. MoveFilesToLevel(6);
  2955. ASSERT_OK(Put(Key(1), rnd.RandomString(4 << 10)));
  2956. ASSERT_OK(Put(Key(3), rnd.RandomString(4 << 10)));
  2957. // So Key(3) does not get dropped.
  2958. const Snapshot* snapshot = db_->GetSnapshot();
  2959. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  2960. Key(4)));
  2961. ASSERT_OK(Flush());
  2962. ASSERT_OK(Put(Key(3), rnd.RandomString(4 << 10)));
  2963. ASSERT_OK(Put(Key(4), rnd.RandomString(4 << 10)));
  2964. ASSERT_OK(Flush());
  2965. MoveFilesToLevel(1);
  2966. // Each file will have two keys, with Key(3) straddle between two files.
  2967. // File 1: Key(1)@1, Key(3)@6, DeleteRange ends at Key(3)@6
  2968. // File 2: Key(3)@4, Key(4)@7, DeleteRange start from Key(3)@4
  2969. ASSERT_EQ(NumTableFilesAtLevel(1), 2);
  2970. std::vector<std::vector<FileMetaData>> files;
  2971. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  2972. ASSERT_EQ(files[1][0].largest.Encode(),
  2973. InternalKey(Key(3), 6, kTypeValue).Encode());
  2974. ASSERT_EQ(files[1][1].smallest.Encode(),
  2975. InternalKey(Key(3), 4, kTypeValue).Encode());
  2976. // Manually update compaction output file cutting decisions
  2977. // to cut before range tombstone sentinel Key(3)@4
  2978. // and the point key Key(3)@4 itself
  2979. SyncPoint::GetInstance()->SetCallBack(
  2980. "CompactionOutputs::ShouldStopBefore::manual_decision", [opts](void* p) {
  2981. auto* pair = (std::pair<bool*, const Slice>*)p;
  2982. if ((opts.comparator->Compare(ExtractUserKey(pair->second), Key(3)) ==
  2983. 0) &&
  2984. (GetInternalKeySeqno(pair->second) <= 4)) {
  2985. *(pair->first) = true;
  2986. }
  2987. });
  2988. SyncPoint::GetInstance()->EnableProcessing();
  2989. std::string begin_key = Key(0);
  2990. std::string end_key = Key(5);
  2991. Slice begin_slice{begin_key};
  2992. Slice end_slice{end_key};
  2993. ASSERT_OK(dbfull()->RunManualCompaction(
  2994. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  2995. ->cfd(),
  2996. 1, 2, CompactRangeOptions(), &begin_slice, &end_slice, true,
  2997. true /* disallow_trivial_move */,
  2998. std::numeric_limits<uint64_t>::max() /*max_file_num_to_ignore*/,
  2999. "" /*trim_ts*/));
  3000. // iterate through to check if any assertion breaks
  3001. std::unique_ptr<Iterator> iter{db_->NewIterator(ReadOptions())};
  3002. iter->SeekToFirst();
  3003. std::vector<int> expected{1, 3, 4};
  3004. for (auto i : expected) {
  3005. ASSERT_TRUE(iter->Valid());
  3006. ASSERT_EQ(iter->key(), Key(i));
  3007. iter->Next();
  3008. }
  3009. ASSERT_TRUE(iter->status().ok() && !iter->Valid());
  3010. db_->ReleaseSnapshot(snapshot);
  3011. }
  3012. TEST_F(DBRangeDelTest, AddRangeDelsSingleUserKeyTombstoneOnlyFile) {
  3013. // Test for an edge case where CompactionOutputs::AddRangeDels()
  3014. // is called with an SST file that has no point keys, and that
  3015. // the lower bound and upper bound have the same user key.
  3016. // This could cause a file's smallest and largest key to be incorrectly set
  3017. // such that smallest > largest, and fail some assertions in iterator and/or
  3018. // assertion in VersionSet::ApproximateSize().
  3019. Options opts = CurrentOptions();
  3020. opts.disable_auto_compactions = true;
  3021. opts.target_file_size_base = 1 << 10;
  3022. DestroyAndReopen(opts);
  3023. Random rnd(301);
  3024. // Create file at bottommost level so the manual compaction below is
  3025. // non-bottommost level and goes through code path like compensate range
  3026. // tombstone size.
  3027. ASSERT_OK(Put(Key(1), "v1"));
  3028. ASSERT_OK(Put(Key(4), "v2"));
  3029. ASSERT_OK(Flush());
  3030. MoveFilesToLevel(6);
  3031. ASSERT_OK(Put(Key(1), rnd.RandomString(10)));
  3032. // Key(3)@4
  3033. ASSERT_OK(Put(Key(3), rnd.RandomString(10)));
  3034. const Snapshot* snapshot1 = db_->GetSnapshot();
  3035. // Key(3)@5
  3036. ASSERT_OK(Put(Key(3), rnd.RandomString(10)));
  3037. const Snapshot* snapshot2 = db_->GetSnapshot();
  3038. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(2),
  3039. Key(4)));
  3040. // Key(3)@7
  3041. ASSERT_OK(Put(Key(3), rnd.RandomString(10)));
  3042. ASSERT_OK(Flush());
  3043. // L0 -> L1 compaction: cut output into two files:
  3044. // File 1: Key(1), Key(3)@7, Range tombstone ends at Key(3)@7
  3045. // File 2: Key(3)@5, Key(3)@4, Range tombstone starts from Key(3)@5
  3046. SyncPoint::GetInstance()->SetCallBack(
  3047. "CompactionOutputs::ShouldStopBefore::manual_decision", [opts](void* p) {
  3048. auto* pair = (std::pair<bool*, const Slice>*)p;
  3049. if ((opts.comparator->Compare(ExtractUserKey(pair->second), Key(3)) ==
  3050. 0) &&
  3051. (GetInternalKeySeqno(pair->second) <= 6)) {
  3052. *(pair->first) = true;
  3053. SyncPoint::GetInstance()->DisableProcessing();
  3054. }
  3055. });
  3056. SyncPoint::GetInstance()->EnableProcessing();
  3057. std::string begin_key = Key(0);
  3058. std::string end_key = Key(5);
  3059. Slice begin_slice{begin_key};
  3060. Slice end_slice{end_key};
  3061. ASSERT_OK(dbfull()->RunManualCompaction(
  3062. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  3063. ->cfd(),
  3064. 0, 1, CompactRangeOptions(), &begin_slice, &end_slice, true,
  3065. true /* disallow_trivial_move */,
  3066. std::numeric_limits<uint64_t>::max() /*max_file_num_to_ignore*/,
  3067. "" /*trim_ts*/));
  3068. ASSERT_EQ(NumTableFilesAtLevel(1), 2);
  3069. // L1 -> L2 compaction, drop the snapshot protecting Key(3)@5.
  3070. // Let ShouldStopBefore() return true for Key(3)@5 (delete range sentinel)
  3071. // and Key(3)@4.
  3072. // Output should have two files:
  3073. // File 1: Key(1), Key(3)@7, range tombstone ends at Key(3)@7
  3074. // File dropped: range tombstone only file (from Key(3)@5 to Key(3)@4)
  3075. // File 2: Range tombstone starting from Key(3)@4, Key(3)@4
  3076. db_->ReleaseSnapshot(snapshot2);
  3077. SyncPoint::GetInstance()->SetCallBack(
  3078. "CompactionOutputs::ShouldStopBefore::manual_decision", [opts](void* p) {
  3079. auto* pair = (std::pair<bool*, const Slice>*)p;
  3080. if ((opts.comparator->Compare(ExtractUserKey(pair->second), Key(3)) ==
  3081. 0) &&
  3082. (GetInternalKeySeqno(pair->second) <= 6)) {
  3083. *(pair->first) = true;
  3084. }
  3085. });
  3086. SyncPoint::GetInstance()->EnableProcessing();
  3087. ASSERT_OK(dbfull()->RunManualCompaction(
  3088. static_cast_with_check<ColumnFamilyHandleImpl>(db_->DefaultColumnFamily())
  3089. ->cfd(),
  3090. 1, 2, CompactRangeOptions(), &begin_slice, &end_slice, true,
  3091. true /* disallow_trivial_move */,
  3092. std::numeric_limits<uint64_t>::max() /*max_file_num_to_ignore*/,
  3093. "" /*trim_ts*/));
  3094. ASSERT_EQ(NumTableFilesAtLevel(2), 2);
  3095. // iterate through to check if any assertion breaks
  3096. std::unique_ptr<Iterator> iter{db_->NewIterator(ReadOptions())};
  3097. iter->SeekToFirst();
  3098. std::vector<int> expected{1, 3, 4};
  3099. for (auto i : expected) {
  3100. ASSERT_TRUE(iter->Valid());
  3101. ASSERT_EQ(iter->key(), Key(i));
  3102. iter->Next();
  3103. }
  3104. ASSERT_TRUE(iter->status().ok() && !iter->Valid());
  3105. db_->ReleaseSnapshot(snapshot1);
  3106. }
  3107. TEST_F(DBRangeDelTest, NonBottommostCompactionDropRangetombstone) {
  3108. // L0: file 1: [DeleteRange[4, 5)], file 2: [3, 6, DeleteRange[8, 9)]
  3109. // L6 file 1: [2, 3], file 2: [7, 8]
  3110. // When compacting the two L0 files to L1, the compaction is non-bottommost
  3111. // since the compaction key range overlaps with L6 file 1. The range tombstone
  3112. // [4, 5) should be dropped since it does not overlap with any file in lower
  3113. // levels. The range tombstone [8, 9) should not be dropped.
  3114. Options opts = CurrentOptions();
  3115. opts.level_compaction_dynamic_level_bytes = false;
  3116. opts.num_levels = 7;
  3117. opts.level0_file_num_compaction_trigger = 3;
  3118. DestroyAndReopen(opts);
  3119. Random rnd(301);
  3120. // L6 file 1
  3121. ASSERT_OK(Put(Key(2), rnd.RandomString(100)));
  3122. ASSERT_OK(Put(Key(3), rnd.RandomString(100)));
  3123. ASSERT_OK(Flush());
  3124. // L6 file 2
  3125. ASSERT_OK(Put(Key(7), rnd.RandomString(100)));
  3126. ASSERT_OK(Put(Key(8), rnd.RandomString(100)));
  3127. ASSERT_OK(Flush());
  3128. MoveFilesToLevel(6);
  3129. ASSERT_EQ(NumTableFilesAtLevel(6), 2);
  3130. // L0 file 1
  3131. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(4),
  3132. Key(5)));
  3133. ASSERT_OK(Flush());
  3134. // L0 file 2
  3135. ASSERT_OK(Put(Key(3), rnd.RandomString(100)));
  3136. ASSERT_OK(Put(Key(6), rnd.RandomString(100)));
  3137. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(8),
  3138. Key(9)));
  3139. ASSERT_OK(Flush());
  3140. // nothing is dropped during flush
  3141. std::string property;
  3142. db_->GetProperty(DB::Properties::kAggregatedTableProperties, &property);
  3143. TableProperties output_tp;
  3144. ParseTablePropertiesString(property, &output_tp);
  3145. ASSERT_EQ(output_tp.num_range_deletions, 2);
  3146. // Add one more L0 file to trigger L0->L1 compaction
  3147. ASSERT_OK(Put(Key(1), rnd.RandomString(100)));
  3148. ASSERT_OK(Put(Key(9), rnd.RandomString(100)));
  3149. ASSERT_OK(Flush());
  3150. ASSERT_OK(dbfull()->TEST_WaitForCompact());
  3151. ASSERT_EQ(NumTableFilesAtLevel(1), 1);
  3152. db_->GetProperty(DB::Properties::kAggregatedTableProperties, &property);
  3153. ParseTablePropertiesString(property, &output_tp);
  3154. ASSERT_EQ(output_tp.num_range_deletions, 1);
  3155. // Now create a snapshot protected range tombstone [4, 5), it should not
  3156. // be dropped.
  3157. ASSERT_OK(Put(Key(4), rnd.RandomString(100)));
  3158. const Snapshot* snapshot = db_->GetSnapshot();
  3159. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(4),
  3160. Key(5)));
  3161. CompactRangeOptions cro;
  3162. cro.bottommost_level_compaction = BottommostLevelCompaction::kForceOptimized;
  3163. ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr));
  3164. // All compacted to L6
  3165. ASSERT_EQ("0,0,0,0,0,0,1", FilesPerLevel(0));
  3166. db_->GetProperty(DB::Properties::kAggregatedTableProperties, &property);
  3167. ParseTablePropertiesString(property, &output_tp);
  3168. ASSERT_EQ(output_tp.num_range_deletions, 1);
  3169. db_->ReleaseSnapshot(snapshot);
  3170. }
  3171. TEST_F(DBRangeDelTest, MemtableMaxRangeDeletions) {
  3172. // Tests option `memtable_max_range_deletions`.
  3173. Options options = CurrentOptions();
  3174. options.memtable_max_range_deletions = 50;
  3175. options.level0_file_num_compaction_trigger = 5;
  3176. DestroyAndReopen(options);
  3177. for (int i = 0; i < 50; ++i) {
  3178. // Intentionally delete overlapping ranges to see if the option
  3179. // checks number of range tombstone fragments instead.
  3180. ASSERT_OK(Put(Key(i), "val1"));
  3181. ASSERT_OK(Put(Key(i + 1), "val2"));
  3182. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  3183. Key(i), Key(i + 2)));
  3184. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3185. ASSERT_EQ(0, NumTableFilesAtLevel(0));
  3186. }
  3187. // One more write to trigger flush.
  3188. ASSERT_OK(Put(Key(50), "val"));
  3189. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3190. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  3191. // This should take effect for the next new memtable.
  3192. ASSERT_OK(db_->SetOptions({{"memtable_max_range_deletions", "1"}}));
  3193. ASSERT_OK(Flush());
  3194. ASSERT_EQ(2, NumTableFilesAtLevel(0));
  3195. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  3196. Key(50), Key(100)));
  3197. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3198. ASSERT_EQ(2, NumTableFilesAtLevel(0));
  3199. // One more write to trigger flush.
  3200. ASSERT_OK(Put(Key(50), "new val"));
  3201. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3202. ASSERT_EQ(3, NumTableFilesAtLevel(0));
  3203. }
  3204. TEST_F(DBRangeDelTest, RangeDelReseekAfterFileReadError) {
  3205. // This is to test a bug that is fixed in
  3206. // https://github.com/facebook/rocksdb/pull/11786.
  3207. Options opts = CurrentOptions();
  3208. opts.num_levels = 7;
  3209. // Set up LSM
  3210. //
  3211. // L4: F1: [key1] F2: [key2]
  3212. // L5: F3:[DeleteRange(key3, key6)]
  3213. // L6: F4:[key3, key6]
  3214. // Will inject error when reading from F2.
  3215. // SeekToFirst() should land on key1.
  3216. // Next() should encounter error when reading from F2,
  3217. // and range del reseek should not reset this status.
  3218. Random rnd(301);
  3219. // L6
  3220. ASSERT_OK(Put(Key(3), rnd.RandomString(100)));
  3221. ASSERT_OK(Put(Key(6), rnd.RandomString(100)));
  3222. ASSERT_OK(Flush());
  3223. MoveFilesToLevel(6);
  3224. // L5
  3225. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  3226. Key(6)));
  3227. ASSERT_OK(Flush());
  3228. MoveFilesToLevel(5);
  3229. // L4
  3230. ASSERT_OK(Put(Key(2), rnd.RandomString(100)));
  3231. ASSERT_OK(Flush());
  3232. MoveFilesToLevel(4);
  3233. std::string fname;
  3234. std::vector<LiveFileMetaData> live_files;
  3235. db_->GetLiveFilesMetaData(&live_files);
  3236. for (auto& meta : live_files) {
  3237. if (meta.level == 4) {
  3238. fname = meta.name;
  3239. break;
  3240. }
  3241. }
  3242. ASSERT_TRUE(!fname.empty());
  3243. ASSERT_OK(Put(Key(1), rnd.RandomString(100)));
  3244. ASSERT_OK(Flush());
  3245. MoveFilesToLevel(4);
  3246. SyncPoint::GetInstance()->SetCallBack(
  3247. "RandomAccessFileReader::Read::BeforeReturn", [&fname](void* pair_ptr) {
  3248. auto p = static_cast<std::pair<std::string*, IOStatus*>*>(pair_ptr);
  3249. if (p->first->find(fname) != std::string::npos) {
  3250. *p->second = IOStatus::IOError();
  3251. p->second->SetRetryable(true);
  3252. }
  3253. });
  3254. SyncPoint::GetInstance()->EnableProcessing();
  3255. std::unique_ptr<Iterator> iter{db_->NewIterator(ReadOptions())};
  3256. iter->SeekToFirst();
  3257. ASSERT_TRUE(iter->Valid());
  3258. ASSERT_OK(iter->status());
  3259. ASSERT_EQ(iter->key(), Key(1));
  3260. iter->Next();
  3261. ASSERT_FALSE(iter->Valid());
  3262. ASSERT_NOK(iter->status());
  3263. ASSERT_TRUE(iter->status().IsIOError());
  3264. iter.reset();
  3265. SyncPoint::GetInstance()->ClearAllCallBacks();
  3266. SyncPoint::GetInstance()->DisableProcessing();
  3267. // Reverse scan
  3268. // LSM setup
  3269. // L4: F1: [key2] F2: [key7, key8]
  3270. // L5: F3:[[key3, key6)]
  3271. // L6: F4:[key1, key5]
  3272. // Ingest error when read from F1.
  3273. // SeekToLast() should land on key8.
  3274. // During Prev(), MergingIterator will encounter error when reading from F1
  3275. // and do a range del reseek (it sees key5 covered by a range tombstone).
  3276. DestroyAndReopen(opts);
  3277. // L6
  3278. ASSERT_OK(Put(Key(1), rnd.RandomString(100)));
  3279. ASSERT_OK(Put(Key(5), rnd.RandomString(100)));
  3280. ASSERT_OK(Flush());
  3281. MoveFilesToLevel(6);
  3282. // L5
  3283. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  3284. Key(6)));
  3285. ASSERT_OK(Flush());
  3286. MoveFilesToLevel(5);
  3287. // L4
  3288. ASSERT_OK(Put(Key(2), rnd.RandomString(100)));
  3289. ASSERT_OK(Flush());
  3290. MoveFilesToLevel(4);
  3291. live_files.clear();
  3292. db_->GetLiveFilesMetaData(&live_files);
  3293. for (auto& meta : live_files) {
  3294. if (meta.level == 4) {
  3295. fname = meta.name;
  3296. break;
  3297. }
  3298. }
  3299. ASSERT_TRUE(!fname.empty());
  3300. ASSERT_OK(Put(Key(7), rnd.RandomString(100)));
  3301. ASSERT_OK(Put(Key(8), rnd.RandomString(100)));
  3302. ASSERT_OK(Flush());
  3303. MoveFilesToLevel(4);
  3304. SyncPoint::GetInstance()->SetCallBack(
  3305. "RandomAccessFileReader::Read::AnyOffset", [&fname](void* pair_ptr) {
  3306. auto p = static_cast<std::pair<std::string*, IOStatus*>*>(pair_ptr);
  3307. if (p->first->find(fname) != std::string::npos) {
  3308. *p->second = IOStatus::IOError();
  3309. p->second->SetRetryable(true);
  3310. }
  3311. });
  3312. SyncPoint::GetInstance()->EnableProcessing();
  3313. iter.reset(db_->NewIterator(ReadOptions()));
  3314. iter->SeekToLast();
  3315. ASSERT_TRUE(iter->Valid());
  3316. ASSERT_OK(iter->status());
  3317. ASSERT_EQ(iter->key(), Key(8));
  3318. // Note that for reverse scan, DBIter will need to ensure
  3319. // the key it returns is the one with the highest sequence number.
  3320. // To return key7, it internally calls MergingIterator::Prev()
  3321. // until it reaches a previous user key.
  3322. iter->Prev();
  3323. ASSERT_FALSE(iter->Valid());
  3324. ASSERT_NOK(iter->status());
  3325. ASSERT_TRUE(iter->status().IsIOError());
  3326. iter.reset();
  3327. }
  3328. TEST_F(DBRangeDelTest, ReleaseSnapshotAfterIteratorCreation) {
  3329. // Test that range tombstone code path in LevelIterator
  3330. // does access ReadOptions::snapshot after Iterator creation.
  3331. //
  3332. // Put some data in L2 so that range tombstone in L1 will not be dropped.
  3333. ASSERT_OK(Put(Key(0), "v"));
  3334. ASSERT_OK(Put(Key(100), "v"));
  3335. ASSERT_OK(Flush());
  3336. MoveFilesToLevel(2);
  3337. // two L1 file with range del
  3338. ASSERT_OK(Put(Key(1), "v"));
  3339. ASSERT_OK(Put(Key(2), "v"));
  3340. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  3341. Key(4)));
  3342. ASSERT_OK(Flush());
  3343. MoveFilesToLevel(1);
  3344. ASSERT_OK(Put(Key(5), "v"));
  3345. ASSERT_OK(Put(Key(6), "v"));
  3346. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(5),
  3347. Key(6)));
  3348. ASSERT_OK(Flush());
  3349. MoveFilesToLevel(1);
  3350. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  3351. ASSERT_EQ(1, NumTableFilesAtLevel(2));
  3352. const Snapshot* snapshot = db_->GetSnapshot();
  3353. ReadOptions ro;
  3354. ro.snapshot = snapshot;
  3355. Iterator* iter = db_->NewIterator(ro);
  3356. db_->ReleaseSnapshot(snapshot);
  3357. iter->Seek(Key(1));
  3358. std::vector<int> expected_keys{1, 2, 6, 100};
  3359. for (int i : expected_keys) {
  3360. ASSERT_OK(iter->status());
  3361. ASSERT_TRUE(iter->Valid());
  3362. ASSERT_EQ(iter->key(), Key(i));
  3363. iter->Next();
  3364. }
  3365. ASSERT_TRUE(!iter->Valid() && iter->status().ok());
  3366. delete iter;
  3367. }
  3368. TEST_F(DBRangeDelTest, RefreshWithSnapshot) {
  3369. ASSERT_OK(Put(Key(4), "4"));
  3370. ASSERT_OK(Put(Key(6), "6"));
  3371. const Snapshot* snapshot = db_->GetSnapshot();
  3372. ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), Key(3),
  3373. Key(5)));
  3374. std::unique_ptr<Iterator> iter{db_->NewIterator(ReadOptions())};
  3375. // Live Memtable
  3376. iter->SeekToFirst();
  3377. ASSERT_TRUE(iter->Valid());
  3378. ASSERT_EQ(iter->key(), Key(6));
  3379. ASSERT_OK(iter->Refresh(snapshot));
  3380. iter->SeekToFirst();
  3381. ASSERT_TRUE(iter->Valid());
  3382. ASSERT_EQ(iter->key(), Key(4));
  3383. // Immutable Memtable
  3384. ASSERT_OK(dbfull()->TEST_SwitchMemtable());
  3385. ASSERT_OK(iter->Refresh(nullptr));
  3386. iter->SeekToFirst();
  3387. ASSERT_TRUE(iter->Valid());
  3388. ASSERT_EQ(iter->key(), Key(6));
  3389. ASSERT_OK(iter->Refresh(snapshot));
  3390. iter->SeekToFirst();
  3391. ASSERT_TRUE(iter->Valid());
  3392. ASSERT_EQ(iter->key(), Key(4));
  3393. // L0
  3394. ASSERT_OK(Flush());
  3395. ASSERT_EQ(1, NumTableFilesAtLevel(0));
  3396. ASSERT_OK(iter->Refresh(nullptr));
  3397. iter->SeekToFirst();
  3398. ASSERT_TRUE(iter->Valid());
  3399. ASSERT_EQ(iter->key(), Key(6));
  3400. ASSERT_OK(iter->Refresh(snapshot));
  3401. iter->SeekToFirst();
  3402. ASSERT_TRUE(iter->Valid());
  3403. ASSERT_EQ(iter->key(), Key(4));
  3404. // L1
  3405. MoveFilesToLevel(1);
  3406. ASSERT_EQ(1, NumTableFilesAtLevel(1));
  3407. ASSERT_OK(iter->Refresh(nullptr));
  3408. iter->SeekToFirst();
  3409. ASSERT_TRUE(iter->Valid());
  3410. ASSERT_EQ(iter->key(), Key(6));
  3411. ASSERT_OK(iter->Refresh(snapshot));
  3412. iter->SeekToFirst();
  3413. ASSERT_TRUE(iter->Valid());
  3414. ASSERT_EQ(iter->key(), Key(4));
  3415. // L1 with two file.
  3416. // Test that when LevelIterator enters a new file,
  3417. // it remembers which snapshot sequence number to use.
  3418. ASSERT_OK(Put(Key(2), "2"));
  3419. ASSERT_OK(Flush());
  3420. MoveFilesToLevel(1);
  3421. ASSERT_EQ(2, NumTableFilesAtLevel(1));
  3422. ASSERT_OK(iter->Refresh(nullptr));
  3423. iter->SeekToFirst();
  3424. ASSERT_TRUE(iter->Valid());
  3425. // LevelIterator is at the first file
  3426. ASSERT_EQ(iter->key(), Key(2));
  3427. ASSERT_OK(iter->Refresh(snapshot));
  3428. // Will enter the second file, and create a new range tombstone iterator.
  3429. // It should use the snapshot sequence number.
  3430. iter->SeekToFirst();
  3431. ASSERT_TRUE(iter->Valid());
  3432. ASSERT_EQ(iter->key(), Key(4));
  3433. iter.reset();
  3434. db_->ReleaseSnapshot(snapshot);
  3435. }
  3436. TEST_F(DBRangeDelTest, RowCache) {
  3437. Options options = CurrentOptions();
  3438. options.row_cache = NewLRUCache(8 << 10);
  3439. DestroyAndReopen(options);
  3440. ASSERT_OK(Put(Key(3), "val"));
  3441. ASSERT_TRUE(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(),
  3442. Key(3), Key(5))
  3443. .IsNotSupported());
  3444. WriteBatch wb;
  3445. ASSERT_OK(wb.Put(Key(6), "abc"));
  3446. ASSERT_OK(wb.DeleteRange(Key(1), Key(5)));
  3447. ASSERT_TRUE(db_->Write(WriteOptions(), &wb).IsNotSupported());
  3448. ASSERT_EQ(Get(Key(3)), "val");
  3449. // By default, memtable insertion failure will turn the DB to read-only mode.
  3450. // The check for delete range should happen before that to fail early
  3451. // and should not turn db into read-only mdoe.
  3452. ASSERT_OK(Put(Key(5), "foo"));
  3453. }
  3454. } // namespace ROCKSDB_NAMESPACE
  3455. int main(int argc, char** argv) {
  3456. ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
  3457. ::testing::InitGoogleTest(&argc, argv);
  3458. return RUN_ALL_TESTS();
  3459. }