db_stress_test_base.cc 78 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133
  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. //
  10. #ifdef GFLAGS
  11. #include "db_stress_tool/db_stress_common.h"
  12. #include "db_stress_tool/db_stress_driver.h"
  13. #include "rocksdb/convenience.h"
  14. namespace ROCKSDB_NAMESPACE {
  15. StressTest::StressTest()
  16. : cache_(NewCache(FLAGS_cache_size)),
  17. compressed_cache_(NewLRUCache(FLAGS_compressed_cache_size)),
  18. filter_policy_(FLAGS_bloom_bits >= 0
  19. ? FLAGS_use_block_based_filter
  20. ? NewBloomFilterPolicy(FLAGS_bloom_bits, true)
  21. : NewBloomFilterPolicy(FLAGS_bloom_bits, false)
  22. : nullptr),
  23. db_(nullptr),
  24. #ifndef ROCKSDB_LITE
  25. txn_db_(nullptr),
  26. #endif
  27. new_column_family_name_(1),
  28. num_times_reopened_(0),
  29. db_preload_finished_(false),
  30. cmp_db_(nullptr) {
  31. if (FLAGS_destroy_db_initially) {
  32. std::vector<std::string> files;
  33. db_stress_env->GetChildren(FLAGS_db, &files);
  34. for (unsigned int i = 0; i < files.size(); i++) {
  35. if (Slice(files[i]).starts_with("heap-")) {
  36. db_stress_env->DeleteFile(FLAGS_db + "/" + files[i]);
  37. }
  38. }
  39. Options options;
  40. // Remove files without preserving manfiest files
  41. #ifndef ROCKSDB_LITE
  42. const Status s = !FLAGS_use_blob_db
  43. ? DestroyDB(FLAGS_db, options)
  44. : blob_db::DestroyBlobDB(FLAGS_db, options,
  45. blob_db::BlobDBOptions());
  46. #else
  47. const Status s = DestroyDB(FLAGS_db, options);
  48. #endif // !ROCKSDB_LITE
  49. if (!s.ok()) {
  50. fprintf(stderr, "Cannot destroy original db: %s\n", s.ToString().c_str());
  51. exit(1);
  52. }
  53. }
  54. }
  55. StressTest::~StressTest() {
  56. for (auto cf : column_families_) {
  57. delete cf;
  58. }
  59. column_families_.clear();
  60. delete db_;
  61. assert(secondaries_.size() == secondary_cfh_lists_.size());
  62. size_t n = secondaries_.size();
  63. for (size_t i = 0; i != n; ++i) {
  64. for (auto* cf : secondary_cfh_lists_[i]) {
  65. delete cf;
  66. }
  67. secondary_cfh_lists_[i].clear();
  68. delete secondaries_[i];
  69. }
  70. secondaries_.clear();
  71. for (auto* cf : cmp_cfhs_) {
  72. delete cf;
  73. }
  74. cmp_cfhs_.clear();
  75. delete cmp_db_;
  76. }
  77. std::shared_ptr<Cache> StressTest::NewCache(size_t capacity) {
  78. if (capacity <= 0) {
  79. return nullptr;
  80. }
  81. if (FLAGS_use_clock_cache) {
  82. auto cache = NewClockCache((size_t)capacity);
  83. if (!cache) {
  84. fprintf(stderr, "Clock cache not supported.");
  85. exit(1);
  86. }
  87. return cache;
  88. } else {
  89. return NewLRUCache((size_t)capacity);
  90. }
  91. }
  92. bool StressTest::BuildOptionsTable() {
  93. if (FLAGS_set_options_one_in <= 0) {
  94. return true;
  95. }
  96. std::unordered_map<std::string, std::vector<std::string>> options_tbl = {
  97. {"write_buffer_size",
  98. {ToString(options_.write_buffer_size),
  99. ToString(options_.write_buffer_size * 2),
  100. ToString(options_.write_buffer_size * 4)}},
  101. {"max_write_buffer_number",
  102. {ToString(options_.max_write_buffer_number),
  103. ToString(options_.max_write_buffer_number * 2),
  104. ToString(options_.max_write_buffer_number * 4)}},
  105. {"arena_block_size",
  106. {
  107. ToString(options_.arena_block_size),
  108. ToString(options_.write_buffer_size / 4),
  109. ToString(options_.write_buffer_size / 8),
  110. }},
  111. {"memtable_huge_page_size", {"0", ToString(2 * 1024 * 1024)}},
  112. {"max_successive_merges", {"0", "2", "4"}},
  113. {"inplace_update_num_locks", {"100", "200", "300"}},
  114. // TODO(ljin): enable test for this option
  115. // {"disable_auto_compactions", {"100", "200", "300"}},
  116. {"soft_rate_limit", {"0", "0.5", "0.9"}},
  117. {"hard_rate_limit", {"0", "1.1", "2.0"}},
  118. {"level0_file_num_compaction_trigger",
  119. {
  120. ToString(options_.level0_file_num_compaction_trigger),
  121. ToString(options_.level0_file_num_compaction_trigger + 2),
  122. ToString(options_.level0_file_num_compaction_trigger + 4),
  123. }},
  124. {"level0_slowdown_writes_trigger",
  125. {
  126. ToString(options_.level0_slowdown_writes_trigger),
  127. ToString(options_.level0_slowdown_writes_trigger + 2),
  128. ToString(options_.level0_slowdown_writes_trigger + 4),
  129. }},
  130. {"level0_stop_writes_trigger",
  131. {
  132. ToString(options_.level0_stop_writes_trigger),
  133. ToString(options_.level0_stop_writes_trigger + 2),
  134. ToString(options_.level0_stop_writes_trigger + 4),
  135. }},
  136. {"max_compaction_bytes",
  137. {
  138. ToString(options_.target_file_size_base * 5),
  139. ToString(options_.target_file_size_base * 15),
  140. ToString(options_.target_file_size_base * 100),
  141. }},
  142. {"target_file_size_base",
  143. {
  144. ToString(options_.target_file_size_base),
  145. ToString(options_.target_file_size_base * 2),
  146. ToString(options_.target_file_size_base * 4),
  147. }},
  148. {"target_file_size_multiplier",
  149. {
  150. ToString(options_.target_file_size_multiplier),
  151. "1",
  152. "2",
  153. }},
  154. {"max_bytes_for_level_base",
  155. {
  156. ToString(options_.max_bytes_for_level_base / 2),
  157. ToString(options_.max_bytes_for_level_base),
  158. ToString(options_.max_bytes_for_level_base * 2),
  159. }},
  160. {"max_bytes_for_level_multiplier",
  161. {
  162. ToString(options_.max_bytes_for_level_multiplier),
  163. "1",
  164. "2",
  165. }},
  166. {"max_sequential_skip_in_iterations", {"4", "8", "12"}},
  167. };
  168. options_table_ = std::move(options_tbl);
  169. for (const auto& iter : options_table_) {
  170. options_index_.push_back(iter.first);
  171. }
  172. return true;
  173. }
  174. void StressTest::InitDb() {
  175. uint64_t now = db_stress_env->NowMicros();
  176. fprintf(stdout, "%s Initializing db_stress\n",
  177. db_stress_env->TimeToString(now / 1000000).c_str());
  178. PrintEnv();
  179. Open();
  180. BuildOptionsTable();
  181. }
  182. void StressTest::InitReadonlyDb(SharedState* shared) {
  183. uint64_t now = db_stress_env->NowMicros();
  184. fprintf(stdout, "%s Preloading db with %" PRIu64 " KVs\n",
  185. db_stress_env->TimeToString(now / 1000000).c_str(), FLAGS_max_key);
  186. PreloadDbAndReopenAsReadOnly(FLAGS_max_key, shared);
  187. }
  188. bool StressTest::VerifySecondaries() {
  189. #ifndef ROCKSDB_LITE
  190. if (FLAGS_test_secondary) {
  191. uint64_t now = db_stress_env->NowMicros();
  192. fprintf(
  193. stdout, "%s Start to verify secondaries against primary\n",
  194. db_stress_env->TimeToString(static_cast<uint64_t>(now) / 1000000).c_str());
  195. }
  196. for (size_t k = 0; k != secondaries_.size(); ++k) {
  197. Status s = secondaries_[k]->TryCatchUpWithPrimary();
  198. if (!s.ok()) {
  199. fprintf(stderr, "Secondary failed to catch up with primary\n");
  200. return false;
  201. }
  202. ReadOptions ropts;
  203. ropts.total_order_seek = true;
  204. // Verify only the default column family since the primary may have
  205. // dropped other column families after most recent reopen.
  206. std::unique_ptr<Iterator> iter1(db_->NewIterator(ropts));
  207. std::unique_ptr<Iterator> iter2(secondaries_[k]->NewIterator(ropts));
  208. for (iter1->SeekToFirst(), iter2->SeekToFirst();
  209. iter1->Valid() && iter2->Valid(); iter1->Next(), iter2->Next()) {
  210. if (iter1->key().compare(iter2->key()) != 0 ||
  211. iter1->value().compare(iter2->value())) {
  212. fprintf(stderr,
  213. "Secondary %d contains different data from "
  214. "primary.\nPrimary: %s : %s\nSecondary: %s : %s\n",
  215. static_cast<int>(k),
  216. iter1->key().ToString(/*hex=*/true).c_str(),
  217. iter1->value().ToString(/*hex=*/true).c_str(),
  218. iter2->key().ToString(/*hex=*/true).c_str(),
  219. iter2->value().ToString(/*hex=*/true).c_str());
  220. return false;
  221. }
  222. }
  223. if (iter1->Valid() && !iter2->Valid()) {
  224. fprintf(stderr,
  225. "Secondary %d record count is smaller than that of primary\n",
  226. static_cast<int>(k));
  227. return false;
  228. } else if (!iter1->Valid() && iter2->Valid()) {
  229. fprintf(stderr,
  230. "Secondary %d record count is larger than that of primary\n",
  231. static_cast<int>(k));
  232. return false;
  233. }
  234. }
  235. if (FLAGS_test_secondary) {
  236. uint64_t now = db_stress_env->NowMicros();
  237. fprintf(
  238. stdout, "%s Verification of secondaries succeeded\n",
  239. db_stress_env->TimeToString(static_cast<uint64_t>(now) / 1000000).c_str());
  240. }
  241. #endif // ROCKSDB_LITE
  242. return true;
  243. }
  244. Status StressTest::AssertSame(DB* db, ColumnFamilyHandle* cf,
  245. ThreadState::SnapshotState& snap_state) {
  246. Status s;
  247. if (cf->GetName() != snap_state.cf_at_name) {
  248. return s;
  249. }
  250. ReadOptions ropt;
  251. ropt.snapshot = snap_state.snapshot;
  252. PinnableSlice exp_v(&snap_state.value);
  253. exp_v.PinSelf();
  254. PinnableSlice v;
  255. s = db->Get(ropt, cf, snap_state.key, &v);
  256. if (!s.ok() && !s.IsNotFound()) {
  257. return s;
  258. }
  259. if (snap_state.status != s) {
  260. return Status::Corruption(
  261. "The snapshot gave inconsistent results for key " +
  262. ToString(Hash(snap_state.key.c_str(), snap_state.key.size(), 0)) +
  263. " in cf " + cf->GetName() + ": (" + snap_state.status.ToString() +
  264. ") vs. (" + s.ToString() + ")");
  265. }
  266. if (s.ok()) {
  267. if (exp_v != v) {
  268. return Status::Corruption("The snapshot gave inconsistent values: (" +
  269. exp_v.ToString() + ") vs. (" + v.ToString() +
  270. ")");
  271. }
  272. }
  273. if (snap_state.key_vec != nullptr) {
  274. // When `prefix_extractor` is set, seeking to beginning and scanning
  275. // across prefixes are only supported with `total_order_seek` set.
  276. ropt.total_order_seek = true;
  277. std::unique_ptr<Iterator> iterator(db->NewIterator(ropt));
  278. std::unique_ptr<std::vector<bool>> tmp_bitvec(
  279. new std::vector<bool>(FLAGS_max_key));
  280. for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) {
  281. uint64_t key_val;
  282. if (GetIntVal(iterator->key().ToString(), &key_val)) {
  283. (*tmp_bitvec.get())[key_val] = true;
  284. }
  285. }
  286. if (!std::equal(snap_state.key_vec->begin(), snap_state.key_vec->end(),
  287. tmp_bitvec.get()->begin())) {
  288. return Status::Corruption("Found inconsistent keys at this snapshot");
  289. }
  290. }
  291. return Status::OK();
  292. }
  293. void StressTest::VerificationAbort(SharedState* shared, std::string msg,
  294. Status s) const {
  295. fprintf(stderr, "Verification failed: %s. Status is %s\n", msg.c_str(),
  296. s.ToString().c_str());
  297. shared->SetVerificationFailure();
  298. }
  299. void StressTest::VerificationAbort(SharedState* shared, std::string msg, int cf,
  300. int64_t key) const {
  301. fprintf(stderr,
  302. "Verification failed for column family %d key %" PRIi64 ": %s\n", cf,
  303. key, msg.c_str());
  304. shared->SetVerificationFailure();
  305. }
  306. void StressTest::PrintStatistics() {
  307. if (dbstats) {
  308. fprintf(stdout, "STATISTICS:\n%s\n", dbstats->ToString().c_str());
  309. }
  310. if (dbstats_secondaries) {
  311. fprintf(stdout, "Secondary instances STATISTICS:\n%s\n",
  312. dbstats_secondaries->ToString().c_str());
  313. }
  314. }
  315. // Currently PreloadDb has to be single-threaded.
  316. void StressTest::PreloadDbAndReopenAsReadOnly(int64_t number_of_keys,
  317. SharedState* shared) {
  318. WriteOptions write_opts;
  319. write_opts.disableWAL = FLAGS_disable_wal;
  320. if (FLAGS_sync) {
  321. write_opts.sync = true;
  322. }
  323. char value[100];
  324. int cf_idx = 0;
  325. Status s;
  326. for (auto cfh : column_families_) {
  327. for (int64_t k = 0; k != number_of_keys; ++k) {
  328. std::string key_str = Key(k);
  329. Slice key = key_str;
  330. size_t sz = GenerateValue(0 /*value_base*/, value, sizeof(value));
  331. Slice v(value, sz);
  332. shared->Put(cf_idx, k, 0, true /* pending */);
  333. if (FLAGS_use_merge) {
  334. if (!FLAGS_use_txn) {
  335. s = db_->Merge(write_opts, cfh, key, v);
  336. } else {
  337. #ifndef ROCKSDB_LITE
  338. Transaction* txn;
  339. s = NewTxn(write_opts, &txn);
  340. if (s.ok()) {
  341. s = txn->Merge(cfh, key, v);
  342. if (s.ok()) {
  343. s = CommitTxn(txn);
  344. }
  345. }
  346. #endif
  347. }
  348. } else {
  349. if (!FLAGS_use_txn) {
  350. s = db_->Put(write_opts, cfh, key, v);
  351. } else {
  352. #ifndef ROCKSDB_LITE
  353. Transaction* txn;
  354. s = NewTxn(write_opts, &txn);
  355. if (s.ok()) {
  356. s = txn->Put(cfh, key, v);
  357. if (s.ok()) {
  358. s = CommitTxn(txn);
  359. }
  360. }
  361. #endif
  362. }
  363. }
  364. shared->Put(cf_idx, k, 0, false /* pending */);
  365. if (!s.ok()) {
  366. break;
  367. }
  368. }
  369. if (!s.ok()) {
  370. break;
  371. }
  372. ++cf_idx;
  373. }
  374. if (s.ok()) {
  375. s = db_->Flush(FlushOptions(), column_families_);
  376. }
  377. if (s.ok()) {
  378. for (auto cf : column_families_) {
  379. delete cf;
  380. }
  381. column_families_.clear();
  382. delete db_;
  383. db_ = nullptr;
  384. #ifndef ROCKSDB_LITE
  385. txn_db_ = nullptr;
  386. #endif
  387. db_preload_finished_.store(true);
  388. auto now = db_stress_env->NowMicros();
  389. fprintf(stdout, "%s Reopening database in read-only\n",
  390. db_stress_env->TimeToString(now / 1000000).c_str());
  391. // Reopen as read-only, can ignore all options related to updates
  392. Open();
  393. } else {
  394. fprintf(stderr, "Failed to preload db");
  395. exit(1);
  396. }
  397. }
  398. Status StressTest::SetOptions(ThreadState* thread) {
  399. assert(FLAGS_set_options_one_in > 0);
  400. std::unordered_map<std::string, std::string> opts;
  401. std::string name =
  402. options_index_[thread->rand.Next() % options_index_.size()];
  403. int value_idx = thread->rand.Next() % options_table_[name].size();
  404. if (name == "soft_rate_limit" || name == "hard_rate_limit") {
  405. opts["soft_rate_limit"] = options_table_["soft_rate_limit"][value_idx];
  406. opts["hard_rate_limit"] = options_table_["hard_rate_limit"][value_idx];
  407. } else if (name == "level0_file_num_compaction_trigger" ||
  408. name == "level0_slowdown_writes_trigger" ||
  409. name == "level0_stop_writes_trigger") {
  410. opts["level0_file_num_compaction_trigger"] =
  411. options_table_["level0_file_num_compaction_trigger"][value_idx];
  412. opts["level0_slowdown_writes_trigger"] =
  413. options_table_["level0_slowdown_writes_trigger"][value_idx];
  414. opts["level0_stop_writes_trigger"] =
  415. options_table_["level0_stop_writes_trigger"][value_idx];
  416. } else {
  417. opts[name] = options_table_[name][value_idx];
  418. }
  419. int rand_cf_idx = thread->rand.Next() % FLAGS_column_families;
  420. auto cfh = column_families_[rand_cf_idx];
  421. return db_->SetOptions(cfh, opts);
  422. }
  423. #ifndef ROCKSDB_LITE
  424. Status StressTest::NewTxn(WriteOptions& write_opts, Transaction** txn) {
  425. if (!FLAGS_use_txn) {
  426. return Status::InvalidArgument("NewTxn when FLAGS_use_txn is not set");
  427. }
  428. static std::atomic<uint64_t> txn_id = {0};
  429. TransactionOptions txn_options;
  430. *txn = txn_db_->BeginTransaction(write_opts, txn_options);
  431. auto istr = std::to_string(txn_id.fetch_add(1));
  432. Status s = (*txn)->SetName("xid" + istr);
  433. return s;
  434. }
  435. Status StressTest::CommitTxn(Transaction* txn) {
  436. if (!FLAGS_use_txn) {
  437. return Status::InvalidArgument("CommitTxn when FLAGS_use_txn is not set");
  438. }
  439. Status s = txn->Prepare();
  440. if (s.ok()) {
  441. s = txn->Commit();
  442. }
  443. delete txn;
  444. return s;
  445. }
  446. Status StressTest::RollbackTxn(Transaction* txn) {
  447. if (!FLAGS_use_txn) {
  448. return Status::InvalidArgument(
  449. "RollbackTxn when FLAGS_use_txn is not"
  450. " set");
  451. }
  452. Status s = txn->Rollback();
  453. delete txn;
  454. return s;
  455. }
  456. #endif
  457. void StressTest::OperateDb(ThreadState* thread) {
  458. ReadOptions read_opts(FLAGS_verify_checksum, true);
  459. WriteOptions write_opts;
  460. auto shared = thread->shared;
  461. char value[100];
  462. std::string from_db;
  463. if (FLAGS_sync) {
  464. write_opts.sync = true;
  465. }
  466. write_opts.disableWAL = FLAGS_disable_wal;
  467. const int prefixBound = static_cast<int>(FLAGS_readpercent) +
  468. static_cast<int>(FLAGS_prefixpercent);
  469. const int writeBound = prefixBound + static_cast<int>(FLAGS_writepercent);
  470. const int delBound = writeBound + static_cast<int>(FLAGS_delpercent);
  471. const int delRangeBound = delBound + static_cast<int>(FLAGS_delrangepercent);
  472. const uint64_t ops_per_open = FLAGS_ops_per_thread / (FLAGS_reopen + 1);
  473. thread->stats.Start();
  474. for (int open_cnt = 0; open_cnt <= FLAGS_reopen; ++open_cnt) {
  475. if (thread->shared->HasVerificationFailedYet() ||
  476. thread->shared->ShouldStopTest()) {
  477. break;
  478. }
  479. if (open_cnt != 0) {
  480. thread->stats.FinishedSingleOp();
  481. MutexLock l(thread->shared->GetMutex());
  482. while (!thread->snapshot_queue.empty()) {
  483. db_->ReleaseSnapshot(thread->snapshot_queue.front().second.snapshot);
  484. delete thread->snapshot_queue.front().second.key_vec;
  485. thread->snapshot_queue.pop();
  486. }
  487. thread->shared->IncVotedReopen();
  488. if (thread->shared->AllVotedReopen()) {
  489. thread->shared->GetStressTest()->Reopen(thread);
  490. thread->shared->GetCondVar()->SignalAll();
  491. } else {
  492. thread->shared->GetCondVar()->Wait();
  493. }
  494. // Commenting this out as we don't want to reset stats on each open.
  495. // thread->stats.Start();
  496. }
  497. for (uint64_t i = 0; i < ops_per_open; i++) {
  498. if (thread->shared->HasVerificationFailedYet()) {
  499. break;
  500. }
  501. // Change Options
  502. if (thread->rand.OneInOpt(FLAGS_set_options_one_in)) {
  503. SetOptions(thread);
  504. }
  505. if (thread->rand.OneInOpt(FLAGS_set_in_place_one_in)) {
  506. options_.inplace_update_support ^= options_.inplace_update_support;
  507. }
  508. if (thread->tid == 0 && FLAGS_verify_db_one_in > 0 &&
  509. thread->rand.OneIn(FLAGS_verify_db_one_in)) {
  510. ContinuouslyVerifyDb(thread);
  511. if (thread->shared->ShouldStopTest()) {
  512. break;
  513. }
  514. }
  515. MaybeClearOneColumnFamily(thread);
  516. if (thread->rand.OneInOpt(FLAGS_sync_wal_one_in)) {
  517. Status s = db_->SyncWAL();
  518. if (!s.ok() && !s.IsNotSupported()) {
  519. fprintf(stderr, "SyncWAL() failed: %s\n", s.ToString().c_str());
  520. }
  521. }
  522. int rand_column_family = thread->rand.Next() % FLAGS_column_families;
  523. ColumnFamilyHandle* column_family = column_families_[rand_column_family];
  524. if (thread->rand.OneInOpt(FLAGS_compact_files_one_in)) {
  525. TestCompactFiles(thread, column_family);
  526. }
  527. int64_t rand_key = GenerateOneKey(thread, i);
  528. std::string keystr = Key(rand_key);
  529. Slice key = keystr;
  530. std::unique_ptr<MutexLock> lock;
  531. if (ShouldAcquireMutexOnKey()) {
  532. lock.reset(new MutexLock(
  533. shared->GetMutexForKey(rand_column_family, rand_key)));
  534. }
  535. if (thread->rand.OneInOpt(FLAGS_compact_range_one_in)) {
  536. TestCompactRange(thread, rand_key, key, column_family);
  537. if (thread->shared->HasVerificationFailedYet()) {
  538. break;
  539. }
  540. }
  541. std::vector<int> rand_column_families =
  542. GenerateColumnFamilies(FLAGS_column_families, rand_column_family);
  543. if (thread->rand.OneInOpt(FLAGS_flush_one_in)) {
  544. Status status = TestFlush(rand_column_families);
  545. if (!status.ok()) {
  546. fprintf(stdout, "Unable to perform Flush(): %s\n",
  547. status.ToString().c_str());
  548. }
  549. }
  550. #ifndef ROCKSDB_LITE
  551. // Every 1 in N verify the one of the following: 1) GetLiveFiles
  552. // 2) GetSortedWalFiles 3) GetCurrentWalFile. Each time, randomly select
  553. // one of them to run the test.
  554. if (thread->rand.OneInOpt(FLAGS_get_live_files_and_wal_files_one_in)) {
  555. Status status = VerifyGetLiveAndWalFiles(thread);
  556. if (!status.ok()) {
  557. VerificationAbort(shared, "VerifyGetLiveAndWalFiles status not OK",
  558. status);
  559. }
  560. }
  561. #endif // !ROCKSDB_LITE
  562. if (thread->rand.OneInOpt(FLAGS_pause_background_one_in)) {
  563. Status status = TestPauseBackground(thread);
  564. if (!status.ok()) {
  565. VerificationAbort(
  566. shared, "Pause/ContinueBackgroundWork status not OK", status);
  567. }
  568. }
  569. #ifndef ROCKSDB_LITE
  570. if (thread->rand.OneInOpt(FLAGS_verify_checksum_one_in)) {
  571. Status status = db_->VerifyChecksum();
  572. if (!status.ok()) {
  573. VerificationAbort(shared, "VerifyChecksum status not OK", status);
  574. }
  575. }
  576. #endif
  577. std::vector<int64_t> rand_keys = GenerateKeys(rand_key);
  578. if (thread->rand.OneInOpt(FLAGS_ingest_external_file_one_in)) {
  579. TestIngestExternalFile(thread, rand_column_families, rand_keys, lock);
  580. }
  581. if (thread->rand.OneInOpt(FLAGS_backup_one_in)) {
  582. Status s = TestBackupRestore(thread, rand_column_families, rand_keys);
  583. if (!s.ok()) {
  584. VerificationAbort(shared, "Backup/restore gave inconsistent state",
  585. s);
  586. }
  587. }
  588. if (thread->rand.OneInOpt(FLAGS_checkpoint_one_in)) {
  589. Status s = TestCheckpoint(thread, rand_column_families, rand_keys);
  590. if (!s.ok()) {
  591. VerificationAbort(shared, "Checkpoint gave inconsistent state", s);
  592. }
  593. }
  594. #ifndef ROCKSDB_LITE
  595. if (thread->rand.OneInOpt(FLAGS_approximate_size_one_in)) {
  596. Status s =
  597. TestApproximateSize(thread, i, rand_column_families, rand_keys);
  598. if (!s.ok()) {
  599. VerificationAbort(shared, "ApproximateSize Failed", s);
  600. }
  601. }
  602. #endif // !ROCKSDB_LITE
  603. if (thread->rand.OneInOpt(FLAGS_acquire_snapshot_one_in)) {
  604. TestAcquireSnapshot(thread, rand_column_family, keystr, i);
  605. }
  606. /*always*/ {
  607. Status s = MaybeReleaseSnapshots(thread, i);
  608. if (!s.ok()) {
  609. VerificationAbort(shared, "Snapshot gave inconsistent state", s);
  610. }
  611. }
  612. int prob_op = thread->rand.Uniform(100);
  613. // Reset this in case we pick something other than a read op. We don't
  614. // want to use a stale value when deciding at the beginning of the loop
  615. // whether to vote to reopen
  616. if (prob_op >= 0 && prob_op < static_cast<int>(FLAGS_readpercent)) {
  617. assert(0 <= prob_op);
  618. // OPERATION read
  619. if (FLAGS_use_multiget) {
  620. // Leave room for one more iteration of the loop with a single key
  621. // batch. This is to ensure that each thread does exactly the same
  622. // number of ops
  623. int multiget_batch_size = static_cast<int>(
  624. std::min(static_cast<uint64_t>(thread->rand.Uniform(64)),
  625. FLAGS_ops_per_thread - i - 1));
  626. // If its the last iteration, ensure that multiget_batch_size is 1
  627. multiget_batch_size = std::max(multiget_batch_size, 1);
  628. rand_keys = GenerateNKeys(thread, multiget_batch_size, i);
  629. TestMultiGet(thread, read_opts, rand_column_families, rand_keys);
  630. i += multiget_batch_size - 1;
  631. } else {
  632. TestGet(thread, read_opts, rand_column_families, rand_keys);
  633. }
  634. } else if (prob_op < prefixBound) {
  635. assert(static_cast<int>(FLAGS_readpercent) <= prob_op);
  636. // OPERATION prefix scan
  637. // keys are 8 bytes long, prefix size is FLAGS_prefix_size. There are
  638. // (8 - FLAGS_prefix_size) bytes besides the prefix. So there will
  639. // be 2 ^ ((8 - FLAGS_prefix_size) * 8) possible keys with the same
  640. // prefix
  641. TestPrefixScan(thread, read_opts, rand_column_families, rand_keys);
  642. } else if (prob_op < writeBound) {
  643. assert(prefixBound <= prob_op);
  644. // OPERATION write
  645. TestPut(thread, write_opts, read_opts, rand_column_families, rand_keys,
  646. value, lock);
  647. } else if (prob_op < delBound) {
  648. assert(writeBound <= prob_op);
  649. // OPERATION delete
  650. TestDelete(thread, write_opts, rand_column_families, rand_keys, lock);
  651. } else if (prob_op < delRangeBound) {
  652. assert(delBound <= prob_op);
  653. // OPERATION delete range
  654. TestDeleteRange(thread, write_opts, rand_column_families, rand_keys,
  655. lock);
  656. } else {
  657. assert(delRangeBound <= prob_op);
  658. // OPERATION iterate
  659. int num_seeks = static_cast<int>(
  660. std::min(static_cast<uint64_t>(thread->rand.Uniform(4)),
  661. FLAGS_ops_per_thread - i - 1));
  662. rand_keys = GenerateNKeys(thread, num_seeks, i);
  663. i += num_seeks - 1;
  664. TestIterate(thread, read_opts, rand_column_families, rand_keys);
  665. }
  666. thread->stats.FinishedSingleOp();
  667. #ifndef ROCKSDB_LITE
  668. uint32_t tid = thread->tid;
  669. assert(secondaries_.empty() ||
  670. static_cast<size_t>(tid) < secondaries_.size());
  671. if (thread->rand.OneInOpt(FLAGS_secondary_catch_up_one_in)) {
  672. Status s = secondaries_[tid]->TryCatchUpWithPrimary();
  673. if (!s.ok()) {
  674. VerificationAbort(shared, "Secondary instance failed to catch up", s);
  675. break;
  676. }
  677. }
  678. #endif
  679. }
  680. }
  681. while (!thread->snapshot_queue.empty()) {
  682. db_->ReleaseSnapshot(thread->snapshot_queue.front().second.snapshot);
  683. delete thread->snapshot_queue.front().second.key_vec;
  684. thread->snapshot_queue.pop();
  685. }
  686. thread->stats.Stop();
  687. }
  688. #ifndef ROCKSDB_LITE
  689. // Generated a list of keys that close to boundaries of SST keys.
  690. // If there isn't any SST file in the DB, return empty list.
  691. std::vector<std::string> StressTest::GetWhiteBoxKeys(ThreadState* thread,
  692. DB* db,
  693. ColumnFamilyHandle* cfh,
  694. size_t num_keys) {
  695. ColumnFamilyMetaData cfmd;
  696. db->GetColumnFamilyMetaData(cfh, &cfmd);
  697. std::vector<std::string> boundaries;
  698. for (const LevelMetaData& lmd : cfmd.levels) {
  699. for (const SstFileMetaData& sfmd : lmd.files) {
  700. boundaries.push_back(sfmd.smallestkey);
  701. boundaries.push_back(sfmd.largestkey);
  702. }
  703. }
  704. if (boundaries.empty()) {
  705. return {};
  706. }
  707. std::vector<std::string> ret;
  708. for (size_t j = 0; j < num_keys; j++) {
  709. std::string k =
  710. boundaries[thread->rand.Uniform(static_cast<int>(boundaries.size()))];
  711. if (thread->rand.OneIn(3)) {
  712. // Reduce one byte from the string
  713. for (int i = static_cast<int>(k.length()) - 1; i >= 0; i--) {
  714. uint8_t cur = k[i];
  715. if (cur > 0) {
  716. k[i] = static_cast<char>(cur - 1);
  717. break;
  718. } else if (i > 0) {
  719. k[i] = 0xFFu;
  720. }
  721. }
  722. } else if (thread->rand.OneIn(2)) {
  723. // Add one byte to the string
  724. for (int i = static_cast<int>(k.length()) - 1; i >= 0; i--) {
  725. uint8_t cur = k[i];
  726. if (cur < 255) {
  727. k[i] = static_cast<char>(cur + 1);
  728. break;
  729. } else if (i > 0) {
  730. k[i] = 0x00;
  731. }
  732. }
  733. }
  734. ret.push_back(k);
  735. }
  736. return ret;
  737. }
  738. #endif // !ROCKSDB_LITE
  739. // Given a key K, this creates an iterator which scans to K and then
  740. // does a random sequence of Next/Prev operations.
  741. Status StressTest::TestIterate(ThreadState* thread,
  742. const ReadOptions& read_opts,
  743. const std::vector<int>& rand_column_families,
  744. const std::vector<int64_t>& rand_keys) {
  745. Status s;
  746. const Snapshot* snapshot = db_->GetSnapshot();
  747. ReadOptions readoptionscopy = read_opts;
  748. readoptionscopy.snapshot = snapshot;
  749. bool expect_total_order = false;
  750. if (thread->rand.OneIn(16)) {
  751. // When prefix extractor is used, it's useful to cover total order seek.
  752. readoptionscopy.total_order_seek = true;
  753. expect_total_order = true;
  754. } else if (thread->rand.OneIn(4)) {
  755. readoptionscopy.total_order_seek = false;
  756. readoptionscopy.auto_prefix_mode = true;
  757. expect_total_order = true;
  758. } else if (options_.prefix_extractor.get() == nullptr) {
  759. expect_total_order = true;
  760. }
  761. std::string upper_bound_str;
  762. Slice upper_bound;
  763. if (thread->rand.OneIn(16)) {
  764. // in 1/16 chance, set a iterator upper bound
  765. int64_t rand_upper_key = GenerateOneKey(thread, FLAGS_ops_per_thread);
  766. upper_bound_str = Key(rand_upper_key);
  767. upper_bound = Slice(upper_bound_str);
  768. // uppder_bound can be smaller than seek key, but the query itself
  769. // should not crash either.
  770. readoptionscopy.iterate_upper_bound = &upper_bound;
  771. }
  772. std::string lower_bound_str;
  773. Slice lower_bound;
  774. if (thread->rand.OneIn(16)) {
  775. // in 1/16 chance, enable iterator lower bound
  776. int64_t rand_lower_key = GenerateOneKey(thread, FLAGS_ops_per_thread);
  777. lower_bound_str = Key(rand_lower_key);
  778. lower_bound = Slice(lower_bound_str);
  779. // uppder_bound can be smaller than seek key, but the query itself
  780. // should not crash either.
  781. readoptionscopy.iterate_lower_bound = &lower_bound;
  782. }
  783. auto cfh = column_families_[rand_column_families[0]];
  784. std::unique_ptr<Iterator> iter(db_->NewIterator(readoptionscopy, cfh));
  785. std::vector<std::string> key_str;
  786. if (thread->rand.OneIn(16)) {
  787. // Generate keys close to lower or upper bound of SST files.
  788. key_str = GetWhiteBoxKeys(thread, db_, cfh, rand_keys.size());
  789. }
  790. if (key_str.empty()) {
  791. // If key string is not geneerated using white block keys,
  792. // Use randomized key passe in.
  793. for (int64_t rkey : rand_keys) {
  794. key_str.push_back(Key(rkey));
  795. }
  796. }
  797. std::string op_logs;
  798. const size_t kOpLogsLimit = 10000;
  799. for (const std::string& skey : key_str) {
  800. if (op_logs.size() > kOpLogsLimit) {
  801. // Shouldn't take too much memory for the history log. Clear it.
  802. op_logs = "(cleared...)\n";
  803. }
  804. Slice key = skey;
  805. if (readoptionscopy.iterate_upper_bound != nullptr &&
  806. thread->rand.OneIn(2)) {
  807. // 1/2 chance, change the upper bound.
  808. // It is possible that it is changed without first use, but there is no
  809. // problem with that.
  810. int64_t rand_upper_key = GenerateOneKey(thread, FLAGS_ops_per_thread);
  811. upper_bound_str = Key(rand_upper_key);
  812. upper_bound = Slice(upper_bound_str);
  813. } else if (readoptionscopy.iterate_lower_bound != nullptr &&
  814. thread->rand.OneIn(4)) {
  815. // 1/4 chance, change the lower bound.
  816. // It is possible that it is changed without first use, but there is no
  817. // problem with that.
  818. int64_t rand_lower_key = GenerateOneKey(thread, FLAGS_ops_per_thread);
  819. lower_bound_str = Key(rand_lower_key);
  820. lower_bound = Slice(lower_bound_str);
  821. }
  822. // Record some options to op_logs;
  823. op_logs += "total_order_seek: ";
  824. op_logs += (readoptionscopy.total_order_seek ? "1 " : "0 ");
  825. op_logs += "auto_prefix_mode: ";
  826. op_logs += (readoptionscopy.auto_prefix_mode ? "1 " : "0 ");
  827. if (readoptionscopy.iterate_upper_bound != nullptr) {
  828. op_logs += "ub: " + upper_bound.ToString(true) + " ";
  829. }
  830. if (readoptionscopy.iterate_lower_bound != nullptr) {
  831. op_logs += "lb: " + lower_bound.ToString(true) + " ";
  832. }
  833. // Set up an iterator and does the same without bounds and with total
  834. // order seek and compare the results. This is to identify bugs related
  835. // to bounds, prefix extractor or reseeking. Sometimes we are comparing
  836. // iterators with the same set-up, and it doesn't hurt to check them
  837. // to be equal.
  838. ReadOptions cmp_ro;
  839. cmp_ro.snapshot = snapshot;
  840. cmp_ro.total_order_seek = true;
  841. ColumnFamilyHandle* cmp_cfh =
  842. GetControlCfh(thread, rand_column_families[0]);
  843. std::unique_ptr<Iterator> cmp_iter(db_->NewIterator(cmp_ro, cmp_cfh));
  844. bool diverged = false;
  845. bool support_seek_first_or_last = expect_total_order;
  846. LastIterateOp last_op;
  847. if (support_seek_first_or_last && thread->rand.OneIn(100)) {
  848. iter->SeekToFirst();
  849. cmp_iter->SeekToFirst();
  850. last_op = kLastOpSeekToFirst;
  851. op_logs += "STF ";
  852. } else if (support_seek_first_or_last && thread->rand.OneIn(100)) {
  853. iter->SeekToLast();
  854. cmp_iter->SeekToLast();
  855. last_op = kLastOpSeekToLast;
  856. op_logs += "STL ";
  857. } else if (thread->rand.OneIn(8)) {
  858. iter->SeekForPrev(key);
  859. cmp_iter->SeekForPrev(key);
  860. last_op = kLastOpSeekForPrev;
  861. op_logs += "SFP " + key.ToString(true) + " ";
  862. } else {
  863. iter->Seek(key);
  864. cmp_iter->Seek(key);
  865. last_op = kLastOpSeek;
  866. op_logs += "S " + key.ToString(true) + " ";
  867. }
  868. VerifyIterator(thread, cmp_cfh, readoptionscopy, iter.get(), cmp_iter.get(),
  869. last_op, key, op_logs, &diverged);
  870. bool no_reverse =
  871. (FLAGS_memtablerep == "prefix_hash" && !expect_total_order);
  872. for (uint64_t i = 0; i < FLAGS_num_iterations && iter->Valid(); i++) {
  873. if (no_reverse || thread->rand.OneIn(2)) {
  874. iter->Next();
  875. if (!diverged) {
  876. assert(cmp_iter->Valid());
  877. cmp_iter->Next();
  878. }
  879. op_logs += "N";
  880. } else {
  881. iter->Prev();
  882. if (!diverged) {
  883. assert(cmp_iter->Valid());
  884. cmp_iter->Prev();
  885. }
  886. op_logs += "P";
  887. }
  888. last_op = kLastOpNextOrPrev;
  889. VerifyIterator(thread, cmp_cfh, readoptionscopy, iter.get(),
  890. cmp_iter.get(), last_op, key, op_logs, &diverged);
  891. }
  892. if (s.ok()) {
  893. thread->stats.AddIterations(1);
  894. } else {
  895. fprintf(stderr, "TestIterate error: %s\n", s.ToString().c_str());
  896. thread->stats.AddErrors(1);
  897. break;
  898. }
  899. op_logs += "; ";
  900. }
  901. db_->ReleaseSnapshot(snapshot);
  902. return s;
  903. }
  904. #ifndef ROCKSDB_LITE
  905. // Test the return status of GetLiveFiles, GetSortedWalFiles, and
  906. // GetCurrentWalFile. Each time, randomly select one of them to run
  907. // and return the status.
  908. Status StressTest::VerifyGetLiveAndWalFiles(ThreadState* thread) {
  909. int case_num = thread->rand.Uniform(3);
  910. if (case_num == 0) {
  911. std::vector<std::string> live_file;
  912. uint64_t manifest_size;
  913. return db_->GetLiveFiles(live_file, &manifest_size);
  914. }
  915. if (case_num == 1) {
  916. VectorLogPtr log_ptr;
  917. return db_->GetSortedWalFiles(log_ptr);
  918. }
  919. if (case_num == 2) {
  920. std::unique_ptr<LogFile> cur_wal_file;
  921. return db_->GetCurrentWalFile(&cur_wal_file);
  922. }
  923. assert(false);
  924. return Status::Corruption("Undefined case happens!");
  925. }
  926. #endif // !ROCKSDB_LITE
  927. // Compare the two iterator, iter and cmp_iter are in the same position,
  928. // unless iter might be made invalidate or undefined because of
  929. // upper or lower bounds, or prefix extractor.
  930. // Will flag failure if the verification fails.
  931. // diverged = true if the two iterator is already diverged.
  932. // True if verification passed, false if not.
  933. void StressTest::VerifyIterator(ThreadState* thread,
  934. ColumnFamilyHandle* cmp_cfh,
  935. const ReadOptions& ro, Iterator* iter,
  936. Iterator* cmp_iter, LastIterateOp op,
  937. const Slice& seek_key,
  938. const std::string& op_logs, bool* diverged) {
  939. if (*diverged) {
  940. return;
  941. }
  942. if (op == kLastOpSeekToFirst && ro.iterate_lower_bound != nullptr) {
  943. // SeekToFirst() with lower bound is not well defined.
  944. *diverged = true;
  945. return;
  946. } else if (op == kLastOpSeekToLast && ro.iterate_upper_bound != nullptr) {
  947. // SeekToLast() with higher bound is not well defined.
  948. *diverged = true;
  949. return;
  950. } else if (op == kLastOpSeek && ro.iterate_lower_bound != nullptr &&
  951. (options_.comparator->Compare(*ro.iterate_lower_bound, seek_key) >=
  952. 0 ||
  953. (ro.iterate_upper_bound != nullptr &&
  954. options_.comparator->Compare(*ro.iterate_lower_bound,
  955. *ro.iterate_upper_bound) >= 0))) {
  956. // Lower bound behavior is not well defined if it is larger than
  957. // seek key or upper bound. Disable the check for now.
  958. *diverged = true;
  959. return;
  960. } else if (op == kLastOpSeekForPrev && ro.iterate_upper_bound != nullptr &&
  961. (options_.comparator->Compare(*ro.iterate_upper_bound, seek_key) <=
  962. 0 ||
  963. (ro.iterate_lower_bound != nullptr &&
  964. options_.comparator->Compare(*ro.iterate_lower_bound,
  965. *ro.iterate_upper_bound) >= 0))) {
  966. // Uppder bound behavior is not well defined if it is smaller than
  967. // seek key or lower bound. Disable the check for now.
  968. *diverged = true;
  969. return;
  970. }
  971. const SliceTransform* pe = (ro.total_order_seek || ro.auto_prefix_mode)
  972. ? nullptr
  973. : options_.prefix_extractor.get();
  974. const Comparator* cmp = options_.comparator;
  975. if (iter->Valid() && !cmp_iter->Valid()) {
  976. if (pe != nullptr) {
  977. if (!pe->InDomain(seek_key)) {
  978. // Prefix seek a non-in-domain key is undefined. Skip checking for
  979. // this scenario.
  980. *diverged = true;
  981. return;
  982. } else if (!pe->InDomain(iter->key())) {
  983. // out of range is iterator key is not in domain anymore.
  984. *diverged = true;
  985. return;
  986. } else if (pe->Transform(iter->key()) != pe->Transform(seek_key)) {
  987. *diverged = true;
  988. return;
  989. }
  990. }
  991. fprintf(stderr,
  992. "Control interator is invalid but iterator has key %s "
  993. "%s\n",
  994. iter->key().ToString(true).c_str(), op_logs.c_str());
  995. *diverged = true;
  996. } else if (cmp_iter->Valid()) {
  997. // Iterator is not valid. It can be legimate if it has already been
  998. // out of upper or lower bound, or filtered out by prefix iterator.
  999. const Slice& total_order_key = cmp_iter->key();
  1000. if (pe != nullptr) {
  1001. if (!pe->InDomain(seek_key)) {
  1002. // Prefix seek a non-in-domain key is undefined. Skip checking for
  1003. // this scenario.
  1004. *diverged = true;
  1005. return;
  1006. }
  1007. if (!pe->InDomain(total_order_key) ||
  1008. pe->Transform(total_order_key) != pe->Transform(seek_key)) {
  1009. // If the prefix is exhausted, the only thing needs to check
  1010. // is the iterator isn't return a position in prefix.
  1011. // Either way, checking can stop from here.
  1012. *diverged = true;
  1013. if (!iter->Valid() || !pe->InDomain(iter->key()) ||
  1014. pe->Transform(iter->key()) != pe->Transform(seek_key)) {
  1015. return;
  1016. }
  1017. fprintf(stderr,
  1018. "Iterator stays in prefix but contol doesn't"
  1019. " iterator key %s control iterator key %s %s\n",
  1020. iter->key().ToString(true).c_str(),
  1021. cmp_iter->key().ToString(true).c_str(), op_logs.c_str());
  1022. }
  1023. }
  1024. // Check upper or lower bounds.
  1025. if (!*diverged) {
  1026. if ((iter->Valid() && iter->key() != cmp_iter->key()) ||
  1027. (!iter->Valid() &&
  1028. (ro.iterate_upper_bound == nullptr ||
  1029. cmp->Compare(total_order_key, *ro.iterate_upper_bound) < 0) &&
  1030. (ro.iterate_lower_bound == nullptr ||
  1031. cmp->Compare(total_order_key, *ro.iterate_lower_bound) > 0))) {
  1032. fprintf(stderr,
  1033. "Iterator diverged from control iterator which"
  1034. " has value %s %s\n",
  1035. total_order_key.ToString(true).c_str(), op_logs.c_str());
  1036. if (iter->Valid()) {
  1037. fprintf(stderr, "iterator has value %s\n",
  1038. iter->key().ToString(true).c_str());
  1039. } else {
  1040. fprintf(stderr, "iterator is not valid\n");
  1041. }
  1042. *diverged = true;
  1043. }
  1044. }
  1045. }
  1046. if (*diverged) {
  1047. fprintf(stderr, "Control CF %s\n", cmp_cfh->GetName().c_str());
  1048. thread->stats.AddErrors(1);
  1049. // Fail fast to preserve the DB state.
  1050. thread->shared->SetVerificationFailure();
  1051. }
  1052. }
  1053. #ifdef ROCKSDB_LITE
  1054. Status StressTest::TestBackupRestore(
  1055. ThreadState* /* thread */,
  1056. const std::vector<int>& /* rand_column_families */,
  1057. const std::vector<int64_t>& /* rand_keys */) {
  1058. assert(false);
  1059. fprintf(stderr,
  1060. "RocksDB lite does not support "
  1061. "TestBackupRestore\n");
  1062. std::terminate();
  1063. }
  1064. Status StressTest::TestCheckpoint(
  1065. ThreadState* /* thread */,
  1066. const std::vector<int>& /* rand_column_families */,
  1067. const std::vector<int64_t>& /* rand_keys */) {
  1068. assert(false);
  1069. fprintf(stderr,
  1070. "RocksDB lite does not support "
  1071. "TestCheckpoint\n");
  1072. std::terminate();
  1073. }
  1074. void StressTest::TestCompactFiles(ThreadState* /* thread */,
  1075. ColumnFamilyHandle* /* column_family */) {
  1076. assert(false);
  1077. fprintf(stderr,
  1078. "RocksDB lite does not support "
  1079. "CompactFiles\n");
  1080. std::terminate();
  1081. }
  1082. #else // ROCKSDB_LITE
  1083. Status StressTest::TestBackupRestore(
  1084. ThreadState* thread, const std::vector<int>& rand_column_families,
  1085. const std::vector<int64_t>& rand_keys) {
  1086. // Note the column families chosen by `rand_column_families` cannot be
  1087. // dropped while the locks for `rand_keys` are held. So we should not have
  1088. // to worry about accessing those column families throughout this function.
  1089. assert(rand_column_families.size() == rand_keys.size());
  1090. std::string backup_dir = FLAGS_db + "/.backup" + ToString(thread->tid);
  1091. std::string restore_dir = FLAGS_db + "/.restore" + ToString(thread->tid);
  1092. BackupableDBOptions backup_opts(backup_dir);
  1093. BackupEngine* backup_engine = nullptr;
  1094. Status s = BackupEngine::Open(db_stress_env, backup_opts, &backup_engine);
  1095. if (s.ok()) {
  1096. s = backup_engine->CreateNewBackup(db_);
  1097. }
  1098. if (s.ok()) {
  1099. delete backup_engine;
  1100. backup_engine = nullptr;
  1101. s = BackupEngine::Open(db_stress_env, backup_opts, &backup_engine);
  1102. }
  1103. if (s.ok()) {
  1104. s = backup_engine->RestoreDBFromLatestBackup(restore_dir /* db_dir */,
  1105. restore_dir /* wal_dir */);
  1106. }
  1107. if (s.ok()) {
  1108. s = backup_engine->PurgeOldBackups(0 /* num_backups_to_keep */);
  1109. }
  1110. DB* restored_db = nullptr;
  1111. std::vector<ColumnFamilyHandle*> restored_cf_handles;
  1112. if (s.ok()) {
  1113. Options restore_options(options_);
  1114. restore_options.listeners.clear();
  1115. std::vector<ColumnFamilyDescriptor> cf_descriptors;
  1116. // TODO(ajkr): `column_family_names_` is not safe to access here when
  1117. // `clear_column_family_one_in != 0`. But we can't easily switch to
  1118. // `ListColumnFamilies` to get names because it won't necessarily give
  1119. // the same order as `column_family_names_`.
  1120. assert(FLAGS_clear_column_family_one_in == 0);
  1121. for (auto name : column_family_names_) {
  1122. cf_descriptors.emplace_back(name, ColumnFamilyOptions(restore_options));
  1123. }
  1124. s = DB::Open(DBOptions(restore_options), restore_dir, cf_descriptors,
  1125. &restored_cf_handles, &restored_db);
  1126. }
  1127. // for simplicity, currently only verifies existence/non-existence of a few
  1128. // keys
  1129. for (size_t i = 0; s.ok() && i < rand_column_families.size(); ++i) {
  1130. std::string key_str = Key(rand_keys[i]);
  1131. Slice key = key_str;
  1132. std::string restored_value;
  1133. Status get_status = restored_db->Get(
  1134. ReadOptions(), restored_cf_handles[rand_column_families[i]], key,
  1135. &restored_value);
  1136. bool exists = thread->shared->Exists(rand_column_families[i], rand_keys[i]);
  1137. if (get_status.ok()) {
  1138. if (!exists) {
  1139. s = Status::Corruption("key exists in restore but not in original db");
  1140. }
  1141. } else if (get_status.IsNotFound()) {
  1142. if (exists) {
  1143. s = Status::Corruption("key exists in original db but not in restore");
  1144. }
  1145. } else {
  1146. s = get_status;
  1147. }
  1148. }
  1149. if (backup_engine != nullptr) {
  1150. delete backup_engine;
  1151. backup_engine = nullptr;
  1152. }
  1153. if (restored_db != nullptr) {
  1154. for (auto* cf_handle : restored_cf_handles) {
  1155. restored_db->DestroyColumnFamilyHandle(cf_handle);
  1156. }
  1157. delete restored_db;
  1158. restored_db = nullptr;
  1159. }
  1160. if (!s.ok()) {
  1161. fprintf(stderr, "A backup/restore operation failed with: %s\n",
  1162. s.ToString().c_str());
  1163. }
  1164. return s;
  1165. }
  1166. #ifndef ROCKSDB_LITE
  1167. Status StressTest::TestApproximateSize(
  1168. ThreadState* thread, uint64_t iteration,
  1169. const std::vector<int>& rand_column_families,
  1170. const std::vector<int64_t>& rand_keys) {
  1171. // rand_keys likely only has one key. Just use the first one.
  1172. assert(!rand_keys.empty());
  1173. assert(!rand_column_families.empty());
  1174. int64_t key1 = rand_keys[0];
  1175. int64_t key2;
  1176. if (thread->rand.OneIn(2)) {
  1177. // Two totally random keys. This tends to cover large ranges.
  1178. key2 = GenerateOneKey(thread, iteration);
  1179. if (key2 < key1) {
  1180. std::swap(key1, key2);
  1181. }
  1182. } else {
  1183. // Unless users pass a very large FLAGS_max_key, it we should not worry
  1184. // about overflow. It is for testing, so we skip the overflow checking
  1185. // for simplicity.
  1186. key2 = key1 + static_cast<int64_t>(thread->rand.Uniform(1000));
  1187. }
  1188. std::string key1_str = Key(key1);
  1189. std::string key2_str = Key(key2);
  1190. Range range{Slice(key1_str), Slice(key2_str)};
  1191. SizeApproximationOptions sao;
  1192. sao.include_memtabtles = thread->rand.OneIn(2);
  1193. if (sao.include_memtabtles) {
  1194. sao.include_files = thread->rand.OneIn(2);
  1195. }
  1196. if (thread->rand.OneIn(2)) {
  1197. if (thread->rand.OneIn(2)) {
  1198. sao.files_size_error_margin = 0.0;
  1199. } else {
  1200. sao.files_size_error_margin =
  1201. static_cast<double>(thread->rand.Uniform(3));
  1202. }
  1203. }
  1204. uint64_t result;
  1205. return db_->GetApproximateSizes(
  1206. sao, column_families_[rand_column_families[0]], &range, 1, &result);
  1207. }
  1208. #endif // ROCKSDB_LITE
  1209. Status StressTest::TestCheckpoint(ThreadState* thread,
  1210. const std::vector<int>& rand_column_families,
  1211. const std::vector<int64_t>& rand_keys) {
  1212. // Note the column families chosen by `rand_column_families` cannot be
  1213. // dropped while the locks for `rand_keys` are held. So we should not have
  1214. // to worry about accessing those column families throughout this function.
  1215. assert(rand_column_families.size() == rand_keys.size());
  1216. std::string checkpoint_dir =
  1217. FLAGS_db + "/.checkpoint" + ToString(thread->tid);
  1218. Options tmp_opts(options_);
  1219. tmp_opts.listeners.clear();
  1220. tmp_opts.env = db_stress_env->target();
  1221. DestroyDB(checkpoint_dir, tmp_opts);
  1222. Checkpoint* checkpoint = nullptr;
  1223. Status s = Checkpoint::Create(db_, &checkpoint);
  1224. if (s.ok()) {
  1225. s = checkpoint->CreateCheckpoint(checkpoint_dir);
  1226. }
  1227. std::vector<ColumnFamilyHandle*> cf_handles;
  1228. DB* checkpoint_db = nullptr;
  1229. if (s.ok()) {
  1230. delete checkpoint;
  1231. checkpoint = nullptr;
  1232. Options options(options_);
  1233. options.listeners.clear();
  1234. std::vector<ColumnFamilyDescriptor> cf_descs;
  1235. // TODO(ajkr): `column_family_names_` is not safe to access here when
  1236. // `clear_column_family_one_in != 0`. But we can't easily switch to
  1237. // `ListColumnFamilies` to get names because it won't necessarily give
  1238. // the same order as `column_family_names_`.
  1239. if (FLAGS_clear_column_family_one_in == 0) {
  1240. for (const auto& name : column_family_names_) {
  1241. cf_descs.emplace_back(name, ColumnFamilyOptions(options));
  1242. }
  1243. s = DB::OpenForReadOnly(DBOptions(options), checkpoint_dir, cf_descs,
  1244. &cf_handles, &checkpoint_db);
  1245. }
  1246. }
  1247. if (checkpoint_db != nullptr) {
  1248. for (size_t i = 0; s.ok() && i < rand_column_families.size(); ++i) {
  1249. std::string key_str = Key(rand_keys[i]);
  1250. Slice key = key_str;
  1251. std::string value;
  1252. Status get_status = checkpoint_db->Get(
  1253. ReadOptions(), cf_handles[rand_column_families[i]], key, &value);
  1254. bool exists =
  1255. thread->shared->Exists(rand_column_families[i], rand_keys[i]);
  1256. if (get_status.ok()) {
  1257. if (!exists) {
  1258. s = Status::Corruption(
  1259. "key exists in checkpoint but not in original db");
  1260. }
  1261. } else if (get_status.IsNotFound()) {
  1262. if (exists) {
  1263. s = Status::Corruption(
  1264. "key exists in original db but not in checkpoint");
  1265. }
  1266. } else {
  1267. s = get_status;
  1268. }
  1269. }
  1270. for (auto cfh : cf_handles) {
  1271. delete cfh;
  1272. }
  1273. cf_handles.clear();
  1274. delete checkpoint_db;
  1275. checkpoint_db = nullptr;
  1276. }
  1277. DestroyDB(checkpoint_dir, tmp_opts);
  1278. if (!s.ok()) {
  1279. fprintf(stderr, "A checkpoint operation failed with: %s\n",
  1280. s.ToString().c_str());
  1281. }
  1282. return s;
  1283. }
  1284. void StressTest::TestCompactFiles(ThreadState* thread,
  1285. ColumnFamilyHandle* column_family) {
  1286. ROCKSDB_NAMESPACE::ColumnFamilyMetaData cf_meta_data;
  1287. db_->GetColumnFamilyMetaData(column_family, &cf_meta_data);
  1288. // Randomly compact up to three consecutive files from a level
  1289. const int kMaxRetry = 3;
  1290. for (int attempt = 0; attempt < kMaxRetry; ++attempt) {
  1291. size_t random_level =
  1292. thread->rand.Uniform(static_cast<int>(cf_meta_data.levels.size()));
  1293. const auto& files = cf_meta_data.levels[random_level].files;
  1294. if (files.size() > 0) {
  1295. size_t random_file_index =
  1296. thread->rand.Uniform(static_cast<int>(files.size()));
  1297. if (files[random_file_index].being_compacted) {
  1298. // Retry as the selected file is currently being compacted
  1299. continue;
  1300. }
  1301. std::vector<std::string> input_files;
  1302. input_files.push_back(files[random_file_index].name);
  1303. if (random_file_index > 0 &&
  1304. !files[random_file_index - 1].being_compacted) {
  1305. input_files.push_back(files[random_file_index - 1].name);
  1306. }
  1307. if (random_file_index + 1 < files.size() &&
  1308. !files[random_file_index + 1].being_compacted) {
  1309. input_files.push_back(files[random_file_index + 1].name);
  1310. }
  1311. size_t output_level =
  1312. std::min(random_level + 1, cf_meta_data.levels.size() - 1);
  1313. auto s = db_->CompactFiles(CompactionOptions(), column_family,
  1314. input_files, static_cast<int>(output_level));
  1315. if (!s.ok()) {
  1316. fprintf(stdout, "Unable to perform CompactFiles(): %s\n",
  1317. s.ToString().c_str());
  1318. thread->stats.AddNumCompactFilesFailed(1);
  1319. } else {
  1320. thread->stats.AddNumCompactFilesSucceed(1);
  1321. }
  1322. break;
  1323. }
  1324. }
  1325. }
  1326. #endif // ROCKSDB_LITE
  1327. Status StressTest::TestFlush(const std::vector<int>& rand_column_families) {
  1328. FlushOptions flush_opts;
  1329. std::vector<ColumnFamilyHandle*> cfhs;
  1330. std::for_each(rand_column_families.begin(), rand_column_families.end(),
  1331. [this, &cfhs](int k) { cfhs.push_back(column_families_[k]); });
  1332. return db_->Flush(flush_opts, cfhs);
  1333. }
  1334. Status StressTest::TestPauseBackground(ThreadState* thread) {
  1335. Status status = db_->PauseBackgroundWork();
  1336. if (!status.ok()) {
  1337. return status;
  1338. }
  1339. // To avoid stalling/deadlocking ourself in this thread, just
  1340. // sleep here during pause and let other threads do db operations.
  1341. // Sleep up to ~16 seconds (2**24 microseconds), but very skewed
  1342. // toward short pause. (1 chance in 25 of pausing >= 1s;
  1343. // 1 chance in 625 of pausing full 16s.)
  1344. int pwr2_micros =
  1345. std::min(thread->rand.Uniform(25), thread->rand.Uniform(25));
  1346. db_stress_env->SleepForMicroseconds(1 << pwr2_micros);
  1347. return db_->ContinueBackgroundWork();
  1348. }
  1349. void StressTest::TestAcquireSnapshot(ThreadState* thread,
  1350. int rand_column_family,
  1351. const std::string& keystr, uint64_t i) {
  1352. Slice key = keystr;
  1353. ColumnFamilyHandle* column_family = column_families_[rand_column_family];
  1354. #ifndef ROCKSDB_LITE
  1355. auto db_impl = reinterpret_cast<DBImpl*>(db_->GetRootDB());
  1356. const bool ww_snapshot = thread->rand.OneIn(10);
  1357. const Snapshot* snapshot =
  1358. ww_snapshot ? db_impl->GetSnapshotForWriteConflictBoundary()
  1359. : db_->GetSnapshot();
  1360. #else
  1361. const Snapshot* snapshot = db_->GetSnapshot();
  1362. #endif // !ROCKSDB_LITE
  1363. ReadOptions ropt;
  1364. ropt.snapshot = snapshot;
  1365. std::string value_at;
  1366. // When taking a snapshot, we also read a key from that snapshot. We
  1367. // will later read the same key before releasing the snapshot and
  1368. // verify that the results are the same.
  1369. auto status_at = db_->Get(ropt, column_family, key, &value_at);
  1370. std::vector<bool>* key_vec = nullptr;
  1371. if (FLAGS_compare_full_db_state_snapshot && (thread->tid == 0)) {
  1372. key_vec = new std::vector<bool>(FLAGS_max_key);
  1373. // When `prefix_extractor` is set, seeking to beginning and scanning
  1374. // across prefixes are only supported with `total_order_seek` set.
  1375. ropt.total_order_seek = true;
  1376. std::unique_ptr<Iterator> iterator(db_->NewIterator(ropt));
  1377. for (iterator->SeekToFirst(); iterator->Valid(); iterator->Next()) {
  1378. uint64_t key_val;
  1379. if (GetIntVal(iterator->key().ToString(), &key_val)) {
  1380. (*key_vec)[key_val] = true;
  1381. }
  1382. }
  1383. }
  1384. ThreadState::SnapshotState snap_state = {
  1385. snapshot, rand_column_family, column_family->GetName(),
  1386. keystr, status_at, value_at,
  1387. key_vec};
  1388. uint64_t hold_for = FLAGS_snapshot_hold_ops;
  1389. if (FLAGS_long_running_snapshots) {
  1390. // Hold 10% of snapshots for 10x more
  1391. if (thread->rand.OneIn(10)) {
  1392. assert(hold_for < port::kMaxInt64 / 10);
  1393. hold_for *= 10;
  1394. // Hold 1% of snapshots for 100x more
  1395. if (thread->rand.OneIn(10)) {
  1396. assert(hold_for < port::kMaxInt64 / 10);
  1397. hold_for *= 10;
  1398. }
  1399. }
  1400. }
  1401. uint64_t release_at = std::min(FLAGS_ops_per_thread - 1, i + hold_for);
  1402. thread->snapshot_queue.emplace(release_at, snap_state);
  1403. }
  1404. Status StressTest::MaybeReleaseSnapshots(ThreadState* thread, uint64_t i) {
  1405. while (!thread->snapshot_queue.empty() &&
  1406. i >= thread->snapshot_queue.front().first) {
  1407. auto snap_state = thread->snapshot_queue.front().second;
  1408. assert(snap_state.snapshot);
  1409. // Note: this is unsafe as the cf might be dropped concurrently. But
  1410. // it is ok since unclean cf drop is cunnrently not supported by write
  1411. // prepared transactions.
  1412. Status s = AssertSame(db_, column_families_[snap_state.cf_at], snap_state);
  1413. db_->ReleaseSnapshot(snap_state.snapshot);
  1414. delete snap_state.key_vec;
  1415. thread->snapshot_queue.pop();
  1416. if (!s.ok()) {
  1417. return s;
  1418. }
  1419. }
  1420. return Status::OK();
  1421. }
  1422. void StressTest::TestCompactRange(ThreadState* thread, int64_t rand_key,
  1423. const Slice& start_key,
  1424. ColumnFamilyHandle* column_family) {
  1425. int64_t end_key_num;
  1426. if (port::kMaxInt64 - rand_key < FLAGS_compact_range_width) {
  1427. end_key_num = port::kMaxInt64;
  1428. } else {
  1429. end_key_num = FLAGS_compact_range_width + rand_key;
  1430. }
  1431. std::string end_key_buf = Key(end_key_num);
  1432. Slice end_key(end_key_buf);
  1433. CompactRangeOptions cro;
  1434. cro.exclusive_manual_compaction = static_cast<bool>(thread->rand.Next() % 2);
  1435. cro.change_level = static_cast<bool>(thread->rand.Next() % 2);
  1436. std::vector<BottommostLevelCompaction> bottom_level_styles = {
  1437. BottommostLevelCompaction::kSkip,
  1438. BottommostLevelCompaction::kIfHaveCompactionFilter,
  1439. BottommostLevelCompaction::kForce,
  1440. BottommostLevelCompaction::kForceOptimized};
  1441. cro.bottommost_level_compaction =
  1442. bottom_level_styles[thread->rand.Next() %
  1443. static_cast<uint32_t>(bottom_level_styles.size())];
  1444. cro.allow_write_stall = static_cast<bool>(thread->rand.Next() % 2);
  1445. cro.max_subcompactions = static_cast<uint32_t>(thread->rand.Next() % 4);
  1446. const Snapshot* pre_snapshot = nullptr;
  1447. uint32_t pre_hash = 0;
  1448. if (thread->rand.OneIn(2)) {
  1449. // Do some validation by declaring a snapshot and compare the data before
  1450. // and after the compaction
  1451. pre_snapshot = db_->GetSnapshot();
  1452. pre_hash =
  1453. GetRangeHash(thread, pre_snapshot, column_family, start_key, end_key);
  1454. }
  1455. Status status = db_->CompactRange(cro, column_family, &start_key, &end_key);
  1456. if (!status.ok()) {
  1457. fprintf(stdout, "Unable to perform CompactRange(): %s\n",
  1458. status.ToString().c_str());
  1459. }
  1460. if (pre_snapshot != nullptr) {
  1461. uint32_t post_hash =
  1462. GetRangeHash(thread, pre_snapshot, column_family, start_key, end_key);
  1463. if (pre_hash != post_hash) {
  1464. fprintf(stderr,
  1465. "Data hash different before and after compact range "
  1466. "start_key %s end_key %s\n",
  1467. start_key.ToString(true).c_str(), end_key.ToString(true).c_str());
  1468. thread->stats.AddErrors(1);
  1469. // Fail fast to preserve the DB state.
  1470. thread->shared->SetVerificationFailure();
  1471. }
  1472. db_->ReleaseSnapshot(pre_snapshot);
  1473. }
  1474. }
  1475. uint32_t StressTest::GetRangeHash(ThreadState* thread, const Snapshot* snapshot,
  1476. ColumnFamilyHandle* column_family,
  1477. const Slice& start_key,
  1478. const Slice& end_key) {
  1479. const std::string kCrcCalculatorSepearator = ";";
  1480. uint32_t crc = 0;
  1481. ReadOptions ro;
  1482. ro.snapshot = snapshot;
  1483. ro.total_order_seek = true;
  1484. std::unique_ptr<Iterator> it(db_->NewIterator(ro, column_family));
  1485. for (it->Seek(start_key);
  1486. it->Valid() && options_.comparator->Compare(it->key(), end_key) <= 0;
  1487. it->Next()) {
  1488. crc = crc32c::Extend(crc, it->key().data(), it->key().size());
  1489. crc = crc32c::Extend(crc, kCrcCalculatorSepearator.data(), 1);
  1490. crc = crc32c::Extend(crc, it->value().data(), it->value().size());
  1491. crc = crc32c::Extend(crc, kCrcCalculatorSepearator.data(), 1);
  1492. }
  1493. if (!it->status().ok()) {
  1494. fprintf(stderr, "Iterator non-OK when calculating range CRC: %s\n",
  1495. it->status().ToString().c_str());
  1496. thread->stats.AddErrors(1);
  1497. // Fail fast to preserve the DB state.
  1498. thread->shared->SetVerificationFailure();
  1499. }
  1500. return crc;
  1501. }
  1502. void StressTest::PrintEnv() const {
  1503. fprintf(stdout, "RocksDB version : %d.%d\n", kMajorVersion,
  1504. kMinorVersion);
  1505. fprintf(stdout, "Format version : %d\n", FLAGS_format_version);
  1506. fprintf(stdout, "TransactionDB : %s\n",
  1507. FLAGS_use_txn ? "true" : "false");
  1508. #ifndef ROCKSDB_LITE
  1509. fprintf(stdout, "BlobDB : %s\n",
  1510. FLAGS_use_blob_db ? "true" : "false");
  1511. #endif // !ROCKSDB_LITE
  1512. fprintf(stdout, "Read only mode : %s\n",
  1513. FLAGS_read_only ? "true" : "false");
  1514. fprintf(stdout, "Atomic flush : %s\n",
  1515. FLAGS_atomic_flush ? "true" : "false");
  1516. fprintf(stdout, "Column families : %d\n", FLAGS_column_families);
  1517. if (!FLAGS_test_batches_snapshots) {
  1518. fprintf(stdout, "Clear CFs one in : %d\n",
  1519. FLAGS_clear_column_family_one_in);
  1520. }
  1521. fprintf(stdout, "Number of threads : %d\n", FLAGS_threads);
  1522. fprintf(stdout, "Ops per thread : %lu\n",
  1523. (unsigned long)FLAGS_ops_per_thread);
  1524. std::string ttl_state("unused");
  1525. if (FLAGS_ttl > 0) {
  1526. ttl_state = NumberToString(FLAGS_ttl);
  1527. }
  1528. fprintf(stdout, "Time to live(sec) : %s\n", ttl_state.c_str());
  1529. fprintf(stdout, "Read percentage : %d%%\n", FLAGS_readpercent);
  1530. fprintf(stdout, "Prefix percentage : %d%%\n", FLAGS_prefixpercent);
  1531. fprintf(stdout, "Write percentage : %d%%\n", FLAGS_writepercent);
  1532. fprintf(stdout, "Delete percentage : %d%%\n", FLAGS_delpercent);
  1533. fprintf(stdout, "Delete range percentage : %d%%\n", FLAGS_delrangepercent);
  1534. fprintf(stdout, "No overwrite percentage : %d%%\n",
  1535. FLAGS_nooverwritepercent);
  1536. fprintf(stdout, "Iterate percentage : %d%%\n", FLAGS_iterpercent);
  1537. fprintf(stdout, "DB-write-buffer-size : %" PRIu64 "\n",
  1538. FLAGS_db_write_buffer_size);
  1539. fprintf(stdout, "Write-buffer-size : %d\n", FLAGS_write_buffer_size);
  1540. fprintf(stdout, "Iterations : %lu\n",
  1541. (unsigned long)FLAGS_num_iterations);
  1542. fprintf(stdout, "Max key : %lu\n",
  1543. (unsigned long)FLAGS_max_key);
  1544. fprintf(stdout, "Ratio #ops/#keys : %f\n",
  1545. (1.0 * FLAGS_ops_per_thread * FLAGS_threads) / FLAGS_max_key);
  1546. fprintf(stdout, "Num times DB reopens : %d\n", FLAGS_reopen);
  1547. fprintf(stdout, "Batches/snapshots : %d\n",
  1548. FLAGS_test_batches_snapshots);
  1549. fprintf(stdout, "Do update in place : %d\n", FLAGS_in_place_update);
  1550. fprintf(stdout, "Num keys per lock : %d\n",
  1551. 1 << FLAGS_log2_keys_per_lock);
  1552. std::string compression = CompressionTypeToString(compression_type_e);
  1553. fprintf(stdout, "Compression : %s\n", compression.c_str());
  1554. std::string bottommost_compression =
  1555. CompressionTypeToString(bottommost_compression_type_e);
  1556. fprintf(stdout, "Bottommost Compression : %s\n",
  1557. bottommost_compression.c_str());
  1558. std::string checksum = ChecksumTypeToString(checksum_type_e);
  1559. fprintf(stdout, "Checksum type : %s\n", checksum.c_str());
  1560. fprintf(stdout, "Bloom bits / key : %s\n",
  1561. FormatDoubleParam(FLAGS_bloom_bits).c_str());
  1562. fprintf(stdout, "Max subcompactions : %" PRIu64 "\n",
  1563. FLAGS_subcompactions);
  1564. fprintf(stdout, "Use MultiGet : %s\n",
  1565. FLAGS_use_multiget ? "true" : "false");
  1566. const char* memtablerep = "";
  1567. switch (FLAGS_rep_factory) {
  1568. case kSkipList:
  1569. memtablerep = "skip_list";
  1570. break;
  1571. case kHashSkipList:
  1572. memtablerep = "prefix_hash";
  1573. break;
  1574. case kVectorRep:
  1575. memtablerep = "vector";
  1576. break;
  1577. }
  1578. fprintf(stdout, "Memtablerep : %s\n", memtablerep);
  1579. fprintf(stdout, "Test kill odd : %d\n", rocksdb_kill_odds);
  1580. if (!rocksdb_kill_prefix_blacklist.empty()) {
  1581. fprintf(stdout, "Skipping kill points prefixes:\n");
  1582. for (auto& p : rocksdb_kill_prefix_blacklist) {
  1583. fprintf(stdout, " %s\n", p.c_str());
  1584. }
  1585. }
  1586. fprintf(stdout, "Periodic Compaction Secs : %" PRIu64 "\n",
  1587. FLAGS_periodic_compaction_seconds);
  1588. fprintf(stdout, "Compaction TTL : %" PRIu64 "\n",
  1589. FLAGS_compaction_ttl);
  1590. fprintf(stdout, "Background Purge : %d\n",
  1591. static_cast<int>(FLAGS_avoid_unnecessary_blocking_io));
  1592. fprintf(stdout, "Write DB ID to manifest : %d\n",
  1593. static_cast<int>(FLAGS_write_dbid_to_manifest));
  1594. fprintf(stdout, "Max Write Batch Group Size: %" PRIu64 "\n",
  1595. FLAGS_max_write_batch_group_size_bytes);
  1596. fprintf(stdout, "Use dynamic level : %d\n",
  1597. static_cast<int>(FLAGS_level_compaction_dynamic_level_bytes));
  1598. fprintf(stdout, "------------------------------------------------\n");
  1599. }
  1600. void StressTest::Open() {
  1601. assert(db_ == nullptr);
  1602. #ifndef ROCKSDB_LITE
  1603. assert(txn_db_ == nullptr);
  1604. #endif
  1605. if (FLAGS_options_file.empty()) {
  1606. BlockBasedTableOptions block_based_options;
  1607. block_based_options.block_cache = cache_;
  1608. block_based_options.cache_index_and_filter_blocks =
  1609. FLAGS_cache_index_and_filter_blocks;
  1610. block_based_options.block_cache_compressed = compressed_cache_;
  1611. block_based_options.checksum = checksum_type_e;
  1612. block_based_options.block_size = FLAGS_block_size;
  1613. block_based_options.format_version =
  1614. static_cast<uint32_t>(FLAGS_format_version);
  1615. block_based_options.index_block_restart_interval =
  1616. static_cast<int32_t>(FLAGS_index_block_restart_interval);
  1617. block_based_options.filter_policy = filter_policy_;
  1618. block_based_options.partition_filters = FLAGS_partition_filters;
  1619. block_based_options.index_type =
  1620. static_cast<BlockBasedTableOptions::IndexType>(FLAGS_index_type);
  1621. options_.table_factory.reset(
  1622. NewBlockBasedTableFactory(block_based_options));
  1623. options_.db_write_buffer_size = FLAGS_db_write_buffer_size;
  1624. options_.write_buffer_size = FLAGS_write_buffer_size;
  1625. options_.max_write_buffer_number = FLAGS_max_write_buffer_number;
  1626. options_.min_write_buffer_number_to_merge =
  1627. FLAGS_min_write_buffer_number_to_merge;
  1628. options_.max_write_buffer_number_to_maintain =
  1629. FLAGS_max_write_buffer_number_to_maintain;
  1630. options_.max_write_buffer_size_to_maintain =
  1631. FLAGS_max_write_buffer_size_to_maintain;
  1632. options_.memtable_prefix_bloom_size_ratio =
  1633. FLAGS_memtable_prefix_bloom_size_ratio;
  1634. options_.memtable_whole_key_filtering = FLAGS_memtable_whole_key_filtering;
  1635. options_.max_background_compactions = FLAGS_max_background_compactions;
  1636. options_.max_background_flushes = FLAGS_max_background_flushes;
  1637. options_.compaction_style =
  1638. static_cast<ROCKSDB_NAMESPACE::CompactionStyle>(FLAGS_compaction_style);
  1639. if (FLAGS_prefix_size >= 0) {
  1640. options_.prefix_extractor.reset(
  1641. NewFixedPrefixTransform(FLAGS_prefix_size));
  1642. }
  1643. options_.max_open_files = FLAGS_open_files;
  1644. options_.statistics = dbstats;
  1645. options_.env = db_stress_env;
  1646. options_.use_fsync = FLAGS_use_fsync;
  1647. options_.compaction_readahead_size = FLAGS_compaction_readahead_size;
  1648. options_.allow_mmap_reads = FLAGS_mmap_read;
  1649. options_.allow_mmap_writes = FLAGS_mmap_write;
  1650. options_.use_direct_reads = FLAGS_use_direct_reads;
  1651. options_.use_direct_io_for_flush_and_compaction =
  1652. FLAGS_use_direct_io_for_flush_and_compaction;
  1653. options_.recycle_log_file_num =
  1654. static_cast<size_t>(FLAGS_recycle_log_file_num);
  1655. options_.target_file_size_base = FLAGS_target_file_size_base;
  1656. options_.target_file_size_multiplier = FLAGS_target_file_size_multiplier;
  1657. options_.max_bytes_for_level_base = FLAGS_max_bytes_for_level_base;
  1658. options_.max_bytes_for_level_multiplier =
  1659. FLAGS_max_bytes_for_level_multiplier;
  1660. options_.level0_stop_writes_trigger = FLAGS_level0_stop_writes_trigger;
  1661. options_.level0_slowdown_writes_trigger =
  1662. FLAGS_level0_slowdown_writes_trigger;
  1663. options_.level0_file_num_compaction_trigger =
  1664. FLAGS_level0_file_num_compaction_trigger;
  1665. options_.compression = compression_type_e;
  1666. options_.bottommost_compression = bottommost_compression_type_e;
  1667. options_.compression_opts.max_dict_bytes = FLAGS_compression_max_dict_bytes;
  1668. options_.compression_opts.zstd_max_train_bytes =
  1669. FLAGS_compression_zstd_max_train_bytes;
  1670. options_.create_if_missing = true;
  1671. options_.max_manifest_file_size = FLAGS_max_manifest_file_size;
  1672. options_.inplace_update_support = FLAGS_in_place_update;
  1673. options_.max_subcompactions = static_cast<uint32_t>(FLAGS_subcompactions);
  1674. options_.allow_concurrent_memtable_write =
  1675. FLAGS_allow_concurrent_memtable_write;
  1676. options_.periodic_compaction_seconds = FLAGS_periodic_compaction_seconds;
  1677. options_.ttl = FLAGS_compaction_ttl;
  1678. options_.enable_pipelined_write = FLAGS_enable_pipelined_write;
  1679. options_.enable_write_thread_adaptive_yield =
  1680. FLAGS_enable_write_thread_adaptive_yield;
  1681. options_.compaction_options_universal.size_ratio =
  1682. FLAGS_universal_size_ratio;
  1683. options_.compaction_options_universal.min_merge_width =
  1684. FLAGS_universal_min_merge_width;
  1685. options_.compaction_options_universal.max_merge_width =
  1686. FLAGS_universal_max_merge_width;
  1687. options_.compaction_options_universal.max_size_amplification_percent =
  1688. FLAGS_universal_max_size_amplification_percent;
  1689. options_.atomic_flush = FLAGS_atomic_flush;
  1690. options_.avoid_unnecessary_blocking_io =
  1691. FLAGS_avoid_unnecessary_blocking_io;
  1692. options_.write_dbid_to_manifest = FLAGS_write_dbid_to_manifest;
  1693. options_.max_write_batch_group_size_bytes =
  1694. FLAGS_max_write_batch_group_size_bytes;
  1695. options_.level_compaction_dynamic_level_bytes =
  1696. FLAGS_level_compaction_dynamic_level_bytes;
  1697. } else {
  1698. #ifdef ROCKSDB_LITE
  1699. fprintf(stderr, "--options_file not supported in lite mode\n");
  1700. exit(1);
  1701. #else
  1702. DBOptions db_options;
  1703. std::vector<ColumnFamilyDescriptor> cf_descriptors;
  1704. Status s = LoadOptionsFromFile(FLAGS_options_file, db_stress_env,
  1705. &db_options, &cf_descriptors);
  1706. db_options.env = new DbStressEnvWrapper(db_stress_env);
  1707. if (!s.ok()) {
  1708. fprintf(stderr, "Unable to load options file %s --- %s\n",
  1709. FLAGS_options_file.c_str(), s.ToString().c_str());
  1710. exit(1);
  1711. }
  1712. options_ = Options(db_options, cf_descriptors[0].options);
  1713. #endif // ROCKSDB_LITE
  1714. }
  1715. if (FLAGS_rate_limiter_bytes_per_sec > 0) {
  1716. options_.rate_limiter.reset(NewGenericRateLimiter(
  1717. FLAGS_rate_limiter_bytes_per_sec, 1000 /* refill_period_us */,
  1718. 10 /* fairness */,
  1719. FLAGS_rate_limit_bg_reads ? RateLimiter::Mode::kReadsOnly
  1720. : RateLimiter::Mode::kWritesOnly));
  1721. if (FLAGS_rate_limit_bg_reads) {
  1722. options_.new_table_reader_for_compaction_inputs = true;
  1723. }
  1724. }
  1725. if (FLAGS_prefix_size == 0 && FLAGS_rep_factory == kHashSkipList) {
  1726. fprintf(stderr,
  1727. "prefeix_size cannot be zero if memtablerep == prefix_hash\n");
  1728. exit(1);
  1729. }
  1730. if (FLAGS_prefix_size != 0 && FLAGS_rep_factory != kHashSkipList) {
  1731. fprintf(stderr,
  1732. "WARNING: prefix_size is non-zero but "
  1733. "memtablerep != prefix_hash\n");
  1734. }
  1735. switch (FLAGS_rep_factory) {
  1736. case kSkipList:
  1737. // no need to do anything
  1738. break;
  1739. #ifndef ROCKSDB_LITE
  1740. case kHashSkipList:
  1741. options_.memtable_factory.reset(NewHashSkipListRepFactory(10000));
  1742. break;
  1743. case kVectorRep:
  1744. options_.memtable_factory.reset(new VectorRepFactory());
  1745. break;
  1746. #else
  1747. default:
  1748. fprintf(stderr,
  1749. "RocksdbLite only supports skip list mem table. Skip "
  1750. "--rep_factory\n");
  1751. #endif // ROCKSDB_LITE
  1752. }
  1753. if (FLAGS_use_full_merge_v1) {
  1754. options_.merge_operator = MergeOperators::CreateDeprecatedPutOperator();
  1755. } else {
  1756. options_.merge_operator = MergeOperators::CreatePutOperator();
  1757. }
  1758. fprintf(stdout, "DB path: [%s]\n", FLAGS_db.c_str());
  1759. Status s;
  1760. if (FLAGS_ttl == -1) {
  1761. std::vector<std::string> existing_column_families;
  1762. s = DB::ListColumnFamilies(DBOptions(options_), FLAGS_db,
  1763. &existing_column_families); // ignore errors
  1764. if (!s.ok()) {
  1765. // DB doesn't exist
  1766. assert(existing_column_families.empty());
  1767. assert(column_family_names_.empty());
  1768. column_family_names_.push_back(kDefaultColumnFamilyName);
  1769. } else if (column_family_names_.empty()) {
  1770. // this is the first call to the function Open()
  1771. column_family_names_ = existing_column_families;
  1772. } else {
  1773. // this is a reopen. just assert that existing column_family_names are
  1774. // equivalent to what we remember
  1775. auto sorted_cfn = column_family_names_;
  1776. std::sort(sorted_cfn.begin(), sorted_cfn.end());
  1777. std::sort(existing_column_families.begin(),
  1778. existing_column_families.end());
  1779. if (sorted_cfn != existing_column_families) {
  1780. fprintf(stderr, "Expected column families differ from the existing:\n");
  1781. fprintf(stderr, "Expected: {");
  1782. for (auto cf : sorted_cfn) {
  1783. fprintf(stderr, "%s ", cf.c_str());
  1784. }
  1785. fprintf(stderr, "}\n");
  1786. fprintf(stderr, "Existing: {");
  1787. for (auto cf : existing_column_families) {
  1788. fprintf(stderr, "%s ", cf.c_str());
  1789. }
  1790. fprintf(stderr, "}\n");
  1791. }
  1792. assert(sorted_cfn == existing_column_families);
  1793. }
  1794. std::vector<ColumnFamilyDescriptor> cf_descriptors;
  1795. for (auto name : column_family_names_) {
  1796. if (name != kDefaultColumnFamilyName) {
  1797. new_column_family_name_ =
  1798. std::max(new_column_family_name_.load(), std::stoi(name) + 1);
  1799. }
  1800. cf_descriptors.emplace_back(name, ColumnFamilyOptions(options_));
  1801. }
  1802. while (cf_descriptors.size() < (size_t)FLAGS_column_families) {
  1803. std::string name = ToString(new_column_family_name_.load());
  1804. new_column_family_name_++;
  1805. cf_descriptors.emplace_back(name, ColumnFamilyOptions(options_));
  1806. column_family_names_.push_back(name);
  1807. }
  1808. options_.listeners.clear();
  1809. options_.listeners.emplace_back(
  1810. new DbStressListener(FLAGS_db, options_.db_paths, cf_descriptors));
  1811. options_.create_missing_column_families = true;
  1812. if (!FLAGS_use_txn) {
  1813. #ifndef ROCKSDB_LITE
  1814. if (FLAGS_use_blob_db) {
  1815. blob_db::BlobDBOptions blob_db_options;
  1816. blob_db_options.min_blob_size = FLAGS_blob_db_min_blob_size;
  1817. blob_db_options.bytes_per_sync = FLAGS_blob_db_bytes_per_sync;
  1818. blob_db_options.blob_file_size = FLAGS_blob_db_file_size;
  1819. blob_db_options.enable_garbage_collection = FLAGS_blob_db_enable_gc;
  1820. blob_db_options.garbage_collection_cutoff = FLAGS_blob_db_gc_cutoff;
  1821. blob_db::BlobDB* blob_db = nullptr;
  1822. s = blob_db::BlobDB::Open(options_, blob_db_options, FLAGS_db,
  1823. cf_descriptors, &column_families_, &blob_db);
  1824. if (s.ok()) {
  1825. db_ = blob_db;
  1826. }
  1827. } else
  1828. #endif // !ROCKSDB_LITE
  1829. {
  1830. if (db_preload_finished_.load() && FLAGS_read_only) {
  1831. s = DB::OpenForReadOnly(DBOptions(options_), FLAGS_db, cf_descriptors,
  1832. &column_families_, &db_);
  1833. } else {
  1834. s = DB::Open(DBOptions(options_), FLAGS_db, cf_descriptors,
  1835. &column_families_, &db_);
  1836. }
  1837. }
  1838. } else {
  1839. #ifndef ROCKSDB_LITE
  1840. TransactionDBOptions txn_db_options;
  1841. assert(FLAGS_txn_write_policy <= TxnDBWritePolicy::WRITE_UNPREPARED);
  1842. txn_db_options.write_policy =
  1843. static_cast<TxnDBWritePolicy>(FLAGS_txn_write_policy);
  1844. if (FLAGS_unordered_write) {
  1845. assert(txn_db_options.write_policy == TxnDBWritePolicy::WRITE_PREPARED);
  1846. options_.unordered_write = true;
  1847. options_.two_write_queues = true;
  1848. txn_db_options.skip_concurrency_control = true;
  1849. }
  1850. s = TransactionDB::Open(options_, txn_db_options, FLAGS_db,
  1851. cf_descriptors, &column_families_, &txn_db_);
  1852. if (!s.ok()) {
  1853. fprintf(stderr, "Error in opening the TransactionDB [%s]\n",
  1854. s.ToString().c_str());
  1855. fflush(stderr);
  1856. }
  1857. assert(s.ok());
  1858. db_ = txn_db_;
  1859. // after a crash, rollback to commit recovered transactions
  1860. std::vector<Transaction*> trans;
  1861. txn_db_->GetAllPreparedTransactions(&trans);
  1862. Random rand(static_cast<uint32_t>(FLAGS_seed));
  1863. for (auto txn : trans) {
  1864. if (rand.OneIn(2)) {
  1865. s = txn->Commit();
  1866. assert(s.ok());
  1867. } else {
  1868. s = txn->Rollback();
  1869. assert(s.ok());
  1870. }
  1871. delete txn;
  1872. }
  1873. trans.clear();
  1874. txn_db_->GetAllPreparedTransactions(&trans);
  1875. assert(trans.size() == 0);
  1876. #endif
  1877. }
  1878. assert(!s.ok() || column_families_.size() ==
  1879. static_cast<size_t>(FLAGS_column_families));
  1880. if (FLAGS_test_secondary) {
  1881. #ifndef ROCKSDB_LITE
  1882. secondaries_.resize(FLAGS_threads);
  1883. std::fill(secondaries_.begin(), secondaries_.end(), nullptr);
  1884. secondary_cfh_lists_.clear();
  1885. secondary_cfh_lists_.resize(FLAGS_threads);
  1886. Options tmp_opts;
  1887. // TODO(yanqin) support max_open_files != -1 for secondary instance.
  1888. tmp_opts.max_open_files = -1;
  1889. tmp_opts.statistics = dbstats_secondaries;
  1890. tmp_opts.env = db_stress_env;
  1891. for (size_t i = 0; i != static_cast<size_t>(FLAGS_threads); ++i) {
  1892. const std::string secondary_path =
  1893. FLAGS_secondaries_base + "/" + std::to_string(i);
  1894. s = DB::OpenAsSecondary(tmp_opts, FLAGS_db, secondary_path,
  1895. cf_descriptors, &secondary_cfh_lists_[i],
  1896. &secondaries_[i]);
  1897. if (!s.ok()) {
  1898. break;
  1899. }
  1900. }
  1901. assert(s.ok());
  1902. #else
  1903. fprintf(stderr, "Secondary is not supported in RocksDBLite\n");
  1904. exit(1);
  1905. #endif
  1906. }
  1907. if (FLAGS_continuous_verification_interval > 0 && !cmp_db_) {
  1908. Options tmp_opts;
  1909. // TODO(yanqin) support max_open_files != -1 for secondary instance.
  1910. tmp_opts.max_open_files = -1;
  1911. tmp_opts.env = db_stress_env;
  1912. std::string secondary_path = FLAGS_secondaries_base + "/cmp_database";
  1913. s = DB::OpenAsSecondary(tmp_opts, FLAGS_db, secondary_path,
  1914. cf_descriptors, &cmp_cfhs_, &cmp_db_);
  1915. assert(!s.ok() ||
  1916. cmp_cfhs_.size() == static_cast<size_t>(FLAGS_column_families));
  1917. }
  1918. } else {
  1919. #ifndef ROCKSDB_LITE
  1920. DBWithTTL* db_with_ttl;
  1921. s = DBWithTTL::Open(options_, FLAGS_db, &db_with_ttl, FLAGS_ttl);
  1922. db_ = db_with_ttl;
  1923. if (FLAGS_test_secondary) {
  1924. secondaries_.resize(FLAGS_threads);
  1925. std::fill(secondaries_.begin(), secondaries_.end(), nullptr);
  1926. Options tmp_opts;
  1927. tmp_opts.env = options_.env;
  1928. // TODO(yanqin) support max_open_files != -1 for secondary instance.
  1929. tmp_opts.max_open_files = -1;
  1930. for (size_t i = 0; i != static_cast<size_t>(FLAGS_threads); ++i) {
  1931. const std::string secondary_path =
  1932. FLAGS_secondaries_base + "/" + std::to_string(i);
  1933. s = DB::OpenAsSecondary(tmp_opts, FLAGS_db, secondary_path,
  1934. &secondaries_[i]);
  1935. if (!s.ok()) {
  1936. break;
  1937. }
  1938. }
  1939. }
  1940. #else
  1941. fprintf(stderr, "TTL is not supported in RocksDBLite\n");
  1942. exit(1);
  1943. #endif
  1944. }
  1945. if (!s.ok()) {
  1946. fprintf(stderr, "open error: %s\n", s.ToString().c_str());
  1947. exit(1);
  1948. }
  1949. }
  1950. void StressTest::Reopen(ThreadState* thread) {
  1951. #ifndef ROCKSDB_LITE
  1952. // BG jobs in WritePrepared must be canceled first because i) they can access
  1953. // the db via a callbac ii) they hold on to a snapshot and the upcoming
  1954. // ::Close would complain about it.
  1955. const bool write_prepared = FLAGS_use_txn && FLAGS_txn_write_policy != 0;
  1956. bool bg_canceled = false;
  1957. if (write_prepared || thread->rand.OneIn(2)) {
  1958. const bool wait =
  1959. write_prepared || static_cast<bool>(thread->rand.OneIn(2));
  1960. CancelAllBackgroundWork(db_, wait);
  1961. bg_canceled = wait;
  1962. }
  1963. assert(!write_prepared || bg_canceled);
  1964. (void) bg_canceled;
  1965. #else
  1966. (void) thread;
  1967. #endif
  1968. for (auto cf : column_families_) {
  1969. delete cf;
  1970. }
  1971. column_families_.clear();
  1972. #ifndef ROCKSDB_LITE
  1973. if (thread->rand.OneIn(2)) {
  1974. Status s = db_->Close();
  1975. if (!s.ok()) {
  1976. fprintf(stderr, "Non-ok close status: %s\n", s.ToString().c_str());
  1977. fflush(stderr);
  1978. }
  1979. assert(s.ok());
  1980. }
  1981. #endif
  1982. delete db_;
  1983. db_ = nullptr;
  1984. #ifndef ROCKSDB_LITE
  1985. txn_db_ = nullptr;
  1986. #endif
  1987. assert(secondaries_.size() == secondary_cfh_lists_.size());
  1988. size_t n = secondaries_.size();
  1989. for (size_t i = 0; i != n; ++i) {
  1990. for (auto* cf : secondary_cfh_lists_[i]) {
  1991. delete cf;
  1992. }
  1993. secondary_cfh_lists_[i].clear();
  1994. delete secondaries_[i];
  1995. }
  1996. secondaries_.clear();
  1997. num_times_reopened_++;
  1998. auto now = db_stress_env->NowMicros();
  1999. fprintf(stdout, "%s Reopening database for the %dth time\n",
  2000. db_stress_env->TimeToString(now / 1000000).c_str(),
  2001. num_times_reopened_);
  2002. Open();
  2003. }
  2004. } // namespace ROCKSDB_NAMESPACE
  2005. #endif // GFLAGS