Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Eliminates a no-op compaction upon snapshot release when disabling auto compactions #7267

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions HISTORY.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
* Fix a bug where a level refitting in CompactRange() might race with an automatic compaction that puts the data to the target level of the refitting. The bug has been there for years.
* BackupEngine::CreateNewBackup could fail intermittently with non-OK status when backing up a read-write DB configured with a DBOptions::file_checksum_gen_factory. This issue has been worked-around such that CreateNewBackup should succeed, but (until fully fixed) BackupEngine might not see all checksums available in the DB.
* Fix a bug where immutable flushed memtable is never destroyed because a memtable is not added to delete list because of refernce hold by super version and super version doesn't switch because of empty delete list. So memory usage increases beyond write_buffer_size + max_write_buffer_size_to_maintain.
* Fix useless no-op compactions scheduled upon snapshot release when options.disable-auto-compactions = true.

### New Features
* A new option `std::shared_ptr<FileChecksumGenFactory> file_checksum_gen_factory` is added to `BackupableDBOptions`. The default value for this option is `nullptr`. If this option is null, the default backup engine checksum function (crc32c) will be used for creating, verifying, or restoring backups. If it is not null and is set to the DB custom checksum factory, the custom checksum function used in DB will also be used for creating, verifying, or restoring backups, in addition to the default checksum function (crc32c). If it is not null and is set to a custom checksum factory different than the DB custom checksum factory (which may be null), BackupEngine will return `Status::InvalidArgument()`.
Expand Down
3 changes: 2 additions & 1 deletion db/column_family.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1054,7 +1054,8 @@ void ColumnFamilyData::CreateNewMemtable(
}

bool ColumnFamilyData::NeedsCompaction() const {
return compaction_picker_->NeedsCompaction(current_->storage_info());
return !mutable_cf_options_.disable_auto_compactions &&
compaction_picker_->NeedsCompaction(current_->storage_info());
}

Compaction* ColumnFamilyData::PickCompaction(
Expand Down
70 changes: 70 additions & 0 deletions db/db_compaction_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3600,6 +3600,76 @@ TEST_F(DBCompactionTest, CompactBottomLevelFilesWithDeletions) {
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
}

TEST_F(DBCompactionTest, NoCompactBottomLevelFilesWithDeletions) {
// bottom-level files may contain deletions due to snapshots protecting the
// deleted keys. Once the snapshot is released, we should see files with many
// such deletions undergo single-file compactions. But when disabling auto
// compactions, it shouldn't be triggered which may causing too many
// background jobs.
const int kNumKeysPerFile = 1024;
const int kNumLevelFiles = 4;
const int kValueSize = 128;
Options options = CurrentOptions();
options.compression = kNoCompression;
options.disable_auto_compactions = true;
options.level0_file_num_compaction_trigger = kNumLevelFiles;
// inflate it a bit to account for key/metadata overhead
options.target_file_size_base = 120 * kNumKeysPerFile * kValueSize / 100;
Reopen(options);

Random rnd(301);
const Snapshot* snapshot = nullptr;
for (int i = 0; i < kNumLevelFiles; ++i) {
for (int j = 0; j < kNumKeysPerFile; ++j) {
ASSERT_OK(
Put(Key(i * kNumKeysPerFile + j), rnd.RandomString(kValueSize)));
}
if (i == kNumLevelFiles - 1) {
snapshot = db_->GetSnapshot();
// delete every other key after grabbing a snapshot, so these deletions
// and the keys they cover can't be dropped until after the snapshot is
// released.
for (int j = 0; j < kNumLevelFiles * kNumKeysPerFile; j += 2) {
ASSERT_OK(Delete(Key(j)));
}
}
Flush();
if (i < kNumLevelFiles - 1) {
ASSERT_EQ(i + 1, NumTableFilesAtLevel(0));
}
}
dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr);
ASSERT_EQ(kNumLevelFiles, NumTableFilesAtLevel(1));

std::vector<LiveFileMetaData> pre_release_metadata, post_release_metadata;
db_->GetLiveFilesMetaData(&pre_release_metadata);
// just need to bump seqnum so ReleaseSnapshot knows the newest key in the SST
// files does not need to be preserved in case of a future snapshot.
ASSERT_OK(Put(Key(0), "val"));

// release snapshot and no compaction should be triggered.
std::atomic<int> num_compactions{0};
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack(
"DBImpl::BackgroundCompaction:Start",
[&](void* /*arg*/) { num_compactions.fetch_add(1); });
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
db_->ReleaseSnapshot(snapshot);
dbfull()->TEST_WaitForCompact();
ASSERT_EQ(0, num_compactions);
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();

db_->GetLiveFilesMetaData(&post_release_metadata);
ASSERT_EQ(pre_release_metadata.size(), post_release_metadata.size());
for (size_t i = 0; i < pre_release_metadata.size(); ++i) {
const auto& pre_file = pre_release_metadata[i];
const auto& post_file = post_release_metadata[i];
ASSERT_EQ(1, pre_file.level);
ASSERT_EQ(1, post_file.level);
// each file is same as before with deletion markers/deleted keys.
ASSERT_EQ(post_file.size, pre_file.size);
}
}

TEST_F(DBCompactionTest, LevelCompactExpiredTtlFiles) {
const int kNumKeysPerFile = 32;
const int kNumLevelFiles = 2;
Expand Down