rocksdb/db/db_sst_test.cc

1892 lines
65 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/db_test_util.h"
#include "env/mock_env.h"
#include "file/sst_file_manager_impl.h"
#include "port/port.h"
#include "port/stack_trace.h"
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
#include "rocksdb/cache.h"
#include "rocksdb/sst_file_manager.h"
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
#include "rocksdb/table.h"
#include "util/random.h"
namespace ROCKSDB_NAMESPACE {
class DBSSTTest : public DBTestBase {
public:
DBSSTTest() : DBTestBase("db_sst_test", /*env_do_fsync=*/true) {}
};
// A class which remembers the name of each flushed file.
class FlushedFileCollector : public EventListener {
public:
FlushedFileCollector() = default;
~FlushedFileCollector() override = default;
void OnFlushCompleted(DB* /*db*/, const FlushJobInfo& info) override {
std::lock_guard<std::mutex> lock(mutex_);
flushed_files_.push_back(info.file_path);
}
std::vector<std::string> GetFlushedFiles() {
std::lock_guard<std::mutex> lock(mutex_);
std::vector<std::string> result;
for (const auto& fname : flushed_files_) {
result.push_back(fname);
}
return result;
}
void ClearFlushedFiles() {
std::lock_guard<std::mutex> lock(mutex_);
flushed_files_.clear();
}
private:
std::vector<std::string> flushed_files_;
std::mutex mutex_;
};
TEST_F(DBSSTTest, DontDeletePendingOutputs) {
Options options;
options.env = env_;
options.create_if_missing = true;
DestroyAndReopen(options);
// Every time we write to a table file, call FOF/POF with full DB scan. This
// will make sure our pending_outputs_ protection work correctly
std::function<void()> purge_obsolete_files_function = [&]() {
JobContext job_context(0);
dbfull()->TEST_LockMutex();
dbfull()->FindObsoleteFiles(&job_context, true /*force*/);
dbfull()->TEST_UnlockMutex();
dbfull()->PurgeObsoleteFiles(job_context);
job_context.Clean();
};
env_->table_write_callback_ = &purge_obsolete_files_function;
for (int i = 0; i < 2; ++i) {
ASSERT_OK(Put("a", "begin"));
ASSERT_OK(Put("z", "end"));
ASSERT_OK(Flush());
}
// If pending output guard does not work correctly, PurgeObsoleteFiles() will
// delete the file that Compaction is trying to create, causing this: error
// db/db_test.cc:975: IO error:
// /tmp/rocksdbtest-1552237650/db_test/000009.sst: No such file or directory
Compact("a", "b");
}
// 1 Create some SST files by inserting K-V pairs into DB
// 2 Close DB and change suffix from ".sst" to ".ldb" for every other SST file
// 3 Open DB and check if all key can be read
TEST_F(DBSSTTest, SSTsWithLdbSuffixHandling) {
Options options = CurrentOptions();
options.write_buffer_size = 110 << 10; // 110KB
options.num_levels = 4;
DestroyAndReopen(options);
Random rnd(301);
int key_id = 0;
for (int i = 0; i < 10; ++i) {
GenerateNewFile(&rnd, &key_id, false);
}
ASSERT_OK(Flush());
Close();
int const num_files = GetSstFileCount(dbname_);
ASSERT_GT(num_files, 0);
Reopen(options);
std::vector<std::string> values;
values.reserve(key_id);
for (int k = 0; k < key_id; ++k) {
values.push_back(Get(Key(k)));
}
Close();
std::vector<std::string> filenames;
GetSstFiles(env_, dbname_, &filenames);
int num_ldb_files = 0;
for (size_t i = 0; i < filenames.size(); ++i) {
if (i & 1) {
continue;
}
std::string const rdb_name = dbname_ + "/" + filenames[i];
std::string const ldb_name = Rocks2LevelTableFileName(rdb_name);
ASSERT_TRUE(env_->RenameFile(rdb_name, ldb_name).ok());
++num_ldb_files;
}
ASSERT_GT(num_ldb_files, 0);
ASSERT_EQ(num_files, GetSstFileCount(dbname_));
Reopen(options);
for (int k = 0; k < key_id; ++k) {
ASSERT_EQ(values[k], Get(Key(k)));
}
Destroy(options);
}
// Check that we don't crash when opening DB with
// DBOptions::skip_checking_sst_file_sizes_on_db_open = true.
TEST_F(DBSSTTest, SkipCheckingSSTFileSizesOnDBOpen) {
ASSERT_OK(Put("pika", "choo"));
ASSERT_OK(Flush());
// Just open the DB with the option set to true and check that we don't crash.
Options options;
Fix many tests to run with MEM_ENV and ENCRYPTED_ENV; Introduce a MemoryFileSystem class (#7566) Summary: This PR does a few things: 1. The MockFileSystem class was split out from the MockEnv. This change would theoretically allow a MockFileSystem to be used by other Environments as well (if we created a means of constructing one). The MockFileSystem implements a FileSystem in its entirety and does not rely on any Wrapper implementation. 2. Make the RocksDB test suite work when MOCK_ENV=1 and ENCRYPTED_ENV=1 are set. To accomplish this, a few things were needed: - The tests that tried to use the "wrong" environment (Env::Default() instead of env_) were updated - The MockFileSystem was changed to support the features it was missing or mishandled (such as recursively deleting files in a directory or supporting renaming of a directory). 3. Updated the test framework to have a ROCKSDB_GTEST_SKIP macro. This can be used to flag tests that are skipped. Currently, this defaults to doing nothing (marks the test as SUCCESS) but will mark the tests as SKIPPED when RocksDB is upgraded to a version of gtest that supports this (gtest-1.10). I have run a full "make check" with MEM_ENV, ENCRYPTED_ENV, both, and neither under both MacOS and RedHat. A few tests were disabled/skipped for the MEM/ENCRYPTED cases. The error_handler_fs_test fails/hangs for MEM_ENV (presumably a timing problem) and I will introduce another PR/issue to track that problem. (I will also push a change to disable those tests soon). There is one more test in DBTest2 that also fails which I need to investigate or skip before this PR is merged. Theoretically, this PR should also allow the test suite to run against an Env loaded from the registry, though I do not have one to try it with currently. Finally, once this is accepted, it would be nice if there was a CircleCI job to run these tests on a checkin so this effort does not become stale. I do not know how to do that, so if someone could write that job, it would be appreciated :) Pull Request resolved: https://github.com/facebook/rocksdb/pull/7566 Reviewed By: zhichao-cao Differential Revision: D24408980 Pulled By: jay-zhuang fbshipit-source-id: 911b1554a4d0da06fd51feca0c090a4abdcb4a5f
2020-10-27 17:31:34 +00:00
options.env = env_;
options.skip_checking_sst_file_sizes_on_db_open = true;
Reopen(options);
ASSERT_EQ("choo", Get("pika"));
}
TEST_F(DBSSTTest, DontDeleteMovedFile) {
// This test triggers move compaction and verifies that the file is not
// deleted when it's part of move compaction
Options options = CurrentOptions();
options.env = env_;
options.create_if_missing = true;
options.max_bytes_for_level_base = 1024 * 1024; // 1 MB
options.level0_file_num_compaction_trigger =
2; // trigger compaction when we have 2 files
DestroyAndReopen(options);
Random rnd(301);
// Create two 1MB sst files
for (int i = 0; i < 2; ++i) {
// Create 1MB sst file
for (int j = 0; j < 100; ++j) {
ASSERT_OK(Put(Key(i * 50 + j), rnd.RandomString(10 * 1024)));
}
ASSERT_OK(Flush());
}
// this should execute both L0->L1 and L1->(move)->L2 compactions
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ("0,0,1", FilesPerLevel(0));
// If the moved file is actually deleted (the move-safeguard in
// ~Version::Version() is not there), we get this failure:
// Corruption: Can't access /000009.sst
Reopen(options);
}
// This reproduces a bug where we don't delete a file because when it was
// supposed to be deleted, it was blocked by pending_outputs
// Consider:
// 1. current file_number is 13
// 2. compaction (1) starts, blocks deletion of all files starting with 13
// (pending outputs)
// 3. file 13 is created by compaction (2)
// 4. file 13 is consumed by compaction (3) and file 15 was created. Since file
// 13 has no references, it is put into VersionSet::obsolete_files_
// 5. FindObsoleteFiles() gets file 13 from VersionSet::obsolete_files_. File 13
// is deleted from obsolete_files_ set.
// 6. PurgeObsoleteFiles() tries to delete file 13, but this file is blocked by
// pending outputs since compaction (1) is still running. It is not deleted and
// it is not present in obsolete_files_ anymore. Therefore, we never delete it.
TEST_F(DBSSTTest, DeleteObsoleteFilesPendingOutputs) {
Options options = CurrentOptions();
options.env = env_;
options.write_buffer_size = 2 * 1024 * 1024; // 2 MB
options.max_bytes_for_level_base = 1024 * 1024; // 1 MB
options.level0_file_num_compaction_trigger =
2; // trigger compaction when we have 2 files
options.max_background_flushes = 2;
options.max_background_compactions = 2;
OnFileDeletionListener* listener = new OnFileDeletionListener();
options.listeners.emplace_back(listener);
Reopen(options);
Random rnd(301);
// Create two 1MB sst files
for (int i = 0; i < 2; ++i) {
// Create 1MB sst file
for (int j = 0; j < 100; ++j) {
ASSERT_OK(Put(Key(i * 50 + j), rnd.RandomString(10 * 1024)));
}
ASSERT_OK(Flush());
}
// this should execute both L0->L1 and L1->(move)->L2 compactions
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ("0,0,1", FilesPerLevel(0));
test::SleepingBackgroundTask blocking_thread;
port::Mutex mutex_;
bool already_blocked(false);
// block the flush
std::function<void()> block_first_time = [&]() {
bool blocking = false;
{
MutexLock l(&mutex_);
if (!already_blocked) {
blocking = true;
already_blocked = true;
}
}
if (blocking) {
blocking_thread.DoSleep();
}
};
env_->table_write_callback_ = &block_first_time;
// Insert 2.5MB data, which should trigger a flush because we exceed
// write_buffer_size. The flush will be blocked with block_first_time
// pending_file is protecting all the files created after
for (int j = 0; j < 256; ++j) {
ASSERT_OK(Put(Key(j), rnd.RandomString(10 * 1024)));
}
blocking_thread.WaitUntilSleeping();
ASSERT_OK(dbfull()->TEST_CompactRange(2, nullptr, nullptr));
ASSERT_EQ("0,0,0,1", FilesPerLevel(0));
std::vector<LiveFileMetaData> metadata;
db_->GetLiveFilesMetaData(&metadata);
ASSERT_EQ(metadata.size(), 1U);
auto file_on_L2 = metadata[0].name;
listener->SetExpectedFileName(dbname_ + file_on_L2);
ASSERT_OK(dbfull()->TEST_CompactRange(3, nullptr, nullptr, nullptr,
true /* disallow trivial move */));
ASSERT_EQ("0,0,0,0,1", FilesPerLevel(0));
// finish the flush!
blocking_thread.WakeUp();
blocking_thread.WaitUntilDone();
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
// File just flushed is too big for L0 and L1 so gets moved to L2.
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ("0,0,1,0,1", FilesPerLevel(0));
metadata.clear();
db_->GetLiveFilesMetaData(&metadata);
ASSERT_EQ(metadata.size(), 2U);
// This file should have been deleted during last compaction
ASSERT_EQ(Status::NotFound(), env_->FileExists(dbname_ + file_on_L2));
listener->VerifyMatchedCount(1);
}
// Test that producing an empty .sst file does not write it out to
// disk, and that the DeleteFile() env method is not called for
// removing the non-existing file later.
TEST_F(DBSSTTest, DeleteFileNotCalledForNotCreatedSSTFile) {
Options options = CurrentOptions();
options.env = env_;
OnFileDeletionListener* listener = new OnFileDeletionListener();
options.listeners.emplace_back(listener);
Reopen(options);
// Flush the empty database.
ASSERT_OK(Flush());
ASSERT_EQ("", FilesPerLevel(0));
// We expect no .sst files.
std::vector<LiveFileMetaData> metadata;
db_->GetLiveFilesMetaData(&metadata);
ASSERT_EQ(metadata.size(), 0U);
// We expect no file deletions.
listener->VerifyMatchedCount(0);
}
// Test that producing a non-empty .sst file does write it out to
// disk, and that the DeleteFile() env method is not called for removing
// the file later.
TEST_F(DBSSTTest, DeleteFileNotCalledForCreatedSSTFile) {
Options options = CurrentOptions();
options.env = env_;
OnFileDeletionListener* listener = new OnFileDeletionListener();
options.listeners.emplace_back(listener);
Reopen(options);
ASSERT_OK(Put("pika", "choo"));
// Flush the non-empty database.
ASSERT_OK(Flush());
ASSERT_EQ("1", FilesPerLevel(0));
// We expect 1 .sst files.
std::vector<LiveFileMetaData> metadata;
db_->GetLiveFilesMetaData(&metadata);
ASSERT_EQ(metadata.size(), 1U);
// We expect no file deletions.
listener->VerifyMatchedCount(0);
}
TEST_F(DBSSTTest, DBWithSstFileManager) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
int files_added = 0;
int files_deleted = 0;
int files_moved = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnAddFile", [&](void* /*arg*/) { files_added++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnDeleteFile",
[&](void* /*arg*/) { files_deleted++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnMoveFile", [&](void* /*arg*/) { files_moved++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
DestroyAndReopen(options);
Random rnd(301);
for (int i = 0; i < 25; i++) {
GenerateNewRandomFile(&rnd);
ASSERT_OK(Flush());
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
ASSERT_OK(dbfull()->TEST_WaitForCompact());
// Verify that we are tracking all sst files in dbname_
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
}
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_OK(dbfull()->TEST_WaitForCompact());
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
// Verify that we are tracking all sst files in dbname_
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
// Verify the total files size
uint64_t total_files_size = 0;
for (auto& file_to_size : files_in_db) {
total_files_size += file_to_size.second;
}
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
// We flushed at least 25 files
ASSERT_GE(files_added, 25);
// Compaction must have deleted some files
ASSERT_GT(files_deleted, 0);
// No files were moved
ASSERT_EQ(files_moved, 0);
Close();
Reopen(options);
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
// Verify that we track all the files again after the DB is closed and opened
Close();
sst_file_manager.reset(NewSstFileManager(env_));
options.sst_file_manager = sst_file_manager;
sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Reopen(options);
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, DBWithSstFileManagerForBlobFiles) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
int files_added = 0;
int files_deleted = 0;
int files_moved = 0;
int files_scheduled_to_delete = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnAddFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
files_added++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnDeleteFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
files_deleted++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::ScheduleFileDeletion", [&](void* arg) {
assert(arg);
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
++files_scheduled_to_delete;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnMoveFile", [&](void* /*arg*/) { files_moved++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.enable_blob_files = true;
options.blob_file_size = 32; // create one blob per file
DestroyAndReopen(options);
Random rnd(301);
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put("Key_" + std::to_string(i), "Value_" + std::to_string(i)));
ASSERT_OK(Flush());
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
ASSERT_OK(dbfull()->TEST_WaitForCompact());
// Verify that we are tracking all sst and blob files in dbname_
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
ASSERT_OK(GetAllDataFiles(kBlobFile, &files_in_db));
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
}
std::vector<uint64_t> blob_files = GetBlobFileNumbers();
ASSERT_EQ(files_added, blob_files.size());
// No blob file is obsoleted.
ASSERT_EQ(files_deleted, 0);
ASSERT_EQ(files_scheduled_to_delete, 0);
// No files were moved.
ASSERT_EQ(files_moved, 0);
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_OK(dbfull()->TEST_WaitForCompact());
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
ASSERT_OK(GetAllDataFiles(kBlobFile, &files_in_db));
// Verify that we are tracking all sst and blob files in dbname_
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
// Verify the total files size
uint64_t total_files_size = 0;
for (auto& file_to_size : files_in_db) {
total_files_size += file_to_size.second;
}
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
Close();
Reopen(options);
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
// Verify that we track all the files again after the DB is closed and opened.
Close();
sst_file_manager.reset(NewSstFileManager(env_));
options.sst_file_manager = sst_file_manager;
sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Reopen(options);
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
// Destroy DB and it will remove all the blob files from sst file manager and
// blob files deletion will go through ScheduleFileDeletion.
ASSERT_EQ(files_deleted, 0);
ASSERT_EQ(files_scheduled_to_delete, 0);
Close();
ASSERT_OK(DestroyDB(dbname_, options));
ASSERT_EQ(files_deleted, blob_files.size());
ASSERT_EQ(files_scheduled_to_delete, blob_files.size());
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
}
TEST_F(DBSSTTest, DBWithSstFileManagerForBlobFilesWithGC) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.enable_blob_files = true;
options.blob_file_size = 32; // create one blob per file
options.disable_auto_compactions = true;
options.enable_blob_garbage_collection = true;
options.blob_garbage_collection_age_cutoff = 0.5;
int files_added = 0;
int files_deleted = 0;
int files_moved = 0;
int files_scheduled_to_delete = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnAddFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
files_added++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnDeleteFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
files_deleted++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::ScheduleFileDeletion", [&](void* arg) {
assert(arg);
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (file_path->find(".blob") != std::string::npos) {
++files_scheduled_to_delete;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnMoveFile", [&](void* /*arg*/) { files_moved++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
DestroyAndReopen(options);
Random rnd(301);
constexpr char first_key[] = "first_key";
constexpr char first_value[] = "first_value";
constexpr char second_key[] = "second_key";
constexpr char second_value[] = "second_value";
ASSERT_OK(Put(first_key, first_value));
ASSERT_OK(Put(second_key, second_value));
ASSERT_OK(Flush());
constexpr char third_key[] = "third_key";
constexpr char third_value[] = "third_value";
constexpr char fourth_key[] = "fourth_key";
constexpr char fourth_value[] = "fourth_value";
constexpr char fifth_key[] = "fifth_key";
constexpr char fifth_value[] = "fifth_value";
ASSERT_OK(Put(third_key, third_value));
ASSERT_OK(Put(fourth_key, fourth_value));
ASSERT_OK(Put(fifth_key, fifth_value));
ASSERT_OK(Flush());
const std::vector<uint64_t> original_blob_files = GetBlobFileNumbers();
ASSERT_EQ(original_blob_files.size(), 5);
ASSERT_EQ(files_added, 5);
ASSERT_EQ(files_deleted, 0);
ASSERT_EQ(files_scheduled_to_delete, 0);
ASSERT_EQ(files_moved, 0);
{
// Verify that we are tracking all sst and blob files in dbname_
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
ASSERT_OK(GetAllDataFiles(kBlobFile, &files_in_db));
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
}
const size_t cutoff_index = static_cast<size_t>(
options.blob_garbage_collection_age_cutoff * original_blob_files.size());
size_t expected_number_of_files = original_blob_files.size();
// Note: turning off enable_blob_files before the compaction results in
// garbage collected values getting inlined.
ASSERT_OK(db_->SetOptions({{"enable_blob_files", "false"}}));
expected_number_of_files -= cutoff_index;
files_added = 0;
constexpr Slice* begin = nullptr;
constexpr Slice* end = nullptr;
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), begin, end));
ASSERT_OK(dbfull()->TEST_WaitForCompact());
sfm->WaitForEmptyTrash();
ASSERT_EQ(Get(first_key), first_value);
ASSERT_EQ(Get(second_key), second_value);
ASSERT_EQ(Get(third_key), third_value);
ASSERT_EQ(Get(fourth_key), fourth_value);
ASSERT_EQ(Get(fifth_key), fifth_value);
const std::vector<uint64_t> new_blob_files = GetBlobFileNumbers();
ASSERT_EQ(new_blob_files.size(), expected_number_of_files);
// No new file is added.
ASSERT_EQ(files_added, 0);
ASSERT_EQ(files_deleted, cutoff_index);
ASSERT_EQ(files_scheduled_to_delete, cutoff_index);
ASSERT_EQ(files_moved, 0);
// Original blob files below the cutoff should be gone, original blob files at
// or above the cutoff should be still there
for (size_t i = cutoff_index; i < original_blob_files.size(); ++i) {
ASSERT_EQ(new_blob_files[i - cutoff_index], original_blob_files[i]);
}
{
// Verify that we are tracking all sst and blob files in dbname_
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db));
ASSERT_OK(GetAllDataFiles(kBlobFile, &files_in_db));
ASSERT_EQ(sfm->GetTrackedFiles(), files_in_db);
}
Close();
ASSERT_OK(DestroyDB(dbname_, options));
sfm->WaitForEmptyTrash();
ASSERT_EQ(files_deleted, 5);
ASSERT_EQ(files_scheduled_to_delete, 5);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
}
class DBSSTTestRateLimit : public DBSSTTest,
public ::testing::WithParamInterface<bool> {
public:
DBSSTTestRateLimit() : DBSSTTest() {}
~DBSSTTestRateLimit() override = default;
};
TEST_P(DBSSTTestRateLimit, RateLimitedDelete) {
Destroy(last_options_);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
{"DBSSTTest::RateLimitedDelete:1",
"DeleteScheduler::BackgroundEmptyTrash"},
});
std::vector<uint64_t> penalties;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::BackgroundEmptyTrash:Wait",
[&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"InstrumentedCondVar::TimedWaitInternal", [&](void* arg) {
// Turn timed wait into a simulated sleep
uint64_t* abs_time_us = static_cast<uint64_t*>(arg);
uint64_t cur_time = env_->NowMicros();
if (*abs_time_us > cur_time) {
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
env_->MockSleepForMicroseconds(*abs_time_us - cur_time);
}
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
// Plus an additional short, random amount
env_->MockSleepForMicroseconds(Random::GetTLSInstance()->Uniform(10));
// Set wait until time to before (actual) current time to force not
// to sleep
*abs_time_us = Env::Default()->NowMicros();
});
// Disable PeriodicTaskScheduler as it also has TimedWait, which could update
// the simulated sleep time
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::StartPeriodicTaskScheduler:DisableScheduler", [&](void* arg) {
bool* disable_scheduler = static_cast<bool*>(arg);
*disable_scheduler = true;
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
bool different_wal_dir = GetParam();
Options options = CurrentOptions();
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
SetTimeElapseOnlySleepOnReopen(&options);
options.disable_auto_compactions = true;
options.env = env_;
options.statistics = CreateDBStatistics();
if (different_wal_dir) {
options.wal_dir = alternative_wal_dir_;
}
int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
Status s;
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
ASSERT_OK(s);
options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
sfm->delete_scheduler()->SetMaxTrashDBRatio(1.1);
WriteOptions wo;
if (!different_wal_dir) {
wo.disableWAL = true;
}
Reopen(options);
// Create 4 files in L0
for (char v = 'a'; v <= 'd'; v++) {
ASSERT_OK(Put("Key2", DummyString(1024, v), wo));
ASSERT_OK(Put("Key3", DummyString(1024, v), wo));
ASSERT_OK(Put("Key4", DummyString(1024, v), wo));
ASSERT_OK(Put("Key1", DummyString(1024, v), wo));
ASSERT_OK(Put("Key4", DummyString(1024, v), wo));
ASSERT_OK(Flush());
}
// We created 4 sst files in L0
ASSERT_EQ("4", FilesPerLevel(0));
std::vector<LiveFileMetaData> metadata;
db_->GetLiveFilesMetaData(&metadata);
// Compaction will move the 4 files in L0 to trash and create 1 L1 file
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr));
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ("0,1", FilesPerLevel(0));
uint64_t delete_start_time = env_->NowMicros();
// Hold BackgroundEmptyTrash
TEST_SYNC_POINT("DBSSTTest::RateLimitedDelete:1");
sfm->WaitForEmptyTrash();
uint64_t time_spent_deleting = env_->NowMicros() - delete_start_time;
uint64_t total_files_size = 0;
uint64_t expected_penlty = 0;
ASSERT_EQ(penalties.size(), metadata.size());
for (size_t i = 0; i < metadata.size(); i++) {
total_files_size += metadata[i].size;
expected_penlty = ((total_files_size * 1000000) / rate_bytes_per_sec);
ASSERT_EQ(expected_penlty, penalties[i]);
}
ASSERT_GT(time_spent_deleting, expected_penlty * 0.9);
ASSERT_LT(time_spent_deleting, expected_penlty * 1.1);
ASSERT_EQ(4, options.statistics->getAndResetTickerCount(FILES_MARKED_TRASH));
ASSERT_EQ(
0, options.statistics->getAndResetTickerCount(FILES_DELETED_IMMEDIATELY));
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
INSTANTIATE_TEST_CASE_P(RateLimitedDelete, DBSSTTestRateLimit,
::testing::Bool());
TEST_F(DBSSTTest, RateLimitedWALDelete) {
Destroy(last_options_);
std::vector<uint64_t> penalties;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::BackgroundEmptyTrash:Wait",
[&](void* arg) { penalties.push_back(*(static_cast<uint64_t*>(arg))); });
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.compression = kNoCompression;
options.env = env_;
int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
Status s;
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
ASSERT_OK(s);
options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
sfm->delete_scheduler()->SetMaxTrashDBRatio(3.1);
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
SetTimeElapseOnlySleepOnReopen(&options);
ASSERT_OK(TryReopen(options));
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
// Create 4 files in L0
for (char v = 'a'; v <= 'd'; v++) {
ASSERT_OK(Put("Key2", DummyString(1024, v)));
ASSERT_OK(Put("Key3", DummyString(1024, v)));
ASSERT_OK(Put("Key4", DummyString(1024, v)));
ASSERT_OK(Put("Key1", DummyString(1024, v)));
ASSERT_OK(Put("Key4", DummyString(1024, v)));
ASSERT_OK(Flush());
}
// We created 4 sst files in L0
ASSERT_EQ("4", FilesPerLevel(0));
Always allow L0->L1 trivial move during manual compaction (#11375) Summary: during manual compaction (CompactRange()), L0->L1 trivial move is disabled when only L0 overlaps with compacting key range (introduced in https://github.com/facebook/rocksdb/issues/7368 to enforce kForce* contract). This can cause large memory usage due to compaction readahead when number of L0 files is large. This PR allows L0->L1 trivial move in this case, and will do a L1 -> L1 intra-level compaction when needed (`bottommost_level_compaction` is kForce*). In brief, consider a DB with only L0 file, and user calls CompactRange(kForce, nullptr, nullptr), - before this PR, RocksDB does a L0 -> L1 compaction (disallow trivial move), - after this PR, RocksDB does a L0 -> L1 compaction (allow trivial move), and a L1 -> L1 compaction. Users can use kForceOptimized to avoid this extra L1->L1 compaction overhead when L0s are overlapping and cannot be trivial moved. This PR also fixed a bug (see previous discussion in https://github.com/facebook/rocksdb/issues/11041) where `final_output_level` of a manual compaction can be miscalculated when `level_compaction_dynamic_level_bytes=true`. This bug could cause incorrect level being moved when CompactRangeOptions::change_level is specified. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11375 Test Plan: - Added new unit tests to test that L0 -> L1 compaction allows trivial move and L1 -> L1 compaction is done when needed. Reviewed By: ajkr Differential Revision: D44943518 Pulled By: cbi42 fbshipit-source-id: e9fb770d17b163c18a623e1d1bd6b81159192708
2023-04-20 18:10:48 +00:00
// Compaction will move the 4 files in L0 to trash and create 1 L1 file.
// Use kForceOptimized to not rewrite the new L1 file.
CompactRangeOptions cro;
Always allow L0->L1 trivial move during manual compaction (#11375) Summary: during manual compaction (CompactRange()), L0->L1 trivial move is disabled when only L0 overlaps with compacting key range (introduced in https://github.com/facebook/rocksdb/issues/7368 to enforce kForce* contract). This can cause large memory usage due to compaction readahead when number of L0 files is large. This PR allows L0->L1 trivial move in this case, and will do a L1 -> L1 intra-level compaction when needed (`bottommost_level_compaction` is kForce*). In brief, consider a DB with only L0 file, and user calls CompactRange(kForce, nullptr, nullptr), - before this PR, RocksDB does a L0 -> L1 compaction (disallow trivial move), - after this PR, RocksDB does a L0 -> L1 compaction (allow trivial move), and a L1 -> L1 compaction. Users can use kForceOptimized to avoid this extra L1->L1 compaction overhead when L0s are overlapping and cannot be trivial moved. This PR also fixed a bug (see previous discussion in https://github.com/facebook/rocksdb/issues/11041) where `final_output_level` of a manual compaction can be miscalculated when `level_compaction_dynamic_level_bytes=true`. This bug could cause incorrect level being moved when CompactRangeOptions::change_level is specified. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11375 Test Plan: - Added new unit tests to test that L0 -> L1 compaction allows trivial move and L1 -> L1 compaction is done when needed. Reviewed By: ajkr Differential Revision: D44943518 Pulled By: cbi42 fbshipit-source-id: e9fb770d17b163c18a623e1d1bd6b81159192708
2023-04-20 18:10:48 +00:00
cro.bottommost_level_compaction = BottommostLevelCompaction::kForceOptimized;
ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr));
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ("0,1", FilesPerLevel(0));
sfm->WaitForEmptyTrash();
ASSERT_EQ(penalties.size(), 8);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
class DBWALTestWithParam
: public DBTestBase,
public testing::WithParamInterface<std::tuple<std::string, bool>> {
public:
explicit DBWALTestWithParam()
: DBTestBase("db_wal_test_with_params", /*env_do_fsync=*/true) {
wal_dir_ = std::get<0>(GetParam());
wal_dir_same_as_dbname_ = std::get<1>(GetParam());
}
std::string wal_dir_;
bool wal_dir_same_as_dbname_;
};
TEST_P(DBWALTestWithParam, WALTrashCleanupOnOpen) {
class MyEnv : public EnvWrapper {
public:
MyEnv(Env* t) : EnvWrapper(t), fake_log_delete(false) {}
const char* Name() const override { return "MyEnv"; }
Status DeleteFile(const std::string& fname) override {
if (fname.find(".log.trash") != std::string::npos && fake_log_delete) {
return Status::OK();
}
return target()->DeleteFile(fname);
}
void set_fake_log_delete(bool fake) { fake_log_delete = fake; }
private:
bool fake_log_delete;
};
Fix many tests to run with MEM_ENV and ENCRYPTED_ENV; Introduce a MemoryFileSystem class (#7566) Summary: This PR does a few things: 1. The MockFileSystem class was split out from the MockEnv. This change would theoretically allow a MockFileSystem to be used by other Environments as well (if we created a means of constructing one). The MockFileSystem implements a FileSystem in its entirety and does not rely on any Wrapper implementation. 2. Make the RocksDB test suite work when MOCK_ENV=1 and ENCRYPTED_ENV=1 are set. To accomplish this, a few things were needed: - The tests that tried to use the "wrong" environment (Env::Default() instead of env_) were updated - The MockFileSystem was changed to support the features it was missing or mishandled (such as recursively deleting files in a directory or supporting renaming of a directory). 3. Updated the test framework to have a ROCKSDB_GTEST_SKIP macro. This can be used to flag tests that are skipped. Currently, this defaults to doing nothing (marks the test as SUCCESS) but will mark the tests as SKIPPED when RocksDB is upgraded to a version of gtest that supports this (gtest-1.10). I have run a full "make check" with MEM_ENV, ENCRYPTED_ENV, both, and neither under both MacOS and RedHat. A few tests were disabled/skipped for the MEM/ENCRYPTED cases. The error_handler_fs_test fails/hangs for MEM_ENV (presumably a timing problem) and I will introduce another PR/issue to track that problem. (I will also push a change to disable those tests soon). There is one more test in DBTest2 that also fails which I need to investigate or skip before this PR is merged. Theoretically, this PR should also allow the test suite to run against an Env loaded from the registry, though I do not have one to try it with currently. Finally, once this is accepted, it would be nice if there was a CircleCI job to run these tests on a checkin so this effort does not become stale. I do not know how to do that, so if someone could write that job, it would be appreciated :) Pull Request resolved: https://github.com/facebook/rocksdb/pull/7566 Reviewed By: zhichao-cao Differential Revision: D24408980 Pulled By: jay-zhuang fbshipit-source-id: 911b1554a4d0da06fd51feca0c090a4abdcb4a5f
2020-10-27 17:31:34 +00:00
std::unique_ptr<MyEnv> env(new MyEnv(env_));
Destroy(last_options_);
env->set_fake_log_delete(true);
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.compression = kNoCompression;
options.env = env.get();
options.wal_dir = dbname_ + wal_dir_;
int64_t rate_bytes_per_sec = 1024 * 10; // 10 Kbs / Sec
Status s;
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
ASSERT_OK(s);
options.sst_file_manager->SetDeleteRateBytesPerSecond(rate_bytes_per_sec);
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
sfm->delete_scheduler()->SetMaxTrashDBRatio(3.1);
Reopen(options);
// Create 4 files in L0
for (char v = 'a'; v <= 'd'; v++) {
if (v == 'c') {
// Maximize the change that the last log file will be preserved in trash
// before restarting the DB.
// We have to set this on the 2nd to last file for it to delay deletion
// on the last file. (Quirk of DeleteScheduler::BackgroundEmptyTrash())
options.sst_file_manager->SetDeleteRateBytesPerSecond(1);
}
ASSERT_OK(Put("Key2", DummyString(1024, v)));
ASSERT_OK(Put("Key3", DummyString(1024, v)));
ASSERT_OK(Put("Key4", DummyString(1024, v)));
ASSERT_OK(Put("Key1", DummyString(1024, v)));
ASSERT_OK(Put("Key4", DummyString(1024, v)));
ASSERT_OK(Flush());
}
// We created 4 sst files in L0
ASSERT_EQ("4", FilesPerLevel(0));
Close();
options.sst_file_manager.reset();
std::vector<std::string> filenames;
int trash_log_count = 0;
if (!wal_dir_same_as_dbname_) {
// Forcibly create some trash log files
std::unique_ptr<WritableFile> result;
ASSERT_OK(env->NewWritableFile(options.wal_dir + "/1000.log.trash", &result,
EnvOptions()));
result.reset();
}
ASSERT_OK(env->GetChildren(options.wal_dir, &filenames));
for (const std::string& fname : filenames) {
if (fname.find(".log.trash") != std::string::npos) {
trash_log_count++;
}
}
ASSERT_GE(trash_log_count, 1);
env->set_fake_log_delete(false);
Reopen(options);
filenames.clear();
trash_log_count = 0;
ASSERT_OK(env->GetChildren(options.wal_dir, &filenames));
for (const std::string& fname : filenames) {
if (fname.find(".log.trash") != std::string::npos) {
trash_log_count++;
}
}
ASSERT_EQ(trash_log_count, 0);
Close();
}
INSTANTIATE_TEST_CASE_P(DBWALTestWithParam, DBWALTestWithParam,
::testing::Values(std::make_tuple("", true),
std::make_tuple("_wal_dir", false)));
TEST_F(DBSSTTest, OpenDBWithExistingTrashAndObsoleteSstFile) {
Options options = CurrentOptions();
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", 1024 * 1024 /* 1 MB/sec */));
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
// Set an extra high trash ratio to prevent immediate/non-rate limited
// deletions
sfm->SetDeleteRateBytesPerSecond(1024 * 1024);
sfm->delete_scheduler()->SetMaxTrashDBRatio(1000.0);
int bg_delete_file = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::DeleteTrashFile:DeleteFile",
[&](void* /*arg*/) { bg_delete_file++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Destroy(last_options_);
// Add some trash files to the db directory so the DB can clean them up
ASSERT_OK(env_->CreateDirIfMissing(dbname_));
Group SST write in flush, compaction and db open with new stats (#11910) Summary: ## Context/Summary Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity. For that, this PR does the following: - Tag different write IOs by passing down and converting WriteOptions to IOOptions - Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS Some related code refactory to make implementation cleaner: - Blob stats - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info. - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write. - Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority - Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification - Build table - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder. This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority ## Test ### db bench Flush ``` ./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100 rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 ``` compaction, db oopen ``` Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279 rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213 rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66 ``` blob stats - just to make sure they aren't broken by this PR ``` Integrated Blob DB Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600 rocksdb.blobdb.blob.file.synced COUNT : 1 rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same) ``` ``` Stacked Blob DB Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876 rocksdb.blobdb.blob.file.synced COUNT : 8 rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same) ``` ### Rehearsal CI stress test Trigger 3 full runs of all our CI stress tests ### Performance Flush ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark; enable_statistics = true Pre-pr: avg 507515519.3 ns 497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908, Post-pr: avg 511971266.5 ns, regressed 0.88% 502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408, ``` Compaction ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 495346098.30 ns 492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846 Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97% 502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007 ``` Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats) ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 3848.10 ns 3814,3838,3839,3848,3854,3854,3854,3860,3860,3860 Post-pr: avg 3874.20 ns, regressed 0.68% 3863,3867,3871,3874,3875,3877,3877,3877,3880,3881 ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910 Reviewed By: ajkr Differential Revision: D49788060 Pulled By: hx235 fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
ASSERT_OK(
WriteStringToFile(env_, "abc", dbname_ + "/" + "001.sst.trash", false));
ASSERT_OK(
WriteStringToFile(env_, "abc", dbname_ + "/" + "002.sst.trash", false));
ASSERT_OK(
WriteStringToFile(env_, "abc", dbname_ + "/" + "003.sst.trash", false));
// Manually add an obsolete sst file. Obsolete SST files are discovered and
// deleted upon recovery.
constexpr uint64_t kSstFileNumber = 100;
const std::string kObsoleteSstFile =
MakeTableFileName(dbname_, kSstFileNumber);
Group SST write in flush, compaction and db open with new stats (#11910) Summary: ## Context/Summary Similar to https://github.com/facebook/rocksdb/pull/11288, https://github.com/facebook/rocksdb/pull/11444, categorizing SST/blob file write according to different io activities allows more insight into the activity. For that, this PR does the following: - Tag different write IOs by passing down and converting WriteOptions to IOOptions - Add new SST_WRITE_MICROS histogram in WritableFileWriter::Append() and breakdown FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS Some related code refactory to make implementation cleaner: - Blob stats - Replace high-level write measurement with low-level WritableFileWriter::Append() measurement for BLOB_DB_BLOB_FILE_WRITE_MICROS. This is to make FILE_WRITE_{FLUSH|COMPACTION|DB_OPEN}_MICROS include blob file. As a consequence, this introduces some behavioral changes on it, see HISTORY and db bench test plan below for more info. - Fix bugs where BLOB_DB_BLOB_FILE_SYNCED/BLOB_DB_BLOB_FILE_BYTES_WRITTEN include file failed to sync and bytes failed to write. - Refactor WriteOptions constructor for easier construction with io_activity and rate_limiter_priority - Refactor DBImpl::~DBImpl()/BlobDBImpl::Close() to bypass thread op verification - Build table - TableBuilderOptions now includes Read/WriteOpitons so BuildTable() do not need to take these two variables - Replace the io_priority passed into BuildTable() with TableBuilderOptions::WriteOpitons::rate_limiter_priority. Similar for BlobFileBuilder. This parameter is used for dynamically changing file io priority for flush, see https://github.com/facebook/rocksdb/pull/9988?fbclid=IwAR1DtKel6c-bRJAdesGo0jsbztRtciByNlvokbxkV6h_L-AE9MACzqRTT5s for more - Update ThreadStatus::FLUSH_BYTES_WRITTEN to use io_activity to track flush IO in flush job and db open instead of io_priority ## Test ### db bench Flush ``` ./db_bench --statistics=1 --benchmarks=fillseq --num=100000 --write_buffer_size=100 rocksdb.sst.write.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.flush.micros P50 : 1.830863 P95 : 4.094720 P99 : 6.578947 P100 : 26.000000 COUNT : 7875 SUM : 20377 rocksdb.file.write.compaction.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.db.open.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 ``` compaction, db oopen ``` Setup: ./db_bench --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 rocksdb.sst.write.micros P50 : 2.675325 P95 : 9.578788 P99 : 18.780000 P100 : 314.000000 COUNT : 638 SUM : 3279 rocksdb.file.write.flush.micros P50 : 0.000000 P95 : 0.000000 P99 : 0.000000 P100 : 0.000000 COUNT : 0 SUM : 0 rocksdb.file.write.compaction.micros P50 : 2.757353 P95 : 9.610687 P99 : 19.316667 P100 : 314.000000 COUNT : 615 SUM : 3213 rocksdb.file.write.db.open.micros P50 : 2.055556 P95 : 3.925000 P99 : 9.000000 P100 : 9.000000 COUNT : 23 SUM : 66 ``` blob stats - just to make sure they aren't broken by this PR ``` Integrated Blob DB Setup: ./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench Run:./db_bench --enable_blob_files=1 --statistics=1 --benchmarks=compact --db=../db_bench --use_existing_db=1 pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 7.298246 P95 : 9.771930 P99 : 9.991813 P100 : 16.000000 COUNT : 235 SUM : 1600 rocksdb.blobdb.blob.file.synced COUNT : 1 rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 2.000000 P95 : 2.829360 P99 : 2.993779 P100 : 9.000000 COUNT : 707 SUM : 1614 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 1 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 34842 (stay the same) ``` ``` Stacked Blob DB Run: ./db_bench --use_blob_db=1 --statistics=1 --benchmarks=fillseq --num=10000 --disable_auto_compactions=1 -write_buffer_size=100 --db=../db_bench pre-PR: rocksdb.blobdb.blob.file.write.micros P50 : 12.808042 P95 : 19.674497 P99 : 28.539683 P100 : 51.000000 COUNT : 10000 SUM : 140876 rocksdb.blobdb.blob.file.synced COUNT : 8 rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 post-PR: rocksdb.blobdb.blob.file.write.micros P50 : 1.657370 P95 : 2.952175 P99 : 3.877519 P100 : 24.000000 COUNT : 30001 SUM : 67924 - COUNT is higher and values are smaller as it includes header and footer write - COUNT is 3X higher due to each Append() count as one post-PR, while in pre-PR, 3 Append()s counts as one. See https://github.com/facebook/rocksdb/pull/11910/files#diff-32b811c0a1c000768cfb2532052b44dc0b3bf82253f3eab078e15ff201a0dabfL157-L164 rocksdb.blobdb.blob.file.synced COUNT : 8 (stay the same) rocksdb.blobdb.blob.file.bytes.written COUNT : 1043445 (stay the same) ``` ### Rehearsal CI stress test Trigger 3 full runs of all our CI stress tests ### Performance Flush ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=ManualFlush/key_num:524288/per_key_size:256 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark; enable_statistics = true Pre-pr: avg 507515519.3 ns 497686074,499444327,500862543,501389862,502994471,503744435,504142123,504224056,505724198,506610393,506837742,506955122,507695561,507929036,508307733,508312691,508999120,509963561,510142147,510698091,510743096,510769317,510957074,511053311,511371367,511409911,511432960,511642385,511691964,511730908, Post-pr: avg 511971266.5 ns, regressed 0.88% 502744835,506502498,507735420,507929724,508313335,509548582,509994942,510107257,510715603,511046955,511352639,511458478,512117521,512317380,512766303,512972652,513059586,513804934,513808980,514059409,514187369,514389494,514447762,514616464,514622882,514641763,514666265,514716377,514990179,515502408, ``` Compaction ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_{pre|post}_pr --benchmark_filter=ManualCompaction/comp_style:0/max_data:134217728/per_key_size:256/enable_statistics:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 495346098.30 ns 492118301,493203526,494201411,494336607,495269217,495404950,496402598,497012157,497358370,498153846 Post-pr: avg 504528077.20, regressed 1.85%. "ManualCompaction" include flush so the isolated regression for compaction should be around 1.85-0.88 = 0.97% 502465338,502485945,502541789,502909283,503438601,504143885,506113087,506629423,507160414,507393007 ``` Put with WAL (in case passing WriteOptions slows down this path even without collecting SST write stats) ``` TEST_TMPDIR=/dev/shm ./db_basic_bench_pre_pr --benchmark_filter=DBPut/comp_style:0/max_data:107374182400/per_key_size:256/enable_statistics:1/wal:1 --benchmark_repetitions=1000 -- default: 1 thread is used to run benchmark Pre-pr: avg 3848.10 ns 3814,3838,3839,3848,3854,3854,3854,3860,3860,3860 Post-pr: avg 3874.20 ns, regressed 0.68% 3863,3867,3871,3874,3875,3877,3877,3877,3880,3881 ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/11910 Reviewed By: ajkr Differential Revision: D49788060 Pulled By: hx235 fbshipit-source-id: 79e73699cda5be3b66461687e5147c2484fc5eff
2023-12-29 23:29:23 +00:00
ASSERT_OK(WriteStringToFile(env_, "abc", kObsoleteSstFile, false));
// Reopen the DB and verify that it deletes existing trash files and obsolete
// SST files with rate limiting.
Reopen(options);
sfm->WaitForEmptyTrash();
ASSERT_NOK(env_->FileExists(dbname_ + "/" + "001.sst.trash"));
ASSERT_NOK(env_->FileExists(dbname_ + "/" + "002.sst.trash"));
ASSERT_NOK(env_->FileExists(dbname_ + "/" + "003.sst.trash"));
ASSERT_NOK(env_->FileExists(kObsoleteSstFile));
ASSERT_EQ(bg_delete_file, 4);
}
// Create a DB with 2 db_paths, and generate multiple files in the 2
// db_paths using CompactRangeOptions, make sure that files that were
// deleted from first db_path were deleted using DeleteScheduler and
// files in the second path were not.
TEST_F(DBSSTTest, DeleteSchedulerMultipleDBPaths) {
std::atomic<int> bg_delete_file(0);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::DeleteTrashFile:DeleteFile",
[&](void* /*arg*/) { bg_delete_file++; });
// The deletion scheduler sometimes skips marking file as trash according to
// a heuristic. In that case the deletion will go through the below SyncPoint.
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::DeleteFile", [&](void* /*arg*/) { bg_delete_file++; });
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.db_paths.emplace_back(dbname_, 1024 * 100);
options.db_paths.emplace_back(dbname_ + "_2", 1024 * 100);
options.env = env_;
int64_t rate_bytes_per_sec = 1024 * 1024; // 1 Mb / Sec
Status s;
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", rate_bytes_per_sec, false, &s,
/* max_trash_db_ratio= */ 1.1));
ASSERT_OK(s);
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
DestroyAndReopen(options);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
WriteOptions wo;
wo.disableWAL = true;
// Create 4 files in L0
for (int i = 0; i < 4; i++) {
ASSERT_OK(Put("Key" + std::to_string(i), DummyString(1024, 'A'), wo));
ASSERT_OK(Flush());
}
// We created 4 sst files in L0
ASSERT_EQ("4", FilesPerLevel(0));
// Compaction will delete files from L0 in first db path and generate a new
// file in L1 in second db path
CompactRangeOptions compact_options;
compact_options.target_path_id = 1;
Slice begin("Key0");
Slice end("Key3");
ASSERT_OK(db_->CompactRange(compact_options, &begin, &end));
ASSERT_EQ("0,1", FilesPerLevel(0));
// Create 4 files in L0
for (int i = 4; i < 8; i++) {
ASSERT_OK(Put("Key" + std::to_string(i), DummyString(1024, 'B'), wo));
ASSERT_OK(Flush());
}
ASSERT_EQ("4,1", FilesPerLevel(0));
// Compaction will delete files from L0 in first db path and generate a new
// file in L1 in second db path
begin = "Key4";
end = "Key7";
ASSERT_OK(db_->CompactRange(compact_options, &begin, &end));
ASSERT_EQ("0,2", FilesPerLevel(0));
sfm->WaitForEmptyTrash();
ASSERT_EQ(bg_delete_file, 8);
// Compaction will delete both files and regenerate a file in L1 in second
// db path. The deleted files should still be cleaned up via delete scheduler.
compact_options.bottommost_level_compaction =
BottommostLevelCompaction::kForceOptimized;
ASSERT_OK(db_->CompactRange(compact_options, nullptr, nullptr));
ASSERT_EQ("0,1", FilesPerLevel(0));
sfm->WaitForEmptyTrash();
ASSERT_EQ(bg_delete_file, 10);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, DestroyDBWithRateLimitedDelete) {
int bg_delete_file = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DeleteScheduler::DeleteTrashFile:DeleteFile",
[&](void* /*arg*/) { bg_delete_file++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Status s;
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.env = env_;
options.sst_file_manager.reset(
NewSstFileManager(env_, nullptr, "", 0, false, &s, 0));
ASSERT_OK(s);
DestroyAndReopen(options);
// Create 4 files in L0
for (int i = 0; i < 4; i++) {
ASSERT_OK(Put("Key" + std::to_string(i), DummyString(1024, 'A')));
ASSERT_OK(Flush());
}
// We created 4 sst files in L0
ASSERT_EQ("4", FilesPerLevel(0));
// Close DB and destroy it using DeleteScheduler
Close();
int num_sst_files = 0;
int num_wal_files = 0;
std::vector<std::string> db_files;
ASSERT_OK(env_->GetChildren(dbname_, &db_files));
for (const std::string& f : db_files) {
if (f.substr(f.find_last_of('.') + 1) == "sst") {
num_sst_files++;
} else if (f.substr(f.find_last_of('.') + 1) == "log") {
num_wal_files++;
}
}
ASSERT_GT(num_sst_files, 0);
ASSERT_GT(num_wal_files, 0);
auto sfm = static_cast<SstFileManagerImpl*>(options.sst_file_manager.get());
sfm->SetDeleteRateBytesPerSecond(1024 * 1024);
// Set an extra high trash ratio to prevent immediate/non-rate limited
// deletions
sfm->delete_scheduler()->SetMaxTrashDBRatio(1000.0);
ASSERT_OK(DestroyDB(dbname_, options));
sfm->WaitForEmptyTrash();
ASSERT_EQ(bg_delete_file, num_sst_files + num_wal_files);
}
TEST_F(DBSSTTest, DBWithMaxSpaceAllowed) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.disable_auto_compactions = true;
DestroyAndReopen(options);
Random rnd(301);
// Generate a file containing 100 keys.
for (int i = 0; i < 100; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
uint64_t first_file_size = 0;
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db, &first_file_size));
ASSERT_EQ(sfm->GetTotalSize(), first_file_size);
// Set the maximum allowed space usage to the current total size
sfm->SetMaxAllowedSpaceUsage(first_file_size + 1);
ASSERT_OK(Put("key1", "val1"));
// This flush will cause bg_error_ and will fail
ASSERT_NOK(Flush());
}
TEST_F(DBSSTTest, DBWithMaxSpaceAllowedWithBlobFiles) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.disable_auto_compactions = true;
options.enable_blob_files = true;
DestroyAndReopen(options);
Random rnd(301);
// Generate a file containing keys.
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
uint64_t files_size = 0;
uint64_t total_files_size = 0;
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kBlobFile, &files_in_db, &files_size));
// Make sure blob files are considered by SSTFileManage in size limits.
ASSERT_GT(files_size, 0);
total_files_size = files_size;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db, &files_size));
total_files_size += files_size;
ASSERT_EQ(sfm->GetTotalSize(), total_files_size);
// Set the maximum allowed space usage to the current total size.
sfm->SetMaxAllowedSpaceUsage(total_files_size + 1);
bool max_allowed_space_reached = false;
bool delete_blob_file = false;
// Sync point called after blob file is closed and max allowed space is
// checked.
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"BlobFileCompletionCallback::CallBack::MaxAllowedSpaceReached",
[&](void* /*arg*/) { max_allowed_space_reached = true; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"BuildTable::AfterDeleteFile",
[&](void* /*arg*/) { delete_blob_file = true; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({
{
"BuildTable::AfterDeleteFile",
"DBSSTTest::DBWithMaxSpaceAllowedWithBlobFiles:1",
},
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
ASSERT_OK(Put("key1", "val1"));
// This flush will fail
ASSERT_NOK(Flush());
ASSERT_TRUE(max_allowed_space_reached);
TEST_SYNC_POINT("DBSSTTest::DBWithMaxSpaceAllowedWithBlobFiles:1");
ASSERT_TRUE(delete_blob_file);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, CancellingCompactionsWorks) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.level0_file_num_compaction_trigger = 2;
options.statistics = CreateDBStatistics();
DestroyAndReopen(options);
int completed_compactions = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::BackgroundCompaction():CancelledCompaction", [&](void* /*arg*/) {
sfm->SetMaxAllowedSpaceUsage(0);
ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::BackgroundCompaction:NonTrivial:AfterRun",
[&](void* /*arg*/) { completed_compactions++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Random rnd(301);
// Generate a file containing 10 keys.
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
uint64_t total_file_size = 0;
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db, &total_file_size));
// Set the maximum allowed space usage to the current total size
sfm->SetMaxAllowedSpaceUsage(2 * total_file_size + 1);
// Generate another file to trigger compaction.
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
// Because we set a callback in CancelledCompaction, we actually
// let the compaction run
ASSERT_GT(completed_compactions, 0);
ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
// Make sure the stat is bumped
ASSERT_GT(dbfull()->immutable_db_options().statistics.get()->getTickerCount(
COMPACTION_CANCELLED),
0);
ASSERT_EQ(0,
dbfull()->immutable_db_options().statistics.get()->getTickerCount(
FILES_MARKED_TRASH));
ASSERT_EQ(4,
dbfull()->immutable_db_options().statistics.get()->getTickerCount(
FILES_DELETED_IMMEDIATELY));
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, CancellingManualCompactionsWorks) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.statistics = CreateDBStatistics();
FlushedFileCollector* collector = new FlushedFileCollector();
options.listeners.emplace_back(collector);
DestroyAndReopen(options);
Random rnd(301);
// Generate a file containing 10 keys.
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
uint64_t total_file_size = 0;
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db, &total_file_size));
// Set the maximum allowed space usage to the current total size
sfm->SetMaxAllowedSpaceUsage(2 * total_file_size + 1);
// Generate another file to trigger compaction.
for (int i = 0; i < 10; i++) {
ASSERT_OK(Put(Key(i), rnd.RandomString(50)));
}
ASSERT_OK(Flush());
// OK, now trigger a manual compaction
ASSERT_TRUE(dbfull()
->CompactRange(CompactRangeOptions(), nullptr, nullptr)
.IsCompactionTooLarge());
// Wait for manual compaction to get scheduled and finish
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
// Make sure the stat is bumped
ASSERT_EQ(dbfull()->immutable_db_options().statistics.get()->getTickerCount(
COMPACTION_CANCELLED),
1);
// Now make sure CompactFiles also gets cancelled
auto l0_files = collector->GetFlushedFiles();
ASSERT_TRUE(
dbfull()
->CompactFiles(ROCKSDB_NAMESPACE::CompactionOptions(), l0_files, 0)
.IsCompactionTooLarge());
// Wait for manual compaction to get scheduled and finish
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ(dbfull()->immutable_db_options().statistics.get()->getTickerCount(
COMPACTION_CANCELLED),
2);
ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
// Now let the flush through and make sure GetCompactionsReservedSize
// returns to normal
sfm->SetMaxAllowedSpaceUsage(0);
int completed_compactions = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"CompactFilesImpl:End", [&](void* /*arg*/) { completed_compactions++; });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
ASSERT_OK(dbfull()->CompactFiles(ROCKSDB_NAMESPACE::CompactionOptions(),
l0_files, 0));
Remove wait_unscheduled from waitForCompact internal API (#11443) Summary: Context: In pull request https://github.com/facebook/rocksdb/issues/11436, we are introducing a new public API `waitForCompact(const WaitForCompactOptions& wait_for_compact_options)`. This API invokes the internal implementation `waitForCompact(bool wait_unscheduled=false)`. The unscheduled parameter indicates the compactions that are not yet scheduled but are required to process items in the queue. In certain cases, we are unable to wait for compactions, such as during a shutdown or when background jobs are paused. It is important to return the appropriate status in these scenarios. For all other cases, we should wait for all compaction and flush jobs, including the unscheduled ones. The primary purpose of this new API is to wait until the system has resolved its compaction debt. Currently, the usage of `wait_unscheduled` is limited to test code. This pull request eliminates the usage of wait_unscheduled. The internal `waitForCompact()` API now waits for unscheduled compactions unless the db is undergoing a shutdown. In the event of a shutdown, the API returns `Status::ShutdownInProgress()`. Additionally, a new parameter, `abort_on_pause`, has been introduced with a default value of `false`. This parameter addresses the possibility of waiting indefinitely for unscheduled jobs if `PauseBackgroundWork()` was called before `waitForCompact()` is invoked. By setting `abort_on_pause` to `true`, the API will immediately return `Status::Aborted`. Furthermore, all tests that previously called `waitForCompact(true)` have been fixed. Pull Request resolved: https://github.com/facebook/rocksdb/pull/11443 Test Plan: Existing tests that involve a shutdown in progress: - DBCompactionTest::CompactRangeShutdownWhileDelayed - DBTestWithParam::PreShutdownMultipleCompaction - DBTestWithParam::PreShutdownCompactionMiddle Reviewed By: pdillinger Differential Revision: D45923426 Pulled By: jaykorean fbshipit-source-id: 7dc93fe6a6841a7d9d2d72866fa647090dba8eae
2023-05-18 01:13:50 +00:00
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ(sfm->GetCompactionsReservedSize(), 0);
ASSERT_GT(completed_compactions, 0);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, DBWithMaxSpaceAllowedRandomized) {
// This test will set a maximum allowed space for the DB, then it will
// keep filling the DB until the limit is reached and bg_error_ is set.
// When bg_error_ is set we will verify that the DB size is greater
// than the limit.
std::vector<int> max_space_limits_mbs = {1, 10};
std::atomic<bool> bg_error_set(false);
std::atomic<int> reached_max_space_on_flush(0);
std::atomic<int> reached_max_space_on_compaction(0);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached",
[&](void* arg) {
Status* bg_error = static_cast<Status*>(arg);
bg_error_set = true;
reached_max_space_on_flush++;
// clear error to ensure compaction callback is called
*bg_error = Status::OK();
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::BackgroundCompaction():CancelledCompaction", [&](void* arg) {
bool* enough_room = static_cast<bool*>(arg);
*enough_room = true;
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"CompactionJob::FinishCompactionOutputFile:MaxAllowedSpaceReached",
[&](void* /*arg*/) {
bg_error_set = true;
reached_max_space_on_compaction++;
});
for (auto limit_mb : max_space_limits_mbs) {
bg_error_set = false;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearTrace();
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.write_buffer_size = 1024 * 512; // 512 Kb
DestroyAndReopen(options);
Random rnd(301);
sfm->SetMaxAllowedSpaceUsage(limit_mb * 1024 * 1024);
// It is easy to detect if the test is stuck in a loop. No need for
// complex termination logic.
while (true) {
auto s = Put(rnd.RandomString(10), rnd.RandomString(50));
if (!s.ok()) {
break;
}
}
ASSERT_TRUE(bg_error_set);
uint64_t total_sst_files_size = 0;
std::unordered_map<std::string, uint64_t> files_in_db;
ASSERT_OK(GetAllDataFiles(kTableFile, &files_in_db, &total_sst_files_size));
ASSERT_GE(total_sst_files_size, limit_mb * 1024 * 1024);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
ASSERT_GT(reached_max_space_on_flush, 0);
ASSERT_GT(reached_max_space_on_compaction, 0);
}
TEST_F(DBSSTTest, OpenDBWithInfiniteMaxOpenFiles) {
// Open DB with infinite max open files
// - First iteration use 1 thread to open files
// - Second iteration use 5 threads to open files
for (int iter = 0; iter < 2; iter++) {
Options options;
options.create_if_missing = true;
options.write_buffer_size = 100000;
options.disable_auto_compactions = true;
options.max_open_files = -1;
if (iter == 0) {
options.max_file_opening_threads = 1;
} else {
options.max_file_opening_threads = 5;
}
options = CurrentOptions(options);
DestroyAndReopen(options);
// Create 12 Files in L0 (then move then to L2)
for (int i = 0; i < 12; i++) {
std::string k = "L2_" + Key(i);
ASSERT_OK(Put(k, k + std::string(1000, 'a')));
ASSERT_OK(Flush());
}
CompactRangeOptions compact_options;
compact_options.change_level = true;
compact_options.target_level = 2;
ASSERT_OK(db_->CompactRange(compact_options, nullptr, nullptr));
// Create 12 Files in L0
for (int i = 0; i < 12; i++) {
std::string k = "L0_" + Key(i);
ASSERT_OK(Put(k, k + std::string(1000, 'a')));
ASSERT_OK(Flush());
}
Close();
// Reopening the DB will load all existing files
Reopen(options);
ASSERT_EQ("12,0,12", FilesPerLevel(0));
std::vector<std::vector<FileMetaData>> files;
dbfull()->TEST_GetFilesMetaData(db_->DefaultColumnFamily(), &files);
for (const auto& level : files) {
for (const auto& file : level) {
ASSERT_TRUE(file.table_reader_handle != nullptr);
}
}
for (int i = 0; i < 12; i++) {
ASSERT_EQ(Get("L0_" + Key(i)), "L0_" + Key(i) + std::string(1000, 'a'));
ASSERT_EQ(Get("L2_" + Key(i)), "L2_" + Key(i) + std::string(1000, 'a'));
}
}
}
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
TEST_F(DBSSTTest, OpenDBWithInfiniteMaxOpenFilesSubjectToMemoryLimit) {
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
2022-05-17 22:01:51 +00:00
for (CacheEntryRoleOptions::Decision charge_table_reader :
{CacheEntryRoleOptions::Decision::kEnabled,
CacheEntryRoleOptions::Decision::kDisabled}) {
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
// Open DB with infinite max open files
// - First iteration use 1 thread to open files
// - Second iteration use 5 threads to open files
for (int iter = 0; iter < 2; iter++) {
Options options;
options.create_if_missing = true;
options.write_buffer_size = 100000;
options.disable_auto_compactions = true;
options.max_open_files = -1;
BlockBasedTableOptions table_options;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
if (iter == 0) {
options.max_file_opening_threads = 1;
} else {
options.max_file_opening_threads = 5;
}
DestroyAndReopen(options);
// Create 5 Files in L0 (then move then to L2)
for (int i = 0; i < 5; i++) {
std::string k = "L2_" + Key(i);
ASSERT_OK(Put(k, k + std::string(1000, 'a')));
ASSERT_OK(Flush()) << i;
}
CompactRangeOptions compact_options;
compact_options.change_level = true;
compact_options.target_level = 2;
ASSERT_OK(db_->CompactRange(compact_options, nullptr, nullptr));
// Create 5 Files in L0
for (int i = 0; i < 5; i++) {
std::string k = "L0_" + Key(i);
ASSERT_OK(Put(k, k + std::string(1000, 'a')));
ASSERT_OK(Flush());
}
Close();
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
2022-05-17 22:01:51 +00:00
table_options.cache_usage_options.options_overrides.insert(
{CacheEntryRole::kBlockBasedTableReader,
{/*.charged = */ charge_table_reader}});
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
table_options.block_cache =
NewLRUCache(1024 /* capacity */, 0 /* num_shard_bits */,
true /* strict_capacity_limit */);
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
// Reopening the DB will try to load all existing files, conditionally
// subject to memory limit
Status s = TryReopen(options);
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
2022-05-17 22:01:51 +00:00
if (charge_table_reader == CacheEntryRoleOptions::Decision::kEnabled) {
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
EXPECT_TRUE(s.IsMemoryLimit());
Rewrite memory-charging feature's option API (#9926) Summary: **Context:** Previous PR https://github.com/facebook/rocksdb/pull/9748, https://github.com/facebook/rocksdb/pull/9073, https://github.com/facebook/rocksdb/pull/8428 added separate flag for each charged memory area. Such API design is not scalable as we charge more and more memory areas. Also, we foresee an opportunity to consolidate this feature with other cache usage related features such as `cache_index_and_filter_blocks` using `CacheEntryRole`. Therefore we decided to consolidate all these flags with `CacheUsageOptions cache_usage_options` and this PR serves as the first step by consolidating memory-charging related flags. **Summary:** - Replaced old API reference with new ones, including making `kCompressionDictionaryBuildingBuffer` opt-out and added a unit test for that - Added missing db bench/stress test for some memory charging features - Renamed related test suite to indicate they are under the same theme of memory charging - Refactored a commonly used mocked cache component in memory charging related tests to reduce code duplication - Replaced the phrases "memory tracking" / "cache reservation" (other than CacheReservationManager-related ones) with "memory charging" for standard description of this feature. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9926 Test Plan: - New unit test for opt-out `kCompressionDictionaryBuildingBuffer` `TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic)` - New unit test for option validation/sanitization `TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions)` - CI - db bench (in case querying new options introduces regression) **+0.5% micros/op**: `TEST_TMPDIR=/dev/shm/testdb ./db_bench -benchmarks=fillseq -db=$TEST_TMPDIR -charge_compression_dictionary_building_buffer=1(remove this for comparison) -compression_max_dict_bytes=10000 -disable_auto_compactions=1 -write_buffer_size=100000 -num=4000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 3.9711 | 0.264408 | 3.9914 | 0.254563 | 0.5111933721 20 | 3.83905 | 0.0664488 | 3.8251 | 0.0695456 | **-0.3633711465** 40 | 3.86625 | 0.136669 | 3.8867 | 0.143765 | **0.5289363078** - db_stress: `python3 tools/db_crashtest.py blackbox -charge_compression_dictionary_building_buffer=1 -charge_filter_construction=1 -charge_table_reader=1 -cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D36054712 Pulled By: hx235 fbshipit-source-id: d406e90f5e0c5ea4dbcb585a484ad9302d4302af
2022-05-17 22:01:51 +00:00
EXPECT_TRUE(s.ToString().find(
kCacheEntryRoleToCamelString[static_cast<std::uint32_t>(
CacheEntryRole::kBlockBasedTableReader)]) !=
std::string::npos);
Account memory of big memory users in BlockBasedTable in global memory limit (#9748) Summary: **Context:** Through heap profiling, we discovered that `BlockBasedTableReader` objects can accumulate and lead to high memory usage (e.g, `max_open_file = -1`). These memories are currently not saved, not tracked, not constrained and not cache evict-able. As a first step to improve this, similar to https://github.com/facebook/rocksdb/pull/8428, this PR is to track an estimate of `BlockBasedTableReader` object's memory in block cache and fail future creation if the memory usage exceeds the available space of cache at the time of creation. **Summary:** - Approximate big memory users (`BlockBasedTable::Rep` and `TableProperties` )' memory usage in addition to the existing estimated ones (filter block/index block/un-compression dictionary) - Charge all of these memory usages to block cache on `BlockBasedTable::Open()` and release them on `~BlockBasedTable()` as there is no memory usage fluctuation of concern in between - Refactor on CacheReservationManager (and its call-sites) to add concurrent support for BlockBasedTable used in this PR. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9748 Test Plan: - New unit tests - db bench: `OpenDb` : **-0.52% in ms** - Setup `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -write_buffer_size=1048576` - Repeated run with pre-change w/o feature and post-change with feature, benchmark `OpenDb`: `./db_bench -benchmarks=readrandom -use_existing_db=1 -db=/dev/shm/testdb -reserve_table_reader_memory=true (remove this when running w/o feature) -file_opening_threads=3 -open_files=-1 -report_open_timing=true| egrep 'OpenDb:'` #-run | (feature-off) avg milliseconds | std milliseconds | (feature-on) avg milliseconds | std milliseconds | change (%) -- | -- | -- | -- | -- | -- 10 | 11.4018 | 5.95173 | 9.47788 | 1.57538 | -16.87382694 20 | 9.23746 | 0.841053 | 9.32377 | 1.14074 | 0.9343477536 40 | 9.0876 | 0.671129 | 9.35053 | 1.11713 | 2.893283155 80 | 9.72514 | 2.28459 | 9.52013 | 1.0894 | -2.108041632 160 | 9.74677 | 0.991234 | 9.84743 | 1.73396 | 1.032752389 320 | 10.7297 | 5.11555 | 10.547 | 1.97692 | **-1.70275031** 640 | 11.7092 | 2.36565 | 11.7869 | 2.69377 | **0.6635807741** - db bench on write with cost to cache in WriteBufferManager (just in case this PR's CRM refactoring accidentally slows down anything in WBM) : `fillseq` : **+0.54% in micros/op** `./db_bench -benchmarks=fillseq -db=/dev/shm/testdb -disable_auto_compactions=1 -cost_write_buffer_to_cache=true -write_buffer_size=10000000000 | egrep 'fillseq'` #-run | (pre-PR) avg micros/op | std micros/op | (post-PR) avg micros/op | std micros/op | change (%) -- | -- | -- | -- | -- | -- 10 | 6.15 | 0.260187 | 6.289 | 0.371192 | 2.260162602 20 | 7.28025 | 0.465402 | 7.37255 | 0.451256 | 1.267813605 40 | 7.06312 | 0.490654 | 7.13803 | 0.478676 | **1.060579461** 80 | 7.14035 | 0.972831 | 7.14196 | 0.92971 | **0.02254791432** - filter bench: `bloom filter`: **-0.78% in ms/key** - ` ./filter_bench -impl=2 -quick -reserve_table_builder_memory=true | grep 'Build avg'` #-run | (pre-PR) avg ns/key | std ns/key | (post-PR) ns/key | std ns/key | change (%) -- | -- | -- | -- | -- | -- 10 | 26.4369 | 0.442182 | 26.3273 | 0.422919 | **-0.4145720565** 20 | 26.4451 | 0.592787 | 26.1419 | 0.62451 | **-1.1465262** - Crash test `python3 tools/db_crashtest.py blackbox --reserve_table_reader_memory=1 --cache_size=1` killed as normal Reviewed By: ajkr Differential Revision: D35136549 Pulled By: hx235 fbshipit-source-id: 146978858d0f900f43f4eb09bfd3e83195e3be28
2022-04-06 17:33:00 +00:00
EXPECT_TRUE(s.ToString().find("memory limit based on cache capacity") !=
std::string::npos);
} else {
EXPECT_TRUE(s.ok());
ASSERT_EQ("5,0,5", FilesPerLevel(0));
}
}
}
}
TEST_F(DBSSTTest, GetTotalSstFilesSize) {
// FIXME: L0 file and L1+ file also differ in size of `oldest_key_time`.
// L0 file has non-zero `oldest_key_time` while L1+ files have 0.
// The test passes since L1+ file uses current time instead of 0
// as oldest_ancestor_time.
//
// We don't propagate oldest-key-time table property on compaction and
// just write 0 as default value. This affect the exact table size, since
// we encode table properties as varint64. Force time to be 0 to work around
// it. Should remove the workaround after we propagate the property on
// compaction.
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"FlushJob::WriteLevel0Table:oldest_ancester_time", [&](void* arg) {
uint64_t* current_time = static_cast<uint64_t*>(arg);
*current_time = 0;
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.compression = kNoCompression;
DestroyAndReopen(options);
// Generate 5 files in L0
for (int i = 0; i < 5; i++) {
for (int j = 0; j < 10; j++) {
std::string val = "val_file_" + std::to_string(i);
ASSERT_OK(Put(Key(j), val));
}
ASSERT_OK(Flush());
}
ASSERT_EQ("5", FilesPerLevel(0));
std::vector<LiveFileMetaData> live_files_meta;
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 5);
uint64_t single_file_size = live_files_meta[0].size;
uint64_t live_sst_files_size = 0;
uint64_t total_sst_files_size = 0;
for (const auto& file_meta : live_files_meta) {
live_sst_files_size += file_meta.size;
}
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 5
// Total SST files = 5
ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
// hold current version
std::unique_ptr<Iterator> iter1(dbfull()->NewIterator(ReadOptions()));
ASSERT_OK(iter1->status());
// Compact 5 files into 1 file in L0
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_EQ("0,1", FilesPerLevel(0));
live_files_meta.clear();
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 1);
live_sst_files_size = 0;
total_sst_files_size = 0;
for (const auto& file_meta : live_files_meta) {
live_sst_files_size += file_meta.size;
}
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 1 (compacted file)
// Total SST files = 6 (5 original files + compacted file)
ASSERT_EQ(live_sst_files_size, 1 * single_file_size);
ASSERT_EQ(total_sst_files_size, 6 * single_file_size);
// hold current version
std::unique_ptr<Iterator> iter2(dbfull()->NewIterator(ReadOptions()));
ASSERT_OK(iter2->status());
// Delete all keys and compact, this will delete all live files
for (int i = 0; i < 10; i++) {
ASSERT_OK(Delete(Key(i)));
}
ASSERT_OK(Flush());
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_EQ("", FilesPerLevel(0));
live_files_meta.clear();
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 0);
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 0
// Total SST files = 6 (5 original files + compacted file)
ASSERT_EQ(total_sst_files_size, 6 * single_file_size);
ASSERT_OK(iter1->status());
iter1.reset();
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 0
// Total SST files = 1 (compacted file)
ASSERT_EQ(total_sst_files_size, 1 * single_file_size);
ASSERT_OK(iter2->status());
iter2.reset();
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 0
// Total SST files = 0
ASSERT_EQ(total_sst_files_size, 0);
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 19:39:49 +00:00
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBSSTTest, OpenDBWithoutGetFileSizeInvocations) {
Options options = CurrentOptions();
std::unique_ptr<MockEnv> env{MockEnv::Create(Env::Default())};
options.env = env.get();
options.disable_auto_compactions = true;
options.compression = kNoCompression;
options.enable_blob_files = true;
options.blob_file_size = 32; // create one blob per file
options.skip_checking_sst_file_sizes_on_db_open = true;
DestroyAndReopen(options);
// Generate 5 files in L0
for (int i = 0; i < 5; i++) {
for (int j = 0; j < 10; j++) {
std::string val = "val_file_" + std::to_string(i);
ASSERT_OK(Put(Key(j), val));
}
ASSERT_OK(Flush());
}
Close();
bool is_get_file_size_called = false;
SyncPoint::GetInstance()->SetCallBack(
"MockFileSystem::GetFileSize:CheckFileType", [&](void* arg) {
Prefer static_cast in place of most reinterpret_cast (#12308) Summary: The following are risks associated with pointer-to-pointer reinterpret_cast: * Can produce the "wrong result" (crash or memory corruption). IIRC, in theory this can happen for any up-cast or down-cast for a non-standard-layout type, though in practice would only happen for multiple inheritance cases (where the base class pointer might be "inside" the derived object). We don't use multiple inheritance a lot, but we do. * Can mask useful compiler errors upon code change, including converting between unrelated pointer types that you are expecting to be related, and converting between pointer and scalar types unintentionally. I can only think of some obscure cases where static_cast could be troublesome when it compiles as a replacement: * Going through `void*` could plausibly cause unnecessary or broken pointer arithmetic. Suppose we have `struct Derived: public Base1, public Base2`. If we have `Derived*` -> `void*` -> `Base2*` -> `Derived*` through reinterpret casts, this could plausibly work (though technical UB) assuming the `Base2*` is not dereferenced. Changing to static cast could introduce breaking pointer arithmetic. * Unnecessary (but safe) pointer arithmetic could arise in a case like `Derived*` -> `Base2*` -> `Derived*` where before the Base2 pointer might not have been dereferenced. This could potentially affect performance. With some light scripting, I tried replacing pointer-to-pointer reinterpret_casts with static_cast and kept the cases that still compile. Most occurrences of reinterpret_cast have successfully been changed (except for java/ and third-party/). 294 changed, 257 remain. A couple of related interventions included here: * Previously Cache::Handle was not actually derived from in the implementations and just used as a `void*` stand-in with reinterpret_cast. Now there is a relationship to allow static_cast. In theory, this could introduce pointer arithmetic (as described above) but is unlikely without multiple inheritance AND non-empty Cache::Handle. * Remove some unnecessary casts to void* as this is allowed to be implicit (for better or worse). Most of the remaining reinterpret_casts are for converting to/from raw bytes of objects. We could consider better idioms for these patterns in follow-up work. I wish there were a way to implement a template variant of static_cast that would only compile if no pointer arithmetic is generated, but best I can tell, this is not possible. AFAIK the best you could do is a dynamic check that the void* conversion after the static cast is unchanged. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12308 Test Plan: existing tests, CI Reviewed By: ltamasi Differential Revision: D53204947 Pulled By: pdillinger fbshipit-source-id: 9de23e618263b0d5b9820f4e15966876888a16e2
2024-02-07 18:44:11 +00:00
std::string* filename = static_cast<std::string*>(arg);
if (filename->find(".blob") != std::string::npos) {
is_get_file_size_called = true;
}
});
SyncPoint::GetInstance()->EnableProcessing();
Reopen(options);
ASSERT_FALSE(is_get_file_size_called);
SyncPoint::GetInstance()->DisableProcessing();
SyncPoint::GetInstance()->ClearAllCallBacks();
Destroy(options);
}
TEST_F(DBSSTTest, GetTotalSstFilesSizeVersionsFilesShared) {
Options options = CurrentOptions();
options.disable_auto_compactions = true;
options.compression = kNoCompression;
DestroyAndReopen(options);
// Generate 5 files in L0
for (int i = 0; i < 5; i++) {
ASSERT_OK(Put(Key(i), "val"));
ASSERT_OK(Flush());
}
ASSERT_EQ("5", FilesPerLevel(0));
std::vector<LiveFileMetaData> live_files_meta;
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 5);
uint64_t single_file_size = live_files_meta[0].size;
uint64_t live_sst_files_size = 0;
uint64_t total_sst_files_size = 0;
for (const auto& file_meta : live_files_meta) {
live_sst_files_size += file_meta.size;
}
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 5
// Total SST files = 5
ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
// hold current version
std::unique_ptr<Iterator> iter1(dbfull()->NewIterator(ReadOptions()));
ASSERT_OK(iter1->status());
// Compaction will do trivial move from L0 to L1
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_EQ("0,5", FilesPerLevel(0));
live_files_meta.clear();
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 5);
live_sst_files_size = 0;
total_sst_files_size = 0;
for (const auto& file_meta : live_files_meta) {
live_sst_files_size += file_meta.size;
}
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 5
// Total SST files = 5 (used in 2 version)
ASSERT_EQ(live_sst_files_size, 5 * single_file_size);
ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
// hold current version
std::unique_ptr<Iterator> iter2(dbfull()->NewIterator(ReadOptions()));
ASSERT_OK(iter2->status());
// Delete all keys and compact, this will delete all live files
for (int i = 0; i < 5; i++) {
ASSERT_OK(Delete(Key(i)));
}
ASSERT_OK(Flush());
ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr));
ASSERT_EQ("", FilesPerLevel(0));
live_files_meta.clear();
dbfull()->GetLiveFilesMetaData(&live_files_meta);
ASSERT_EQ(live_files_meta.size(), 0);
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 0
// Total SST files = 5 (used in 2 version)
ASSERT_EQ(total_sst_files_size, 5 * single_file_size);
ASSERT_OK(iter1->status());
iter1.reset();
ASSERT_OK(iter2->status());
iter2.reset();
ASSERT_TRUE(dbfull()->GetIntProperty("rocksdb.total-sst-files-size",
&total_sst_files_size));
// Live SST files = 0
// Total SST files = 0
ASSERT_EQ(total_sst_files_size, 0);
}
// This test if blob files are recorded by SST File Manager when Compaction job
// creates/delete them and in case of AtomicFlush.
TEST_F(DBSSTTest, DBWithSFMForBlobFilesAtomicFlush) {
std::shared_ptr<SstFileManager> sst_file_manager(NewSstFileManager(env_));
auto sfm = static_cast<SstFileManagerImpl*>(sst_file_manager.get());
Options options = CurrentOptions();
options.sst_file_manager = sst_file_manager;
options.enable_blob_files = true;
options.min_blob_size = 0;
options.disable_auto_compactions = true;
options.enable_blob_garbage_collection = true;
options.blob_garbage_collection_age_cutoff = 0.5;
options.atomic_flush = true;
int files_added = 0;
int files_deleted = 0;
int files_scheduled_to_delete = 0;
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnAddFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (EndsWith(*file_path, ".blob")) {
files_added++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::OnDeleteFile", [&](void* arg) {
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (EndsWith(*file_path, ".blob")) {
files_deleted++;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"SstFileManagerImpl::ScheduleFileDeletion", [&](void* arg) {
assert(arg);
const std::string* const file_path =
static_cast<const std::string*>(arg);
if (EndsWith(*file_path, ".blob")) {
++files_scheduled_to_delete;
}
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
DestroyAndReopen(options);
Random rnd(301);
ASSERT_OK(Put("key_1", "value_1"));
ASSERT_OK(Put("key_2", "value_2"));
ASSERT_OK(Put("key_3", "value_3"));
ASSERT_OK(Put("key_4", "value_4"));
ASSERT_OK(Flush());
// Overwrite will create the garbage data.
ASSERT_OK(Put("key_3", "new_value_3"));
ASSERT_OK(Put("key_4", "new_value_4"));
ASSERT_OK(Flush());
ASSERT_OK(Put("Key5", "blob_value5"));
ASSERT_OK(Put("Key6", "blob_value6"));
ASSERT_OK(Flush());
ASSERT_EQ(files_added, 3);
ASSERT_EQ(files_deleted, 0);
ASSERT_EQ(files_scheduled_to_delete, 0);
files_added = 0;
constexpr Slice* begin = nullptr;
constexpr Slice* end = nullptr;
// Compaction job will create a new file and delete the older files.
ASSERT_OK(db_->CompactRange(CompactRangeOptions(), begin, end));
ASSERT_OK(dbfull()->TEST_WaitForCompact());
ASSERT_EQ(files_added, 1);
ASSERT_EQ(files_scheduled_to_delete, 1);
sfm->WaitForEmptyTrash();
ASSERT_EQ(files_deleted, 1);
Close();
ASSERT_OK(DestroyDB(dbname_, options));
ASSERT_EQ(files_scheduled_to_delete, 4);
sfm->WaitForEmptyTrash();
ASSERT_EQ(files_deleted, 4);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();
}
} // namespace ROCKSDB_NAMESPACE
int main(int argc, char** argv) {
ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
::testing::InitGoogleTest(&argc, argv);
RegisterCustomObjects(argc, argv);
return RUN_ALL_TESTS();
}