db_sst_test.cc 42 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227
  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 "db/db_test_util.h"
  10. #include "file/sst_file_manager_impl.h"
  11. #include "port/port.h"
  12. #include "port/stack_trace.h"
  13. #include "rocksdb/sst_file_manager.h"
  14. namespace ROCKSDB_NAMESPACE {
  15. class DBSSTTest : public DBTestBase {
  16. public:
  17. DBSSTTest() : DBTestBase("/db_sst_test") {}
  18. };
  19. #ifndef ROCKSDB_LITE
  20. // A class which remembers the name of each flushed file.
  21. class FlushedFileCollector : public EventListener {
  22. public:
  23. FlushedFileCollector() {}
  24. ~FlushedFileCollector() override {}
  25. void OnFlushCompleted(DB* /*db*/, const FlushJobInfo& info) override {
  26. std::lock_guard<std::mutex> lock(mutex_);
  27. flushed_files_.push_back(info.file_path);
  28. }
  29. std::vector<std::string> GetFlushedFiles() {
  30. std::lock_guard<std::mutex> lock(mutex_);
  31. std::vector<std::string> result;
  32. for (auto fname : flushed_files_) {
  33. result.push_back(fname);
  34. }
  35. return result;
  36. }
  37. void ClearFlushedFiles() {
  38. std::lock_guard<std::mutex> lock(mutex_);
  39. flushed_files_.clear();
  40. }
  41. private:
  42. std::vector<std::string> flushed_files_;
  43. std::mutex mutex_;
  44. };
  45. #endif // ROCKSDB_LITE
  46. TEST_F(DBSSTTest, DontDeletePendingOutputs) {
  47. Options options;
  48. options.env = env_;
  49. options.create_if_missing = true;
  50. DestroyAndReopen(options);
  51. // Every time we write to a table file, call FOF/POF with full DB scan. This
  52. // will make sure our pending_outputs_ protection work correctly
  53. std::function<void()> purge_obsolete_files_function = [&]() {
  54. JobContext job_context(0);
  55. dbfull()->TEST_LockMutex();
  56. dbfull()->FindObsoleteFiles(&job_context, true /*force*/);
  57. dbfull()->TEST_UnlockMutex();
  58. dbfull()->PurgeObsoleteFiles(job_context);
  59. job_context.Clean();
  60. };
  61. env_->table_write_callback_ = &purge_obsolete_files_function;
  62. for (int i = 0; i < 2; ++i) {
  63. ASSERT_OK(Put("a", "begin"));
  64. ASSERT_OK(Put("z", "end"));
  65. ASSERT_OK(Flush());
  66. }
  67. // If pending output guard does not work correctly, PurgeObsoleteFiles() will
  68. // delete the file that Compaction is trying to create, causing this: error
  69. // db/db_test.cc:975: IO error:
  70. // /tmp/rocksdbtest-1552237650/db_test/000009.sst: No such file or directory
  71. Compact("a", "b");
  72. }
  73. // 1 Create some SST files by inserting K-V pairs into DB
  74. // 2 Close DB and change suffix from ".sst" to ".ldb" for every other SST file
  75. // 3 Open DB and check if all key can be read
  76. TEST_F(DBSSTTest, SSTsWithLdbSuffixHandling) {
  77. Options options = CurrentOptions();
  78. options.write_buffer_size = 110 << 10; // 110KB
  79. options.num_levels = 4;
  80. DestroyAndReopen(options);
  81. Random rnd(301);
  82. int key_id = 0;
  83. for (int i = 0; i < 10; ++i) {
  84. GenerateNewFile(&rnd, &key_id, false);
  85. }
  86. Flush();
  87. Close();
  88. int const num_files = GetSstFileCount(dbname_);
  89. ASSERT_GT(num_files, 0);
  90. Reopen(options);
  91. std::vector<std::string> values;
  92. values.reserve(key_id);
  93. for (int k = 0; k < key_id; ++k) {
  94. values.push_back(Get(Key(k)));
  95. }
  96. Close();
  97. std::vector<std::string> filenames;
  98. GetSstFiles(env_, dbname_, &filenames);
  99. int num_ldb_files = 0;
  100. for (size_t i = 0; i < filenames.size(); ++i) {
  101. if (i & 1) {
  102. continue;
  103. }
  104. std::string const rdb_name = dbname_ + "/" + filenames[i];
  105. std::string const ldb_name = Rocks2LevelTableFileName(rdb_name);
  106. ASSERT_TRUE(env_->RenameFile(rdb_name, ldb_name).ok());
  107. ++num_ldb_files;
  108. }
  109. ASSERT_GT(num_ldb_files, 0);
  110. ASSERT_EQ(num_files, GetSstFileCount(dbname_));
  111. Reopen(options);
  112. for (int k = 0; k < key_id; ++k) {
  113. ASSERT_EQ(values[k], Get(Key(k)));
  114. }
  115. Destroy(options);
  116. }
  117. // Check that we don't crash when opening DB with
  118. // DBOptions::skip_checking_sst_file_sizes_on_db_open = true.
  119. TEST_F(DBSSTTest, SkipCheckingSSTFileSizesOnDBOpen) {
  120. ASSERT_OK(Put("pika", "choo"));
  121. ASSERT_OK(Flush());
  122. // Just open the DB with the option set to true and check that we don't crash.
  123. Options options;
  124. options.skip_checking_sst_file_sizes_on_db_open = true;
  125. Reopen(options);
  126. ASSERT_EQ("choo", Get("pika"));
  127. }
  128. #ifndef ROCKSDB_LITE
  129. TEST_F(DBSSTTest, DontDeleteMovedFile) {
  130. // This test triggers move compaction and verifies that the file is not
  131. // deleted when it's part of move compaction
  132. Options options = CurrentOptions();
  133. options.env = env_;
  134. options.create_if_missing = true;
  135. options.max_bytes_for_level_base = 1024 * 1024; // 1 MB
  136. options.level0_file_num_compaction_trigger =
  137. 2; // trigger compaction when we have 2 files
  138. DestroyAndReopen(options);
  139. Random rnd(301);
  140. // Create two 1MB sst files
  141. for (int i = 0; i < 2; ++i) {
  142. // Create 1MB sst file
  143. for (int j = 0; j < 100; ++j) {
  144. ASSERT_OK(Put(Key(i * 50 + j), RandomString(&rnd, 10 * 1024)));
  145. }
  146. ASSERT_OK(Flush());
  147. }
  148. // this should execute both L0->L1 and L1->(move)->L2 compactions
  149. dbfull()->TEST_WaitForCompact();
  150. ASSERT_EQ("0,0,1", FilesPerLevel(0));
  151. // If the moved file is actually deleted (the move-safeguard in
  152. // ~Version::Version() is not there), we get this failure:
  153. // Corruption: Can't access /000009.sst
  154. Reopen(options);
  155. }
  156. // This reproduces a bug where we don't delete a file because when it was
  157. // supposed to be deleted, it was blocked by pending_outputs
  158. // Consider:
  159. // 1. current file_number is 13
  160. // 2. compaction (1) starts, blocks deletion of all files starting with 13
  161. // (pending outputs)
  162. // 3. file 13 is created by compaction (2)
  163. // 4. file 13 is consumed by compaction (3) and file 15 was created. Since file
  164. // 13 has no references, it is put into VersionSet::obsolete_files_
  165. // 5. FindObsoleteFiles() gets file 13 from VersionSet::obsolete_files_. File 13
  166. // is deleted from obsolete_files_ set.
  167. // 6. PurgeObsoleteFiles() tries to delete file 13, but this file is blocked by
  168. // pending outputs since compaction (1) is still running. It is not deleted and
  169. // it is not present in obsolete_files_ anymore. Therefore, we never delete it.
  170. TEST_F(DBSSTTest, DeleteObsoleteFilesPendingOutputs) {
  171. Options options = CurrentOptions();
  172. options.env = env_;
  173. options.write_buffer_size = 2 * 1024 * 1024; // 2 MB
  174. options.max_bytes_for_level_base = 1024 * 1024; // 1 MB
  175. options.level0_file_num_compaction_trigger =
  176. 2; // trigger compaction when we have 2 files
  177. options.max_background_flushes = 2;
  178. options.max_background_compactions = 2;
  179. OnFileDeletionListener* listener = new OnFileDeletionListener();
  180. options.listeners.emplace_back(listener);
  181. Reopen(options);
  182. Random rnd(301);
  183. // Create two 1MB sst files
  184. for (int i = 0; i < 2; ++i) {
  185. // Create 1MB sst file
  186. for (int j = 0; j < 100; ++j) {
  187. ASSERT_OK(Put(Key(i * 50 + j), RandomString(&rnd, 10 * 1024)));
  188. }
  189. ASSERT_OK(Flush());
  190. }
  191. // this should execute both L0->L1 and L1->(move)->L2 compactions
  192. dbfull()->TEST_WaitForCompact();
  193. ASSERT_EQ("0,0,1", FilesPerLevel(0));
  194. test::SleepingBackgroundTask blocking_thread;
  195. port::Mutex mutex_;
  196. bool already_blocked(false);
  197. // block the flush
  198. std::function<void()> block_first_time = [&]() {
  199. bool blocking = false;
  200. {
  201. MutexLock l(&mutex_);
  202. if (!already_blocked) {
  203. blocking = true;
  204. already_blocked = true;
  205. }
  206. }
  207. if (blocking) {
  208. blocking_thread.DoSleep();
  209. }
  210. };
  211. env_->table_write_callback_ = &block_first_time;
  212. // Insert 2.5MB data, which should trigger a flush because we exceed
  213. // write_buffer_size. The flush will be blocked with block_first_time
  214. // pending_file is protecting all the files created after
  215. for (int j = 0; j < 256; ++j) {
  216. ASSERT_OK(Put(Key(j), RandomString(&rnd, 10 * 1024)));
  217. }
  218. blocking_thread.WaitUntilSleeping();
  219. ASSERT_OK(dbfull()->TEST_CompactRange(2, nullptr, nullptr));
  220. ASSERT_EQ("0,0,0,1", FilesPerLevel(0));
  221. std::vector<LiveFileMetaData> metadata;
  222. db_->GetLiveFilesMetaData(&metadata);
  223. ASSERT_EQ(metadata.size(), 1U);
  224. auto file_on_L2 = metadata[0].name;
  225. listener->SetExpectedFileName(dbname_ + file_on_L2);
  226. ASSERT_OK(dbfull()->TEST_CompactRange(3, nullptr, nullptr, nullptr,
  227. true /* disallow trivial move */));
  228. ASSERT_EQ("0,0,0,0,1", FilesPerLevel(0));
  229. // finish the flush!
  230. blocking_thread.WakeUp();
  231. blocking_thread.WaitUntilDone();
  232. dbfull()->TEST_WaitForFlushMemTable();
  233. // File just flushed is too big for L0 and L1 so gets moved to L2.
  234. dbfull()->TEST_WaitForCompact();
  235. ASSERT_EQ("0,0,1,0,1", FilesPerLevel(0));
  236. metadata.clear();
  237. db_->GetLiveFilesMetaData(&metadata);
  238. ASSERT_EQ(metadata.size(), 2U);
  239. // This file should have been deleted during last compaction
  240. ASSERT_EQ(Status::NotFound(), env_->FileExists(dbname_ + file_on_L2));
  241. listener->VerifyMatchedCount(1);
  242. }
  243. TEST_F(DBSSTTest, DBWithSstFileManager) {
  244. std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
  245. auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  246. int files_added = 0;
  247. int files_deleted = 0;
  248. int files_moved = 0;
  249. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  250. "SstFileManagerImpl::OnAddFile", [&](void* /*arg*/) { files_added++; });
  251. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  252. "SstFileManagerImpl::OnDeleteFile",
  253. [&](void* /*arg*/) { files_deleted++; });
  254. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  255. "SstFileManagerImpl::OnMoveFile", [&](void* /*arg*/) { files_moved++; });
  256. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  257. Options options = CurrentOptions();
  258. options.sst_file_manager = sst_file_manager;
  259. DestroyAndReopen(options);
  260. Random rnd(301);
  261. for (int i = 0; i < 25; i++) {
  262. GenerateNewRandomFile(&rnd);
  263. ASSERT_OK(Flush());
  264. dbfull()->TEST_WaitForFlushMemTable();
  265. dbfull()->TEST_WaitForCompact();
  266. // Verify that we are tracking all sst files in dbname_
  267. ASSERT_EQ(sfm->GetTrackedFiles(), GetAllSSTFiles());
  268. }
  269. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  270. auto files_in_db = GetAllSSTFiles();
  271. // Verify that we are tracking all sst files in dbname_
  272. ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
  273. // Verify the total files size
  274. uint64_t total_files_size = 0;
  275. for (auto& file_to_size : files_in_db) {
  276. total_files_size += file_to_size.second;
  277. }
  278. ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
  279. // We flushed at least 25 files
  280. ASSERT_GE(files_added, 25);
  281. // Compaction must have deleted some files
  282. ASSERT_GT(files_deleted, 0);
  283. // No files were moved
  284. ASSERT_EQ(files_moved, 0);
  285. Close();
  286. Reopen(options);
  287. ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
  288. ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
  289. // Verify that we track all the files again after the DB is closed and opened
  290. Close();
  291. sst_file_manager.reset(NewSstFileManager(env_));
  292. options.sst_file_manager = sst_file_manager;
  293. sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  294. Reopen(options);
  295. ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
  296. ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
  297. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  298. }
  299. TEST_F(DBSSTTest, RateLimitedDelete) {
  300. Destroy(last_options_);
  301. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
  302. {"DBSSTTest::RateLimitedDelete:1",
  303. "DeleteScheduler::BackgroundEmptyTrash"},
  304. });
  305. std::vector<uint64_t> penalties;
  306. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  307. "DeleteScheduler::BackgroundEmptyTrash:Wait",
  308. [&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
  309. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  310. "InstrumentedCondVar::TimedWaitInternal", [&](void* arg) {
  311. // Turn timed wait into a simulated sleep
  312. uint64_t* abs_time_us = static_cast<uint64_t*>(arg);
  313. int64_t cur_time = 0;
  314. env_->GetCurrentTime(&cur_time);
  315. if (*abs_time_us > static_cast<uint64_t>(cur_time)) {
  316. env_->addon_time_.fetch_add(*abs_time_us -
  317. static_cast<uint64_t>(cur_time));
  318. }
  319. // Randomly sleep shortly
  320. env_->addon_time_.fetch_add(
  321. static_cast<uint64_t>(Random::GetTLSInstance()->Uniform(10)));
  322. // Set wait until time to before current to force not to sleep.
  323. int64_t real_cur_time = 0;
  324. Env::Default()->GetCurrentTime(&real_cur_time);
  325. *abs_time_us = static_cast<uint64_t>(real_cur_time);
  326. });
  327. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  328. env_->no_slowdown_ = true;
  329. env_->time_elapse_only_sleep_ = true;
  330. Options options = CurrentOptions();
  331. options.disable_auto_compactions = true;
  332. // Need to disable stats dumping and persisting which also use
  333. // RepeatableThread, one of whose member variables is of type
  334. // InstrumentedCondVar. The callback for
  335. // InstrumentedCondVar::TimedWaitInternal can be triggered by stats dumping
  336. // and persisting threads and cause time_spent_deleting measurement to become
  337. // incorrect.
  338. options.stats_dump_period_sec = 0;
  339. options.stats_persist_period_sec = 0;
  340. options.env = env_;
  341. int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
  342. Status s;
  343. options.sst_file_manager.reset(
  344. NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
  345. ASSERT_OK(s);
  346. options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
  347. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  348. sfm->delete_scheduler()->SetMaxTrashDBRatio(1.1);
  349. WriteOptions wo;
  350. wo.disableWAL = true;
  351. ASSERT_OK(TryReopen(options));
  352. // Create 4 files in L0
  353. for (char v = 'a'; v <= 'd'; v++) {
  354. ASSERT_OK(Put("Key2", DummyString(1024, v), wo));
  355. ASSERT_OK(Put("Key3", DummyString(1024, v), wo));
  356. ASSERT_OK(Put("Key4", DummyString(1024, v), wo));
  357. ASSERT_OK(Put("Key1", DummyString(1024, v), wo));
  358. ASSERT_OK(Put("Key4", DummyString(1024, v), wo));
  359. ASSERT_OK(Flush());
  360. }
  361. // We created 4 sst files in L0
  362. ASSERT_EQ("4", FilesPerLevel(0));
  363. std::vector<LiveFileMetaData> metadata;
  364. db_->GetLiveFilesMetaData(&metadata);
  365. // Compaction will move the 4 files in L0 to trash and create 1 L1 file
  366. ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  367. ASSERT_OK(dbfull()->TEST_WaitForCompact(true));
  368. ASSERT_EQ("0,1", FilesPerLevel(0));
  369. uint64_t delete_start_time = env_->NowMicros();
  370. // Hold BackgroundEmptyTrash
  371. TEST_SYNC_POINT("DBSSTTest::RateLimitedDelete:1");
  372. sfm->WaitForEmptyTrash();
  373. uint64_t time_spent_deleting = env_->NowMicros() - delete_start_time;
  374. uint64_t total_files_size = 0;
  375. uint64_t expected_penlty = 0;
  376. ASSERT_EQ(penalties.size(), metadata.size());
  377. for (size_t i = 0; i < metadata.size(); i++) {
  378. total_files_size += metadata[i].size;
  379. expected_penlty = ((total_files_size * 1000000) / rate_bytes_per_sec);
  380. ASSERT_EQ(expected_penlty, penalties[i]);
  381. }
  382. ASSERT_GT(time_spent_deleting, expected_penlty * 0.9);
  383. ASSERT_LT(time_spent_deleting, expected_penlty * 1.1);
  384. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  385. }
  386. TEST_F(DBSSTTest, RateLimitedWALDelete) {
  387. Destroy(last_options_);
  388. std::vector<uint64_t> penalties;
  389. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  390. "DeleteScheduler::BackgroundEmptyTrash:Wait",
  391. [&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
  392. env_->no_slowdown_ = true;
  393. env_->time_elapse_only_sleep_ = true;
  394. Options options = CurrentOptions();
  395. options.disable_auto_compactions = true;
  396. options.compression = kNoCompression;
  397. options.env = env_;
  398. int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
  399. Status s;
  400. options.sst_file_manager.reset(
  401. NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
  402. ASSERT_OK(s);
  403. options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
  404. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  405. sfm->delete_scheduler()->SetMaxTrashDBRatio(3.1);
  406. ASSERT_OK(TryReopen(options));
  407. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  408. // Create 4 files in L0
  409. for (char v = 'a'; v <= 'd'; v++) {
  410. ASSERT_OK(Put("Key2", DummyString(1024, v)));
  411. ASSERT_OK(Put("Key3", DummyString(1024, v)));
  412. ASSERT_OK(Put("Key4", DummyString(1024, v)));
  413. ASSERT_OK(Put("Key1", DummyString(1024, v)));
  414. ASSERT_OK(Put("Key4", DummyString(1024, v)));
  415. ASSERT_OK(Flush());
  416. }
  417. // We created 4 sst files in L0
  418. ASSERT_EQ("4", FilesPerLevel(0));
  419. // Compaction will move the 4 files in L0 to trash and create 1 L1 file
  420. CompactRangeOptions cro;
  421. cro.bottommost_level_compaction = BottommostLevelCompaction::kForce;
  422. ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr));
  423. ASSERT_OK(dbfull()->TEST_WaitForCompact(true));
  424. ASSERT_EQ("0,1", FilesPerLevel(0));
  425. sfm->WaitForEmptyTrash();
  426. ASSERT_EQ(penalties.size(), 8);
  427. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  428. }
  429. class DBWALTestWithParam
  430. : public DBSSTTest,
  431. public testing::WithParamInterface<std::tuple<std::string, bool>> {
  432. public:
  433. DBWALTestWithParam() {
  434. wal_dir_ = std::get<0>(GetParam());
  435. wal_dir_same_as_dbname_ = std::get<1>(GetParam());
  436. }
  437. std::string wal_dir_;
  438. bool wal_dir_same_as_dbname_;
  439. };
  440. TEST_P(DBWALTestWithParam, WALTrashCleanupOnOpen) {
  441. class MyEnv : public EnvWrapper {
  442. public:
  443. MyEnv(Env* t) : EnvWrapper(t), fake_log_delete(false) {}
  444. Status DeleteFile(const std::string& fname) {
  445. if (fname.find(".log.trash") != std::string::npos && fake_log_delete) {
  446. return Status::OK();
  447. }
  448. return target()->DeleteFile(fname);
  449. }
  450. void set_fake_log_delete(bool fake) { fake_log_delete = fake; }
  451. private:
  452. bool fake_log_delete;
  453. };
  454. std::unique_ptr<MyEnv> env(new MyEnv(Env::Default()));
  455. Destroy(last_options_);
  456. env->set_fake_log_delete(true);
  457. Options options = CurrentOptions();
  458. options.disable_auto_compactions = true;
  459. options.compression = kNoCompression;
  460. options.env = env.get();
  461. options.wal_dir = dbname_ + wal_dir_;
  462. int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
  463. Status s;
  464. options.sst_file_manager.reset(
  465. NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
  466. ASSERT_OK(s);
  467. options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
  468. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  469. sfm->delete_scheduler()->SetMaxTrashDBRatio(3.1);
  470. ASSERT_OK(TryReopen(options));
  471. // Create 4 files in L0
  472. for (char v = 'a'; v <= 'd'; v++) {
  473. ASSERT_OK(Put("Key2", DummyString(1024, v)));
  474. ASSERT_OK(Put("Key3", DummyString(1024, v)));
  475. ASSERT_OK(Put("Key4", DummyString(1024, v)));
  476. ASSERT_OK(Put("Key1", DummyString(1024, v)));
  477. ASSERT_OK(Put("Key4", DummyString(1024, v)));
  478. ASSERT_OK(Flush());
  479. }
  480. // We created 4 sst files in L0
  481. ASSERT_EQ("4", FilesPerLevel(0));
  482. Close();
  483. options.sst_file_manager.reset();
  484. std::vector<std::string> filenames;
  485. int trash_log_count = 0;
  486. if (!wal_dir_same_as_dbname_) {
  487. // Forcibly create some trash log files
  488. std::unique_ptr<WritableFile> result;
  489. env->NewWritableFile(options.wal_dir + "/1000.log.trash", &result,
  490. EnvOptions());
  491. result.reset();
  492. }
  493. env->GetChildren(options.wal_dir, &filenames);
  494. for (const std::string& fname : filenames) {
  495. if (fname.find(".log.trash") != std::string::npos) {
  496. trash_log_count++;
  497. }
  498. }
  499. ASSERT_GE(trash_log_count, 1);
  500. env->set_fake_log_delete(false);
  501. ASSERT_OK(TryReopen(options));
  502. filenames.clear();
  503. trash_log_count = 0;
  504. env->GetChildren(options.wal_dir, &filenames);
  505. for (const std::string& fname : filenames) {
  506. if (fname.find(".log.trash") != std::string::npos) {
  507. trash_log_count++;
  508. }
  509. }
  510. ASSERT_EQ(trash_log_count, 0);
  511. Close();
  512. }
  513. INSTANTIATE_TEST_CASE_P(DBWALTestWithParam, DBWALTestWithParam,
  514. ::testing::Values(std::make_tuple("", true),
  515. std::make_tuple("_wal_dir", false)));
  516. TEST_F(DBSSTTest, OpenDBWithExistingTrash) {
  517. Options options = CurrentOptions();
  518. options.sst_file_manager.reset(
  519. NewSstFileManager(env_, nullptr, "", 1024 * 1024 /* 1 MB/sec */));
  520. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  521. Destroy(last_options_);
  522. // Add some trash files to the db directory so the DB can clean them up
  523. env_->CreateDirIfMissing(dbname_);
  524. ASSERT_OK(WriteStringToFile(env_, "abc", dbname_ + "/" + "001.sst.trash"));
  525. ASSERT_OK(WriteStringToFile(env_, "abc", dbname_ + "/" + "002.sst.trash"));
  526. ASSERT_OK(WriteStringToFile(env_, "abc", dbname_ + "/" + "003.sst.trash"));
  527. // Reopen the DB and verify that it deletes existing trash files
  528. ASSERT_OK(TryReopen(options));
  529. sfm->WaitForEmptyTrash();
  530. ASSERT_NOK(env_->FileExists(dbname_ + "/" + "001.sst.trash"));
  531. ASSERT_NOK(env_->FileExists(dbname_ + "/" + "002.sst.trash"));
  532. ASSERT_NOK(env_->FileExists(dbname_ + "/" + "003.sst.trash"));
  533. }
  534. // Create a DB with 2 db_paths, and generate multiple files in the 2
  535. // db_paths using CompactRangeOptions, make sure that files that were
  536. // deleted from first db_path were deleted using DeleteScheduler and
  537. // files in the second path were not.
  538. TEST_F(DBSSTTest, DeleteSchedulerMultipleDBPaths) {
  539. std::atomic<int> bg_delete_file(0);
  540. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  541. "DeleteScheduler::DeleteTrashFile:DeleteFile",
  542. [&](void* /*arg*/) { bg_delete_file++; });
  543. // The deletion scheduler sometimes skips marking file as trash according to
  544. // a heuristic. In that case the deletion will go through the below SyncPoint.
  545. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  546. "DeleteScheduler::DeleteFile", [&](void* /*arg*/) { bg_delete_file++; });
  547. Options options = CurrentOptions();
  548. options.disable_auto_compactions = true;
  549. options.db_paths.emplace_back(dbname_, 1024 * 100);
  550. options.db_paths.emplace_back(dbname_ + "_2", 1024 * 100);
  551. options.env = env_;
  552. int64_t rate_bytes_per_sec = 1024 * 1024; // 1 Mb / Sec
  553. Status s;
  554. options.sst_file_manager.reset(
  555. NewSstFileManager(env_, nullptr, "", rate_bytes_per_sec, false, &s,
  556. /* max_trash_db_ratio= */ 1.1));
  557. ASSERT_OK(s);
  558. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  559. DestroyAndReopen(options);
  560. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  561. WriteOptions wo;
  562. wo.disableWAL = true;
  563. // Create 4 files in L0
  564. for (int i = 0; i < 4; i++) {
  565. ASSERT_OK(Put("Key" + ToString(i), DummyString(1024, 'A'), wo));
  566. ASSERT_OK(Flush());
  567. }
  568. // We created 4 sst files in L0
  569. ASSERT_EQ("4", FilesPerLevel(0));
  570. // Compaction will delete files from L0 in first db path and generate a new
  571. // file in L1 in second db path
  572. CompactRangeOptions compact_options;
  573. compact_options.target_path_id = 1;
  574. Slice begin("Key0");
  575. Slice end("Key3");
  576. ASSERT_OK(db_->CompactRange(compact_options, &begin, &end));
  577. ASSERT_EQ("0,1", FilesPerLevel(0));
  578. // Create 4 files in L0
  579. for (int i = 4; i < 8; i++) {
  580. ASSERT_OK(Put("Key" + ToString(i), DummyString(1024, 'B'), wo));
  581. ASSERT_OK(Flush());
  582. }
  583. ASSERT_EQ("4,1", FilesPerLevel(0));
  584. // Compaction will delete files from L0 in first db path and generate a new
  585. // file in L1 in second db path
  586. begin = "Key4";
  587. end = "Key7";
  588. ASSERT_OK(db_->CompactRange(compact_options, &begin, &end));
  589. ASSERT_EQ("0,2", FilesPerLevel(0));
  590. sfm->WaitForEmptyTrash();
  591. ASSERT_EQ(bg_delete_file, 8);
  592. // Compaction will delete both files and regenerate a file in L1 in second
  593. // db path. The deleted files should still be cleaned up via delete scheduler.
  594. compact_options.bottommost_level_compaction =
  595. BottommostLevelCompaction::kForceOptimized;
  596. ASSERT_OK(db_->CompactRange(compact_options, nullptr, nullptr));
  597. ASSERT_EQ("0,1", FilesPerLevel(0));
  598. sfm->WaitForEmptyTrash();
  599. ASSERT_EQ(bg_delete_file, 10);
  600. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  601. }
  602. TEST_F(DBSSTTest, DestroyDBWithRateLimitedDelete) {
  603. int bg_delete_file = 0;
  604. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  605. "DeleteScheduler::DeleteTrashFile:DeleteFile",
  606. [&](void* /*arg*/) { bg_delete_file++; });
  607. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  608. Status s;
  609. Options options = CurrentOptions();
  610. options.disable_auto_compactions = true;
  611. options.env = env_;
  612. options.sst_file_manager.reset(
  613. NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
  614. ASSERT_OK(s);
  615. DestroyAndReopen(options);
  616. // Create 4 files in L0
  617. for (int i = 0; i < 4; i++) {
  618. ASSERT_OK(Put("Key" + ToString(i), DummyString(1024, 'A')));
  619. ASSERT_OK(Flush());
  620. }
  621. // We created 4 sst files in L0
  622. ASSERT_EQ("4", FilesPerLevel(0));
  623. // Close DB and destroy it using DeleteScheduler
  624. Close();
  625. int num_sst_files = 0;
  626. int num_wal_files = 0;
  627. std::vector<std::string> db_files;
  628. env_->GetChildren(dbname_, &db_files);
  629. for (std::string f : db_files) {
  630. if (f.substr(f.find_last_of(".") + 1) == "sst") {
  631. num_sst_files++;
  632. } else if (f.substr(f.find_last_of(".") + 1) == "log") {
  633. num_wal_files++;
  634. }
  635. }
  636. ASSERT_GT(num_sst_files, 0);
  637. ASSERT_GT(num_wal_files, 0);
  638. auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
  639. sfm->SetDeleteRateBytesPerSecond(1024 * 1024);
  640. sfm->delete_scheduler()->SetMaxTrashDBRatio(1.1);
  641. ASSERT_OK(DestroyDB(dbname_, options));
  642. sfm->WaitForEmptyTrash();
  643. ASSERT_EQ(bg_delete_file, num_sst_files + num_wal_files);
  644. }
  645. TEST_F(DBSSTTest, DBWithMaxSpaceAllowed) {
  646. std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
  647. auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  648. Options options = CurrentOptions();
  649. options.sst_file_manager = sst_file_manager;
  650. options.disable_auto_compactions = true;
  651. DestroyAndReopen(options);
  652. Random rnd(301);
  653. // Generate a file containing 100 keys.
  654. for (int i = 0; i < 100; i++) {
  655. ASSERT_OK(Put(Key(i), RandomString(&rnd, 50)));
  656. }
  657. ASSERT_OK(Flush());
  658. uint64_t first_file_size = 0;
  659. auto files_in_db = GetAllSSTFiles(&first_file_size);
  660. ASSERT_EQ(sfm->GetTotalSize(), first_file_size);
  661. // Set the maximum allowed space usage to the current total size
  662. sfm->SetMaxAllowedSpaceUsage(first_file_size + 1);
  663. ASSERT_OK(Put("key1", "val1"));
  664. // This flush will cause bg_error_ and will fail
  665. ASSERT_NOK(Flush());
  666. }
  667. TEST_F(DBSSTTest, CancellingCompactionsWorks) {
  668. std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
  669. auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  670. Options options = CurrentOptions();
  671. options.sst_file_manager = sst_file_manager;
  672. options.level0_file_num_compaction_trigger = 2;
  673. options.statistics = CreateDBStatistics();
  674. DestroyAndReopen(options);
  675. int completed_compactions = 0;
  676. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  677. "DBImpl::BackgroundCompaction():CancelledCompaction", [&](void* /*arg*/) {
  678. sfm->SetMaxAllowedSpaceUsage(0);
  679. ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
  680. });
  681. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  682. "DBImpl::BackgroundCompaction:NonTrivial:AfterRun",
  683. [&](void* /*arg*/) { completed_compactions++; });
  684. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  685. Random rnd(301);
  686. // Generate a file containing 10 keys.
  687. for (int i = 0; i < 10; i++) {
  688. ASSERT_OK(Put(Key(i), RandomString(&rnd, 50)));
  689. }
  690. ASSERT_OK(Flush());
  691. uint64_t total_file_size = 0;
  692. auto files_in_db = GetAllSSTFiles(&total_file_size);
  693. // Set the maximum allowed space usage to the current total size
  694. sfm->SetMaxAllowedSpaceUsage(2 * total_file_size + 1);
  695. // Generate another file to trigger compaction.
  696. for (int i = 0; i < 10; i++) {
  697. ASSERT_OK(Put(Key(i), RandomString(&rnd, 50)));
  698. }
  699. ASSERT_OK(Flush());
  700. dbfull()->TEST_WaitForCompact(true);
  701. // Because we set a callback in CancelledCompaction, we actually
  702. // let the compaction run
  703. ASSERT_GT(completed_compactions, 0);
  704. ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
  705. // Make sure the stat is bumped
  706. ASSERT_GT(dbfull()->immutable_db_options().statistics.get()->getTickerCount(COMPACTION_CANCELLED), 0);
  707. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  708. }
  709. TEST_F(DBSSTTest, CancellingManualCompactionsWorks) {
  710. std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
  711. auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  712. Options options = CurrentOptions();
  713. options.sst_file_manager = sst_file_manager;
  714. options.statistics = CreateDBStatistics();
  715. FlushedFileCollector* collector = new FlushedFileCollector();
  716. options.listeners.emplace_back(collector);
  717. DestroyAndReopen(options);
  718. Random rnd(301);
  719. // Generate a file containing 10 keys.
  720. for (int i = 0; i < 10; i++) {
  721. ASSERT_OK(Put(Key(i), RandomString(&rnd, 50)));
  722. }
  723. ASSERT_OK(Flush());
  724. uint64_t total_file_size = 0;
  725. auto files_in_db = GetAllSSTFiles(&total_file_size);
  726. // Set the maximum allowed space usage to the current total size
  727. sfm->SetMaxAllowedSpaceUsage(2 * total_file_size + 1);
  728. // Generate another file to trigger compaction.
  729. for (int i = 0; i < 10; i++) {
  730. ASSERT_OK(Put(Key(i), RandomString(&rnd, 50)));
  731. }
  732. ASSERT_OK(Flush());
  733. // OK, now trigger a manual compaction
  734. dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr);
  735. // Wait for manual compaction to get scheduled and finish
  736. dbfull()->TEST_WaitForCompact(true);
  737. ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
  738. // Make sure the stat is bumped
  739. ASSERT_EQ(dbfull()->immutable_db_options().statistics.get()->getTickerCount(
  740. COMPACTION_CANCELLED),
  741. 1);
  742. // Now make sure CompactFiles also gets cancelled
  743. auto l0_files = collector->GetFlushedFiles();
  744. dbfull()->CompactFiles(ROCKSDB_NAMESPACE::CompactionOptions(), l0_files, 0);
  745. // Wait for manual compaction to get scheduled and finish
  746. dbfull()->TEST_WaitForCompact(true);
  747. ASSERT_EQ(dbfull()->immutable_db_options().statistics.get()->getTickerCount(
  748. COMPACTION_CANCELLED),
  749. 2);
  750. ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
  751. // Now let the flush through and make sure GetCompactionsReservedSize
  752. // returns to normal
  753. sfm->SetMaxAllowedSpaceUsage(0);
  754. int completed_compactions = 0;
  755. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  756. "CompactFilesImpl:End", [&](void* /*arg*/) { completed_compactions++; });
  757. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  758. dbfull()->CompactFiles(ROCKSDB_NAMESPACE::CompactionOptions(), l0_files, 0);
  759. dbfull()->TEST_WaitForCompact(true);
  760. ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
  761. ASSERT_GT(completed_compactions, 0);
  762. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  763. }
  764. TEST_F(DBSSTTest, DBWithMaxSpaceAllowedRandomized) {
  765. // This test will set a maximum allowed space for the DB, then it will
  766. // keep filling the DB until the limit is reached and bg_error_ is set.
  767. // When bg_error_ is set we will verify that the DB size is greater
  768. // than the limit.
  769. std::vector<int> max_space_limits_mbs = {1, 10};
  770. std::atomic<bool> bg_error_set(false);
  771. std::atomic<int> reached_max_space_on_flush(0);
  772. std::atomic<int> reached_max_space_on_compaction(0);
  773. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  774. "DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached",
  775. [&](void* arg) {
  776. Status* bg_error = static_cast<Status*>(arg);
  777. bg_error_set = true;
  778. reached_max_space_on_flush++;
  779. // clear error to ensure compaction callback is called
  780. *bg_error = Status::OK();
  781. });
  782. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  783. "DBImpl::BackgroundCompaction():CancelledCompaction", [&](void* arg) {
  784. bool* enough_room = static_cast<bool*>(arg);
  785. *enough_room = true;
  786. });
  787. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
  788. "CompactionJob::FinishCompactionOutputFile:MaxAllowedSpaceReached",
  789. [&](void* /*arg*/) {
  790. bg_error_set = true;
  791. reached_max_space_on_compaction++;
  792. });
  793. for (auto limit_mb : max_space_limits_mbs) {
  794. bg_error_set = false;
  795. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearTrace();
  796. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
  797. std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
  798. auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
  799. Options options = CurrentOptions();
  800. options.sst_file_manager = sst_file_manager;
  801. options.write_buffer_size = 1024 * 512; // 512 Kb
  802. DestroyAndReopen(options);
  803. Random rnd(301);
  804. sfm->SetMaxAllowedSpaceUsage(limit_mb * 1024 * 1024);
  805. // It is easy to detect if the test is stuck in a loop. No need for
  806. // complex termination logic.
  807. while (true) {
  808. auto s = Put(RandomString(&rnd, 10), RandomString(&rnd, 50));
  809. if (!s.ok()) {
  810. break;
  811. }
  812. }
  813. ASSERT_TRUE(bg_error_set);
  814. uint64_t total_sst_files_size = 0;
  815. GetAllSSTFiles(&total_sst_files_size);
  816. ASSERT_GE(total_sst_files_size, limit_mb * 1024 * 1024);
  817. ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
  818. }
  819. ASSERT_GT(reached_max_space_on_flush, 0);
  820. ASSERT_GT(reached_max_space_on_compaction, 0);
  821. }
  822. TEST_F(DBSSTTest, OpenDBWithInfiniteMaxOpenFiles) {
  823. // Open DB with infinite max open files
  824. // - First iteration use 1 thread to open files
  825. // - Second iteration use 5 threads to open files
  826. for (int iter = 0; iter < 2; iter++) {
  827. Options options;
  828. options.create_if_missing = true;
  829. options.write_buffer_size = 100000;
  830. options.disable_auto_compactions = true;
  831. options.max_open_files = -1;
  832. if (iter == 0) {
  833. options.max_file_opening_threads = 1;
  834. } else {
  835. options.max_file_opening_threads = 5;
  836. }
  837. options = CurrentOptions(options);
  838. DestroyAndReopen(options);
  839. // Create 12 Files in L0 (then move then to L2)
  840. for (int i = 0; i < 12; i++) {
  841. std::string k = "L2_" + Key(i);
  842. ASSERT_OK(Put(k, k + std::string(1000, 'a')));
  843. ASSERT_OK(Flush());
  844. }
  845. CompactRangeOptions compact_options;
  846. compact_options.change_level = true;
  847. compact_options.target_level = 2;
  848. db_->CompactRange(compact_options, nullptr, nullptr);
  849. // Create 12 Files in L0
  850. for (int i = 0; i < 12; i++) {
  851. std::string k = "L0_" + Key(i);
  852. ASSERT_OK(Put(k, k + std::string(1000, 'a')));
  853. ASSERT_OK(Flush());
  854. }
  855. Close();
  856. // Reopening the DB will load all existing files
  857. Reopen(options);
  858. ASSERT_EQ("12,0,12", FilesPerLevel(0));
  859. std::vector<std::vector<FileMetaData>> files;
  860. dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
  861. for (const auto& level : files) {
  862. for (const auto& file : level) {
  863. ASSERT_TRUE(file.table_reader_handle != nullptr);
  864. }
  865. }
  866. for (int i = 0; i < 12; i++) {
  867. ASSERT_EQ(Get("L0_" + Key(i)), "L0_" + Key(i) + std::string(1000, 'a'));
  868. ASSERT_EQ(Get("L2_" + Key(i)), "L2_" + Key(i) + std::string(1000, 'a'));
  869. }
  870. }
  871. }
  872. TEST_F(DBSSTTest, GetTotalSstFilesSize) {
  873. // We don't propagate oldest-key-time table property on compaction and
  874. // just write 0 as default value. This affect the exact table size, since
  875. // we encode table properties as varint64. Force time to be 0 to work around
  876. // it. Should remove the workaround after we propagate the property on
  877. // compaction.
  878. std::unique_ptr<MockTimeEnv> mock_env(new MockTimeEnv(Env::Default()));
  879. mock_env->set_current_time(0);
  880. Options options = CurrentOptions();
  881. options.disable_auto_compactions = true;
  882. options.compression = kNoCompression;
  883. options.env = mock_env.get();
  884. DestroyAndReopen(options);
  885. // Generate 5 files in L0
  886. for (int i = 0; i < 5; i++) {
  887. for (int j = 0; j < 10; j++) {
  888. std::string val = "val_file_" + ToString(i);
  889. ASSERT_OK(Put(Key(j), val));
  890. }
  891. Flush();
  892. }
  893. ASSERT_EQ("5", FilesPerLevel(0));
  894. std::vector<LiveFileMetaData> live_files_meta;
  895. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  896. ASSERT_EQ(live_files_meta.size(), 5);
  897. uint64_t single_file_size = live_files_meta[0].size;
  898. uint64_t live_sst_files_size = 0;
  899. uint64_t total_sst_files_size = 0;
  900. for (const auto& file_meta : live_files_meta) {
  901. live_sst_files_size += file_meta.size;
  902. }
  903. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  904. &total_sst_files_size));
  905. // Live SST files = 5
  906. // Total SST files = 5
  907. ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
  908. ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
  909. // hold current version
  910. std::unique_ptr<Iterator> iter1(dbfull()->NewIterator(ReadOptions()));
  911. // Compact 5 files into 1 file in L0
  912. ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  913. ASSERT_EQ("0,1", FilesPerLevel(0));
  914. live_files_meta.clear();
  915. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  916. ASSERT_EQ(live_files_meta.size(), 1);
  917. live_sst_files_size = 0;
  918. total_sst_files_size = 0;
  919. for (const auto& file_meta : live_files_meta) {
  920. live_sst_files_size += file_meta.size;
  921. }
  922. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  923. &total_sst_files_size));
  924. // Live SST files = 1 (compacted file)
  925. // Total SST files = 6 (5 original files + compacted file)
  926. ASSERT_EQ(live_sst_files_size, 1 * single_file_size);
  927. ASSERT_EQ(total_sst_files_size, 6 * single_file_size);
  928. // hold current version
  929. std::unique_ptr<Iterator> iter2(dbfull()->NewIterator(ReadOptions()));
  930. // Delete all keys and compact, this will delete all live files
  931. for (int i = 0; i < 10; i++) {
  932. ASSERT_OK(Delete(Key(i)));
  933. }
  934. Flush();
  935. ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  936. ASSERT_EQ("", FilesPerLevel(0));
  937. live_files_meta.clear();
  938. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  939. ASSERT_EQ(live_files_meta.size(), 0);
  940. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  941. &total_sst_files_size));
  942. // Live SST files = 0
  943. // Total SST files = 6 (5 original files + compacted file)
  944. ASSERT_EQ(total_sst_files_size, 6 * single_file_size);
  945. iter1.reset();
  946. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  947. &total_sst_files_size));
  948. // Live SST files = 0
  949. // Total SST files = 1 (compacted file)
  950. ASSERT_EQ(total_sst_files_size, 1 * single_file_size);
  951. iter2.reset();
  952. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  953. &total_sst_files_size));
  954. // Live SST files = 0
  955. // Total SST files = 0
  956. ASSERT_EQ(total_sst_files_size, 0);
  957. // Close db before mock_env destruct.
  958. Close();
  959. }
  960. TEST_F(DBSSTTest, GetTotalSstFilesSizeVersionsFilesShared) {
  961. Options options = CurrentOptions();
  962. options.disable_auto_compactions = true;
  963. options.compression = kNoCompression;
  964. DestroyAndReopen(options);
  965. // Generate 5 files in L0
  966. for (int i = 0; i < 5; i++) {
  967. ASSERT_OK(Put(Key(i), "val"));
  968. Flush();
  969. }
  970. ASSERT_EQ("5", FilesPerLevel(0));
  971. std::vector<LiveFileMetaData> live_files_meta;
  972. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  973. ASSERT_EQ(live_files_meta.size(), 5);
  974. uint64_t single_file_size = live_files_meta[0].size;
  975. uint64_t live_sst_files_size = 0;
  976. uint64_t total_sst_files_size = 0;
  977. for (const auto& file_meta : live_files_meta) {
  978. live_sst_files_size += file_meta.size;
  979. }
  980. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  981. &total_sst_files_size));
  982. // Live SST files = 5
  983. // Total SST files = 5
  984. ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
  985. ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
  986. // hold current version
  987. std::unique_ptr<Iterator> iter1(dbfull()->NewIterator(ReadOptions()));
  988. // Compaction will do trivial move from L0 to L1
  989. ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  990. ASSERT_EQ("0,5", FilesPerLevel(0));
  991. live_files_meta.clear();
  992. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  993. ASSERT_EQ(live_files_meta.size(), 5);
  994. live_sst_files_size = 0;
  995. total_sst_files_size = 0;
  996. for (const auto& file_meta : live_files_meta) {
  997. live_sst_files_size += file_meta.size;
  998. }
  999. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  1000. &total_sst_files_size));
  1001. // Live SST files = 5
  1002. // Total SST files = 5 (used in 2 version)
  1003. ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
  1004. ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
  1005. // hold current version
  1006. std::unique_ptr<Iterator> iter2(dbfull()->NewIterator(ReadOptions()));
  1007. // Delete all keys and compact, this will delete all live files
  1008. for (int i = 0; i < 5; i++) {
  1009. ASSERT_OK(Delete(Key(i)));
  1010. }
  1011. Flush();
  1012. ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
  1013. ASSERT_EQ("", FilesPerLevel(0));
  1014. live_files_meta.clear();
  1015. dbfull()->GetLiveFilesMetaData(&live_files_meta);
  1016. ASSERT_EQ(live_files_meta.size(), 0);
  1017. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  1018. &total_sst_files_size));
  1019. // Live SST files = 0
  1020. // Total SST files = 5 (used in 2 version)
  1021. ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
  1022. iter1.reset();
  1023. iter2.reset();
  1024. ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
  1025. &total_sst_files_size));
  1026. // Live SST files = 0
  1027. // Total SST files = 0
  1028. ASSERT_EQ(total_sst_files_size, 0);
  1029. }
  1030. #endif // ROCKSDB_LITE
  1031. } // namespace ROCKSDB_NAMESPACE
  1032. int main(int argc, char** argv) {
  1033. ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
  1034. ::testing::InitGoogleTest(&argc, argv);
  1035. return RUN_ALL_TESTS();
  1036. }