Auto enable Periodic Compactions if a Compaction Filter is used (#5865)
Summary: - Periodic compactions are auto-enabled if a compaction filter or a compaction filter factory is set, in Level Compaction. - The default value of `periodic_compaction_seconds` is changed to UINT64_MAX, which lets RocksDB auto-tune periodic compactions as needed. An explicit value of 0 will still work as before ie. to disable periodic compactions completely. For now, on seeing a compaction filter along with a UINT64_MAX value for `periodic_compaction_seconds`, RocksDB will make SST files older than 30 days to go through periodic copmactions. Some RocksDB users make use of compaction filters to control when their data can be deleted, usually with a custom TTL logic. But it is occasionally possible that the compactions get delayed by considerable time due to factors like low writes to a key range, data reaching bottom level, etc before the TTL expiry. Periodic Compactions feature was originally built to help such cases. Now periodic compactions are auto enabled by default when compaction filters or compaction filter factories are used, as it is generally helpful to all cases to collect garbage. `periodic_compaction_seconds` is set to a large value, 30 days, in `SanitizeOptions` when RocksDB sees that a `compaction_filter` or `compaction_filter_factory` is used. This is done only for Level Compaction style. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5865 Test Plan: - Added a new test `DBCompactionTest.LevelPeriodicCompactionWithCompactionFilters` to make sure that `periodic_compaction_seconds` is set if either `compaction_filter` or `compaction_filter_factory` options are set. - `COMPILE_WITH_ASAN=1 make check` Differential Revision: D17659180 Pulled By: sagar0 fbshipit-source-id: 4887b9cf2e53cf2dc93a7b658c6b15e1181217ee
This commit is contained in:
parent
26dc29633e
commit
4c9aa30a62
@ -1,6 +1,7 @@
|
||||
# Rocksdb Change Log
|
||||
## Unreleased
|
||||
### Public API Change
|
||||
* Changed the default value of periodic_compaction_seconds to `UINT64_MAX` which allows RocksDB to auto-tune periodic compaction scheduling. When using the default value, periodic compactions are now auto-enabled if a compaction filter is used. A value of `0` will turn off the feature completely.
|
||||
* Added an API GetCreationTimeOfOldestFile(uint64_t* creation_time) to get the
|
||||
file_creation_time of the oldest SST file in the DB.
|
||||
|
||||
|
@ -30,6 +30,7 @@
|
||||
#include "memtable/hash_skiplist_rep.h"
|
||||
#include "monitoring/thread_status_util.h"
|
||||
#include "options/options_helper.h"
|
||||
#include "port/port.h"
|
||||
#include "table/block_based/block_based_table_factory.h"
|
||||
#include "table/merging_iterator.h"
|
||||
#include "util/autovector.h"
|
||||
@ -342,6 +343,16 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
|
||||
result.max_compaction_bytes = result.target_file_size_base * 25;
|
||||
}
|
||||
|
||||
// Turn on periodic compactions and set them to occur once every 30 days if
|
||||
// compaction filters are used and periodic_compaction_seconds is set to the
|
||||
// default value.
|
||||
if (result.compaction_style == kCompactionStyleLevel &&
|
||||
(result.compaction_filter != nullptr ||
|
||||
result.compaction_filter_factory != nullptr) &&
|
||||
result.periodic_compaction_seconds == port::kMaxUint64) {
|
||||
result.periodic_compaction_seconds = 30 * 24 * 60 * 60;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -1180,12 +1191,8 @@ Status ColumnFamilyData::ValidateOptions(
|
||||
}
|
||||
}
|
||||
|
||||
if (cf_options.periodic_compaction_seconds > 0) {
|
||||
if (db_options.max_open_files != -1) {
|
||||
return Status::NotSupported(
|
||||
"Periodic Compaction is only supported when files are always "
|
||||
"kept open (set max_open_files = -1). ");
|
||||
}
|
||||
if (cf_options.periodic_compaction_seconds > 0 &&
|
||||
cf_options.periodic_compaction_seconds < port::kMaxUint64) {
|
||||
if (cf_options.table_factory->Name() != BlockBasedTableFactory().Name()) {
|
||||
return Status::NotSupported(
|
||||
"Periodic Compaction is only supported in "
|
||||
|
@ -3748,6 +3748,91 @@ TEST_F(DBCompactionTest, LevelPeriodicAndTtlCompaction) {
|
||||
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
|
||||
}
|
||||
|
||||
TEST_F(DBCompactionTest, LevelPeriodicCompactionWithCompactionFilters) {
|
||||
class TestCompactionFilter : public CompactionFilter {
|
||||
const char* Name() const override { return "TestCompactionFilter"; }
|
||||
};
|
||||
class TestCompactionFilterFactory : public CompactionFilterFactory {
|
||||
const char* Name() const override { return "TestCompactionFilterFactory"; }
|
||||
std::unique_ptr<CompactionFilter> CreateCompactionFilter(
|
||||
const CompactionFilter::Context& /*context*/) override {
|
||||
return std::unique_ptr<CompactionFilter>(new TestCompactionFilter());
|
||||
}
|
||||
};
|
||||
|
||||
const int kNumKeysPerFile = 32;
|
||||
const int kNumLevelFiles = 2;
|
||||
const int kValueSize = 100;
|
||||
|
||||
Random rnd(301);
|
||||
|
||||
Options options = CurrentOptions();
|
||||
TestCompactionFilter test_compaction_filter;
|
||||
env_->time_elapse_only_sleep_ = false;
|
||||
options.env = env_;
|
||||
env_->addon_time_.store(0);
|
||||
|
||||
enum CompactionFilterType {
|
||||
kUseCompactionFilter,
|
||||
kUseCompactionFilterFactory
|
||||
};
|
||||
|
||||
for (CompactionFilterType comp_filter_type :
|
||||
{kUseCompactionFilter, kUseCompactionFilterFactory}) {
|
||||
// Assert that periodic compactions are not enabled.
|
||||
ASSERT_EQ(port::kMaxUint64, options.periodic_compaction_seconds);
|
||||
|
||||
if (comp_filter_type == kUseCompactionFilter) {
|
||||
options.compaction_filter = &test_compaction_filter;
|
||||
options.compaction_filter_factory.reset();
|
||||
} else if (comp_filter_type == kUseCompactionFilterFactory) {
|
||||
options.compaction_filter = nullptr;
|
||||
options.compaction_filter_factory.reset(
|
||||
new TestCompactionFilterFactory());
|
||||
}
|
||||
DestroyAndReopen(options);
|
||||
|
||||
// periodic_compaction_seconds should be set to the sanitized value when
|
||||
// a compaction filter or a compaction filter factory is used.
|
||||
ASSERT_EQ(30 * 24 * 60 * 60,
|
||||
dbfull()->GetOptions().periodic_compaction_seconds);
|
||||
|
||||
int periodic_compactions = 0;
|
||||
rocksdb::SyncPoint::GetInstance()->SetCallBack(
|
||||
"LevelCompactionPicker::PickCompaction:Return", [&](void* arg) {
|
||||
Compaction* compaction = reinterpret_cast<Compaction*>(arg);
|
||||
auto compaction_reason = compaction->compaction_reason();
|
||||
if (compaction_reason == CompactionReason::kPeriodicCompaction) {
|
||||
periodic_compactions++;
|
||||
}
|
||||
});
|
||||
rocksdb::SyncPoint::GetInstance()->EnableProcessing();
|
||||
|
||||
for (int i = 0; i < kNumLevelFiles; ++i) {
|
||||
for (int j = 0; j < kNumKeysPerFile; ++j) {
|
||||
ASSERT_OK(
|
||||
Put(Key(i * kNumKeysPerFile + j), RandomString(&rnd, kValueSize)));
|
||||
}
|
||||
Flush();
|
||||
}
|
||||
dbfull()->TEST_WaitForCompact();
|
||||
|
||||
ASSERT_EQ("2", FilesPerLevel());
|
||||
ASSERT_EQ(0, periodic_compactions);
|
||||
|
||||
// Add 31 days and do a write
|
||||
env_->addon_time_.fetch_add(31 * 24 * 60 * 60);
|
||||
ASSERT_OK(Put("a", "1"));
|
||||
Flush();
|
||||
dbfull()->TEST_WaitForCompact();
|
||||
// Assert that the files stay in the same level
|
||||
ASSERT_EQ("3", FilesPerLevel());
|
||||
// The two old files go through the periodic compaction process
|
||||
ASSERT_EQ(2, periodic_compactions);
|
||||
|
||||
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(DBCompactionTest, CompactRangeDelayedByL0FileCount) {
|
||||
// Verify that, when `CompactRangeOptions::allow_write_stall == false`, manual
|
||||
|
@ -2445,7 +2445,8 @@ void VersionStorageInfo::ComputeCompactionScore(
|
||||
if (mutable_cf_options.ttl > 0) {
|
||||
ComputeExpiredTtlFiles(immutable_cf_options, mutable_cf_options.ttl);
|
||||
}
|
||||
if (mutable_cf_options.periodic_compaction_seconds > 0) {
|
||||
if (mutable_cf_options.periodic_compaction_seconds > 0 &&
|
||||
mutable_cf_options.periodic_compaction_seconds < port::kMaxUint64) {
|
||||
ComputeFilesMarkedForPeriodicCompaction(
|
||||
immutable_cf_options, mutable_cf_options.periodic_compaction_seconds);
|
||||
}
|
||||
@ -2505,7 +2506,8 @@ void VersionStorageInfo::ComputeExpiredTtlFiles(
|
||||
void VersionStorageInfo::ComputeFilesMarkedForPeriodicCompaction(
|
||||
const ImmutableCFOptions& ioptions,
|
||||
const uint64_t periodic_compaction_seconds) {
|
||||
assert(periodic_compaction_seconds > 0);
|
||||
assert(periodic_compaction_seconds > 0 &&
|
||||
periodic_compaction_seconds < port::kMaxUint64);
|
||||
|
||||
files_marked_for_periodic_compaction_.clear();
|
||||
|
||||
@ -2515,6 +2517,13 @@ void VersionStorageInfo::ComputeFilesMarkedForPeriodicCompaction(
|
||||
return;
|
||||
}
|
||||
const uint64_t current_time = static_cast<uint64_t>(temp_current_time);
|
||||
|
||||
assert(periodic_compaction_seconds <= current_time);
|
||||
// Disable periodic compaction if periodic_compaction_seconds > current_time.
|
||||
// This also help handle the underflow case.
|
||||
if (periodic_compaction_seconds > current_time) {
|
||||
return;
|
||||
}
|
||||
const uint64_t allowed_time_limit =
|
||||
current_time - periodic_compaction_seconds;
|
||||
|
||||
|
@ -670,10 +670,18 @@ struct AdvancedColumnFamilyOptions {
|
||||
// Only supported in Level compaction.
|
||||
// Pre-req: max_open_file == -1.
|
||||
// unit: seconds. Ex: 7 days = 7 * 24 * 60 * 60
|
||||
// Default: 0 (disabled)
|
||||
//
|
||||
// Values:
|
||||
// 0: Turn off Periodic compactions.
|
||||
// UINT64_MAX (i.e 0xffffffffffffffff): Let RocksDB control this feature
|
||||
// as needed. For now, RocksDB will change this value to 30 days
|
||||
// (i.e 30 * 24 * 60 * 60) so that every file goes through the compaction
|
||||
// process at least once every 30 days if not compacted sooner.
|
||||
//
|
||||
// Default: UINT64_MAX (allow RocksDB to auto-tune)
|
||||
//
|
||||
// Dynamically changeable through SetOptions() API
|
||||
uint64_t periodic_compaction_seconds = 0;
|
||||
uint64_t periodic_compaction_seconds = 0xffffffffffffffff;
|
||||
|
||||
// If this option is set then 1 in N blocks are compressed
|
||||
// using a fast (lz4) and slow (zstd) compression algorithm.
|
||||
|
@ -137,6 +137,9 @@ Status BlobDBImpl::Open(std::vector<ColumnFamilyHandle*>* handles) {
|
||||
cf_options_.compaction_filter_factory != nullptr) {
|
||||
return Status::NotSupported("Blob DB doesn't support compaction filter.");
|
||||
}
|
||||
// BlobDB does not support Periodic Compactions. So disable periodic
|
||||
// compactions irrespective of the user set value.
|
||||
cf_options_.periodic_compaction_seconds = 0;
|
||||
|
||||
Status s;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user