column_family_test.cc 141 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659266026612662266326642665266626672668266926702671267226732674267526762677267826792680268126822683268426852686268726882689269026912692269326942695269626972698269927002701270227032704270527062707270827092710271127122713271427152716271727182719272027212722272327242725272627272728272927302731273227332734273527362737273827392740274127422743274427452746274727482749275027512752275327542755275627572758275927602761276227632764276527662767276827692770277127722773277427752776277727782779278027812782278327842785278627872788278927902791279227932794279527962797279827992800280128022803280428052806280728082809281028112812281328142815281628172818281928202821282228232824282528262827282828292830283128322833283428352836283728382839284028412842284328442845284628472848284928502851285228532854285528562857285828592860286128622863286428652866286728682869287028712872287328742875287628772878287928802881288228832884288528862887288828892890289128922893289428952896289728982899290029012902290329042905290629072908290929102911291229132914291529162917291829192920292129222923292429252926292729282929293029312932293329342935293629372938293929402941294229432944294529462947294829492950295129522953295429552956295729582959296029612962296329642965296629672968296929702971297229732974297529762977297829792980298129822983298429852986298729882989299029912992299329942995299629972998299930003001300230033004300530063007300830093010301130123013301430153016301730183019302030213022302330243025302630273028302930303031303230333034303530363037303830393040304130423043304430453046304730483049305030513052305330543055305630573058305930603061306230633064306530663067306830693070307130723073307430753076307730783079308030813082308330843085308630873088308930903091309230933094309530963097309830993100310131023103310431053106310731083109311031113112311331143115311631173118311931203121312231233124312531263127312831293130313131323133313431353136313731383139314031413142314331443145314631473148314931503151315231533154315531563157315831593160316131623163316431653166316731683169317031713172317331743175317631773178317931803181318231833184318531863187318831893190319131923193319431953196319731983199320032013202320332043205320632073208320932103211321232133214321532163217321832193220322132223223322432253226322732283229323032313232323332343235323632373238323932403241324232433244324532463247324832493250325132523253325432553256325732583259326032613262326332643265326632673268326932703271327232733274327532763277327832793280328132823283328432853286328732883289329032913292329332943295329632973298329933003301330233033304330533063307330833093310331133123313331433153316331733183319332033213322332333243325332633273328332933303331333233333334333533363337333833393340334133423343334433453346334733483349335033513352335333543355335633573358335933603361336233633364336533663367336833693370337133723373337433753376337733783379338033813382338333843385338633873388338933903391339233933394339533963397339833993400340134023403340434053406340734083409341034113412341334143415341634173418341934203421342234233424342534263427342834293430343134323433343434353436343734383439344034413442344334443445344634473448344934503451345234533454345534563457345834593460346134623463346434653466346734683469347034713472347334743475347634773478347934803481348234833484348534863487348834893490349134923493349434953496349734983499350035013502350335043505350635073508350935103511351235133514351535163517351835193520352135223523352435253526352735283529353035313532353335343535353635373538353935403541354235433544354535463547354835493550355135523553355435553556355735583559356035613562356335643565356635673568356935703571357235733574357535763577357835793580358135823583358435853586358735883589359035913592359335943595359635973598359936003601360236033604360536063607360836093610361136123613361436153616361736183619362036213622362336243625362636273628362936303631363236333634363536363637363836393640364136423643364436453646364736483649365036513652365336543655365636573658365936603661366236633664366536663667366836693670367136723673367436753676367736783679368036813682368336843685368636873688368936903691369236933694369536963697369836993700370137023703370437053706370737083709371037113712371337143715371637173718371937203721372237233724372537263727372837293730373137323733373437353736373737383739374037413742374337443745374637473748374937503751375237533754375537563757375837593760376137623763376437653766376737683769377037713772377337743775377637773778377937803781378237833784378537863787378837893790379137923793379437953796379737983799380038013802380338043805380638073808380938103811381238133814381538163817381838193820382138223823382438253826382738283829383038313832383338343835383638373838383938403841384238433844384538463847384838493850385138523853385438553856385738583859386038613862386338643865386638673868386938703871387238733874387538763877387838793880388138823883388438853886388738883889389038913892389338943895389638973898389939003901390239033904390539063907390839093910391139123913391439153916391739183919392039213922392339243925392639273928392939303931393239333934393539363937393839393940394139423943394439453946394739483949395039513952395339543955395639573958395939603961396239633964396539663967396839693970397139723973397439753976397739783979398039813982398339843985398639873988398939903991399239933994399539963997399839994000400140024003400440054006400740084009401040114012401340144015401640174018401940204021402240234024402540264027402840294030403140324033403440354036403740384039404040414042404340444045404640474048404940504051405240534054405540564057405840594060406140624063406440654066406740684069407040714072407340744075407640774078407940804081408240834084
  1. // Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
  2. // This source code is licensed under both the GPLv2 (found in the
  3. // COPYING file in the root directory) and Apache 2.0 License
  4. // (found in the LICENSE.Apache file in the root directory).
  5. //
  6. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
  7. // Use of this source code is governed by a BSD-style license that can be
  8. // found in the LICENSE file. See the AUTHORS file for names of contributors.
  9. #include <algorithm>
  10. #include <atomic>
  11. #include <string>
  12. #include <thread>
  13. #include <vector>
  14. #include "db/db_impl/db_impl.h"
  15. #include "db/db_test_util.h"
  16. #include "options/options_parser.h"
  17. #include "port/port.h"
  18. #include "port/stack_trace.h"
  19. #include "rocksdb/comparator.h"
  20. #include "rocksdb/convenience.h"
  21. #include "rocksdb/db.h"
  22. #include "rocksdb/env.h"
  23. #include "rocksdb/iterator.h"
  24. #include "rocksdb/listener.h"
  25. #include "rocksdb/utilities/object_registry.h"
  26. #include "test_util/sync_point.h"
  27. #include "test_util/testharness.h"
  28. #include "test_util/testutil.h"
  29. #include "util/coding.h"
  30. #include "util/defer.h"
  31. #include "util/string_util.h"
  32. #include "utilities/fault_injection_env.h"
  33. #include "utilities/merge_operators.h"
  34. namespace ROCKSDB_NAMESPACE {
  35. namespace {
  36. std::string EncodeAsUint64(uint64_t number) {
  37. std::string result;
  38. PutFixed64(&result, number);
  39. return result;
  40. }
  41. } // namespace
  42. static const int kValueSize = 1000;
  43. // counts how many operations were performed
  44. class EnvCounter : public SpecialEnv {
  45. public:
  46. explicit EnvCounter(Env* base)
  47. : SpecialEnv(base), num_new_writable_file_(0) {}
  48. int GetNumberOfNewWritableFileCalls() { return num_new_writable_file_; }
  49. Status NewWritableFile(const std::string& f, std::unique_ptr<WritableFile>* r,
  50. const EnvOptions& soptions) override {
  51. ++num_new_writable_file_;
  52. return EnvWrapper::NewWritableFile(f, r, soptions);
  53. }
  54. private:
  55. std::atomic<int> num_new_writable_file_;
  56. };
  57. class ColumnFamilyTestBase : public testing::Test {
  58. public:
  59. explicit ColumnFamilyTestBase(uint32_t format) : rnd_(139), format_(format) {
  60. Env* base_env = Env::Default();
  61. EXPECT_OK(
  62. test::CreateEnvFromSystem(ConfigOptions(), &base_env, &env_guard_));
  63. EXPECT_NE(nullptr, base_env);
  64. env_ = new EnvCounter(base_env);
  65. env_->skip_fsync_ = true;
  66. dbname_ = test::PerThreadDBPath("column_family_test");
  67. db_options_.create_if_missing = true;
  68. db_options_.env = env_;
  69. }
  70. void SetUp() override {
  71. EXPECT_OK(DestroyDB(dbname_, Options(db_options_, column_family_options_)));
  72. }
  73. ~ColumnFamilyTestBase() override {
  74. std::vector<ColumnFamilyDescriptor> column_families;
  75. for (auto h : handles_) {
  76. ColumnFamilyDescriptor cfdescriptor;
  77. Status s = h->GetDescriptor(&cfdescriptor);
  78. EXPECT_OK(s);
  79. column_families.push_back(cfdescriptor);
  80. }
  81. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  82. Destroy(column_families);
  83. delete env_;
  84. }
  85. BlockBasedTableOptions GetBlockBasedTableOptions() {
  86. BlockBasedTableOptions options;
  87. options.format_version = format_;
  88. return options;
  89. }
  90. // Return the value to associate with the specified key
  91. Slice Value(int k, std::string* storage) {
  92. if (k == 0) {
  93. // Ugh. Random seed of 0 used to produce no entropy. This code
  94. // preserves the implementation that was in place when all of the
  95. // magic values in this file were picked.
  96. *storage = std::string(kValueSize, ' ');
  97. } else {
  98. Random r(k);
  99. *storage = r.RandomString(kValueSize);
  100. }
  101. return Slice(*storage);
  102. }
  103. void Build(int base, int n, int flush_every = 0) {
  104. std::string key_space, value_space;
  105. WriteBatch batch;
  106. for (int i = 0; i < n; i++) {
  107. if (flush_every != 0 && i != 0 && i % flush_every == 0) {
  108. DBImpl* dbi = static_cast_with_check<DBImpl>(db_);
  109. dbi->TEST_FlushMemTable();
  110. }
  111. int keyi = base + i;
  112. Slice key(DBTestBase::Key(keyi));
  113. batch.Clear();
  114. batch.Put(handles_[0], key, Value(keyi, &value_space));
  115. batch.Put(handles_[1], key, Value(keyi, &value_space));
  116. batch.Put(handles_[2], key, Value(keyi, &value_space));
  117. ASSERT_OK(db_->Write(WriteOptions(), &batch));
  118. }
  119. }
  120. void CheckMissed() {
  121. uint64_t next_expected = 0;
  122. uint64_t missed = 0;
  123. int bad_keys = 0;
  124. int bad_values = 0;
  125. int correct = 0;
  126. std::string value_space;
  127. for (int cf = 0; cf < 3; cf++) {
  128. next_expected = 0;
  129. Iterator* iter = db_->NewIterator(ReadOptions(false, true), handles_[cf]);
  130. for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
  131. uint64_t key;
  132. Slice in(iter->key());
  133. in.remove_prefix(3);
  134. if (!ConsumeDecimalNumber(&in, &key) || !in.empty() ||
  135. key < next_expected) {
  136. bad_keys++;
  137. continue;
  138. }
  139. missed += (key - next_expected);
  140. next_expected = key + 1;
  141. if (iter->value() != Value(static_cast<int>(key), &value_space)) {
  142. bad_values++;
  143. } else {
  144. correct++;
  145. }
  146. }
  147. delete iter;
  148. }
  149. ASSERT_EQ(0, bad_keys);
  150. ASSERT_EQ(0, bad_values);
  151. ASSERT_EQ(0, missed);
  152. (void)correct;
  153. }
  154. void Close() {
  155. for (auto h : handles_) {
  156. if (h) {
  157. ASSERT_OK(db_->DestroyColumnFamilyHandle(h));
  158. }
  159. }
  160. handles_.clear();
  161. names_.clear();
  162. delete db_;
  163. db_ = nullptr;
  164. }
  165. Status TryOpen(std::vector<std::string> cf,
  166. std::vector<ColumnFamilyOptions> options = {}) {
  167. std::vector<ColumnFamilyDescriptor> column_families;
  168. names_.clear();
  169. for (size_t i = 0; i < cf.size(); ++i) {
  170. column_families.emplace_back(
  171. cf[i], options.size() == 0 ? column_family_options_ : options[i]);
  172. names_.push_back(cf[i]);
  173. }
  174. return DB::Open(db_options_, dbname_, column_families, &handles_, &db_);
  175. }
  176. Status OpenReadOnly(std::vector<std::string> cf,
  177. std::vector<ColumnFamilyOptions> options = {}) {
  178. std::vector<ColumnFamilyDescriptor> column_families;
  179. names_.clear();
  180. for (size_t i = 0; i < cf.size(); ++i) {
  181. column_families.emplace_back(
  182. cf[i], options.size() == 0 ? column_family_options_ : options[i]);
  183. names_.push_back(cf[i]);
  184. }
  185. return DB::OpenForReadOnly(db_options_, dbname_, column_families, &handles_,
  186. &db_);
  187. }
  188. void AssertOpenReadOnly(std::vector<std::string> cf,
  189. std::vector<ColumnFamilyOptions> options = {}) {
  190. ASSERT_OK(OpenReadOnly(cf, options));
  191. }
  192. void Open(std::vector<std::string> cf,
  193. std::vector<ColumnFamilyOptions> options = {}) {
  194. ASSERT_OK(TryOpen(cf, options));
  195. }
  196. void Open() { Open({"default"}); }
  197. DBImpl* dbfull() { return static_cast_with_check<DBImpl>(db_); }
  198. int GetProperty(int cf, std::string property) {
  199. std::string value;
  200. EXPECT_TRUE(dbfull()->GetProperty(handles_[cf], property, &value));
  201. #ifndef CYGWIN
  202. return std::stoi(value);
  203. #else
  204. return std::strtol(value.c_str(), 0 /* off */, 10 /* base */);
  205. #endif
  206. }
  207. bool IsDbWriteStopped() {
  208. uint64_t v;
  209. EXPECT_TRUE(dbfull()->GetIntProperty("rocksdb.is-write-stopped", &v));
  210. return (v == 1);
  211. }
  212. uint64_t GetDbDelayedWriteRate() {
  213. uint64_t v;
  214. EXPECT_TRUE(
  215. dbfull()->GetIntProperty("rocksdb.actual-delayed-write-rate", &v));
  216. return v;
  217. }
  218. void Destroy(const std::vector<ColumnFamilyDescriptor>& column_families =
  219. std::vector<ColumnFamilyDescriptor>()) {
  220. Close();
  221. ASSERT_OK(DestroyDB(dbname_, Options(db_options_, column_family_options_),
  222. column_families));
  223. }
  224. void CreateColumnFamilies(
  225. const std::vector<std::string>& cfs,
  226. const std::vector<ColumnFamilyOptions> options = {}) {
  227. int cfi = static_cast<int>(handles_.size());
  228. handles_.resize(cfi + cfs.size());
  229. names_.resize(cfi + cfs.size());
  230. for (size_t i = 0; i < cfs.size(); ++i) {
  231. const auto& current_cf_opt =
  232. options.size() == 0 ? column_family_options_ : options[i];
  233. ASSERT_OK(
  234. db_->CreateColumnFamily(current_cf_opt, cfs[i], &handles_[cfi]));
  235. names_[cfi] = cfs[i];
  236. // Verify the CF options of the returned CF handle.
  237. ColumnFamilyDescriptor desc;
  238. ASSERT_OK(handles_[cfi]->GetDescriptor(&desc));
  239. // Need to sanitize the default column family options before comparing
  240. // them.
  241. ASSERT_OK(RocksDBOptionsParser::VerifyCFOptions(
  242. ConfigOptions(), desc.options,
  243. SanitizeCfOptions(dbfull()->immutable_db_options(),
  244. /*read_only*/ false, current_cf_opt)));
  245. cfi++;
  246. }
  247. }
  248. void Reopen(const std::vector<ColumnFamilyOptions> options = {}) {
  249. std::vector<std::string> names;
  250. for (const auto& name : names_) {
  251. if (name != "") {
  252. names.push_back(name);
  253. }
  254. }
  255. Close();
  256. assert(options.size() == 0 || names.size() == options.size());
  257. Open(names, options);
  258. }
  259. void CreateColumnFamiliesAndReopen(const std::vector<std::string>& cfs) {
  260. CreateColumnFamilies(cfs);
  261. Reopen();
  262. }
  263. void DropColumnFamilies(const std::vector<int>& cfs) {
  264. for (auto cf : cfs) {
  265. ASSERT_OK(db_->DropColumnFamily(handles_[cf]));
  266. ASSERT_OK(db_->DestroyColumnFamilyHandle(handles_[cf]));
  267. handles_[cf] = nullptr;
  268. names_[cf] = "";
  269. }
  270. }
  271. void PutRandomData(int cf, int num, int key_value_size, bool save = false) {
  272. if (cf >= static_cast<int>(keys_.size())) {
  273. keys_.resize(cf + 1);
  274. }
  275. for (int i = 0; i < num; ++i) {
  276. // 10 bytes for key, rest is value
  277. if (!save) {
  278. ASSERT_OK(Put(cf, test::RandomKey(&rnd_, 11),
  279. rnd_.RandomString(key_value_size - 10)));
  280. } else {
  281. std::string key = test::RandomKey(&rnd_, 11);
  282. keys_[cf].insert(key);
  283. ASSERT_OK(Put(cf, key, rnd_.RandomString(key_value_size - 10)));
  284. }
  285. }
  286. ASSERT_OK(db_->FlushWAL(/*sync=*/false));
  287. }
  288. void WaitForFlush(int cf) {
  289. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable(handles_[cf]));
  290. }
  291. void WaitForCompaction() { ASSERT_OK(dbfull()->TEST_WaitForCompact()); }
  292. uint64_t MaxTotalInMemoryState() {
  293. return dbfull()->TEST_MaxTotalInMemoryState();
  294. }
  295. void AssertMaxTotalInMemoryState(uint64_t value) {
  296. ASSERT_EQ(value, MaxTotalInMemoryState());
  297. }
  298. Status Put(int cf, const std::string& key, const std::string& value) {
  299. return db_->Put(WriteOptions(), handles_[cf], Slice(key), Slice(value));
  300. }
  301. Status Merge(int cf, const std::string& key, const std::string& value) {
  302. return db_->Merge(WriteOptions(), handles_[cf], Slice(key), Slice(value));
  303. }
  304. Status Flush(int cf) { return db_->Flush(FlushOptions(), handles_[cf]); }
  305. std::string Get(int cf, const std::string& key) {
  306. ReadOptions options;
  307. options.verify_checksums = true;
  308. std::string result;
  309. Status s = db_->Get(options, handles_[cf], Slice(key), &result);
  310. if (s.IsNotFound()) {
  311. result = "NOT_FOUND";
  312. } else if (!s.ok()) {
  313. result = s.ToString();
  314. }
  315. return result;
  316. }
  317. void CompactAll(int cf) {
  318. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), handles_[cf], nullptr,
  319. nullptr));
  320. }
  321. void Compact(int cf, const Slice& start, const Slice& limit) {
  322. ASSERT_OK(
  323. db_->CompactRange(CompactRangeOptions(), handles_[cf], &start, &limit));
  324. }
  325. int NumTableFilesAtLevel(int level, int cf) {
  326. return GetProperty(cf,
  327. "rocksdb.num-files-at-level" + std::to_string(level));
  328. }
  329. // Return spread of files per level
  330. std::string FilesPerLevel(int cf) {
  331. std::string result;
  332. int last_non_zero_offset = 0;
  333. for (int level = 0; level < dbfull()->NumberLevels(handles_[cf]); level++) {
  334. int f = NumTableFilesAtLevel(level, cf);
  335. char buf[100];
  336. snprintf(buf, sizeof(buf), "%s%d", (level ? "," : ""), f);
  337. result += buf;
  338. if (f > 0) {
  339. last_non_zero_offset = static_cast<int>(result.size());
  340. }
  341. }
  342. result.resize(last_non_zero_offset);
  343. return result;
  344. }
  345. void AssertFilesPerLevel(const std::string& value, int cf) {
  346. ASSERT_EQ(value, FilesPerLevel(cf));
  347. }
  348. int CountLiveFiles() {
  349. std::vector<LiveFileMetaData> metadata;
  350. db_->GetLiveFilesMetaData(&metadata);
  351. return static_cast<int>(metadata.size());
  352. }
  353. void AssertCountLiveFiles(int expected_value) {
  354. ASSERT_EQ(expected_value, CountLiveFiles());
  355. }
  356. // Do n memtable flushes, each of which produces an sstable
  357. // covering the range [small,large].
  358. void MakeTables(int cf, int n, const std::string& small,
  359. const std::string& large) {
  360. for (int i = 0; i < n; i++) {
  361. ASSERT_OK(Put(cf, small, "begin"));
  362. ASSERT_OK(Put(cf, large, "end"));
  363. ASSERT_OK(db_->Flush(FlushOptions(), handles_[cf]));
  364. }
  365. }
  366. int CountLiveLogFiles() {
  367. int micros_wait_for_log_deletion = 20000;
  368. env_->SleepForMicroseconds(micros_wait_for_log_deletion);
  369. int ret = 0;
  370. VectorLogPtr wal_files;
  371. Status s;
  372. // GetSortedWalFiles is a flakey function -- it gets all the wal_dir
  373. // children files and then later checks for their existence. if some of the
  374. // log files doesn't exist anymore, it reports an error. it does all of this
  375. // without DB mutex held, so if a background process deletes the log file
  376. // while the function is being executed, it returns an error. We retry the
  377. // function 10 times to avoid the error failing the test
  378. for (int retries = 0; retries < 10; ++retries) {
  379. wal_files.clear();
  380. s = db_->GetSortedWalFiles(wal_files);
  381. if (s.ok()) {
  382. break;
  383. }
  384. }
  385. EXPECT_OK(s);
  386. for (const auto& wal : wal_files) {
  387. if (wal->Type() == kAliveLogFile) {
  388. ++ret;
  389. }
  390. }
  391. return ret;
  392. return 0;
  393. }
  394. void AssertCountLiveLogFiles(int value) {
  395. ASSERT_EQ(value, CountLiveLogFiles());
  396. }
  397. void AssertNumberOfImmutableMemtables(std::vector<int> num_per_cf) {
  398. assert(num_per_cf.size() == handles_.size());
  399. for (size_t i = 0; i < num_per_cf.size(); ++i) {
  400. ASSERT_EQ(num_per_cf[i], GetProperty(static_cast<int>(i),
  401. "rocksdb.num-immutable-mem-table"));
  402. }
  403. }
  404. void CopyFile(const std::string& source, const std::string& destination,
  405. uint64_t size = 0) {
  406. const EnvOptions soptions;
  407. std::unique_ptr<SequentialFile> srcfile;
  408. ASSERT_OK(env_->NewSequentialFile(source, &srcfile, soptions));
  409. std::unique_ptr<WritableFile> destfile;
  410. ASSERT_OK(env_->NewWritableFile(destination, &destfile, soptions));
  411. if (size == 0) {
  412. // default argument means copy everything
  413. ASSERT_OK(env_->GetFileSize(source, &size));
  414. }
  415. char buffer[4096];
  416. Slice slice;
  417. while (size > 0) {
  418. uint64_t one = std::min(uint64_t(sizeof(buffer)), size);
  419. ASSERT_OK(srcfile->Read(one, &slice, buffer));
  420. ASSERT_OK(destfile->Append(slice));
  421. size -= slice.size();
  422. }
  423. ASSERT_OK(destfile->Close());
  424. }
  425. int GetSstFileCount(std::string path) {
  426. std::vector<std::string> files;
  427. DBTestBase::GetSstFiles(env_, path, &files);
  428. return static_cast<int>(files.size());
  429. }
  430. void RecalculateWriteStallConditions(
  431. ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options) {
  432. // add lock to avoid race condition between
  433. // `RecalculateWriteStallConditions` which writes to CFStats and
  434. // background `DBImpl::DumpStats()` threads which read CFStats
  435. dbfull()->TEST_LockMutex();
  436. cfd->RecalculateWriteStallConditions(mutable_cf_options);
  437. dbfull()->TEST_UnlockMutex();
  438. }
  439. std::vector<ColumnFamilyHandle*> handles_;
  440. std::vector<std::string> names_;
  441. std::vector<std::set<std::string>> keys_;
  442. ColumnFamilyOptions column_family_options_;
  443. DBOptions db_options_;
  444. std::string dbname_;
  445. DB* db_ = nullptr;
  446. EnvCounter* env_;
  447. std::shared_ptr<Env> env_guard_;
  448. Random rnd_;
  449. uint32_t format_;
  450. };
  451. class ColumnFamilyTest
  452. : public ColumnFamilyTestBase,
  453. virtual public ::testing::WithParamInterface<uint32_t> {
  454. public:
  455. ColumnFamilyTest() : ColumnFamilyTestBase(GetParam()) {}
  456. };
  457. INSTANTIATE_TEST_CASE_P(FormatDef, ColumnFamilyTest,
  458. testing::Values(test::kDefaultFormatVersion));
  459. INSTANTIATE_TEST_CASE_P(FormatLatest, ColumnFamilyTest,
  460. testing::Values(kLatestFormatVersion));
  461. TEST_P(ColumnFamilyTest, DontReuseColumnFamilyID) {
  462. for (int iter = 0; iter < 3; ++iter) {
  463. Open();
  464. CreateColumnFamilies({"one", "two", "three"});
  465. for (size_t i = 0; i < handles_.size(); ++i) {
  466. auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(handles_[i]);
  467. ASSERT_EQ(i, cfh->GetID());
  468. }
  469. if (iter == 1) {
  470. Reopen();
  471. }
  472. DropColumnFamilies({3});
  473. Reopen();
  474. if (iter == 2) {
  475. // this tests if max_column_family is correctly persisted with
  476. // WriteSnapshot()
  477. Reopen();
  478. }
  479. CreateColumnFamilies({"three2"});
  480. // ID 3 that was used for dropped column family "three" should not be
  481. // reused
  482. auto cfh3 = static_cast_with_check<ColumnFamilyHandleImpl>(handles_[3]);
  483. ASSERT_EQ(4U, cfh3->GetID());
  484. Close();
  485. Destroy();
  486. }
  487. }
  488. TEST_P(ColumnFamilyTest, CreateCFRaceWithGetAggProperty) {
  489. Open();
  490. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  491. {{"DBImpl::WriteOptionsFile:1",
  492. "ColumnFamilyTest.CreateCFRaceWithGetAggProperty:1"},
  493. {"ColumnFamilyTest.CreateCFRaceWithGetAggProperty:2",
  494. "DBImpl::WriteOptionsFile:2"}});
  495. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  496. ROCKSDB_NAMESPACE::port::Thread thread(
  497. [&] { CreateColumnFamilies({"one"}); });
  498. TEST_SYNC_POINT("ColumnFamilyTest.CreateCFRaceWithGetAggProperty:1");
  499. uint64_t pv;
  500. db_->GetAggregatedIntProperty(DB::Properties::kEstimateTableReadersMem, &pv);
  501. TEST_SYNC_POINT("ColumnFamilyTest.CreateCFRaceWithGetAggProperty:2");
  502. thread.join();
  503. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  504. }
  505. class FlushEmptyCFTestWithParam
  506. : public ColumnFamilyTestBase,
  507. virtual public testing::WithParamInterface<std::tuple<uint32_t, bool>> {
  508. public:
  509. FlushEmptyCFTestWithParam()
  510. : ColumnFamilyTestBase(std::get<0>(GetParam())),
  511. allow_2pc_(std::get<1>(GetParam())) {}
  512. // Required if inheriting from testing::WithParamInterface<>
  513. static void SetUpTestCase() {}
  514. static void TearDownTestCase() {}
  515. bool allow_2pc_;
  516. };
  517. TEST_P(FlushEmptyCFTestWithParam, FlushEmptyCFTest) {
  518. std::unique_ptr<FaultInjectionTestEnv> fault_env(
  519. new FaultInjectionTestEnv(env_));
  520. db_options_.env = fault_env.get();
  521. db_options_.allow_2pc = allow_2pc_;
  522. Open();
  523. CreateColumnFamilies({"one", "two"});
  524. // Generate log file A.
  525. ASSERT_OK(Put(1, "foo", "v1")); // seqID 1
  526. Reopen();
  527. // Log file A is not dropped after reopening because default column family's
  528. // min log number is 0.
  529. // It flushes to SST file X
  530. ASSERT_OK(Put(1, "foo", "v1")); // seqID 2
  531. ASSERT_OK(Put(1, "bar", "v2")); // seqID 3
  532. // Current log file is file B now. While flushing, a new log file C is created
  533. // and is set to current. Boths' min log number is set to file C in memory, so
  534. // after flushing file B is deleted. At the same time, the min log number of
  535. // default CF is not written to manifest. Log file A still remains.
  536. // Flushed to SST file Y.
  537. ASSERT_OK(Flush(1));
  538. ASSERT_OK(Flush(0));
  539. ASSERT_OK(Put(1, "bar", "v3")); // seqID 4
  540. ASSERT_OK(Put(1, "foo", "v4")); // seqID 5
  541. ASSERT_OK(db_->FlushWAL(/*sync=*/false));
  542. // Preserve file system state up to here to simulate a crash condition.
  543. fault_env->SetFilesystemActive(false);
  544. std::vector<std::string> names;
  545. for (const auto& name : names_) {
  546. if (name != "") {
  547. names.push_back(name);
  548. }
  549. }
  550. Close();
  551. fault_env->ResetState();
  552. // Before opening, there are four files:
  553. // Log file A contains seqID 1
  554. // Log file C contains seqID 4, 5
  555. // SST file X contains seqID 1
  556. // SST file Y contains seqID 2, 3
  557. // Min log number:
  558. // default CF: 0
  559. // CF one, two: C
  560. // When opening the DB, all the seqID should be preserved.
  561. Open(names, {});
  562. ASSERT_EQ("v4", Get(1, "foo"));
  563. ASSERT_EQ("v3", Get(1, "bar"));
  564. Close();
  565. db_options_.env = env_;
  566. }
  567. TEST_P(FlushEmptyCFTestWithParam, FlushEmptyCFTest2) {
  568. std::unique_ptr<FaultInjectionTestEnv> fault_env(
  569. new FaultInjectionTestEnv(env_));
  570. db_options_.env = fault_env.get();
  571. db_options_.allow_2pc = allow_2pc_;
  572. Open();
  573. CreateColumnFamilies({"one", "two"});
  574. // Generate log file A.
  575. ASSERT_OK(Put(1, "foo", "v1")); // seqID 1
  576. Reopen();
  577. // Log file A is not dropped after reopening because default column family's
  578. // min log number is 0.
  579. // It flushes to SST file X
  580. ASSERT_OK(Put(1, "foo", "v1")); // seqID 2
  581. ASSERT_OK(Put(1, "bar", "v2")); // seqID 3
  582. // Current log file is file B now. While flushing, a new log file C is created
  583. // and is set to current. Both CFs' min log number is set to file C so after
  584. // flushing file B is deleted. Log file A still remains.
  585. // Flushed to SST file Y.
  586. ASSERT_OK(Flush(1));
  587. ASSERT_OK(Put(0, "bar", "v2")); // seqID 4
  588. ASSERT_OK(Put(2, "bar", "v2")); // seqID 5
  589. ASSERT_OK(Put(1, "bar", "v3")); // seqID 6
  590. // Flushing all column families. This forces all CFs' min log to current. This
  591. // is written to the manifest file. Log file C is cleared.
  592. ASSERT_OK(Flush(0));
  593. ASSERT_OK(Flush(1));
  594. ASSERT_OK(Flush(2));
  595. // Write to log file D
  596. ASSERT_OK(Put(1, "bar", "v4")); // seqID 7
  597. ASSERT_OK(Put(1, "bar", "v5")); // seqID 8
  598. ASSERT_OK(db_->FlushWAL(/*sync=*/false));
  599. // Preserve file system state up to here to simulate a crash condition.
  600. fault_env->SetFilesystemActive(false);
  601. std::vector<std::string> names;
  602. for (const auto& name : names_) {
  603. if (name != "") {
  604. names.push_back(name);
  605. }
  606. }
  607. Close();
  608. fault_env->ResetState();
  609. // Before opening, there are two logfiles:
  610. // Log file A contains seqID 1
  611. // Log file D contains seqID 7, 8
  612. // Min log number:
  613. // default CF: D
  614. // CF one, two: D
  615. // When opening the DB, log file D should be replayed using the seqID
  616. // specified in the file.
  617. Open(names, {});
  618. ASSERT_EQ("v1", Get(1, "foo"));
  619. ASSERT_EQ("v5", Get(1, "bar"));
  620. Close();
  621. db_options_.env = env_;
  622. }
  623. INSTANTIATE_TEST_CASE_P(
  624. FormatDef, FlushEmptyCFTestWithParam,
  625. testing::Values(std::make_tuple(test::kDefaultFormatVersion, true),
  626. std::make_tuple(test::kDefaultFormatVersion, false)));
  627. INSTANTIATE_TEST_CASE_P(
  628. FormatLatest, FlushEmptyCFTestWithParam,
  629. testing::Values(std::make_tuple(kLatestFormatVersion, true),
  630. std::make_tuple(kLatestFormatVersion, false)));
  631. TEST_P(ColumnFamilyTest, AddDrop) {
  632. Open();
  633. CreateColumnFamilies({"one", "two", "three"});
  634. ASSERT_EQ("NOT_FOUND", Get(1, "fodor"));
  635. ASSERT_EQ("NOT_FOUND", Get(2, "fodor"));
  636. DropColumnFamilies({2});
  637. ASSERT_EQ("NOT_FOUND", Get(1, "fodor"));
  638. CreateColumnFamilies({"four"});
  639. ASSERT_EQ("NOT_FOUND", Get(3, "fodor"));
  640. ASSERT_OK(Put(1, "fodor", "mirko"));
  641. ASSERT_EQ("mirko", Get(1, "fodor"));
  642. ASSERT_EQ("NOT_FOUND", Get(3, "fodor"));
  643. Close();
  644. ASSERT_TRUE(TryOpen({"default"}).IsInvalidArgument());
  645. Open({"default", "one", "three", "four"});
  646. DropColumnFamilies({1});
  647. Reopen();
  648. Close();
  649. std::vector<std::string> families;
  650. ASSERT_OK(DB::ListColumnFamilies(db_options_, dbname_, &families));
  651. std::sort(families.begin(), families.end());
  652. ASSERT_TRUE(families ==
  653. std::vector<std::string>({"default", "four", "three"}));
  654. }
  655. TEST_P(ColumnFamilyTest, BulkAddDrop) {
  656. constexpr int kNumCF = 1000;
  657. ColumnFamilyOptions cf_options;
  658. WriteOptions write_options;
  659. Open();
  660. std::vector<std::string> cf_names;
  661. std::vector<ColumnFamilyHandle*> cf_handles;
  662. for (int i = 1; i <= kNumCF; i++) {
  663. cf_names.push_back("cf1-" + std::to_string(i));
  664. }
  665. ASSERT_OK(db_->CreateColumnFamilies(cf_options, cf_names, &cf_handles));
  666. for (int i = 1; i <= kNumCF; i++) {
  667. ASSERT_OK(db_->Put(write_options, cf_handles[i - 1], "foo", "bar"));
  668. }
  669. ASSERT_OK(db_->DropColumnFamilies(cf_handles));
  670. std::vector<ColumnFamilyDescriptor> cf_descriptors;
  671. for (auto* handle : cf_handles) {
  672. delete handle;
  673. }
  674. cf_handles.clear();
  675. for (int i = 1; i <= kNumCF; i++) {
  676. cf_descriptors.emplace_back("cf2-" + std::to_string(i),
  677. ColumnFamilyOptions());
  678. }
  679. ASSERT_OK(db_->CreateColumnFamilies(cf_descriptors, &cf_handles));
  680. for (int i = 1; i <= kNumCF; i++) {
  681. ASSERT_OK(db_->Put(write_options, cf_handles[i - 1], "foo", "bar"));
  682. }
  683. ASSERT_OK(db_->DropColumnFamilies(cf_handles));
  684. for (auto* handle : cf_handles) {
  685. delete handle;
  686. }
  687. Close();
  688. std::vector<std::string> families;
  689. ASSERT_OK(DB::ListColumnFamilies(db_options_, dbname_, &families));
  690. std::sort(families.begin(), families.end());
  691. ASSERT_TRUE(families == std::vector<std::string>({"default"}));
  692. }
  693. TEST_P(ColumnFamilyTest, DropTest) {
  694. // first iteration - don't reopen DB before dropping
  695. // second iteration - reopen DB before dropping
  696. for (int iter = 0; iter < 2; ++iter) {
  697. Open({"default"});
  698. CreateColumnFamiliesAndReopen({"pikachu"});
  699. for (int i = 0; i < 100; ++i) {
  700. ASSERT_OK(Put(1, std::to_string(i), "bar" + std::to_string(i)));
  701. }
  702. ASSERT_OK(Flush(1));
  703. if (iter == 1) {
  704. Reopen();
  705. }
  706. ASSERT_EQ("bar1", Get(1, "1"));
  707. AssertCountLiveFiles(1);
  708. DropColumnFamilies({1});
  709. // make sure that all files are deleted when we drop the column family
  710. AssertCountLiveFiles(0);
  711. Destroy();
  712. }
  713. }
  714. TEST_P(ColumnFamilyTest, WriteBatchFailure) {
  715. Open();
  716. CreateColumnFamiliesAndReopen({"one", "two"});
  717. WriteBatch batch;
  718. ASSERT_OK(batch.Put(handles_[0], Slice("existing"), Slice("column-family")));
  719. ASSERT_OK(
  720. batch.Put(handles_[1], Slice("non-existing"), Slice("column-family")));
  721. ASSERT_OK(db_->Write(WriteOptions(), &batch));
  722. DropColumnFamilies({1});
  723. WriteOptions woptions_ignore_missing_cf;
  724. woptions_ignore_missing_cf.ignore_missing_column_families = true;
  725. ASSERT_OK(
  726. batch.Put(handles_[0], Slice("still here"), Slice("column-family")));
  727. ASSERT_OK(db_->Write(woptions_ignore_missing_cf, &batch));
  728. ASSERT_EQ("column-family", Get(0, "still here"));
  729. Status s = db_->Write(WriteOptions(), &batch);
  730. ASSERT_TRUE(s.IsInvalidArgument());
  731. Close();
  732. }
  733. TEST_P(ColumnFamilyTest, ReadWrite) {
  734. Open();
  735. CreateColumnFamiliesAndReopen({"one", "two"});
  736. ASSERT_OK(Put(0, "foo", "v1"));
  737. ASSERT_OK(Put(0, "bar", "v2"));
  738. ASSERT_OK(Put(1, "mirko", "v3"));
  739. ASSERT_OK(Put(0, "foo", "v2"));
  740. ASSERT_OK(Put(2, "fodor", "v5"));
  741. for (int iter = 0; iter <= 3; ++iter) {
  742. ASSERT_EQ("v2", Get(0, "foo"));
  743. ASSERT_EQ("v2", Get(0, "bar"));
  744. ASSERT_EQ("v3", Get(1, "mirko"));
  745. ASSERT_EQ("v5", Get(2, "fodor"));
  746. ASSERT_EQ("NOT_FOUND", Get(0, "fodor"));
  747. ASSERT_EQ("NOT_FOUND", Get(1, "fodor"));
  748. ASSERT_EQ("NOT_FOUND", Get(2, "foo"));
  749. if (iter <= 1) {
  750. Reopen();
  751. }
  752. }
  753. Close();
  754. }
  755. TEST_P(ColumnFamilyTest, IgnoreRecoveredLog) {
  756. std::string backup_logs = dbname_ + "/backup_logs";
  757. // delete old files in backup_logs directory
  758. ASSERT_OK(env_->CreateDirIfMissing(dbname_));
  759. ASSERT_OK(env_->CreateDirIfMissing(backup_logs));
  760. std::vector<std::string> old_files;
  761. ASSERT_OK(env_->GetChildren(backup_logs, &old_files));
  762. for (auto& file : old_files) {
  763. ASSERT_OK(env_->DeleteFile(backup_logs + "/" + file));
  764. }
  765. column_family_options_.merge_operator =
  766. MergeOperators::CreateUInt64AddOperator();
  767. db_options_.wal_dir = dbname_ + "/logs";
  768. Destroy();
  769. Open();
  770. CreateColumnFamilies({"cf1", "cf2"});
  771. // fill up the DB
  772. std::string one, two, three;
  773. PutFixed64(&one, 1);
  774. PutFixed64(&two, 2);
  775. PutFixed64(&three, 3);
  776. ASSERT_OK(Merge(0, "foo", one));
  777. ASSERT_OK(Merge(1, "mirko", one));
  778. ASSERT_OK(Merge(0, "foo", one));
  779. ASSERT_OK(Merge(2, "bla", one));
  780. ASSERT_OK(Merge(2, "fodor", one));
  781. ASSERT_OK(Merge(0, "bar", one));
  782. ASSERT_OK(Merge(2, "bla", one));
  783. ASSERT_OK(Merge(1, "mirko", two));
  784. ASSERT_OK(Merge(1, "franjo", one));
  785. // copy the logs to backup
  786. std::vector<std::string> logs;
  787. ASSERT_OK(env_->GetChildren(db_options_.wal_dir, &logs));
  788. for (auto& log : logs) {
  789. CopyFile(db_options_.wal_dir + "/" + log, backup_logs + "/" + log);
  790. }
  791. // recover the DB
  792. Close();
  793. // 1. check consistency
  794. // 2. copy the logs from backup back to WAL dir. if the recovery happens
  795. // again on the same log files, this should lead to incorrect results
  796. // due to applying merge operator twice
  797. // 3. check consistency
  798. for (int iter = 0; iter < 2; ++iter) {
  799. // assert consistency
  800. Open({"default", "cf1", "cf2"});
  801. ASSERT_EQ(two, Get(0, "foo"));
  802. ASSERT_EQ(one, Get(0, "bar"));
  803. ASSERT_EQ(three, Get(1, "mirko"));
  804. ASSERT_EQ(one, Get(1, "franjo"));
  805. ASSERT_EQ(one, Get(2, "fodor"));
  806. ASSERT_EQ(two, Get(2, "bla"));
  807. Close();
  808. if (iter == 0) {
  809. // copy the logs from backup back to wal dir
  810. for (auto& log : logs) {
  811. CopyFile(backup_logs + "/" + log, db_options_.wal_dir + "/" + log);
  812. }
  813. }
  814. }
  815. }
  816. TEST_P(ColumnFamilyTest, FlushTest) {
  817. Open();
  818. CreateColumnFamiliesAndReopen({"one", "two"});
  819. ASSERT_OK(Put(0, "foo", "v1"));
  820. ASSERT_OK(Put(0, "bar", "v2"));
  821. ASSERT_OK(Put(1, "mirko", "v3"));
  822. ASSERT_OK(Put(0, "foo", "v2"));
  823. ASSERT_OK(Put(2, "fodor", "v5"));
  824. for (int j = 0; j < 2; j++) {
  825. ReadOptions ro;
  826. std::vector<Iterator*> iterators;
  827. // Hold super version.
  828. if (j == 0) {
  829. ASSERT_OK(db_->NewIterators(ro, handles_, &iterators));
  830. }
  831. for (int i = 0; i < 3; ++i) {
  832. uint64_t max_total_in_memory_state = MaxTotalInMemoryState();
  833. ASSERT_OK(Flush(i));
  834. AssertMaxTotalInMemoryState(max_total_in_memory_state);
  835. }
  836. ASSERT_OK(Put(1, "foofoo", "bar"));
  837. ASSERT_OK(Put(0, "foofoo", "bar"));
  838. for (auto* it : iterators) {
  839. ASSERT_OK(it->status());
  840. delete it;
  841. }
  842. }
  843. Reopen();
  844. for (int iter = 0; iter <= 2; ++iter) {
  845. ASSERT_EQ("v2", Get(0, "foo"));
  846. ASSERT_EQ("v2", Get(0, "bar"));
  847. ASSERT_EQ("v3", Get(1, "mirko"));
  848. ASSERT_EQ("v5", Get(2, "fodor"));
  849. ASSERT_EQ("NOT_FOUND", Get(0, "fodor"));
  850. ASSERT_EQ("NOT_FOUND", Get(1, "fodor"));
  851. ASSERT_EQ("NOT_FOUND", Get(2, "foo"));
  852. if (iter <= 1) {
  853. Reopen();
  854. }
  855. }
  856. Close();
  857. }
  858. // Makes sure that obsolete log files get deleted
  859. TEST_P(ColumnFamilyTest, LogDeletionTest) {
  860. db_options_.max_total_wal_size = std::numeric_limits<uint64_t>::max();
  861. column_family_options_.arena_block_size = 4 * 1024;
  862. column_family_options_.write_buffer_size = 128000; // 128KB
  863. Open();
  864. CreateColumnFamilies({"one", "two", "three", "four"});
  865. // Each bracket is one log file. if number is in (), it means
  866. // we don't need it anymore (it's been flushed)
  867. // []
  868. AssertCountLiveLogFiles(0);
  869. PutRandomData(0, 1, 128);
  870. // [0]
  871. PutRandomData(1, 1, 128);
  872. // [0, 1]
  873. PutRandomData(1, 1000, 128);
  874. WaitForFlush(1);
  875. // [0, (1)] [1]
  876. AssertCountLiveLogFiles(2);
  877. PutRandomData(0, 1, 128);
  878. // [0, (1)] [0, 1]
  879. AssertCountLiveLogFiles(2);
  880. PutRandomData(2, 1, 128);
  881. // [0, (1)] [0, 1, 2]
  882. PutRandomData(2, 1000, 128);
  883. WaitForFlush(2);
  884. // [0, (1)] [0, 1, (2)] [2]
  885. AssertCountLiveLogFiles(3);
  886. PutRandomData(2, 1000, 128);
  887. WaitForFlush(2);
  888. // [0, (1)] [0, 1, (2)] [(2)] [2]
  889. AssertCountLiveLogFiles(4);
  890. PutRandomData(3, 1, 128);
  891. // [0, (1)] [0, 1, (2)] [(2)] [2, 3]
  892. PutRandomData(1, 1, 128);
  893. // [0, (1)] [0, 1, (2)] [(2)] [1, 2, 3]
  894. AssertCountLiveLogFiles(4);
  895. PutRandomData(1, 1000, 128);
  896. WaitForFlush(1);
  897. // [0, (1)] [0, (1), (2)] [(2)] [(1), 2, 3] [1]
  898. AssertCountLiveLogFiles(5);
  899. PutRandomData(0, 1000, 128);
  900. WaitForFlush(0);
  901. // [(0), (1)] [(0), (1), (2)] [(2)] [(1), 2, 3] [1, (0)] [0]
  902. // delete obsolete logs -->
  903. // [(1), 2, 3] [1, (0)] [0]
  904. AssertCountLiveLogFiles(3);
  905. PutRandomData(0, 1000, 128);
  906. WaitForFlush(0);
  907. // [(1), 2, 3] [1, (0)], [(0)] [0]
  908. AssertCountLiveLogFiles(4);
  909. PutRandomData(1, 1000, 128);
  910. WaitForFlush(1);
  911. // [(1), 2, 3] [(1), (0)] [(0)] [0, (1)] [1]
  912. AssertCountLiveLogFiles(5);
  913. PutRandomData(2, 1000, 128);
  914. WaitForFlush(2);
  915. // [(1), (2), 3] [(1), (0)] [(0)] [0, (1)] [1, (2)], [2]
  916. AssertCountLiveLogFiles(6);
  917. PutRandomData(3, 1000, 128);
  918. WaitForFlush(3);
  919. // [(1), (2), (3)] [(1), (0)] [(0)] [0, (1)] [1, (2)], [2, (3)] [3]
  920. // delete obsolete logs -->
  921. // [0, (1)] [1, (2)], [2, (3)] [3]
  922. AssertCountLiveLogFiles(4);
  923. Close();
  924. }
  925. TEST_P(ColumnFamilyTest, CrashAfterFlush) {
  926. std::unique_ptr<FaultInjectionTestEnv> fault_env(
  927. new FaultInjectionTestEnv(env_));
  928. db_options_.env = fault_env.get();
  929. Open();
  930. CreateColumnFamilies({"one"});
  931. WriteBatch batch;
  932. ASSERT_OK(batch.Put(handles_[0], Slice("foo"), Slice("bar")));
  933. ASSERT_OK(batch.Put(handles_[1], Slice("foo"), Slice("bar")));
  934. ASSERT_OK(db_->Write(WriteOptions(), &batch));
  935. ASSERT_OK(Flush(0));
  936. fault_env->SetFilesystemActive(false);
  937. std::vector<std::string> names;
  938. for (const auto& name : names_) {
  939. if (name != "") {
  940. names.push_back(name);
  941. }
  942. }
  943. Close();
  944. ASSERT_OK(fault_env->DropUnsyncedFileData());
  945. fault_env->ResetState();
  946. Open(names, {});
  947. // Write batch should be atomic.
  948. ASSERT_EQ(Get(0, "foo"), Get(1, "foo"));
  949. Close();
  950. db_options_.env = env_;
  951. }
  952. TEST_P(ColumnFamilyTest, OpenNonexistentColumnFamily) {
  953. ASSERT_OK(TryOpen({"default"}));
  954. Close();
  955. ASSERT_TRUE(TryOpen({"default", "dne"}).IsInvalidArgument());
  956. }
  957. // Makes sure that obsolete log files get deleted
  958. TEST_P(ColumnFamilyTest, DifferentWriteBufferSizes) {
  959. // disable flushing stale column families
  960. db_options_.max_total_wal_size = std::numeric_limits<uint64_t>::max();
  961. Open();
  962. CreateColumnFamilies({"one", "two", "three"});
  963. ColumnFamilyOptions default_cf, one, two, three;
  964. // setup options. all column families have max_write_buffer_number setup to 10
  965. // "default" -> 100KB memtable, start flushing immediately
  966. // "one" -> 200KB memtable, start flushing with two immutable memtables
  967. // "two" -> 1MB memtable, start flushing with three immutable memtables
  968. // "three" -> 90KB memtable, start flushing with four immutable memtables
  969. default_cf.write_buffer_size = 100000;
  970. default_cf.arena_block_size = 4 * 4096;
  971. default_cf.max_write_buffer_number = 10;
  972. default_cf.min_write_buffer_number_to_merge = 1;
  973. default_cf.max_write_buffer_size_to_maintain = 0;
  974. one.write_buffer_size = 200000;
  975. one.arena_block_size = 4 * 4096;
  976. one.max_write_buffer_number = 10;
  977. one.min_write_buffer_number_to_merge = 2;
  978. one.max_write_buffer_size_to_maintain =
  979. static_cast<int>(one.write_buffer_size);
  980. two.write_buffer_size = 1000000;
  981. two.arena_block_size = 4 * 4096;
  982. two.max_write_buffer_number = 10;
  983. two.min_write_buffer_number_to_merge = 3;
  984. two.max_write_buffer_size_to_maintain =
  985. static_cast<int>(two.write_buffer_size);
  986. three.write_buffer_size = 4096 * 22;
  987. three.arena_block_size = 4096;
  988. three.max_write_buffer_number = 10;
  989. three.min_write_buffer_number_to_merge = 4;
  990. three.max_write_buffer_size_to_maintain =
  991. static_cast<int>(three.write_buffer_size);
  992. Reopen({default_cf, one, two, three});
  993. int micros_wait_for_flush = 10000;
  994. PutRandomData(0, 100, 1000);
  995. WaitForFlush(0);
  996. AssertNumberOfImmutableMemtables({0, 0, 0, 0});
  997. AssertCountLiveLogFiles(1);
  998. PutRandomData(1, 200, 1000);
  999. env_->SleepForMicroseconds(micros_wait_for_flush);
  1000. AssertNumberOfImmutableMemtables({0, 1, 0, 0});
  1001. AssertCountLiveLogFiles(2);
  1002. PutRandomData(2, 1000, 1000);
  1003. env_->SleepForMicroseconds(micros_wait_for_flush);
  1004. AssertNumberOfImmutableMemtables({0, 1, 1, 0});
  1005. AssertCountLiveLogFiles(3);
  1006. PutRandomData(2, 1000, 1000);
  1007. env_->SleepForMicroseconds(micros_wait_for_flush);
  1008. AssertNumberOfImmutableMemtables({0, 1, 2, 0});
  1009. AssertCountLiveLogFiles(4);
  1010. PutRandomData(3, 93, 990);
  1011. env_->SleepForMicroseconds(micros_wait_for_flush);
  1012. AssertNumberOfImmutableMemtables({0, 1, 2, 1});
  1013. AssertCountLiveLogFiles(5);
  1014. PutRandomData(3, 88, 990);
  1015. env_->SleepForMicroseconds(micros_wait_for_flush);
  1016. AssertNumberOfImmutableMemtables({0, 1, 2, 2});
  1017. AssertCountLiveLogFiles(6);
  1018. PutRandomData(3, 88, 990);
  1019. env_->SleepForMicroseconds(micros_wait_for_flush);
  1020. AssertNumberOfImmutableMemtables({0, 1, 2, 3});
  1021. AssertCountLiveLogFiles(7);
  1022. PutRandomData(0, 100, 1000);
  1023. WaitForFlush(0);
  1024. AssertNumberOfImmutableMemtables({0, 1, 2, 3});
  1025. AssertCountLiveLogFiles(8);
  1026. PutRandomData(2, 100, 10000);
  1027. WaitForFlush(2);
  1028. AssertNumberOfImmutableMemtables({0, 1, 0, 3});
  1029. AssertCountLiveLogFiles(9);
  1030. PutRandomData(3, 88, 990);
  1031. WaitForFlush(3);
  1032. AssertNumberOfImmutableMemtables({0, 1, 0, 0});
  1033. AssertCountLiveLogFiles(10);
  1034. PutRandomData(3, 88, 990);
  1035. env_->SleepForMicroseconds(micros_wait_for_flush);
  1036. AssertNumberOfImmutableMemtables({0, 1, 0, 1});
  1037. AssertCountLiveLogFiles(11);
  1038. PutRandomData(1, 200, 1000);
  1039. WaitForFlush(1);
  1040. AssertNumberOfImmutableMemtables({0, 0, 0, 1});
  1041. AssertCountLiveLogFiles(5);
  1042. PutRandomData(3, 88 * 3, 990);
  1043. WaitForFlush(3);
  1044. PutRandomData(3, 88 * 4, 990);
  1045. WaitForFlush(3);
  1046. AssertNumberOfImmutableMemtables({0, 0, 0, 0});
  1047. AssertCountLiveLogFiles(12);
  1048. PutRandomData(0, 100, 1000);
  1049. WaitForFlush(0);
  1050. AssertNumberOfImmutableMemtables({0, 0, 0, 0});
  1051. AssertCountLiveLogFiles(12);
  1052. PutRandomData(2, 3 * 1000, 1000);
  1053. WaitForFlush(2);
  1054. AssertNumberOfImmutableMemtables({0, 0, 0, 0});
  1055. AssertCountLiveLogFiles(12);
  1056. PutRandomData(1, 2 * 200, 1000);
  1057. WaitForFlush(1);
  1058. AssertNumberOfImmutableMemtables({0, 0, 0, 0});
  1059. AssertCountLiveLogFiles(7);
  1060. Close();
  1061. }
  1062. // The test is commented out because we want to test that snapshot is
  1063. // not created for memtables not supported it, but There isn't a memtable
  1064. // that doesn't support snapshot right now. If we have one later, we can
  1065. // re-enable the test.
  1066. //
  1067. // TEST_P(ColumnFamilyTest, MemtableNotSupportSnapshot) {
  1068. // db_options_.allow_concurrent_memtable_write = false;
  1069. // Open();
  1070. // auto* s1 = dbfull()->GetSnapshot();
  1071. // ASSERT_TRUE(s1 != nullptr);
  1072. // dbfull()->ReleaseSnapshot(s1);
  1073. // // Add a column family that doesn't support snapshot
  1074. // ColumnFamilyOptions first;
  1075. // first.memtable_factory.reset(new DummyMemtableNotSupportingSnapshot());
  1076. // CreateColumnFamilies({"first"}, {first});
  1077. // auto* s2 = dbfull()->GetSnapshot();
  1078. // ASSERT_TRUE(s2 == nullptr);
  1079. // // Add a column family that supports snapshot. Snapshot stays not
  1080. // supported. ColumnFamilyOptions second; CreateColumnFamilies({"second"},
  1081. // {second}); auto* s3 = dbfull()->GetSnapshot(); ASSERT_TRUE(s3 == nullptr);
  1082. // Close();
  1083. // }
  1084. class TestComparator : public Comparator {
  1085. int Compare(const ROCKSDB_NAMESPACE::Slice& /*a*/,
  1086. const ROCKSDB_NAMESPACE::Slice& /*b*/) const override {
  1087. return 0;
  1088. }
  1089. const char* Name() const override { return "Test"; }
  1090. void FindShortestSeparator(
  1091. std::string* /*start*/,
  1092. const ROCKSDB_NAMESPACE::Slice& /*limit*/) const override {}
  1093. void FindShortSuccessor(std::string* /*key*/) const override {}
  1094. };
  1095. static TestComparator third_comparator;
  1096. static TestComparator fourth_comparator;
  1097. // Test that we can retrieve the comparator from a created CF
  1098. TEST_P(ColumnFamilyTest, GetComparator) {
  1099. Open();
  1100. // Add a column family with no comparator specified
  1101. CreateColumnFamilies({"first"});
  1102. const Comparator* comp = handles_[0]->GetComparator();
  1103. ASSERT_EQ(comp, BytewiseComparator());
  1104. // Add three column families - one with no comparator and two
  1105. // with comparators specified
  1106. ColumnFamilyOptions second, third, fourth;
  1107. second.comparator = &third_comparator;
  1108. third.comparator = &fourth_comparator;
  1109. CreateColumnFamilies({"second", "third", "fourth"}, {second, third, fourth});
  1110. ASSERT_EQ(handles_[1]->GetComparator(), BytewiseComparator());
  1111. ASSERT_EQ(handles_[2]->GetComparator(), &third_comparator);
  1112. ASSERT_EQ(handles_[3]->GetComparator(), &fourth_comparator);
  1113. Close();
  1114. }
  1115. TEST_P(ColumnFamilyTest, DifferentMergeOperators) {
  1116. Open();
  1117. CreateColumnFamilies({"first", "second"});
  1118. ColumnFamilyOptions default_cf, first, second;
  1119. first.merge_operator = MergeOperators::CreateUInt64AddOperator();
  1120. second.merge_operator = MergeOperators::CreateStringAppendOperator();
  1121. Reopen({default_cf, first, second});
  1122. std::string one, two, three;
  1123. PutFixed64(&one, 1);
  1124. PutFixed64(&two, 2);
  1125. PutFixed64(&three, 3);
  1126. ASSERT_OK(Put(0, "foo", two));
  1127. ASSERT_OK(Put(0, "foo", one));
  1128. ASSERT_TRUE(Merge(0, "foo", two).IsNotSupported());
  1129. ASSERT_EQ(Get(0, "foo"), one);
  1130. ASSERT_OK(Put(1, "foo", two));
  1131. ASSERT_OK(Put(1, "foo", one));
  1132. ASSERT_OK(Merge(1, "foo", two));
  1133. ASSERT_EQ(Get(1, "foo"), three);
  1134. ASSERT_OK(Put(2, "foo", two));
  1135. ASSERT_OK(Put(2, "foo", one));
  1136. ASSERT_OK(Merge(2, "foo", two));
  1137. ASSERT_EQ(Get(2, "foo"), one + "," + two);
  1138. Close();
  1139. }
  1140. TEST_P(ColumnFamilyTest, DifferentCompactionStyles) {
  1141. Open();
  1142. CreateColumnFamilies({"one", "two"});
  1143. ColumnFamilyOptions default_cf, one, two;
  1144. db_options_.max_open_files = 20; // only 10 files in file cache
  1145. default_cf.level_compaction_dynamic_level_bytes = false;
  1146. default_cf.compaction_style = kCompactionStyleLevel;
  1147. default_cf.num_levels = 3;
  1148. default_cf.write_buffer_size = 64 << 10; // 64KB
  1149. default_cf.target_file_size_base = 30 << 10;
  1150. default_cf.max_compaction_bytes = static_cast<uint64_t>(1) << 60;
  1151. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1152. table_options.no_block_cache = true;
  1153. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1154. one.compaction_style = kCompactionStyleUniversal;
  1155. one.num_levels = 1;
  1156. // trigger compaction if there are >= 4 files
  1157. one.level0_file_num_compaction_trigger = 4;
  1158. one.write_buffer_size = 120000;
  1159. two.level_compaction_dynamic_level_bytes = false;
  1160. two.compaction_style = kCompactionStyleLevel;
  1161. two.num_levels = 4;
  1162. two.level0_file_num_compaction_trigger = 3;
  1163. two.write_buffer_size = 100000;
  1164. Reopen({default_cf, one, two});
  1165. // SETUP column family "one" -- universal style
  1166. for (int i = 0; i < one.level0_file_num_compaction_trigger - 1; ++i) {
  1167. PutRandomData(1, 10, 12000);
  1168. PutRandomData(1, 1, 10);
  1169. WaitForFlush(1);
  1170. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1171. }
  1172. // SETUP column family "two" -- level style with 4 levels
  1173. for (int i = 0; i < two.level0_file_num_compaction_trigger - 1; ++i) {
  1174. PutRandomData(2, 10, 12000);
  1175. PutRandomData(2, 1, 10);
  1176. WaitForFlush(2);
  1177. AssertFilesPerLevel(std::to_string(i + 1), 2);
  1178. }
  1179. // TRIGGER compaction "one"
  1180. PutRandomData(1, 10, 12000);
  1181. PutRandomData(1, 1, 10);
  1182. // TRIGGER compaction "two"
  1183. PutRandomData(2, 10, 12000);
  1184. PutRandomData(2, 1, 10);
  1185. // WAIT for compactions
  1186. WaitForCompaction();
  1187. // VERIFY compaction "one"
  1188. AssertFilesPerLevel("1", 1);
  1189. // VERIFY compaction "two"
  1190. AssertFilesPerLevel("0,1", 2);
  1191. CompactAll(2);
  1192. AssertFilesPerLevel("0,1", 2);
  1193. Close();
  1194. }
  1195. // Sync points not supported in RocksDB Lite
  1196. TEST_P(ColumnFamilyTest, MultipleManualCompactions) {
  1197. Open();
  1198. CreateColumnFamilies({"one", "two"});
  1199. ColumnFamilyOptions default_cf, one, two;
  1200. db_options_.max_open_files = 20; // only 10 files in file cache
  1201. db_options_.max_background_compactions = 3;
  1202. default_cf.level_compaction_dynamic_level_bytes = false;
  1203. default_cf.compaction_style = kCompactionStyleLevel;
  1204. default_cf.num_levels = 3;
  1205. default_cf.write_buffer_size = 64 << 10; // 64KB
  1206. default_cf.target_file_size_base = 30 << 10;
  1207. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1208. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1209. table_options.no_block_cache = true;
  1210. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1211. one.compaction_style = kCompactionStyleUniversal;
  1212. one.num_levels = 1;
  1213. // trigger compaction if there are >= 4 files
  1214. one.level0_file_num_compaction_trigger = 4;
  1215. one.write_buffer_size = 120000;
  1216. two.level_compaction_dynamic_level_bytes = false;
  1217. two.compaction_style = kCompactionStyleLevel;
  1218. two.num_levels = 4;
  1219. two.level0_file_num_compaction_trigger = 3;
  1220. two.write_buffer_size = 100000;
  1221. Reopen({default_cf, one, two});
  1222. // SETUP column family "one" -- universal style
  1223. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1224. PutRandomData(1, 10, 12000, true);
  1225. PutRandomData(1, 1, 10, true);
  1226. WaitForFlush(1);
  1227. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1228. }
  1229. std::atomic_bool cf_1_1{true};
  1230. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1231. {{"ColumnFamilyTest::MultiManual:4", "ColumnFamilyTest::MultiManual:1"},
  1232. {"ColumnFamilyTest::MultiManual:2", "ColumnFamilyTest::MultiManual:5"},
  1233. {"ColumnFamilyTest::MultiManual:2", "ColumnFamilyTest::MultiManual:3"}});
  1234. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1235. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1236. if (cf_1_1.exchange(false)) {
  1237. TEST_SYNC_POINT("ColumnFamilyTest::MultiManual:4");
  1238. TEST_SYNC_POINT("ColumnFamilyTest::MultiManual:3");
  1239. }
  1240. });
  1241. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1242. std::vector<port::Thread> threads;
  1243. threads.emplace_back([&] {
  1244. CompactRangeOptions compact_options;
  1245. compact_options.exclusive_manual_compaction = false;
  1246. ASSERT_OK(
  1247. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1248. });
  1249. // SETUP column family "two" -- level style with 4 levels
  1250. for (int i = 0; i < two.level0_file_num_compaction_trigger - 2; ++i) {
  1251. PutRandomData(2, 10, 12000);
  1252. PutRandomData(2, 1, 10);
  1253. WaitForFlush(2);
  1254. AssertFilesPerLevel(std::to_string(i + 1), 2);
  1255. }
  1256. threads.emplace_back([&] {
  1257. TEST_SYNC_POINT("ColumnFamilyTest::MultiManual:1");
  1258. CompactRangeOptions compact_options;
  1259. compact_options.exclusive_manual_compaction = false;
  1260. ASSERT_OK(
  1261. db_->CompactRange(compact_options, handles_[2], nullptr, nullptr));
  1262. TEST_SYNC_POINT("ColumnFamilyTest::MultiManual:2");
  1263. });
  1264. TEST_SYNC_POINT("ColumnFamilyTest::MultiManual:5");
  1265. for (auto& t : threads) {
  1266. t.join();
  1267. }
  1268. // VERIFY compaction "one"
  1269. AssertFilesPerLevel("1", 1);
  1270. // VERIFY compaction "two"
  1271. AssertFilesPerLevel("0,1", 2);
  1272. CompactAll(2);
  1273. AssertFilesPerLevel("0,1", 2);
  1274. // Compare against saved keys
  1275. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1276. while (key_iter != keys_[1].end()) {
  1277. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1278. key_iter++;
  1279. }
  1280. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1281. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1282. Close();
  1283. }
  1284. TEST_P(ColumnFamilyTest, AutomaticAndManualCompactions) {
  1285. Open();
  1286. CreateColumnFamilies({"one", "two"});
  1287. ColumnFamilyOptions default_cf, one, two;
  1288. db_options_.max_open_files = 20; // only 10 files in file cache
  1289. db_options_.max_background_compactions = 3;
  1290. default_cf.level_compaction_dynamic_level_bytes = false;
  1291. default_cf.compaction_style = kCompactionStyleLevel;
  1292. default_cf.num_levels = 3;
  1293. default_cf.write_buffer_size = 64 << 10; // 64KB
  1294. default_cf.target_file_size_base = 30 << 10;
  1295. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1296. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1297. table_options.no_block_cache = true;
  1298. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1299. one.compaction_style = kCompactionStyleUniversal;
  1300. one.num_levels = 1;
  1301. // trigger compaction if there are >= 4 files
  1302. one.level0_file_num_compaction_trigger = 4;
  1303. one.write_buffer_size = 120000;
  1304. two.level_compaction_dynamic_level_bytes = false;
  1305. two.compaction_style = kCompactionStyleLevel;
  1306. two.num_levels = 4;
  1307. two.level0_file_num_compaction_trigger = 3;
  1308. two.write_buffer_size = 100000;
  1309. Reopen({default_cf, one, two});
  1310. // make sure all background compaction jobs can be scheduled
  1311. auto stop_token =
  1312. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1313. std::atomic_bool cf_1_1{true};
  1314. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1315. {{"ColumnFamilyTest::AutoManual:4", "ColumnFamilyTest::AutoManual:1"},
  1316. {"ColumnFamilyTest::AutoManual:2", "ColumnFamilyTest::AutoManual:5"},
  1317. {"ColumnFamilyTest::AutoManual:2", "ColumnFamilyTest::AutoManual:3"}});
  1318. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1319. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1320. if (cf_1_1.exchange(false)) {
  1321. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:4");
  1322. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:3");
  1323. }
  1324. });
  1325. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1326. // SETUP column family "one" -- universal style
  1327. for (int i = 0; i < one.level0_file_num_compaction_trigger; ++i) {
  1328. PutRandomData(1, 10, 12000, true);
  1329. PutRandomData(1, 1, 10, true);
  1330. WaitForFlush(1);
  1331. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1332. }
  1333. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:1");
  1334. // SETUP column family "two" -- level style with 4 levels
  1335. for (int i = 0; i < two.level0_file_num_compaction_trigger - 2; ++i) {
  1336. PutRandomData(2, 10, 12000);
  1337. PutRandomData(2, 1, 10);
  1338. WaitForFlush(2);
  1339. AssertFilesPerLevel(std::to_string(i + 1), 2);
  1340. }
  1341. ROCKSDB_NAMESPACE::port::Thread threads([&] {
  1342. CompactRangeOptions compact_options;
  1343. compact_options.exclusive_manual_compaction = false;
  1344. ASSERT_OK(
  1345. db_->CompactRange(compact_options, handles_[2], nullptr, nullptr));
  1346. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:2");
  1347. });
  1348. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:5");
  1349. threads.join();
  1350. // WAIT for compactions
  1351. WaitForCompaction();
  1352. // VERIFY compaction "one"
  1353. AssertFilesPerLevel("1", 1);
  1354. // VERIFY compaction "two"
  1355. AssertFilesPerLevel("0,1", 2);
  1356. CompactAll(2);
  1357. AssertFilesPerLevel("0,1", 2);
  1358. // Compare against saved keys
  1359. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1360. while (key_iter != keys_[1].end()) {
  1361. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1362. key_iter++;
  1363. }
  1364. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1365. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1366. }
  1367. TEST_P(ColumnFamilyTest, ManualAndAutomaticCompactions) {
  1368. Open();
  1369. CreateColumnFamilies({"one", "two"});
  1370. ColumnFamilyOptions default_cf, one, two;
  1371. db_options_.max_open_files = 20; // only 10 files in file cache
  1372. db_options_.max_background_compactions = 3;
  1373. default_cf.level_compaction_dynamic_level_bytes = false;
  1374. default_cf.compaction_style = kCompactionStyleLevel;
  1375. default_cf.num_levels = 3;
  1376. default_cf.write_buffer_size = 64 << 10; // 64KB
  1377. default_cf.target_file_size_base = 30 << 10;
  1378. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1379. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1380. table_options.no_block_cache = true;
  1381. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1382. one.compaction_style = kCompactionStyleUniversal;
  1383. one.num_levels = 1;
  1384. // trigger compaction if there are >= 4 files
  1385. one.level0_file_num_compaction_trigger = 4;
  1386. one.write_buffer_size = 120000;
  1387. two.level_compaction_dynamic_level_bytes = false;
  1388. two.compaction_style = kCompactionStyleLevel;
  1389. two.num_levels = 4;
  1390. two.level0_file_num_compaction_trigger = 3;
  1391. two.write_buffer_size = 100000;
  1392. Reopen({default_cf, one, two});
  1393. // make sure all background compaction jobs can be scheduled
  1394. auto stop_token =
  1395. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1396. // SETUP column family "one" -- universal style
  1397. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1398. PutRandomData(1, 10, 12000, true);
  1399. PutRandomData(1, 1, 10, true);
  1400. WaitForFlush(1);
  1401. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1402. }
  1403. std::atomic_bool cf_1_1{true};
  1404. std::atomic_bool cf_1_2{true};
  1405. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1406. {{"ColumnFamilyTest::ManualAuto:4", "ColumnFamilyTest::ManualAuto:1"},
  1407. {"ColumnFamilyTest::ManualAuto:5", "ColumnFamilyTest::ManualAuto:2"},
  1408. {"ColumnFamilyTest::ManualAuto:2", "ColumnFamilyTest::ManualAuto:3"}});
  1409. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1410. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1411. if (cf_1_1.exchange(false)) {
  1412. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:4");
  1413. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:3");
  1414. } else if (cf_1_2.exchange(false)) {
  1415. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:2");
  1416. }
  1417. });
  1418. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1419. ROCKSDB_NAMESPACE::port::Thread threads([&] {
  1420. CompactRangeOptions compact_options;
  1421. compact_options.exclusive_manual_compaction = false;
  1422. ASSERT_OK(
  1423. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1424. });
  1425. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:1");
  1426. // SETUP column family "two" -- level style with 4 levels
  1427. for (int i = 0; i < two.level0_file_num_compaction_trigger; ++i) {
  1428. PutRandomData(2, 10, 12000);
  1429. PutRandomData(2, 1, 10);
  1430. WaitForFlush(2);
  1431. AssertFilesPerLevel(std::to_string(i + 1), 2);
  1432. }
  1433. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:5");
  1434. threads.join();
  1435. // WAIT for compactions
  1436. WaitForCompaction();
  1437. // VERIFY compaction "one"
  1438. AssertFilesPerLevel("1", 1);
  1439. // VERIFY compaction "two"
  1440. AssertFilesPerLevel("0,1", 2);
  1441. CompactAll(2);
  1442. AssertFilesPerLevel("0,1", 2);
  1443. // Compare against saved keys
  1444. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1445. while (key_iter != keys_[1].end()) {
  1446. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1447. key_iter++;
  1448. }
  1449. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1450. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1451. }
  1452. TEST_P(ColumnFamilyTest, SameCFManualManualCompactions) {
  1453. Open();
  1454. CreateColumnFamilies({"one"});
  1455. ColumnFamilyOptions default_cf, one;
  1456. db_options_.max_open_files = 20; // only 10 files in file cache
  1457. db_options_.max_background_compactions = 3;
  1458. default_cf.compaction_style = kCompactionStyleLevel;
  1459. default_cf.num_levels = 3;
  1460. default_cf.write_buffer_size = 64 << 10; // 64KB
  1461. default_cf.target_file_size_base = 30 << 10;
  1462. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1463. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1464. ;
  1465. table_options.no_block_cache = true;
  1466. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1467. one.compaction_style = kCompactionStyleUniversal;
  1468. one.num_levels = 1;
  1469. // trigger compaction if there are >= 4 files
  1470. one.level0_file_num_compaction_trigger = 4;
  1471. one.write_buffer_size = 120000;
  1472. Reopen({default_cf, one});
  1473. // make sure all background compaction jobs can be scheduled
  1474. auto stop_token =
  1475. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1476. // SETUP column family "one" -- universal style
  1477. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1478. PutRandomData(1, 10, 12000, true);
  1479. PutRandomData(1, 1, 10, true);
  1480. WaitForFlush(1);
  1481. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1482. }
  1483. std::atomic_bool cf_1_1{true};
  1484. std::atomic_bool cf_1_2{true};
  1485. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1486. {{"ColumnFamilyTest::ManualManual:4", "ColumnFamilyTest::ManualManual:2"},
  1487. {"ColumnFamilyTest::ManualManual:4", "ColumnFamilyTest::ManualManual:5"},
  1488. {"ColumnFamilyTest::ManualManual:1", "ColumnFamilyTest::ManualManual:2"},
  1489. {"ColumnFamilyTest::ManualManual:1",
  1490. "ColumnFamilyTest::ManualManual:3"}});
  1491. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1492. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1493. if (cf_1_1.exchange(false)) {
  1494. TEST_SYNC_POINT("ColumnFamilyTest::ManualManual:4");
  1495. TEST_SYNC_POINT("ColumnFamilyTest::ManualManual:3");
  1496. } else if (cf_1_2.exchange(false)) {
  1497. TEST_SYNC_POINT("ColumnFamilyTest::ManualManual:2");
  1498. }
  1499. });
  1500. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1501. ROCKSDB_NAMESPACE::port::Thread threads([&] {
  1502. CompactRangeOptions compact_options;
  1503. compact_options.exclusive_manual_compaction = true;
  1504. ASSERT_OK(
  1505. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1506. });
  1507. TEST_SYNC_POINT("ColumnFamilyTest::ManualManual:5");
  1508. WaitForFlush(1);
  1509. // Add more L0 files and force another manual compaction
  1510. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1511. PutRandomData(1, 10, 12000, true);
  1512. PutRandomData(1, 1, 10, true);
  1513. WaitForFlush(1);
  1514. AssertFilesPerLevel(
  1515. std::to_string(one.level0_file_num_compaction_trigger + i), 1);
  1516. }
  1517. ROCKSDB_NAMESPACE::port::Thread threads1([&] {
  1518. CompactRangeOptions compact_options;
  1519. compact_options.exclusive_manual_compaction = false;
  1520. ASSERT_OK(
  1521. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1522. });
  1523. TEST_SYNC_POINT("ColumnFamilyTest::ManualManual:1");
  1524. threads.join();
  1525. threads1.join();
  1526. WaitForCompaction();
  1527. // VERIFY compaction "one"
  1528. ASSERT_LE(NumTableFilesAtLevel(0, 1), 2);
  1529. // Compare against saved keys
  1530. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1531. while (key_iter != keys_[1].end()) {
  1532. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1533. key_iter++;
  1534. }
  1535. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1536. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1537. }
  1538. TEST_P(ColumnFamilyTest, SameCFManualAutomaticCompactions) {
  1539. Open();
  1540. CreateColumnFamilies({"one"});
  1541. ColumnFamilyOptions default_cf, one;
  1542. db_options_.max_open_files = 20; // only 10 files in file cache
  1543. db_options_.max_background_compactions = 3;
  1544. default_cf.compaction_style = kCompactionStyleLevel;
  1545. default_cf.num_levels = 3;
  1546. default_cf.write_buffer_size = 64 << 10; // 64KB
  1547. default_cf.target_file_size_base = 30 << 10;
  1548. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1549. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1550. ;
  1551. table_options.no_block_cache = true;
  1552. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1553. one.compaction_style = kCompactionStyleUniversal;
  1554. one.num_levels = 1;
  1555. // trigger compaction if there are >= 4 files
  1556. one.level0_file_num_compaction_trigger = 4;
  1557. one.write_buffer_size = 120000;
  1558. Reopen({default_cf, one});
  1559. // make sure all background compaction jobs can be scheduled
  1560. auto stop_token =
  1561. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1562. // SETUP column family "one" -- universal style
  1563. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1564. PutRandomData(1, 10, 12000, true);
  1565. PutRandomData(1, 1, 10, true);
  1566. WaitForFlush(1);
  1567. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1568. }
  1569. std::atomic_bool cf_1_1{true};
  1570. std::atomic_bool cf_1_2{true};
  1571. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1572. {{"ColumnFamilyTest::ManualAuto:4", "ColumnFamilyTest::ManualAuto:2"},
  1573. {"ColumnFamilyTest::ManualAuto:4", "ColumnFamilyTest::ManualAuto:5"},
  1574. {"ColumnFamilyTest::ManualAuto:1", "ColumnFamilyTest::ManualAuto:2"},
  1575. {"ColumnFamilyTest::ManualAuto:1", "ColumnFamilyTest::ManualAuto:3"}});
  1576. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1577. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1578. if (cf_1_1.exchange(false)) {
  1579. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:4");
  1580. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:3");
  1581. } else if (cf_1_2.exchange(false)) {
  1582. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:2");
  1583. }
  1584. });
  1585. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1586. ROCKSDB_NAMESPACE::port::Thread threads([&] {
  1587. CompactRangeOptions compact_options;
  1588. compact_options.exclusive_manual_compaction = false;
  1589. ASSERT_OK(
  1590. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1591. });
  1592. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:5");
  1593. WaitForFlush(1);
  1594. // Add more L0 files and force automatic compaction
  1595. for (int i = 0; i < one.level0_file_num_compaction_trigger; ++i) {
  1596. PutRandomData(1, 10, 12000, true);
  1597. PutRandomData(1, 1, 10, true);
  1598. WaitForFlush(1);
  1599. AssertFilesPerLevel(
  1600. std::to_string(one.level0_file_num_compaction_trigger + i), 1);
  1601. }
  1602. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:1");
  1603. threads.join();
  1604. WaitForCompaction();
  1605. // VERIFY compaction "one"
  1606. ASSERT_LE(NumTableFilesAtLevel(0, 1), 2);
  1607. // Compare against saved keys
  1608. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1609. while (key_iter != keys_[1].end()) {
  1610. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1611. key_iter++;
  1612. }
  1613. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1614. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1615. }
  1616. TEST_P(ColumnFamilyTest, SameCFManualAutomaticCompactionsLevel) {
  1617. Open();
  1618. CreateColumnFamilies({"one"});
  1619. ColumnFamilyOptions default_cf, one;
  1620. db_options_.max_open_files = 20; // only 10 files in file cache
  1621. db_options_.max_background_compactions = 3;
  1622. default_cf.compaction_style = kCompactionStyleLevel;
  1623. default_cf.num_levels = 3;
  1624. default_cf.write_buffer_size = 64 << 10; // 64KB
  1625. default_cf.target_file_size_base = 30 << 10;
  1626. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1627. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1628. ;
  1629. table_options.no_block_cache = true;
  1630. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1631. one.compaction_style = kCompactionStyleLevel;
  1632. one.num_levels = 1;
  1633. // trigger compaction if there are >= 4 files
  1634. one.level0_file_num_compaction_trigger = 3;
  1635. one.write_buffer_size = 120000;
  1636. Reopen({default_cf, one});
  1637. // make sure all background compaction jobs can be scheduled
  1638. auto stop_token =
  1639. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1640. // SETUP column family "one" -- level style
  1641. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1642. PutRandomData(1, 10, 12000, true);
  1643. PutRandomData(1, 1, 10, true);
  1644. WaitForFlush(1);
  1645. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1646. }
  1647. std::atomic_bool cf_1_1{true};
  1648. std::atomic_bool cf_1_2{true};
  1649. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1650. {{"ColumnFamilyTest::ManualAuto:4", "ColumnFamilyTest::ManualAuto:2"},
  1651. {"ColumnFamilyTest::ManualAuto:4", "ColumnFamilyTest::ManualAuto:5"},
  1652. {"ColumnFamilyTest::ManualAuto:3", "ColumnFamilyTest::ManualAuto:2"},
  1653. {"LevelCompactionPicker::PickCompactionBySize:0",
  1654. "ColumnFamilyTest::ManualAuto:3"},
  1655. {"ColumnFamilyTest::ManualAuto:1", "ColumnFamilyTest::ManualAuto:3"}});
  1656. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1657. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1658. if (cf_1_1.exchange(false)) {
  1659. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:4");
  1660. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:3");
  1661. } else if (cf_1_2.exchange(false)) {
  1662. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:2");
  1663. }
  1664. });
  1665. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1666. ROCKSDB_NAMESPACE::port::Thread threads([&] {
  1667. CompactRangeOptions compact_options;
  1668. compact_options.exclusive_manual_compaction = false;
  1669. ASSERT_OK(
  1670. db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1671. });
  1672. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:5");
  1673. // Add more L0 files and force automatic compaction
  1674. for (int i = 0; i < one.level0_file_num_compaction_trigger; ++i) {
  1675. PutRandomData(1, 10, 12000, true);
  1676. PutRandomData(1, 1, 10, true);
  1677. WaitForFlush(1);
  1678. AssertFilesPerLevel(
  1679. std::to_string(one.level0_file_num_compaction_trigger + i), 1);
  1680. }
  1681. TEST_SYNC_POINT("ColumnFamilyTest::ManualAuto:1");
  1682. threads.join();
  1683. WaitForCompaction();
  1684. // VERIFY compaction "one"
  1685. AssertFilesPerLevel("0,1", 1);
  1686. // Compare against saved keys
  1687. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1688. while (key_iter != keys_[1].end()) {
  1689. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1690. key_iter++;
  1691. }
  1692. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1693. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1694. }
  1695. // In this test, we generate enough files to trigger automatic compactions.
  1696. // The automatic compaction waits in NonTrivial:AfterRun
  1697. // We generate more files and then trigger an automatic compaction
  1698. // This will wait because the automatic compaction has files it needs.
  1699. // Once the conflict is hit, the automatic compaction starts and ends
  1700. // Then the manual will run and end.
  1701. TEST_P(ColumnFamilyTest, SameCFAutomaticManualCompactions) {
  1702. Open();
  1703. CreateColumnFamilies({"one"});
  1704. ColumnFamilyOptions default_cf, one;
  1705. db_options_.max_open_files = 20; // only 10 files in file cache
  1706. db_options_.max_background_compactions = 3;
  1707. default_cf.compaction_style = kCompactionStyleLevel;
  1708. default_cf.num_levels = 3;
  1709. default_cf.write_buffer_size = 64 << 10; // 64KB
  1710. default_cf.target_file_size_base = 30 << 10;
  1711. default_cf.max_compaction_bytes = default_cf.target_file_size_base * 1100;
  1712. BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
  1713. ;
  1714. table_options.no_block_cache = true;
  1715. default_cf.table_factory.reset(NewBlockBasedTableFactory(table_options));
  1716. one.compaction_style = kCompactionStyleUniversal;
  1717. one.num_levels = 1;
  1718. // trigger compaction if there are >= 4 files
  1719. one.level0_file_num_compaction_trigger = 4;
  1720. one.write_buffer_size = 120000;
  1721. Reopen({default_cf, one});
  1722. // make sure all background compaction jobs can be scheduled
  1723. auto stop_token =
  1724. dbfull()->TEST_write_controler().GetCompactionPressureToken();
  1725. std::atomic_bool cf_1_1{true};
  1726. std::atomic_bool cf_1_2{true};
  1727. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  1728. {{"ColumnFamilyTest::AutoManual:4", "ColumnFamilyTest::AutoManual:2"},
  1729. {"ColumnFamilyTest::AutoManual:4", "ColumnFamilyTest::AutoManual:5"},
  1730. {"CompactionPicker::CompactRange:Conflict",
  1731. "ColumnFamilyTest::AutoManual:3"}});
  1732. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  1733. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun", [&](void* /*arg*/) {
  1734. if (cf_1_1.exchange(false)) {
  1735. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:4");
  1736. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:3");
  1737. } else if (cf_1_2.exchange(false)) {
  1738. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:2");
  1739. }
  1740. });
  1741. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  1742. // SETUP column family "one" -- universal style
  1743. for (int i = 0; i < one.level0_file_num_compaction_trigger; ++i) {
  1744. PutRandomData(1, 10, 12000, true);
  1745. PutRandomData(1, 1, 10, true);
  1746. WaitForFlush(1);
  1747. AssertFilesPerLevel(std::to_string(i + 1), 1);
  1748. }
  1749. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:5");
  1750. // Add another L0 file and force automatic compaction
  1751. for (int i = 0; i < one.level0_file_num_compaction_trigger - 2; ++i) {
  1752. PutRandomData(1, 10, 12000, true);
  1753. PutRandomData(1, 1, 10, true);
  1754. WaitForFlush(1);
  1755. }
  1756. CompactRangeOptions compact_options;
  1757. compact_options.exclusive_manual_compaction = false;
  1758. ASSERT_OK(db_->CompactRange(compact_options, handles_[1], nullptr, nullptr));
  1759. TEST_SYNC_POINT("ColumnFamilyTest::AutoManual:1");
  1760. WaitForCompaction();
  1761. // VERIFY compaction "one"
  1762. AssertFilesPerLevel("1", 1);
  1763. // Compare against saved keys
  1764. std::set<std::string>::iterator key_iter = keys_[1].begin();
  1765. while (key_iter != keys_[1].end()) {
  1766. ASSERT_NE("NOT_FOUND", Get(1, *key_iter));
  1767. key_iter++;
  1768. }
  1769. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  1770. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  1771. }
  1772. namespace {
  1773. std::string IterStatus(Iterator* iter) {
  1774. std::string result;
  1775. if (iter->Valid()) {
  1776. result = iter->key().ToString() + "->" + iter->value().ToString();
  1777. } else {
  1778. EXPECT_OK(iter->status());
  1779. result = "(invalid)";
  1780. }
  1781. return result;
  1782. }
  1783. } // anonymous namespace
  1784. TEST_P(ColumnFamilyTest, NewIteratorsTest) {
  1785. // iter == 0 -- no tailing
  1786. // iter == 2 -- tailing
  1787. for (int iter = 0; iter < 2; ++iter) {
  1788. Open();
  1789. CreateColumnFamiliesAndReopen({"one", "two"});
  1790. ASSERT_OK(Put(0, "a", "b"));
  1791. ASSERT_OK(Put(1, "b", "a"));
  1792. ASSERT_OK(Put(2, "c", "m"));
  1793. ASSERT_OK(Put(2, "v", "t"));
  1794. std::vector<Iterator*> iterators;
  1795. ReadOptions options;
  1796. options.tailing = (iter == 1);
  1797. ASSERT_OK(db_->NewIterators(options, handles_, &iterators));
  1798. for (auto it : iterators) {
  1799. it->SeekToFirst();
  1800. }
  1801. ASSERT_EQ(IterStatus(iterators[0]), "a->b");
  1802. ASSERT_EQ(IterStatus(iterators[1]), "b->a");
  1803. ASSERT_EQ(IterStatus(iterators[2]), "c->m");
  1804. ASSERT_OK(Put(1, "x", "x"));
  1805. for (auto it : iterators) {
  1806. it->Next();
  1807. }
  1808. ASSERT_EQ(IterStatus(iterators[0]), "(invalid)");
  1809. if (iter == 0) {
  1810. // no tailing
  1811. ASSERT_EQ(IterStatus(iterators[1]), "(invalid)");
  1812. } else {
  1813. // tailing
  1814. ASSERT_EQ(IterStatus(iterators[1]), "x->x");
  1815. }
  1816. ASSERT_EQ(IterStatus(iterators[2]), "v->t");
  1817. for (auto it : iterators) {
  1818. delete it;
  1819. }
  1820. Destroy();
  1821. }
  1822. }
  1823. TEST_P(ColumnFamilyTest, ReadOnlyDBTest) {
  1824. Open();
  1825. CreateColumnFamiliesAndReopen({"one", "two", "three", "four"});
  1826. ASSERT_OK(Put(0, "a", "b"));
  1827. ASSERT_OK(Put(1, "foo", "bla"));
  1828. ASSERT_OK(Put(2, "foo", "blabla"));
  1829. ASSERT_OK(Put(3, "foo", "blablabla"));
  1830. ASSERT_OK(Put(4, "foo", "blablablabla"));
  1831. DropColumnFamilies({2});
  1832. Close();
  1833. // open only a subset of column families
  1834. AssertOpenReadOnly({"default", "one", "four"});
  1835. ASSERT_EQ("NOT_FOUND", Get(0, "foo"));
  1836. ASSERT_EQ("bla", Get(1, "foo"));
  1837. ASSERT_EQ("blablablabla", Get(2, "foo"));
  1838. // test newiterators
  1839. {
  1840. std::vector<Iterator*> iterators;
  1841. ASSERT_OK(db_->NewIterators(ReadOptions(), handles_, &iterators));
  1842. for (auto it : iterators) {
  1843. it->SeekToFirst();
  1844. }
  1845. ASSERT_EQ(IterStatus(iterators[0]), "a->b");
  1846. ASSERT_EQ(IterStatus(iterators[1]), "foo->bla");
  1847. ASSERT_EQ(IterStatus(iterators[2]), "foo->blablablabla");
  1848. for (auto it : iterators) {
  1849. it->Next();
  1850. }
  1851. ASSERT_EQ(IterStatus(iterators[0]), "(invalid)");
  1852. ASSERT_EQ(IterStatus(iterators[1]), "(invalid)");
  1853. ASSERT_EQ(IterStatus(iterators[2]), "(invalid)");
  1854. for (auto it : iterators) {
  1855. delete it;
  1856. }
  1857. }
  1858. Close();
  1859. // can't open dropped column family
  1860. Status s = OpenReadOnly({"default", "one", "two"});
  1861. ASSERT_TRUE(!s.ok());
  1862. // Can't open without specifying default column family
  1863. s = OpenReadOnly({"one", "four"});
  1864. ASSERT_TRUE(!s.ok());
  1865. }
  1866. TEST_P(ColumnFamilyTest, DontRollEmptyLogs) {
  1867. Open();
  1868. CreateColumnFamiliesAndReopen({"one", "two", "three", "four"});
  1869. for (size_t i = 0; i < handles_.size(); ++i) {
  1870. PutRandomData(static_cast<int>(i), 10, 100);
  1871. }
  1872. int num_writable_file_start = env_->GetNumberOfNewWritableFileCalls();
  1873. // this will trigger the flushes
  1874. for (int i = 0; i <= 4; ++i) {
  1875. ASSERT_OK(Flush(i));
  1876. }
  1877. for (int i = 0; i < 4; ++i) {
  1878. WaitForFlush(i);
  1879. }
  1880. int total_new_writable_files =
  1881. env_->GetNumberOfNewWritableFileCalls() - num_writable_file_start;
  1882. ASSERT_EQ(static_cast<size_t>(total_new_writable_files), handles_.size() + 1);
  1883. Close();
  1884. }
  1885. TEST_P(ColumnFamilyTest, FlushStaleColumnFamilies) {
  1886. Open();
  1887. CreateColumnFamilies({"one", "two"});
  1888. ColumnFamilyOptions default_cf, one, two;
  1889. default_cf.write_buffer_size = 100000; // small write buffer size
  1890. default_cf.arena_block_size = 4096;
  1891. default_cf.disable_auto_compactions = true;
  1892. one.disable_auto_compactions = true;
  1893. two.disable_auto_compactions = true;
  1894. db_options_.max_total_wal_size = 210000;
  1895. Reopen({default_cf, one, two});
  1896. PutRandomData(2, 1, 10); // 10 bytes
  1897. for (int i = 0; i < 2; ++i) {
  1898. PutRandomData(0, 100, 1000); // flush
  1899. WaitForFlush(0);
  1900. AssertCountLiveFiles(i + 1);
  1901. }
  1902. // third flush. now, CF [two] should be detected as stale and flushed
  1903. // column family 1 should not be flushed since it's empty
  1904. PutRandomData(0, 100, 1000); // flush
  1905. WaitForFlush(0);
  1906. WaitForFlush(2);
  1907. // at least 3 files for default column families, 1 file for column family
  1908. // [two], zero files for column family [one], because it's empty
  1909. std::vector<LiveFileMetaData> metadata;
  1910. db_->GetLiveFilesMetaData(&metadata);
  1911. ASSERT_GE(metadata.size(), 4);
  1912. bool has_cf1_sst = false;
  1913. bool has_cf2_sst = false;
  1914. for (const auto& file : metadata) {
  1915. if (file.column_family_name == "one") {
  1916. has_cf1_sst = true;
  1917. } else if (file.column_family_name == "two") {
  1918. has_cf2_sst = true;
  1919. }
  1920. }
  1921. ASSERT_FALSE(has_cf1_sst);
  1922. ASSERT_TRUE(has_cf2_sst);
  1923. ASSERT_OK(Flush(0));
  1924. ASSERT_EQ(0, dbfull()->TEST_wals_total_size());
  1925. Close();
  1926. }
  1927. namespace {
  1928. struct CountOptionsFilesFs : public FileSystemWrapper {
  1929. explicit CountOptionsFilesFs(const std::shared_ptr<FileSystem>& t)
  1930. : FileSystemWrapper(t) {}
  1931. const char* Name() const override { return "CountOptionsFilesFs"; }
  1932. IOStatus NewWritableFile(const std::string& f, const FileOptions& file_opts,
  1933. std::unique_ptr<FSWritableFile>* r,
  1934. IODebugContext* dbg) override {
  1935. if (f.find("OPTIONS-") != std::string::npos) {
  1936. options_files_created.fetch_add(1, std::memory_order_relaxed);
  1937. }
  1938. return FileSystemWrapper::NewWritableFile(f, file_opts, r, dbg);
  1939. }
  1940. std::atomic<int> options_files_created{};
  1941. };
  1942. } // namespace
  1943. TEST_P(ColumnFamilyTest, CreateMissingColumnFamilies) {
  1944. // Can't accidentally add CFs to an existing DB
  1945. Open();
  1946. Close();
  1947. ASSERT_FALSE(db_options_.create_missing_column_families);
  1948. ASSERT_NOK(TryOpen({"one", "two"}));
  1949. // Nor accidentally create in a new DB
  1950. Destroy();
  1951. db_options_.create_if_missing = true;
  1952. ASSERT_NOK(TryOpen({"one", "two"}));
  1953. // Only with the option (new DB case)
  1954. db_options_.create_missing_column_families = true;
  1955. // Also setup to count number of options files created (see check below)
  1956. auto my_fs =
  1957. std::make_shared<CountOptionsFilesFs>(db_options_.env->GetFileSystem());
  1958. auto my_env = std::make_unique<CompositeEnvWrapper>(db_options_.env, my_fs);
  1959. SaveAndRestore<Env*> save_restore_env(&db_options_.env, my_env.get());
  1960. ASSERT_OK(TryOpen({"default", "one", "two"}));
  1961. Close();
  1962. // An older version would write an updated options file for each column
  1963. // family created under create_missing_column_families, which would be
  1964. // quadratic I/O in the number of column families.
  1965. ASSERT_EQ(my_fs->options_files_created.load(), 1);
  1966. // Add to existing DB case
  1967. ASSERT_OK(TryOpen({"default", "one", "two", "three", "four"}));
  1968. Close();
  1969. ASSERT_EQ(my_fs->options_files_created.load(), 2);
  1970. }
  1971. TEST_P(ColumnFamilyTest, SanitizeCfOptions) {
  1972. DBOptions db_options;
  1973. for (int s = kCompactionStyleLevel; s <= kCompactionStyleUniversal; ++s) {
  1974. for (int l = 0; l <= 2; l++) {
  1975. for (int i = 1; i <= 3; i++) {
  1976. for (int j = 1; j <= 3; j++) {
  1977. for (int k = 1; k <= 3; k++) {
  1978. ColumnFamilyOptions original;
  1979. original.compaction_style = static_cast<CompactionStyle>(s);
  1980. original.num_levels = l;
  1981. original.level0_stop_writes_trigger = i;
  1982. original.level0_slowdown_writes_trigger = j;
  1983. original.level0_file_num_compaction_trigger = k;
  1984. original.write_buffer_size =
  1985. l * 4 * 1024 * 1024 + i * 1024 * 1024 + j * 1024 + k;
  1986. ColumnFamilyOptions result = SanitizeCfOptions(
  1987. ImmutableDBOptions(db_options), /*read_only*/ false, original);
  1988. ASSERT_TRUE(result.level0_stop_writes_trigger >=
  1989. result.level0_slowdown_writes_trigger);
  1990. ASSERT_TRUE(result.level0_slowdown_writes_trigger >=
  1991. result.level0_file_num_compaction_trigger);
  1992. ASSERT_TRUE(result.level0_file_num_compaction_trigger ==
  1993. original.level0_file_num_compaction_trigger);
  1994. if (s == kCompactionStyleLevel) {
  1995. ASSERT_GE(result.num_levels, 2);
  1996. } else {
  1997. ASSERT_GE(result.num_levels, 1);
  1998. if (original.num_levels >= 1) {
  1999. ASSERT_EQ(result.num_levels, original.num_levels);
  2000. }
  2001. }
  2002. // Make sure Sanitize options sets arena_block_size to 1/8 of
  2003. // the write_buffer_size, rounded up to a multiple of 4k.
  2004. size_t expected_arena_block_size =
  2005. l * 4 * 1024 * 1024 / 8 + i * 1024 * 1024 / 8;
  2006. if (j + k != 0) {
  2007. // not a multiple of 4k, round up 4k
  2008. expected_arena_block_size += 4 * 1024;
  2009. }
  2010. expected_arena_block_size =
  2011. std::min(size_t{1024 * 1024}, expected_arena_block_size);
  2012. ASSERT_EQ(expected_arena_block_size, result.arena_block_size);
  2013. }
  2014. }
  2015. }
  2016. }
  2017. }
  2018. }
  2019. TEST_P(ColumnFamilyTest, ReadDroppedColumnFamily) {
  2020. // iter 0 -- drop CF, don't reopen
  2021. // iter 1 -- delete CF, reopen
  2022. for (int iter = 0; iter < 2; ++iter) {
  2023. db_options_.create_missing_column_families = true;
  2024. db_options_.max_open_files = 20;
  2025. // delete obsolete files always
  2026. db_options_.delete_obsolete_files_period_micros = 0;
  2027. Open({"default", "one", "two"});
  2028. ColumnFamilyOptions options;
  2029. options.level0_file_num_compaction_trigger = 100;
  2030. options.level0_slowdown_writes_trigger = 200;
  2031. options.level0_stop_writes_trigger = 200;
  2032. options.write_buffer_size = 100000; // small write buffer size
  2033. Reopen({options, options, options});
  2034. // 1MB should create ~10 files for each CF
  2035. int kKeysNum = 10000;
  2036. PutRandomData(0, kKeysNum, 100);
  2037. PutRandomData(1, kKeysNum, 100);
  2038. PutRandomData(2, kKeysNum, 100);
  2039. {
  2040. std::unique_ptr<Iterator> iterator(
  2041. db_->NewIterator(ReadOptions(), handles_[2]));
  2042. iterator->SeekToFirst();
  2043. if (iter == 0) {
  2044. // Drop CF two
  2045. ASSERT_OK(db_->DropColumnFamily(handles_[2]));
  2046. } else {
  2047. // delete CF two
  2048. ASSERT_OK(db_->DestroyColumnFamilyHandle(handles_[2]));
  2049. handles_[2] = nullptr;
  2050. }
  2051. // Make sure iterator created can still be used.
  2052. int count = 0;
  2053. for (; iterator->Valid(); iterator->Next()) {
  2054. ASSERT_OK(iterator->status());
  2055. ++count;
  2056. }
  2057. ASSERT_OK(iterator->status());
  2058. ASSERT_EQ(count, kKeysNum);
  2059. }
  2060. // Add bunch more data to other CFs
  2061. PutRandomData(0, kKeysNum, 100);
  2062. PutRandomData(1, kKeysNum, 100);
  2063. if (iter == 1) {
  2064. Reopen();
  2065. }
  2066. // Since we didn't delete CF handle, RocksDB's contract guarantees that
  2067. // we're still able to read dropped CF
  2068. for (int i = 0; i < 3; ++i) {
  2069. std::unique_ptr<Iterator> iterator(
  2070. db_->NewIterator(ReadOptions(), handles_[i]));
  2071. int count = 0;
  2072. for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) {
  2073. ASSERT_OK(iterator->status());
  2074. ++count;
  2075. }
  2076. ASSERT_OK(iterator->status());
  2077. ASSERT_EQ(count, kKeysNum * ((i == 2) ? 1 : 2));
  2078. }
  2079. Close();
  2080. Destroy();
  2081. }
  2082. }
  2083. TEST_P(ColumnFamilyTest, LiveIteratorWithDroppedColumnFamily) {
  2084. db_options_.create_missing_column_families = true;
  2085. db_options_.max_open_files = 20;
  2086. // delete obsolete files always
  2087. db_options_.delete_obsolete_files_period_micros = 0;
  2088. Open({"default", "one", "two"});
  2089. ColumnFamilyOptions options;
  2090. options.level0_file_num_compaction_trigger = 100;
  2091. options.level0_slowdown_writes_trigger = 200;
  2092. options.level0_stop_writes_trigger = 200;
  2093. options.write_buffer_size = 100000; // small write buffer size
  2094. Reopen({options, options, options});
  2095. // 1MB should create ~10 files for each CF
  2096. int kKeysNum = 10000;
  2097. PutRandomData(1, kKeysNum, 100);
  2098. {
  2099. std::unique_ptr<Iterator> iterator(
  2100. db_->NewIterator(ReadOptions(), handles_[1]));
  2101. iterator->SeekToFirst();
  2102. DropColumnFamilies({1});
  2103. // Make sure iterator created can still be used.
  2104. int count = 0;
  2105. for (; iterator->Valid(); iterator->Next()) {
  2106. ASSERT_OK(iterator->status());
  2107. ++count;
  2108. }
  2109. ASSERT_OK(iterator->status());
  2110. ASSERT_EQ(count, kKeysNum);
  2111. }
  2112. Reopen();
  2113. Close();
  2114. Destroy();
  2115. }
  2116. TEST_P(ColumnFamilyTest, FlushAndDropRaceCondition) {
  2117. db_options_.create_missing_column_families = true;
  2118. Open({"default", "one"});
  2119. ColumnFamilyOptions options;
  2120. options.level0_file_num_compaction_trigger = 100;
  2121. options.level0_slowdown_writes_trigger = 200;
  2122. options.level0_stop_writes_trigger = 200;
  2123. options.max_write_buffer_number = 20;
  2124. options.write_buffer_size = 100000; // small write buffer size
  2125. Reopen({options, options});
  2126. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  2127. {{"VersionSet::LogAndApply::ColumnFamilyDrop:0",
  2128. "FlushJob::WriteLevel0Table"},
  2129. {"VersionSet::LogAndApply::ColumnFamilyDrop:1",
  2130. "FlushJob::InstallResults"},
  2131. {"FlushJob::InstallResults",
  2132. "VersionSet::LogAndApply::ColumnFamilyDrop:2"}});
  2133. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2134. test::SleepingBackgroundTask sleeping_task;
  2135. env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask, &sleeping_task,
  2136. Env::Priority::HIGH);
  2137. // Make sure the task is sleeping. Otherwise, it might start to execute
  2138. // after sleeping_task.WaitUntilDone() and cause TSAN warning.
  2139. sleeping_task.WaitUntilSleeping();
  2140. // 1MB should create ~10 files for each CF
  2141. int kKeysNum = 10000;
  2142. PutRandomData(1, kKeysNum, 100);
  2143. std::vector<port::Thread> threads;
  2144. threads.emplace_back([&] { ASSERT_OK(db_->DropColumnFamily(handles_[1])); });
  2145. sleeping_task.WakeUp();
  2146. sleeping_task.WaitUntilDone();
  2147. sleeping_task.Reset();
  2148. // now we sleep again. this is just so we're certain that flush job finished
  2149. env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask, &sleeping_task,
  2150. Env::Priority::HIGH);
  2151. // Make sure the task is sleeping. Otherwise, it might start to execute
  2152. // after sleeping_task.WaitUntilDone() and cause TSAN warning.
  2153. sleeping_task.WaitUntilSleeping();
  2154. sleeping_task.WakeUp();
  2155. sleeping_task.WaitUntilDone();
  2156. {
  2157. // Since we didn't delete CF handle, RocksDB's contract guarantees that
  2158. // we're still able to read dropped CF
  2159. std::unique_ptr<Iterator> iterator(
  2160. db_->NewIterator(ReadOptions(), handles_[1]));
  2161. int count = 0;
  2162. for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) {
  2163. ASSERT_OK(iterator->status());
  2164. ++count;
  2165. }
  2166. ASSERT_OK(iterator->status());
  2167. ASSERT_EQ(count, kKeysNum);
  2168. }
  2169. for (auto& t : threads) {
  2170. t.join();
  2171. }
  2172. Close();
  2173. Destroy();
  2174. }
  2175. namespace {
  2176. std::atomic<int> test_stage(0);
  2177. std::atomic<bool> ordered_by_writethread(false);
  2178. const int kMainThreadStartPersistingOptionsFile = 1;
  2179. const int kChildThreadFinishDroppingColumnFamily = 2;
  2180. void DropSingleColumnFamily(ColumnFamilyTest* cf_test, int cf_id,
  2181. std::vector<Comparator*>* comparators) {
  2182. while (test_stage < kMainThreadStartPersistingOptionsFile &&
  2183. !ordered_by_writethread) {
  2184. Env::Default()->SleepForMicroseconds(100);
  2185. }
  2186. cf_test->DropColumnFamilies({cf_id});
  2187. if ((*comparators)[cf_id]) {
  2188. delete (*comparators)[cf_id];
  2189. (*comparators)[cf_id] = nullptr;
  2190. }
  2191. test_stage = kChildThreadFinishDroppingColumnFamily;
  2192. }
  2193. } // anonymous namespace
  2194. // This test attempts to set up a race condition in a way that is no longer
  2195. // possible, causing the test to hang. If DBImpl::options_mutex_ is removed
  2196. // in the future, this test might become relevant again.
  2197. TEST_P(ColumnFamilyTest, DISABLED_CreateAndDropRace) {
  2198. const int kCfCount = 5;
  2199. std::vector<ColumnFamilyOptions> cf_opts;
  2200. std::vector<Comparator*> comparators;
  2201. for (int i = 0; i < kCfCount; ++i) {
  2202. cf_opts.emplace_back();
  2203. comparators.push_back(new test::SimpleSuffixReverseComparator());
  2204. cf_opts.back().comparator = comparators.back();
  2205. }
  2206. db_options_.create_if_missing = true;
  2207. db_options_.create_missing_column_families = true;
  2208. auto main_thread_id = std::this_thread::get_id();
  2209. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  2210. "PersistRocksDBOptions:start", [&](void* /*arg*/) {
  2211. auto current_thread_id = std::this_thread::get_id();
  2212. // If it's the main thread hitting this sync-point, then it
  2213. // will be blocked until some other thread update the test_stage.
  2214. if (main_thread_id == current_thread_id) {
  2215. test_stage = kMainThreadStartPersistingOptionsFile;
  2216. while (test_stage < kChildThreadFinishDroppingColumnFamily &&
  2217. !ordered_by_writethread) {
  2218. Env::Default()->SleepForMicroseconds(100);
  2219. }
  2220. }
  2221. });
  2222. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  2223. "WriteThread::EnterUnbatched:Wait", [&](void* /*arg*/) {
  2224. // This means a thread doing DropColumnFamily() is waiting for
  2225. // other thread to finish persisting options.
  2226. // In such case, we update the test_stage to unblock the main thread.
  2227. ordered_by_writethread = true;
  2228. });
  2229. // Create a database with four column families
  2230. Open({"default", "one", "two", "three"},
  2231. {cf_opts[0], cf_opts[1], cf_opts[2], cf_opts[3]});
  2232. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2233. // Start a thread that will drop the first column family
  2234. // and its comparator
  2235. ROCKSDB_NAMESPACE::port::Thread drop_cf_thread(DropSingleColumnFamily, this,
  2236. 1, &comparators);
  2237. DropColumnFamilies({2});
  2238. drop_cf_thread.join();
  2239. Close();
  2240. Destroy();
  2241. for (auto* comparator : comparators) {
  2242. if (comparator) {
  2243. delete comparator;
  2244. }
  2245. }
  2246. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2247. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  2248. }
  2249. TEST_P(ColumnFamilyTest, CreateAndDropPeriodicRace) {
  2250. // This is a mini-stress test looking for inconsistency between the set of
  2251. // CFs in the DB, particularly whether any use preserve_internal_time_seconds,
  2252. // and whether that is accurately reflected in the periodic task setup.
  2253. constexpr size_t kNumThreads = 12;
  2254. std::vector<std::thread> threads;
  2255. bool last_cf_on = Random::GetTLSInstance()->OneIn(2);
  2256. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  2257. "DBImpl::RegisterRecordSeqnoTimeWorker:BeforePeriodicTaskType",
  2258. [&](void* /*arg*/) { std::this_thread::yield(); });
  2259. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2260. ASSERT_EQ(column_family_options_.preserve_internal_time_seconds, 0U);
  2261. ColumnFamilyOptions other_opts = column_family_options_;
  2262. ColumnFamilyOptions last_opts = column_family_options_;
  2263. (last_cf_on ? last_opts : other_opts).preserve_internal_time_seconds =
  2264. 1000000;
  2265. Open();
  2266. for (size_t i = 0; i < kNumThreads; i++) {
  2267. threads.emplace_back([this, &other_opts, i]() {
  2268. ColumnFamilyHandle* cfh;
  2269. ASSERT_OK(db_->CreateColumnFamily(other_opts, std::to_string(i), &cfh));
  2270. ASSERT_OK(db_->DropColumnFamily(cfh));
  2271. ASSERT_OK(db_->DestroyColumnFamilyHandle(cfh));
  2272. });
  2273. }
  2274. ColumnFamilyHandle* last_cfh;
  2275. ASSERT_OK(db_->CreateColumnFamily(last_opts, "last", &last_cfh));
  2276. for (auto& t : threads) {
  2277. t.join();
  2278. }
  2279. bool task_enabled = dbfull()->TEST_GetPeriodicTaskScheduler().TEST_HasTask(
  2280. PeriodicTaskType::kRecordSeqnoTime);
  2281. ASSERT_EQ(last_cf_on, task_enabled);
  2282. ASSERT_OK(db_->DropColumnFamily(last_cfh));
  2283. ASSERT_OK(db_->DestroyColumnFamilyHandle(last_cfh));
  2284. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2285. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
  2286. }
  2287. TEST_P(ColumnFamilyTest, WriteStallSingleColumnFamily) {
  2288. const uint64_t kBaseRate = 800000u;
  2289. db_options_.delayed_write_rate = kBaseRate;
  2290. db_options_.max_background_compactions = 6;
  2291. Open({"default"});
  2292. ColumnFamilyData* cfd =
  2293. static_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())->cfd();
  2294. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2295. MutableCFOptions mutable_cf_options(column_family_options_);
  2296. mutable_cf_options.level0_slowdown_writes_trigger = 20;
  2297. mutable_cf_options.level0_stop_writes_trigger = 10000;
  2298. mutable_cf_options.soft_pending_compaction_bytes_limit = 200;
  2299. mutable_cf_options.hard_pending_compaction_bytes_limit = 2000;
  2300. mutable_cf_options.disable_auto_compactions = false;
  2301. auto dbmu = dbfull()->TEST_Mutex();
  2302. vstorage->TEST_set_estimated_compaction_needed_bytes(50, dbmu);
  2303. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2304. ASSERT_TRUE(!IsDbWriteStopped());
  2305. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2306. vstorage->TEST_set_estimated_compaction_needed_bytes(201, dbmu);
  2307. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2308. ASSERT_TRUE(!IsDbWriteStopped());
  2309. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2310. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2311. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2312. vstorage->TEST_set_estimated_compaction_needed_bytes(400, dbmu);
  2313. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2314. ASSERT_TRUE(!IsDbWriteStopped());
  2315. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2316. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2317. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2318. vstorage->TEST_set_estimated_compaction_needed_bytes(500, dbmu);
  2319. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2320. ASSERT_TRUE(!IsDbWriteStopped());
  2321. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2322. ASSERT_EQ(kBaseRate / 1.25 / 1.25, GetDbDelayedWriteRate());
  2323. vstorage->TEST_set_estimated_compaction_needed_bytes(450, dbmu);
  2324. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2325. ASSERT_TRUE(!IsDbWriteStopped());
  2326. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2327. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2328. vstorage->TEST_set_estimated_compaction_needed_bytes(205, dbmu);
  2329. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2330. ASSERT_TRUE(!IsDbWriteStopped());
  2331. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2332. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2333. vstorage->TEST_set_estimated_compaction_needed_bytes(202, dbmu);
  2334. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2335. ASSERT_TRUE(!IsDbWriteStopped());
  2336. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2337. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2338. vstorage->TEST_set_estimated_compaction_needed_bytes(201, dbmu);
  2339. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2340. ASSERT_TRUE(!IsDbWriteStopped());
  2341. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2342. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2343. vstorage->TEST_set_estimated_compaction_needed_bytes(198, dbmu);
  2344. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2345. ASSERT_TRUE(!IsDbWriteStopped());
  2346. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2347. vstorage->TEST_set_estimated_compaction_needed_bytes(399, dbmu);
  2348. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2349. ASSERT_TRUE(!IsDbWriteStopped());
  2350. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2351. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2352. vstorage->TEST_set_estimated_compaction_needed_bytes(599, dbmu);
  2353. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2354. ASSERT_TRUE(!IsDbWriteStopped());
  2355. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2356. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2357. vstorage->TEST_set_estimated_compaction_needed_bytes(2001, dbmu);
  2358. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2359. ASSERT_TRUE(IsDbWriteStopped());
  2360. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2361. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2362. vstorage->TEST_set_estimated_compaction_needed_bytes(3001, dbmu);
  2363. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2364. ASSERT_TRUE(IsDbWriteStopped());
  2365. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2366. vstorage->TEST_set_estimated_compaction_needed_bytes(390, dbmu);
  2367. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2368. ASSERT_TRUE(!IsDbWriteStopped());
  2369. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2370. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2371. vstorage->TEST_set_estimated_compaction_needed_bytes(100, dbmu);
  2372. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2373. ASSERT_TRUE(!IsDbWriteStopped());
  2374. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2375. vstorage->set_l0_delay_trigger_count(100);
  2376. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2377. ASSERT_TRUE(!IsDbWriteStopped());
  2378. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2379. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2380. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2381. vstorage->set_l0_delay_trigger_count(101);
  2382. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2383. ASSERT_TRUE(!IsDbWriteStopped());
  2384. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2385. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2386. vstorage->set_l0_delay_trigger_count(0);
  2387. vstorage->TEST_set_estimated_compaction_needed_bytes(300, dbmu);
  2388. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2389. ASSERT_TRUE(!IsDbWriteStopped());
  2390. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2391. ASSERT_EQ(kBaseRate / 1.25 / 1.25, GetDbDelayedWriteRate());
  2392. vstorage->set_l0_delay_trigger_count(101);
  2393. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2394. ASSERT_TRUE(!IsDbWriteStopped());
  2395. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2396. ASSERT_EQ(kBaseRate / 1.25 / 1.25 / 1.25, GetDbDelayedWriteRate());
  2397. vstorage->TEST_set_estimated_compaction_needed_bytes(200, dbmu);
  2398. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2399. ASSERT_TRUE(!IsDbWriteStopped());
  2400. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2401. ASSERT_EQ(kBaseRate / 1.25 / 1.25, GetDbDelayedWriteRate());
  2402. vstorage->set_l0_delay_trigger_count(0);
  2403. vstorage->TEST_set_estimated_compaction_needed_bytes(0, dbmu);
  2404. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2405. ASSERT_TRUE(!IsDbWriteStopped());
  2406. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2407. mutable_cf_options.disable_auto_compactions = true;
  2408. dbfull()->TEST_write_controler().set_delayed_write_rate(kBaseRate);
  2409. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2410. ASSERT_TRUE(!IsDbWriteStopped());
  2411. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2412. vstorage->set_l0_delay_trigger_count(50);
  2413. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2414. ASSERT_TRUE(!IsDbWriteStopped());
  2415. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2416. ASSERT_EQ(0, GetDbDelayedWriteRate());
  2417. ASSERT_EQ(kBaseRate, dbfull()->TEST_write_controler().delayed_write_rate());
  2418. vstorage->set_l0_delay_trigger_count(60);
  2419. vstorage->TEST_set_estimated_compaction_needed_bytes(300, dbmu);
  2420. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2421. ASSERT_TRUE(!IsDbWriteStopped());
  2422. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2423. ASSERT_EQ(0, GetDbDelayedWriteRate());
  2424. ASSERT_EQ(kBaseRate, dbfull()->TEST_write_controler().delayed_write_rate());
  2425. mutable_cf_options.disable_auto_compactions = false;
  2426. vstorage->set_l0_delay_trigger_count(70);
  2427. vstorage->TEST_set_estimated_compaction_needed_bytes(500, dbmu);
  2428. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2429. ASSERT_TRUE(!IsDbWriteStopped());
  2430. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2431. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2432. vstorage->set_l0_delay_trigger_count(71);
  2433. vstorage->TEST_set_estimated_compaction_needed_bytes(501, dbmu);
  2434. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2435. ASSERT_TRUE(!IsDbWriteStopped());
  2436. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2437. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2438. }
  2439. TEST_P(ColumnFamilyTest, CompactionSpeedupSingleColumnFamily) {
  2440. db_options_.max_background_compactions = 6;
  2441. Open({"default"});
  2442. ColumnFamilyData* cfd =
  2443. static_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())->cfd();
  2444. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2445. MutableCFOptions mutable_cf_options(column_family_options_);
  2446. // Speed up threshold = min(4 * 2, 4 + (36 - 4)/4) = 8
  2447. mutable_cf_options.level0_file_num_compaction_trigger = 4;
  2448. mutable_cf_options.level0_slowdown_writes_trigger = 36;
  2449. mutable_cf_options.level0_stop_writes_trigger = 50;
  2450. // Speedup threshold = 200 / 4 = 50
  2451. mutable_cf_options.soft_pending_compaction_bytes_limit = 200;
  2452. mutable_cf_options.hard_pending_compaction_bytes_limit = 2000;
  2453. auto dbmu = dbfull()->TEST_Mutex();
  2454. vstorage->TEST_set_estimated_compaction_needed_bytes(40, dbmu);
  2455. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2456. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2457. vstorage->TEST_set_estimated_compaction_needed_bytes(50, dbmu);
  2458. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2459. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2460. vstorage->TEST_set_estimated_compaction_needed_bytes(300, dbmu);
  2461. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2462. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2463. vstorage->TEST_set_estimated_compaction_needed_bytes(45, dbmu);
  2464. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2465. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2466. vstorage->set_l0_delay_trigger_count(7);
  2467. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2468. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2469. vstorage->set_l0_delay_trigger_count(9);
  2470. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2471. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2472. vstorage->set_l0_delay_trigger_count(6);
  2473. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2474. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2475. // Speed up threshold = min(4 * 2, 4 + (12 - 4)/4) = 6
  2476. mutable_cf_options.level0_file_num_compaction_trigger = 4;
  2477. mutable_cf_options.level0_slowdown_writes_trigger = 16;
  2478. mutable_cf_options.level0_stop_writes_trigger = 30;
  2479. vstorage->set_l0_delay_trigger_count(5);
  2480. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2481. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2482. vstorage->set_l0_delay_trigger_count(7);
  2483. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2484. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2485. vstorage->set_l0_delay_trigger_count(3);
  2486. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2487. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2488. }
  2489. TEST_P(ColumnFamilyTest, WriteStallTwoColumnFamilies) {
  2490. const uint64_t kBaseRate = 810000u;
  2491. db_options_.delayed_write_rate = kBaseRate;
  2492. Open();
  2493. CreateColumnFamilies({"one"});
  2494. ColumnFamilyData* cfd =
  2495. static_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())->cfd();
  2496. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2497. ColumnFamilyData* cfd1 =
  2498. static_cast<ColumnFamilyHandleImpl*>(handles_[1])->cfd();
  2499. VersionStorageInfo* vstorage1 = cfd1->current()->storage_info();
  2500. MutableCFOptions mutable_cf_options(column_family_options_);
  2501. mutable_cf_options.level0_slowdown_writes_trigger = 20;
  2502. mutable_cf_options.level0_stop_writes_trigger = 10000;
  2503. mutable_cf_options.soft_pending_compaction_bytes_limit = 200;
  2504. mutable_cf_options.hard_pending_compaction_bytes_limit = 2000;
  2505. MutableCFOptions mutable_cf_options1 = mutable_cf_options;
  2506. mutable_cf_options1.soft_pending_compaction_bytes_limit = 500;
  2507. auto dbmu = dbfull()->TEST_Mutex();
  2508. vstorage->TEST_set_estimated_compaction_needed_bytes(50, dbmu);
  2509. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2510. ASSERT_TRUE(!IsDbWriteStopped());
  2511. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2512. vstorage1->TEST_set_estimated_compaction_needed_bytes(201, dbmu);
  2513. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2514. ASSERT_TRUE(!IsDbWriteStopped());
  2515. ASSERT_TRUE(!dbfull()->TEST_write_controler().NeedsDelay());
  2516. vstorage1->TEST_set_estimated_compaction_needed_bytes(600, dbmu);
  2517. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2518. ASSERT_TRUE(!IsDbWriteStopped());
  2519. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2520. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2521. vstorage->TEST_set_estimated_compaction_needed_bytes(70, dbmu);
  2522. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2523. ASSERT_TRUE(!IsDbWriteStopped());
  2524. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2525. ASSERT_EQ(kBaseRate, GetDbDelayedWriteRate());
  2526. vstorage1->TEST_set_estimated_compaction_needed_bytes(800, dbmu);
  2527. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2528. ASSERT_TRUE(!IsDbWriteStopped());
  2529. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2530. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2531. vstorage->TEST_set_estimated_compaction_needed_bytes(300, dbmu);
  2532. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2533. ASSERT_TRUE(!IsDbWriteStopped());
  2534. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2535. ASSERT_EQ(kBaseRate / 1.25 / 1.25, GetDbDelayedWriteRate());
  2536. vstorage1->TEST_set_estimated_compaction_needed_bytes(700, dbmu);
  2537. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2538. ASSERT_TRUE(!IsDbWriteStopped());
  2539. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2540. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2541. vstorage->TEST_set_estimated_compaction_needed_bytes(500, dbmu);
  2542. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2543. ASSERT_TRUE(!IsDbWriteStopped());
  2544. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2545. ASSERT_EQ(kBaseRate / 1.25 / 1.25, GetDbDelayedWriteRate());
  2546. vstorage1->TEST_set_estimated_compaction_needed_bytes(600, dbmu);
  2547. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2548. ASSERT_TRUE(!IsDbWriteStopped());
  2549. ASSERT_TRUE(dbfull()->TEST_write_controler().NeedsDelay());
  2550. ASSERT_EQ(kBaseRate / 1.25, GetDbDelayedWriteRate());
  2551. }
  2552. TEST_P(ColumnFamilyTest, CompactionSpeedupTwoColumnFamilies) {
  2553. db_options_.max_background_compactions = 6;
  2554. column_family_options_.soft_pending_compaction_bytes_limit = 200;
  2555. column_family_options_.hard_pending_compaction_bytes_limit = 2000;
  2556. Open();
  2557. CreateColumnFamilies({"one"});
  2558. ColumnFamilyData* cfd =
  2559. static_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())->cfd();
  2560. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2561. ColumnFamilyData* cfd1 =
  2562. static_cast<ColumnFamilyHandleImpl*>(handles_[1])->cfd();
  2563. VersionStorageInfo* vstorage1 = cfd1->current()->storage_info();
  2564. MutableCFOptions mutable_cf_options(column_family_options_);
  2565. // Speed up threshold = min(4 * 2, 4 + (36 - 4)/4) = 8
  2566. mutable_cf_options.level0_file_num_compaction_trigger = 4;
  2567. mutable_cf_options.level0_slowdown_writes_trigger = 36;
  2568. mutable_cf_options.level0_stop_writes_trigger = 30;
  2569. // Speedup threshold = 200 / 4 = 50
  2570. mutable_cf_options.soft_pending_compaction_bytes_limit = 200;
  2571. mutable_cf_options.hard_pending_compaction_bytes_limit = 2000;
  2572. MutableCFOptions mutable_cf_options1 = mutable_cf_options;
  2573. mutable_cf_options1.level0_slowdown_writes_trigger = 16;
  2574. auto dbmu = dbfull()->TEST_Mutex();
  2575. vstorage->TEST_set_estimated_compaction_needed_bytes(40, dbmu);
  2576. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2577. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2578. vstorage->TEST_set_estimated_compaction_needed_bytes(60, dbmu);
  2579. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2580. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2581. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2582. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2583. vstorage1->TEST_set_estimated_compaction_needed_bytes(30, dbmu);
  2584. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2585. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2586. vstorage1->TEST_set_estimated_compaction_needed_bytes(70, dbmu);
  2587. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2588. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2589. vstorage->TEST_set_estimated_compaction_needed_bytes(20, dbmu);
  2590. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2591. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2592. vstorage1->TEST_set_estimated_compaction_needed_bytes(3, dbmu);
  2593. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2594. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2595. vstorage->set_l0_delay_trigger_count(9);
  2596. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2597. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2598. vstorage1->set_l0_delay_trigger_count(2);
  2599. RecalculateWriteStallConditions(cfd1, mutable_cf_options);
  2600. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2601. vstorage->set_l0_delay_trigger_count(0);
  2602. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2603. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2604. }
  2605. TEST_P(ColumnFamilyTest, CompactionSpeedupForCompactionDebt) {
  2606. db_options_.max_background_compactions = 6;
  2607. Open();
  2608. ColumnFamilyData* cfd =
  2609. static_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())->cfd();
  2610. MutableCFOptions mutable_cf_options(column_family_options_);
  2611. mutable_cf_options.soft_pending_compaction_bytes_limit =
  2612. std::numeric_limits<uint64_t>::max();
  2613. auto dbmu = dbfull()->TEST_Mutex();
  2614. {
  2615. // No bottommost data, so debt ratio cannot trigger speedup.
  2616. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2617. vstorage->TEST_set_estimated_compaction_needed_bytes(1048576 /* 1MB */,
  2618. dbmu);
  2619. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2620. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2621. }
  2622. // Add a tiny amount of bottommost data.
  2623. ASSERT_OK(db_->Put(WriteOptions(), "foo", "bar"));
  2624. ASSERT_OK(db_->Flush(FlushOptions()));
  2625. {
  2626. VersionStorageInfo* vstorage = cfd->current()->storage_info();
  2627. // Eight bytes is way smaller than bottommost data so definitely does not
  2628. // trigger speedup.
  2629. vstorage->TEST_set_estimated_compaction_needed_bytes(8, dbmu);
  2630. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2631. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2632. // 1MB is much larger than bottommost level size. However, since it's too
  2633. // small in terms of absolute size, it does not trigger parallel compaction
  2634. // in this case (see GetPendingCompactionBytesForCompactionSpeedup()).
  2635. vstorage->TEST_set_estimated_compaction_needed_bytes(1048576 /* 1MB */,
  2636. dbmu);
  2637. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2638. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2639. vstorage->TEST_set_estimated_compaction_needed_bytes(
  2640. 2 * mutable_cf_options.max_bytes_for_level_base, dbmu);
  2641. RecalculateWriteStallConditions(cfd, mutable_cf_options);
  2642. ASSERT_EQ(6, dbfull()->TEST_BGCompactionsAllowed());
  2643. }
  2644. }
  2645. TEST_P(ColumnFamilyTest, CompactionSpeedupForMarkedFiles) {
  2646. const int kParallelismLimit = 3;
  2647. class AlwaysCompactTpc : public TablePropertiesCollector {
  2648. public:
  2649. Status Finish(UserCollectedProperties* /* properties */) override {
  2650. return Status::OK();
  2651. }
  2652. UserCollectedProperties GetReadableProperties() const override {
  2653. return UserCollectedProperties{};
  2654. }
  2655. const char* Name() const override { return "AlwaysCompactTpc"; }
  2656. bool NeedCompact() const override { return true; }
  2657. };
  2658. class AlwaysCompactTpcf : public TablePropertiesCollectorFactory {
  2659. public:
  2660. TablePropertiesCollector* CreateTablePropertiesCollector(
  2661. TablePropertiesCollectorFactory::Context /* context */) override {
  2662. return new AlwaysCompactTpc();
  2663. }
  2664. const char* Name() const override { return "AlwaysCompactTpcf"; }
  2665. };
  2666. column_family_options_.num_levels = 2;
  2667. column_family_options_.table_properties_collector_factories.emplace_back(
  2668. std::make_shared<AlwaysCompactTpcf>());
  2669. db_options_.max_background_compactions = kParallelismLimit;
  2670. Open();
  2671. // Make a nonempty last level. Only marked files in upper levels count.
  2672. ASSERT_OK(db_->Put(WriteOptions(), "foo", "bar"));
  2673. ASSERT_OK(db_->Flush(FlushOptions()));
  2674. WaitForCompaction();
  2675. AssertFilesPerLevel("0,1", 0 /* cf */);
  2676. // We should calculate the limit by obtaining the number of env background
  2677. // threads, because the current test case will share the same env
  2678. // with another case that may have already increased the number of
  2679. // background threads which is larger than kParallelismLimit
  2680. const auto limit = env_->GetBackgroundThreads(Env::Priority::LOW);
  2681. // Block the compaction thread pool so marked files accumulate in L0.
  2682. std::vector<std::shared_ptr<test::SleepingBackgroundTask>> sleeping_tasks;
  2683. for (int i = 0; i < limit; i++) {
  2684. sleeping_tasks.emplace_back(
  2685. std::make_shared<test::SleepingBackgroundTask>());
  2686. env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask,
  2687. sleeping_tasks[i].get(), Env::Priority::LOW);
  2688. sleeping_tasks[i]->WaitUntilSleeping();
  2689. }
  2690. // Zero marked upper-level files. No speedup.
  2691. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2692. AssertFilesPerLevel("0,1", 0 /* cf */);
  2693. // One marked upper-level file. No speedup.
  2694. ASSERT_OK(db_->Put(WriteOptions(), "foo", "bar"));
  2695. ASSERT_OK(db_->Flush(FlushOptions()));
  2696. ASSERT_EQ(1, dbfull()->TEST_BGCompactionsAllowed());
  2697. AssertFilesPerLevel("1,1", 0 /* cf */);
  2698. // Two marked upper-level files. Speedup.
  2699. ASSERT_OK(db_->Put(WriteOptions(), "foo", "bar"));
  2700. ASSERT_OK(db_->Flush(FlushOptions()));
  2701. ASSERT_EQ(kParallelismLimit, dbfull()->TEST_BGCompactionsAllowed());
  2702. AssertFilesPerLevel("2,1", 0 /* cf */);
  2703. for (int i = 0; i < limit; i++) {
  2704. sleeping_tasks[i]->WakeUp();
  2705. sleeping_tasks[i]->WaitUntilDone();
  2706. }
  2707. }
  2708. TEST_P(ColumnFamilyTest, CreateAndDestroyOptions) {
  2709. std::unique_ptr<ColumnFamilyOptions> cfo(new ColumnFamilyOptions());
  2710. ColumnFamilyHandle* cfh;
  2711. Open();
  2712. ASSERT_OK(db_->CreateColumnFamily(*(cfo.get()), "yoyo", &cfh));
  2713. cfo.reset();
  2714. ASSERT_OK(db_->Put(WriteOptions(), cfh, "foo", "bar"));
  2715. ASSERT_OK(db_->Flush(FlushOptions(), cfh));
  2716. ASSERT_OK(db_->DropColumnFamily(cfh));
  2717. ASSERT_OK(db_->DestroyColumnFamilyHandle(cfh));
  2718. }
  2719. TEST_P(ColumnFamilyTest, CreateDropAndDestroy) {
  2720. ColumnFamilyHandle* cfh;
  2721. Open();
  2722. ASSERT_OK(db_->CreateColumnFamily(ColumnFamilyOptions(), "yoyo", &cfh));
  2723. ASSERT_OK(db_->Put(WriteOptions(), cfh, "foo", "bar"));
  2724. ASSERT_OK(db_->Flush(FlushOptions(), cfh));
  2725. ASSERT_OK(db_->DropColumnFamily(cfh));
  2726. ASSERT_OK(db_->DestroyColumnFamilyHandle(cfh));
  2727. }
  2728. TEST_P(ColumnFamilyTest, CreateDropAndDestroyWithoutFileDeletion) {
  2729. ColumnFamilyHandle* cfh;
  2730. Open();
  2731. ASSERT_OK(db_->CreateColumnFamily(ColumnFamilyOptions(), "yoyo", &cfh));
  2732. ASSERT_OK(db_->Put(WriteOptions(), cfh, "foo", "bar"));
  2733. ASSERT_OK(db_->Flush(FlushOptions(), cfh));
  2734. ASSERT_OK(db_->DisableFileDeletions());
  2735. ASSERT_OK(db_->DropColumnFamily(cfh));
  2736. ASSERT_OK(db_->DestroyColumnFamilyHandle(cfh));
  2737. }
  2738. TEST_P(ColumnFamilyTest, FlushCloseWALFiles) {
  2739. SpecialEnv env(Env::Default());
  2740. db_options_.env = &env;
  2741. db_options_.max_background_flushes = 1;
  2742. column_family_options_.memtable_factory.reset(
  2743. test::NewSpecialSkipListFactory(2));
  2744. Open();
  2745. CreateColumnFamilies({"one"});
  2746. ASSERT_OK(Put(1, "fodor", "mirko"));
  2747. ASSERT_OK(Put(0, "fodor", "mirko"));
  2748. ASSERT_OK(Put(1, "fodor", "mirko"));
  2749. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
  2750. {"DBImpl::BGWorkFlush:done", "FlushCloseWALFiles:0"},
  2751. });
  2752. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2753. // Block flush jobs from running
  2754. test::SleepingBackgroundTask sleeping_task;
  2755. env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask, &sleeping_task,
  2756. Env::Priority::HIGH);
  2757. // Make sure the task is sleeping. Otherwise, it might start to execute
  2758. // after sleeping_task.WaitUntilDone() and cause TSAN warning.
  2759. sleeping_task.WaitUntilSleeping();
  2760. WriteOptions wo;
  2761. wo.sync = true;
  2762. ASSERT_OK(db_->Put(wo, handles_[1], "fodor", "mirko"));
  2763. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2764. sleeping_task.WakeUp();
  2765. sleeping_task.WaitUntilDone();
  2766. TEST_SYNC_POINT("FlushCloseWALFiles:0");
  2767. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2768. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2769. Reopen();
  2770. ASSERT_EQ("mirko", Get(0, "fodor"));
  2771. ASSERT_EQ("mirko", Get(1, "fodor"));
  2772. db_options_.env = env_;
  2773. Close();
  2774. }
  2775. TEST_P(ColumnFamilyTest, IteratorCloseWALFile1) {
  2776. SpecialEnv env(Env::Default());
  2777. db_options_.env = &env;
  2778. db_options_.max_background_flushes = 1;
  2779. // When this option is removed, the test will need re-engineering
  2780. db_options_.background_close_inactive_wals = true;
  2781. column_family_options_.memtable_factory.reset(
  2782. test::NewSpecialSkipListFactory(2));
  2783. Open();
  2784. CreateColumnFamilies({"one"});
  2785. ASSERT_OK(Put(1, "fodor", "mirko"));
  2786. // Create an iterator holding the current super version.
  2787. Iterator* it = db_->NewIterator(ReadOptions(), handles_[1]);
  2788. ASSERT_OK(it->status());
  2789. // A flush will make `it` hold the last reference of its super version.
  2790. ASSERT_OK(Flush(1));
  2791. ASSERT_OK(Put(1, "fodor", "mirko"));
  2792. ASSERT_OK(Put(0, "fodor", "mirko"));
  2793. ASSERT_OK(Put(1, "fodor", "mirko"));
  2794. // Flush jobs will close previous WAL files after finishing. By
  2795. // block flush jobs from running, we trigger a condition where
  2796. // the iterator destructor should close the WAL files.
  2797. test::SleepingBackgroundTask sleeping_task;
  2798. env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask, &sleeping_task,
  2799. Env::Priority::HIGH);
  2800. // Make sure the task is sleeping. Otherwise, it might start to execute
  2801. // after sleeping_task.WaitUntilDone() and cause TSAN warning.
  2802. sleeping_task.WaitUntilSleeping();
  2803. WriteOptions wo;
  2804. wo.sync = true;
  2805. ASSERT_OK(db_->Put(wo, handles_[1], "fodor", "mirko"));
  2806. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2807. // Deleting the iterator will clear its super version, triggering
  2808. // closing all files
  2809. delete it;
  2810. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2811. sleeping_task.WakeUp();
  2812. sleeping_task.WaitUntilDone();
  2813. WaitForFlush(1);
  2814. Reopen();
  2815. ASSERT_EQ("mirko", Get(0, "fodor"));
  2816. ASSERT_EQ("mirko", Get(1, "fodor"));
  2817. db_options_.env = env_;
  2818. Close();
  2819. }
  2820. TEST_P(ColumnFamilyTest, IteratorCloseWALFile2) {
  2821. SpecialEnv env(Env::Default());
  2822. // Allow both of flush and purge job to schedule.
  2823. env.SetBackgroundThreads(2, Env::HIGH);
  2824. db_options_.env = &env;
  2825. db_options_.max_background_flushes = 1;
  2826. // When this option is removed, the test will need re-engineering
  2827. db_options_.background_close_inactive_wals = true;
  2828. column_family_options_.memtable_factory.reset(
  2829. test::NewSpecialSkipListFactory(2));
  2830. Open();
  2831. CreateColumnFamilies({"one"});
  2832. ASSERT_OK(Put(1, "fodor", "mirko"));
  2833. // Create an iterator holding the current super version.
  2834. ReadOptions ro;
  2835. ro.background_purge_on_iterator_cleanup = true;
  2836. Iterator* it = db_->NewIterator(ro, handles_[1]);
  2837. ASSERT_OK(it->status());
  2838. // A flush will make `it` hold the last reference of its super version.
  2839. ASSERT_OK(Flush(1));
  2840. ASSERT_OK(Put(1, "fodor", "mirko"));
  2841. ASSERT_OK(Put(0, "fodor", "mirko"));
  2842. ASSERT_OK(Put(1, "fodor", "mirko"));
  2843. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
  2844. {"ColumnFamilyTest::IteratorCloseWALFile2:0",
  2845. "DBImpl::BGWorkPurge:start"},
  2846. {"ColumnFamilyTest::IteratorCloseWALFile2:2",
  2847. "DBImpl::BackgroundCallFlush:start"},
  2848. {"DBImpl::BGWorkPurge:end", "ColumnFamilyTest::IteratorCloseWALFile2:1"},
  2849. });
  2850. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2851. WriteOptions wo;
  2852. wo.sync = true;
  2853. ASSERT_OK(db_->Put(wo, handles_[1], "fodor", "mirko"));
  2854. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2855. // Deleting the iterator will clear its super version, triggering
  2856. // closing all files
  2857. delete it;
  2858. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2859. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:0");
  2860. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:1");
  2861. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2862. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:2");
  2863. WaitForFlush(1);
  2864. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2865. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2866. Reopen();
  2867. ASSERT_EQ("mirko", Get(0, "fodor"));
  2868. ASSERT_EQ("mirko", Get(1, "fodor"));
  2869. db_options_.env = env_;
  2870. Close();
  2871. }
  2872. TEST_P(ColumnFamilyTest, ForwardIteratorCloseWALFile) {
  2873. SpecialEnv env(Env::Default());
  2874. // Allow both of flush and purge job to schedule.
  2875. env.SetBackgroundThreads(2, Env::HIGH);
  2876. db_options_.env = &env;
  2877. db_options_.max_background_flushes = 1;
  2878. // When this option is removed, the test will need re-engineering
  2879. db_options_.background_close_inactive_wals = true;
  2880. column_family_options_.memtable_factory.reset(
  2881. test::NewSpecialSkipListFactory(3));
  2882. column_family_options_.level0_file_num_compaction_trigger = 2;
  2883. Open();
  2884. CreateColumnFamilies({"one"});
  2885. ASSERT_OK(Put(1, "fodor", "mirko"));
  2886. ASSERT_OK(Put(1, "fodar2", "mirko"));
  2887. ASSERT_OK(Flush(1));
  2888. // Create an iterator holding the current super version, as well as
  2889. // the SST file just flushed.
  2890. ReadOptions ro;
  2891. ro.tailing = true;
  2892. ro.background_purge_on_iterator_cleanup = true;
  2893. Iterator* it = db_->NewIterator(ro, handles_[1]);
  2894. // A flush will make `it` hold the last reference of its super version.
  2895. ASSERT_OK(Put(1, "fodor", "mirko"));
  2896. ASSERT_OK(Put(1, "fodar2", "mirko"));
  2897. ASSERT_OK(Flush(1));
  2898. WaitForCompaction();
  2899. ASSERT_OK(Put(1, "fodor", "mirko"));
  2900. ASSERT_OK(Put(1, "fodor", "mirko"));
  2901. ASSERT_OK(Put(0, "fodor", "mirko"));
  2902. ASSERT_OK(Put(1, "fodor", "mirko"));
  2903. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
  2904. {"ColumnFamilyTest::IteratorCloseWALFile2:0",
  2905. "DBImpl::BGWorkPurge:start"},
  2906. {"ColumnFamilyTest::IteratorCloseWALFile2:2",
  2907. "DBImpl::BackgroundCallFlush:start"},
  2908. {"DBImpl::BGWorkPurge:end", "ColumnFamilyTest::IteratorCloseWALFile2:1"},
  2909. });
  2910. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2911. WriteOptions wo;
  2912. wo.sync = true;
  2913. ASSERT_OK(db_->Put(wo, handles_[1], "fodor", "mirko"));
  2914. env.delete_count_.store(0);
  2915. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2916. // Deleting the iterator will clear its super version, triggering
  2917. // closing all files
  2918. it->Seek("");
  2919. ASSERT_OK(it->status());
  2920. ASSERT_EQ(2, env.num_open_wal_file_.load());
  2921. ASSERT_EQ(0, env.delete_count_.load());
  2922. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:0");
  2923. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:1");
  2924. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2925. ASSERT_EQ(1, env.delete_count_.load());
  2926. TEST_SYNC_POINT("ColumnFamilyTest::IteratorCloseWALFile2:2");
  2927. WaitForFlush(1);
  2928. ASSERT_EQ(1, env.num_open_wal_file_.load());
  2929. ASSERT_EQ(1, env.delete_count_.load());
  2930. delete it;
  2931. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2932. Reopen();
  2933. ASSERT_EQ("mirko", Get(0, "fodor"));
  2934. ASSERT_EQ("mirko", Get(1, "fodor"));
  2935. db_options_.env = env_;
  2936. Close();
  2937. }
  2938. // Disable on windows because SyncWAL requires env->IsSyncThreadSafe()
  2939. // to return true which is not so in unbuffered mode.
  2940. #ifndef OS_WIN
  2941. TEST_P(ColumnFamilyTest, LogSyncConflictFlush) {
  2942. Open();
  2943. CreateColumnFamiliesAndReopen({"one", "two"});
  2944. ASSERT_OK(Put(0, "", ""));
  2945. ASSERT_OK(Put(1, "foo", "bar"));
  2946. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency(
  2947. {{"DBImpl::SyncWAL:BeforeMarkLogsSynced:1",
  2948. "ColumnFamilyTest::LogSyncConflictFlush:1"},
  2949. {"ColumnFamilyTest::LogSyncConflictFlush:2",
  2950. "DBImpl::SyncWAL:BeforeMarkLogsSynced:2"}});
  2951. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  2952. ROCKSDB_NAMESPACE::port::Thread thread([&] { ASSERT_OK(db_->SyncWAL()); });
  2953. TEST_SYNC_POINT("ColumnFamilyTest::LogSyncConflictFlush:1");
  2954. ASSERT_OK(Flush(1));
  2955. ASSERT_OK(Put(1, "foo", "bar"));
  2956. ASSERT_OK(Flush(1));
  2957. TEST_SYNC_POINT("ColumnFamilyTest::LogSyncConflictFlush:2");
  2958. thread.join();
  2959. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  2960. Close();
  2961. }
  2962. #endif
  2963. // this test is placed here, because the infrastructure for Column Family
  2964. // test is being used to ensure a roll of wal files.
  2965. // Basic idea is to test that WAL truncation is being detected and not
  2966. // ignored
  2967. TEST_P(ColumnFamilyTest, DISABLED_LogTruncationTest) {
  2968. Open();
  2969. CreateColumnFamiliesAndReopen({"one", "two"});
  2970. Build(0, 100);
  2971. // Flush the 0th column family to force a roll of the wal log
  2972. ASSERT_OK(Flush(0));
  2973. // Add some more entries
  2974. Build(100, 100);
  2975. std::vector<std::string> filenames;
  2976. ASSERT_OK(env_->GetChildren(dbname_, &filenames));
  2977. // collect wal files
  2978. std::vector<std::string> logfs;
  2979. for (size_t i = 0; i < filenames.size(); i++) {
  2980. uint64_t number;
  2981. FileType type;
  2982. if (!(ParseFileName(filenames[i], &number, &type))) {
  2983. continue;
  2984. }
  2985. if (type != kWalFile) {
  2986. continue;
  2987. }
  2988. logfs.push_back(filenames[i]);
  2989. }
  2990. std::sort(logfs.begin(), logfs.end());
  2991. ASSERT_GE(logfs.size(), 2);
  2992. // Take the last but one file, and truncate it
  2993. std::string fpath = dbname_ + "/" + logfs[logfs.size() - 2];
  2994. std::vector<std::string> names_save = names_;
  2995. uint64_t fsize;
  2996. ASSERT_OK(env_->GetFileSize(fpath, &fsize));
  2997. ASSERT_GT(fsize, 0);
  2998. Close();
  2999. std::string backup_logs = dbname_ + "/backup_logs";
  3000. std::string t_fpath = backup_logs + "/" + logfs[logfs.size() - 2];
  3001. ASSERT_OK(env_->CreateDirIfMissing(backup_logs));
  3002. // Not sure how easy it is to make this data driven.
  3003. // need to read back the WAL file and truncate last 10
  3004. // entries
  3005. CopyFile(fpath, t_fpath, fsize - 9180);
  3006. ASSERT_OK(env_->DeleteFile(fpath));
  3007. ASSERT_OK(env_->RenameFile(t_fpath, fpath));
  3008. db_options_.wal_recovery_mode = WALRecoveryMode::kPointInTimeRecovery;
  3009. OpenReadOnly(names_save);
  3010. CheckMissed();
  3011. Close();
  3012. Open(names_save);
  3013. CheckMissed();
  3014. Close();
  3015. // cleanup
  3016. ASSERT_OK(env_->DeleteDir(backup_logs));
  3017. }
  3018. TEST_P(ColumnFamilyTest, DefaultCfPathsTest) {
  3019. Open();
  3020. // Leave cf_paths for one column families to be empty.
  3021. // Files should be generated according to db_paths for that
  3022. // column family.
  3023. ColumnFamilyOptions cf_opt1, cf_opt2;
  3024. cf_opt1.cf_paths.emplace_back(dbname_ + "_one_1",
  3025. std::numeric_limits<uint64_t>::max());
  3026. CreateColumnFamilies({"one", "two"}, {cf_opt1, cf_opt2});
  3027. Reopen({ColumnFamilyOptions(), cf_opt1, cf_opt2});
  3028. // Fill Column family 1.
  3029. PutRandomData(1, 100, 100);
  3030. ASSERT_OK(Flush(1));
  3031. ASSERT_EQ(1, GetSstFileCount(cf_opt1.cf_paths[0].path));
  3032. ASSERT_EQ(0, GetSstFileCount(dbname_));
  3033. // Fill column family 2
  3034. PutRandomData(2, 100, 100);
  3035. ASSERT_OK(Flush(2));
  3036. // SST from Column family 2 should be generated in
  3037. // db_paths which is dbname_ in this case.
  3038. ASSERT_EQ(1, GetSstFileCount(dbname_));
  3039. }
  3040. TEST_P(ColumnFamilyTest, MultipleCFPathsTest) {
  3041. Open();
  3042. // Configure Column family specific paths.
  3043. ColumnFamilyOptions cf_opt1, cf_opt2;
  3044. cf_opt1.cf_paths.emplace_back(dbname_ + "_one_1",
  3045. std::numeric_limits<uint64_t>::max());
  3046. cf_opt2.cf_paths.emplace_back(dbname_ + "_two_1",
  3047. std::numeric_limits<uint64_t>::max());
  3048. CreateColumnFamilies({"one", "two"}, {cf_opt1, cf_opt2});
  3049. Reopen({ColumnFamilyOptions(), cf_opt1, cf_opt2});
  3050. PutRandomData(1, 100, 100, true /* save */);
  3051. ASSERT_OK(Flush(1));
  3052. // Check that files are generated in appropriate paths.
  3053. ASSERT_EQ(1, GetSstFileCount(cf_opt1.cf_paths[0].path));
  3054. ASSERT_EQ(0, GetSstFileCount(dbname_));
  3055. PutRandomData(2, 100, 100, true /* save */);
  3056. ASSERT_OK(Flush(2));
  3057. ASSERT_EQ(1, GetSstFileCount(cf_opt2.cf_paths[0].path));
  3058. ASSERT_EQ(0, GetSstFileCount(dbname_));
  3059. // Re-open and verify the keys.
  3060. Reopen({ColumnFamilyOptions(), cf_opt1, cf_opt2});
  3061. DBImpl* dbi = static_cast_with_check<DBImpl>(db_);
  3062. for (int cf = 1; cf != 3; ++cf) {
  3063. ReadOptions read_options;
  3064. read_options.readahead_size = 0;
  3065. auto it = dbi->NewIterator(read_options, handles_[cf]);
  3066. for (it->SeekToFirst(); it->Valid(); it->Next()) {
  3067. ASSERT_OK(it->status());
  3068. Slice key(it->key());
  3069. ASSERT_NE(keys_[cf].end(), keys_[cf].find(key.ToString()));
  3070. }
  3071. ASSERT_OK(it->status());
  3072. delete it;
  3073. for (const auto& key : keys_[cf]) {
  3074. ASSERT_NE("NOT_FOUND", Get(cf, key));
  3075. }
  3076. }
  3077. }
  3078. TEST(ColumnFamilyTest, ValidateBlobGCCutoff) {
  3079. DBOptions db_options;
  3080. ColumnFamilyOptions cf_options;
  3081. cf_options.enable_blob_garbage_collection = true;
  3082. cf_options.blob_garbage_collection_age_cutoff = -0.5;
  3083. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3084. .IsInvalidArgument());
  3085. cf_options.blob_garbage_collection_age_cutoff = 0.0;
  3086. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3087. cf_options.blob_garbage_collection_age_cutoff = 0.5;
  3088. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3089. cf_options.blob_garbage_collection_age_cutoff = 1.0;
  3090. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3091. cf_options.blob_garbage_collection_age_cutoff = 1.5;
  3092. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3093. .IsInvalidArgument());
  3094. }
  3095. TEST(ColumnFamilyTest, ValidateBlobGCForceThreshold) {
  3096. DBOptions db_options;
  3097. ColumnFamilyOptions cf_options;
  3098. cf_options.enable_blob_garbage_collection = true;
  3099. cf_options.blob_garbage_collection_force_threshold = -0.5;
  3100. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3101. .IsInvalidArgument());
  3102. cf_options.blob_garbage_collection_force_threshold = 0.0;
  3103. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3104. cf_options.blob_garbage_collection_force_threshold = 0.5;
  3105. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3106. cf_options.blob_garbage_collection_force_threshold = 1.0;
  3107. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3108. cf_options.blob_garbage_collection_force_threshold = 1.5;
  3109. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3110. .IsInvalidArgument());
  3111. }
  3112. TEST(ColumnFamilyTest, ValidateMemtableKVChecksumOption) {
  3113. DBOptions db_options;
  3114. ColumnFamilyOptions cf_options;
  3115. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3116. cf_options.memtable_protection_bytes_per_key = 5;
  3117. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3118. .IsNotSupported());
  3119. cf_options.memtable_protection_bytes_per_key = 1;
  3120. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3121. cf_options.memtable_protection_bytes_per_key = 16;
  3122. ASSERT_TRUE(ColumnFamilyData::ValidateOptions(db_options, cf_options)
  3123. .IsNotSupported());
  3124. cf_options.memtable_protection_bytes_per_key = 0;
  3125. ASSERT_OK(ColumnFamilyData::ValidateOptions(db_options, cf_options));
  3126. }
  3127. // Tests the flushing behavior of a column family to retain user-defined
  3128. // timestamp when `persist_user_defined_timestamp` is false. The behavior of
  3129. // auto flush is it makes some effort to retain user-defined timestamps while
  3130. // the behavior of manual flush is that it skips retaining UDTs.
  3131. class ColumnFamilyRetainUDTTest : public ColumnFamilyTestBase {
  3132. public:
  3133. ColumnFamilyRetainUDTTest() : ColumnFamilyTestBase(kLatestFormatVersion) {}
  3134. void SetUp() override {
  3135. db_options_.allow_concurrent_memtable_write = false;
  3136. column_family_options_.comparator =
  3137. test::BytewiseComparatorWithU64TsWrapper();
  3138. column_family_options_.persist_user_defined_timestamps = false;
  3139. ColumnFamilyTestBase::SetUp();
  3140. }
  3141. Status Put(int cf, const std::string& key, const std::string& ts,
  3142. const std::string& value) {
  3143. return db_->Put(WriteOptions(), handles_[cf], Slice(key), Slice(ts),
  3144. Slice(value));
  3145. }
  3146. std::string Get(int cf, const std::string& key, const std::string& read_ts) {
  3147. ReadOptions ropts;
  3148. Slice timestamp = read_ts;
  3149. ropts.timestamp = &timestamp;
  3150. std::string value;
  3151. Status s = db_->Get(ropts, handles_[cf], Slice(key), &value);
  3152. if (s.IsNotFound()) {
  3153. return "NOT_FOUND";
  3154. } else if (s.ok()) {
  3155. return value;
  3156. }
  3157. return "";
  3158. }
  3159. void CheckEffectiveCutoffTime(uint64_t expected_cutoff) {
  3160. std::string effective_full_history_ts_low;
  3161. EXPECT_OK(
  3162. db_->GetFullHistoryTsLow(handles_[0], &effective_full_history_ts_low));
  3163. EXPECT_EQ(EncodeAsUint64(expected_cutoff), effective_full_history_ts_low);
  3164. }
  3165. };
  3166. class TestTsComparator : public Comparator {
  3167. public:
  3168. TestTsComparator() : Comparator(8 /*ts_sz*/) {}
  3169. int Compare(const ROCKSDB_NAMESPACE::Slice& /*a*/,
  3170. const ROCKSDB_NAMESPACE::Slice& /*b*/) const override {
  3171. return 0;
  3172. }
  3173. const char* Name() const override { return "TestTs"; }
  3174. void FindShortestSeparator(
  3175. std::string* /*start*/,
  3176. const ROCKSDB_NAMESPACE::Slice& /*limit*/) const override {}
  3177. void FindShortSuccessor(std::string* /*key*/) const override {}
  3178. };
  3179. TEST_F(ColumnFamilyRetainUDTTest, SanityCheck) {
  3180. Open();
  3181. ColumnFamilyOptions cf_options;
  3182. cf_options.persist_user_defined_timestamps = false;
  3183. TestTsComparator test_comparator;
  3184. cf_options.comparator = &test_comparator;
  3185. ColumnFamilyHandle* handle;
  3186. // Not persisting user-defined timestamps feature only supports user-defined
  3187. // timestamps formatted as uint64_t.
  3188. ASSERT_TRUE(
  3189. db_->CreateColumnFamily(cf_options, "pikachu", &handle).IsNotSupported());
  3190. Destroy();
  3191. // Not persisting user-defined timestamps feature doesn't work in combination
  3192. // with atomic flush.
  3193. db_options_.atomic_flush = true;
  3194. ASSERT_TRUE(TryOpen({"default"}).IsNotSupported());
  3195. // Not persisting user-defined timestamps feature doesn't work in combination
  3196. // with concurrent memtable write.
  3197. db_options_.atomic_flush = false;
  3198. db_options_.allow_concurrent_memtable_write = true;
  3199. ASSERT_TRUE(TryOpen({"default"}).IsNotSupported());
  3200. Close();
  3201. }
  3202. class AutoFlushRetainUDTTest : public ColumnFamilyRetainUDTTest {};
  3203. TEST_F(AutoFlushRetainUDTTest, FullHistoryTsLowNotSet) {
  3204. SyncPoint::GetInstance()->SetCallBack(
  3205. "DBImpl::BackgroundFlush:CheckFlushRequest:cb", [&](void* arg) {
  3206. ASSERT_NE(nullptr, arg);
  3207. auto reschedule_count = *static_cast<int*>(arg);
  3208. ASSERT_EQ(1, reschedule_count);
  3209. });
  3210. SyncPoint::GetInstance()->EnableProcessing();
  3211. Open();
  3212. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3213. // No `full_history_ts_low` explicitly set by user, auto flush is continued
  3214. // without checking if its UDTs expired.
  3215. ASSERT_OK(dbfull()->TEST_SwitchWAL());
  3216. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3217. // After flush, `full_history_ts_low` should be automatically advanced to
  3218. // the effective cutoff timestamp: write_ts + 1
  3219. CheckEffectiveCutoffTime(2);
  3220. Close();
  3221. SyncPoint::GetInstance()->DisableProcessing();
  3222. SyncPoint::GetInstance()->ClearAllCallBacks();
  3223. }
  3224. TEST_F(AutoFlushRetainUDTTest, AllKeysExpired) {
  3225. SyncPoint::GetInstance()->SetCallBack(
  3226. "DBImpl::BackgroundFlush:CheckFlushRequest:cb", [&](void* arg) {
  3227. ASSERT_NE(nullptr, arg);
  3228. auto reschedule_count = *static_cast<int*>(arg);
  3229. ASSERT_EQ(1, reschedule_count);
  3230. });
  3231. SyncPoint::GetInstance()->EnableProcessing();
  3232. Open();
  3233. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3234. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(3)));
  3235. // All keys expired w.r.t the configured `full_history_ts_low`, auto flush
  3236. // continue without the need for a re-schedule.
  3237. ASSERT_OK(dbfull()->TEST_SwitchWAL());
  3238. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3239. // `full_history_ts_low` stays unchanged after flush.
  3240. CheckEffectiveCutoffTime(3);
  3241. Close();
  3242. SyncPoint::GetInstance()->DisableProcessing();
  3243. SyncPoint::GetInstance()->ClearAllCallBacks();
  3244. }
  3245. TEST_F(AutoFlushRetainUDTTest, NotAllKeysExpiredFlushToAvoidWriteStall) {
  3246. SyncPoint::GetInstance()->SetCallBack(
  3247. "DBImpl::BackgroundFlush:CheckFlushRequest:cb", [&](void* arg) {
  3248. ASSERT_NE(nullptr, arg);
  3249. auto reschedule_count = *static_cast<int*>(arg);
  3250. ASSERT_EQ(1, reschedule_count);
  3251. });
  3252. SyncPoint::GetInstance()->EnableProcessing();
  3253. Open();
  3254. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3255. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(1)));
  3256. ASSERT_OK(db_->SetOptions(handles_[0], {{"max_write_buffer_number", "1"}}));
  3257. // Not all keys expired, but auto flush is continued without a re-schedule
  3258. // because of risk of write stall.
  3259. ASSERT_OK(dbfull()->TEST_SwitchWAL());
  3260. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3261. // After flush, `full_history_ts_low` should be automatically advanced to
  3262. // the effective cutoff timestamp: write_ts + 1
  3263. CheckEffectiveCutoffTime(2);
  3264. Close();
  3265. SyncPoint::GetInstance()->DisableProcessing();
  3266. SyncPoint::GetInstance()->ClearAllCallBacks();
  3267. }
  3268. TEST_F(AutoFlushRetainUDTTest, NotAllKeysExpiredFlushRescheduled) {
  3269. std::atomic<int> local_counter{1};
  3270. SyncPoint::GetInstance()->SetCallBack(
  3271. "DBImpl::AfterRetainUDTReschedule:cb", [&](void* /*arg*/) {
  3272. // Increasing full_history_ts_low so all keys expired after the initial
  3273. // FlushRequest is rescheduled
  3274. ASSERT_OK(
  3275. db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(3)));
  3276. });
  3277. SyncPoint::GetInstance()->SetCallBack(
  3278. "DBImpl::BackgroundFlush:CheckFlushRequest:cb", [&](void* arg) {
  3279. ASSERT_NE(nullptr, arg);
  3280. auto reschedule_count = *static_cast<int*>(arg);
  3281. ASSERT_EQ(2, reschedule_count);
  3282. local_counter.fetch_add(1);
  3283. });
  3284. SyncPoint::GetInstance()->EnableProcessing();
  3285. Open();
  3286. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3287. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(1)));
  3288. // Not all keys expired, and there is no risk of write stall. Flush is
  3289. // rescheduled. The actual flush happens after `full_history_ts_low` is
  3290. // increased to mark all keys expired.
  3291. ASSERT_OK(dbfull()->TEST_SwitchWAL());
  3292. ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3293. // Make sure callback is not skipped.
  3294. ASSERT_EQ(2, local_counter);
  3295. CheckEffectiveCutoffTime(3);
  3296. Close();
  3297. SyncPoint::GetInstance()->DisableProcessing();
  3298. SyncPoint::GetInstance()->ClearAllCallBacks();
  3299. }
  3300. class ManualFlushSkipRetainUDTTest : public ColumnFamilyRetainUDTTest {
  3301. public:
  3302. // Write an entry with timestamp that is not expired w.r.t cutoff timestamp,
  3303. // and make sure automatic flush would be rescheduled to retain UDT.
  3304. void CheckAutomaticFlushRetainUDT(uint64_t write_ts) {
  3305. std::atomic<int> local_counter{1};
  3306. SyncPoint::GetInstance()->SetCallBack(
  3307. "DBImpl::AfterRetainUDTReschedule:cb", [&](void* /*arg*/) {
  3308. // Increasing full_history_ts_low so all keys expired after the
  3309. // initial FlushRequest is rescheduled
  3310. ASSERT_OK(db_->IncreaseFullHistoryTsLow(
  3311. handles_[0], EncodeAsUint64(write_ts + 1)));
  3312. });
  3313. SyncPoint::GetInstance()->SetCallBack(
  3314. "DBImpl::BackgroundFlush:CheckFlushRequest:cb", [&](void* arg) {
  3315. ASSERT_NE(nullptr, arg);
  3316. auto reschedule_count = *static_cast<int*>(arg);
  3317. ASSERT_EQ(2, reschedule_count);
  3318. local_counter.fetch_add(1);
  3319. });
  3320. SyncPoint::GetInstance()->EnableProcessing();
  3321. EXPECT_OK(Put(0, "foo", EncodeAsUint64(write_ts),
  3322. "foo" + std::to_string(write_ts)));
  3323. EXPECT_OK(dbfull()->TEST_SwitchWAL());
  3324. EXPECT_OK(dbfull()->TEST_WaitForFlushMemTable());
  3325. // Make sure callback is not skipped.
  3326. EXPECT_EQ(2, local_counter);
  3327. SyncPoint::GetInstance()->DisableProcessing();
  3328. SyncPoint::GetInstance()->ClearAllCallBacks();
  3329. }
  3330. };
  3331. TEST_F(ManualFlushSkipRetainUDTTest, ManualFlush) {
  3332. Open();
  3333. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3334. // Manual flush proceeds without trying to retain UDT.
  3335. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3336. ASSERT_OK(Flush(0));
  3337. CheckEffectiveCutoffTime(2);
  3338. CheckAutomaticFlushRetainUDT(3);
  3339. Close();
  3340. }
  3341. TEST_F(ManualFlushSkipRetainUDTTest, FlushRemovesStaleEntries) {
  3342. column_family_options_.max_write_buffer_number = 4;
  3343. Open();
  3344. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3345. ColumnFamilyHandle* cfh = db_->DefaultColumnFamily();
  3346. ColumnFamilyData* cfd =
  3347. static_cast_with_check<ColumnFamilyHandleImpl>(cfh)->cfd();
  3348. for (int version = 0; version < 100; version++) {
  3349. if (version == 50) {
  3350. ASSERT_OK(static_cast_with_check<DBImpl>(db_)->TEST_SwitchMemtable(cfd));
  3351. }
  3352. ASSERT_OK(
  3353. Put(0, "foo", EncodeAsUint64(version), "v" + std::to_string(version)));
  3354. }
  3355. ASSERT_OK(Flush(0));
  3356. TablePropertiesCollection tables_properties;
  3357. ASSERT_OK(db_->GetPropertiesOfAllTables(&tables_properties));
  3358. ASSERT_EQ(1, tables_properties.size());
  3359. std::shared_ptr<const TableProperties> table_properties =
  3360. tables_properties.begin()->second;
  3361. ASSERT_EQ(1, table_properties->num_entries);
  3362. ASSERT_EQ(0, table_properties->num_deletions);
  3363. ASSERT_EQ(0, table_properties->num_range_deletions);
  3364. CheckEffectiveCutoffTime(100);
  3365. CheckAutomaticFlushRetainUDT(101);
  3366. Close();
  3367. }
  3368. TEST_F(ManualFlushSkipRetainUDTTest, RangeDeletionFlushRemovesStaleEntries) {
  3369. column_family_options_.max_write_buffer_number = 4;
  3370. Open();
  3371. // TODO(yuzhangyu): a non 0 full history ts low is needed for this garbage
  3372. // collection to kick in. This doesn't work well for the very first flush of
  3373. // the column family. Not a big issue, but would be nice to improve this.
  3374. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(9)));
  3375. for (int i = 10; i < 100; i++) {
  3376. ASSERT_OK(Put(0, "foo" + std::to_string(i), EncodeAsUint64(i),
  3377. "val" + std::to_string(i)));
  3378. if (i % 2 == 1) {
  3379. ASSERT_OK(db_->DeleteRange(WriteOptions(), "foo" + std::to_string(i - 1),
  3380. "foo" + std::to_string(i), EncodeAsUint64(i)));
  3381. }
  3382. }
  3383. ASSERT_OK(Flush(0));
  3384. CheckEffectiveCutoffTime(100);
  3385. std::string read_ts = EncodeAsUint64(100);
  3386. std::string min_ts = EncodeAsUint64(0);
  3387. ReadOptions ropts;
  3388. Slice read_ts_slice = read_ts;
  3389. std::string value;
  3390. ropts.timestamp = &read_ts_slice;
  3391. {
  3392. Iterator* iter = db_->NewIterator(ropts);
  3393. iter->SeekToFirst();
  3394. int i = 11;
  3395. while (iter->Valid()) {
  3396. ASSERT_TRUE(iter->Valid());
  3397. ASSERT_EQ("foo" + std::to_string(i), iter->key());
  3398. ASSERT_EQ("val" + std::to_string(i), iter->value());
  3399. ASSERT_EQ(min_ts, iter->timestamp());
  3400. iter->Next();
  3401. i += 2;
  3402. }
  3403. ASSERT_OK(iter->status());
  3404. delete iter;
  3405. }
  3406. TablePropertiesCollection tables_properties;
  3407. ASSERT_OK(db_->GetPropertiesOfAllTables(&tables_properties));
  3408. ASSERT_EQ(1, tables_properties.size());
  3409. std::shared_ptr<const TableProperties> table_properties =
  3410. tables_properties.begin()->second;
  3411. // 45 point data + 45 range deletions. 45 obsolete point data are garbage
  3412. // collected.
  3413. ASSERT_EQ(90, table_properties->num_entries);
  3414. ASSERT_EQ(45, table_properties->num_deletions);
  3415. ASSERT_EQ(45, table_properties->num_range_deletions);
  3416. Close();
  3417. }
  3418. TEST_F(ManualFlushSkipRetainUDTTest, ManualCompaction) {
  3419. Open();
  3420. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3421. // Manual compaction proceeds without trying to retain UDT.
  3422. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v2"));
  3423. ASSERT_OK(
  3424. db_->CompactRange(CompactRangeOptions(), handles_[0], nullptr, nullptr));
  3425. CheckEffectiveCutoffTime(2);
  3426. CheckAutomaticFlushRetainUDT(3);
  3427. Close();
  3428. }
  3429. TEST_F(ManualFlushSkipRetainUDTTest, BulkLoading) {
  3430. Open();
  3431. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3432. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3433. // Test flush behavior in bulk loading scenarios.
  3434. Options options(db_options_, column_family_options_);
  3435. std::string sst_files_dir = dbname_ + "/sst_files/";
  3436. ASSERT_OK(DestroyDir(env_, sst_files_dir));
  3437. ASSERT_OK(env_->CreateDir(sst_files_dir));
  3438. SstFileWriter sst_file_writer(EnvOptions(), options);
  3439. std::string file1 = sst_files_dir + "file1.sst";
  3440. ASSERT_OK(sst_file_writer.Open(file1));
  3441. ASSERT_OK(sst_file_writer.Put("foo", EncodeAsUint64(0), "v2"));
  3442. ExternalSstFileInfo file1_info;
  3443. ASSERT_OK(sst_file_writer.Finish(&file1_info));
  3444. // Bulk loading in UDT mode doesn't support external file key range overlap
  3445. // with DB key range.
  3446. ASSERT_TRUE(db_->IngestExternalFile({file1}, IngestExternalFileOptions())
  3447. .IsInvalidArgument());
  3448. std::string file2 = sst_files_dir + "file2.sst";
  3449. ASSERT_OK(sst_file_writer.Open(file2));
  3450. ASSERT_OK(sst_file_writer.Put("bar", EncodeAsUint64(0), "val"));
  3451. ExternalSstFileInfo file2_info;
  3452. ASSERT_OK(sst_file_writer.Finish(&file2_info));
  3453. // A successful bulk loading, and it doesn't trigger any flush. As a result
  3454. // the effective cutoff timestamp is also unchanged.
  3455. ASSERT_OK(db_->IngestExternalFile({file2}, IngestExternalFileOptions()));
  3456. ASSERT_EQ(Get(0, "foo", EncodeAsUint64(1)), "v1");
  3457. ASSERT_EQ(Get(0, "bar", EncodeAsUint64(0)), "val");
  3458. CheckEffectiveCutoffTime(0);
  3459. CheckAutomaticFlushRetainUDT(1);
  3460. Close();
  3461. }
  3462. TEST_F(ManualFlushSkipRetainUDTTest, AutomaticFlushQueued) {
  3463. Open();
  3464. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3465. ASSERT_OK(Put(0, "foo", EncodeAsUint64(1), "v1"));
  3466. ASSERT_OK(dbfull()->TEST_SwitchWAL());
  3467. CheckEffectiveCutoffTime(0);
  3468. // Default `max_write_buffer_number=2` used, writing another memtable can get
  3469. // automatic flush to proceed because of memory pressure. Not doing that so
  3470. // we can test automatic flush gets to proceed because of an ongoing manual
  3471. // flush attempt.
  3472. ASSERT_OK(Flush(0));
  3473. CheckEffectiveCutoffTime(2);
  3474. CheckAutomaticFlushRetainUDT(3);
  3475. Close();
  3476. }
  3477. TEST_F(ManualFlushSkipRetainUDTTest, ConcurrentManualFlushes) {
  3478. Open();
  3479. ASSERT_OK(db_->IncreaseFullHistoryTsLow(handles_[0], EncodeAsUint64(0)));
  3480. std::vector<ROCKSDB_NAMESPACE::port::Thread> manual_flush_tds;
  3481. std::atomic<int> next_ts{0};
  3482. std::mutex mtx;
  3483. std::condition_variable cv;
  3484. auto manual_flush = [&](int write_ts) {
  3485. {
  3486. std::unique_lock<std::mutex> lock(mtx);
  3487. cv.wait(lock,
  3488. [&write_ts, &next_ts] { return write_ts == next_ts.load(); });
  3489. ASSERT_OK(Put(0, "foo" + std::to_string(write_ts),
  3490. EncodeAsUint64(write_ts),
  3491. "val_" + std::to_string(write_ts)));
  3492. next_ts.fetch_add(1);
  3493. cv.notify_all();
  3494. }
  3495. if (write_ts % 2 == 0) {
  3496. ASSERT_OK(Flush(0));
  3497. } else {
  3498. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), handles_[0], nullptr,
  3499. nullptr));
  3500. }
  3501. };
  3502. for (int write_ts = 0; write_ts < 10; write_ts++) {
  3503. manual_flush_tds.emplace_back(manual_flush, write_ts);
  3504. }
  3505. for (auto& td : manual_flush_tds) {
  3506. td.join();
  3507. }
  3508. CheckEffectiveCutoffTime(10);
  3509. CheckAutomaticFlushRetainUDT(11);
  3510. Close();
  3511. }
  3512. } // namespace ROCKSDB_NAMESPACE
  3513. int main(int argc, char** argv) {
  3514. ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
  3515. ::testing::InitGoogleTest(&argc, argv);
  3516. RegisterCustomObjects(argc, argv);
  3517. return RUN_ALL_TESTS();
  3518. }