提交 6a40ee5e 编写于 作者: M Maysam Yabandeh 提交者: Facebook Github Bot

Refresh snapshot list during long compactions (2nd attempt) (#5278)

Summary:
Part of compaction cpu goes to processing snapshot list, the larger the list the bigger the overhead. Although the lifetime of most of the snapshots is much shorter than the lifetime of compactions, the compaction conservatively operates on the list of snapshots that it initially obtained. This patch allows the snapshot list to be updated via a callback if the compaction is taking long. This should let the compaction to continue more efficiently with much smaller snapshot list.
For simplicity, to avoid the feature is disabled in two cases: i) When more than one sub-compaction are sharing the same snapshot list, ii) when Range Delete is used in which the range delete aggregator has its own copy of snapshot list.
This fixes the reverted https://github.com/facebook/rocksdb/pull/5099 issue with range deletes.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5278

Differential Revision: D15203291

Pulled By: maysamyabandeh

fbshipit-source-id: fa645611e606aa222c7ce53176dc5bb6f259c258
上级 5d27d65b
......@@ -5,6 +5,7 @@
### New Features
* Reduce binary search when iterator reseek into the same data block.
* Add an option `snap_refresh_nanos` (default to 0.1s) to periodically refresh the snapshot list in compaction jobs. Assign to 0 to disable the feature.
## 6.2.0 (4/30/2019)
### New Features
......
......@@ -2226,6 +2226,11 @@ void rocksdb_options_set_max_bytes_for_level_base(
opt->rep.max_bytes_for_level_base = n;
}
void rocksdb_options_set_snap_refresh_nanos(rocksdb_options_t* opt,
uint64_t n) {
opt->rep.snap_refresh_nanos = n;
}
void rocksdb_options_set_level_compaction_dynamic_level_bytes(
rocksdb_options_t* opt, unsigned char v) {
opt->rep.level_compaction_dynamic_level_bytes = v;
......
......@@ -38,14 +38,16 @@ CompactionIterator::CompactionIterator(
CompactionRangeDelAggregator* range_del_agg, const Compaction* compaction,
const CompactionFilter* compaction_filter,
const std::atomic<bool>* shutting_down,
const SequenceNumber preserve_deletes_seqnum)
const SequenceNumber preserve_deletes_seqnum,
SnapshotListFetchCallback* snap_list_callback)
: CompactionIterator(
input, cmp, merge_helper, last_sequence, snapshots,
earliest_write_conflict_snapshot, snapshot_checker, env,
report_detailed_time, expect_valid_internal_key, range_del_agg,
std::unique_ptr<CompactionProxy>(
compaction ? new CompactionProxy(compaction) : nullptr),
compaction_filter, shutting_down, preserve_deletes_seqnum) {}
compaction_filter, shutting_down, preserve_deletes_seqnum,
snap_list_callback) {}
CompactionIterator::CompactionIterator(
InternalIterator* input, const Comparator* cmp, MergeHelper* merge_helper,
......@@ -57,7 +59,8 @@ CompactionIterator::CompactionIterator(
std::unique_ptr<CompactionProxy> compaction,
const CompactionFilter* compaction_filter,
const std::atomic<bool>* shutting_down,
const SequenceNumber preserve_deletes_seqnum)
const SequenceNumber preserve_deletes_seqnum,
SnapshotListFetchCallback* snap_list_callback)
: input_(input),
cmp_(cmp),
merge_helper_(merge_helper),
......@@ -75,7 +78,8 @@ CompactionIterator::CompactionIterator(
current_user_key_sequence_(0),
current_user_key_snapshot_(0),
merge_out_iter_(merge_helper_),
current_key_committed_(false) {
current_key_committed_(false),
snap_list_callback_(snap_list_callback) {
assert(compaction_filter_ == nullptr || compaction_ != nullptr);
assert(snapshots_ != nullptr);
bottommost_level_ =
......@@ -83,24 +87,7 @@ CompactionIterator::CompactionIterator(
if (compaction_ != nullptr) {
level_ptrs_ = std::vector<size_t>(compaction_->number_levels(), 0);
}
if (snapshots_->size() == 0) {
// optimize for fast path if there are no snapshots
visible_at_tip_ = true;
earliest_snapshot_iter_ = snapshots_->end();
earliest_snapshot_ = kMaxSequenceNumber;
latest_snapshot_ = 0;
} else {
visible_at_tip_ = false;
earliest_snapshot_iter_ = snapshots_->begin();
earliest_snapshot_ = snapshots_->at(0);
latest_snapshot_ = snapshots_->back();
}
#ifndef NDEBUG
// findEarliestVisibleSnapshot assumes this ordering.
for (size_t i = 1; i < snapshots_->size(); ++i) {
assert(snapshots_->at(i - 1) < snapshots_->at(i));
}
#endif
ProcessSnapshotList();
input_->SetPinnedItersMgr(&pinned_iters_mgr_);
TEST_SYNC_POINT_CALLBACK("CompactionIterator:AfterInit", compaction_.get());
}
......@@ -222,6 +209,28 @@ void CompactionIterator::InvokeFilterIfNeeded(bool* need_skip,
}
}
void CompactionIterator::ProcessSnapshotList() {
#ifndef NDEBUG
// findEarliestVisibleSnapshot assumes this ordering.
for (size_t i = 1; i < snapshots_->size(); ++i) {
assert(snapshots_->at(i - 1) < snapshots_->at(i));
}
#endif
if (snapshots_->size() == 0) {
// optimize for fast path if there are no snapshots
visible_at_tip_ = true;
earliest_snapshot_iter_ = snapshots_->end();
earliest_snapshot_ = kMaxSequenceNumber;
latest_snapshot_ = 0;
} else {
visible_at_tip_ = false;
earliest_snapshot_iter_ = snapshots_->begin();
earliest_snapshot_ = snapshots_->at(0);
latest_snapshot_ = snapshots_->back();
}
released_snapshots_.clear();
}
void CompactionIterator::NextFromInput() {
at_next_ = false;
valid_ = false;
......@@ -269,6 +278,13 @@ void CompactionIterator::NextFromInput() {
// compaction filter). ikey_.user_key is pointing to the copy.
if (!has_current_user_key_ ||
!cmp_->Equal(ikey_.user_key, current_user_key_)) {
num_keys_++;
// Use num_keys_ to reduce the overhead of reading current time
if (snap_list_callback_ && snapshots_->size() &&
snap_list_callback_->TimeToRefresh(num_keys_)) {
snap_list_callback_->Refresh(snapshots_, latest_snapshot_);
ProcessSnapshotList();
}
// First occurrence of this user key
// Copy key for output
key_ = current_key_.SetInternalKey(key_, &ikey_);
......
......@@ -21,6 +21,53 @@
namespace rocksdb {
// This callback can be used to refresh the snapshot list from the db. It
// includes logics to exponentially decrease the refresh rate to limit the
// overhead of refresh.
class SnapshotListFetchCallback {
public:
SnapshotListFetchCallback(Env* env, uint64_t snap_refresh_nanos,
size_t every_nth_key = 1024)
: timer_(env, /*auto restart*/ true),
snap_refresh_nanos_(snap_refresh_nanos),
every_nth_key_minus_one_(every_nth_key - 1) {
assert(every_nth_key > 0);
assert((ceil(log2(every_nth_key)) == floor(log2(every_nth_key))));
}
// Refresh the snapshot list. snapshots will bre replacted with the new list.
// max is the upper bound. Note: this function will acquire the db_mutex_.
virtual void Refresh(std::vector<SequenceNumber>* snapshots,
SequenceNumber max) = 0;
inline bool TimeToRefresh(const size_t key_index) {
// skip the key if key_index % every_nth_key (which is of power 2) is not 0.
if ((key_index & every_nth_key_minus_one_) != 0) {
return false;
}
const uint64_t elapsed = timer_.ElapsedNanos();
auto ret = elapsed > snap_refresh_nanos_;
// pre-compute the next time threshold
if (ret) {
// inc next refresh period exponentially (by x4)
auto next_refresh_threshold = snap_refresh_nanos_ << 2;
// make sure the shift has not overflown the highest 1 bit
snap_refresh_nanos_ =
std::max(snap_refresh_nanos_, next_refresh_threshold);
}
return ret;
}
static constexpr SnapshotListFetchCallback* kDisabled = nullptr;
virtual ~SnapshotListFetchCallback() {}
private:
// Time since the callback was created
StopWatchNano timer_;
// The delay before calling ::Refresh. To be increased exponentially.
uint64_t snap_refresh_nanos_;
// Skip evey nth key. Number n if of power 2. The math will require n-1.
const uint64_t every_nth_key_minus_one_;
};
class CompactionIterator {
public:
// A wrapper around Compaction. Has a much smaller interface, only what
......@@ -69,7 +116,8 @@ class CompactionIterator {
const Compaction* compaction = nullptr,
const CompactionFilter* compaction_filter = nullptr,
const std::atomic<bool>* shutting_down = nullptr,
const SequenceNumber preserve_deletes_seqnum = 0);
const SequenceNumber preserve_deletes_seqnum = 0,
SnapshotListFetchCallback* snap_list_callback = nullptr);
// Constructor with custom CompactionProxy, used for tests.
CompactionIterator(InternalIterator* input, const Comparator* cmp,
......@@ -82,7 +130,8 @@ class CompactionIterator {
std::unique_ptr<CompactionProxy> compaction,
const CompactionFilter* compaction_filter = nullptr,
const std::atomic<bool>* shutting_down = nullptr,
const SequenceNumber preserve_deletes_seqnum = 0);
const SequenceNumber preserve_deletes_seqnum = 0,
SnapshotListFetchCallback* snap_list_callback = nullptr);
~CompactionIterator();
......@@ -110,6 +159,8 @@ class CompactionIterator {
private:
// Processes the input stream to find the next output
void NextFromInput();
// Process snapshots_ and assign related variables
void ProcessSnapshotList();
// Do last preparations before presenting the output to the callee. At this
// point this only zeroes out the sequence number if possible for better
......@@ -144,7 +195,7 @@ class CompactionIterator {
InternalIterator* input_;
const Comparator* cmp_;
MergeHelper* merge_helper_;
const std::vector<SequenceNumber>* snapshots_;
std::vector<SequenceNumber>* snapshots_;
// List of snapshots released during compaction.
// findEarliestVisibleSnapshot() find them out from return of
// snapshot_checker, and make sure they will not be returned as
......@@ -219,6 +270,9 @@ class CompactionIterator {
// Used to avoid purging uncommitted values. The application can specify
// uncommitted values by providing a SnapshotChecker object.
bool current_key_committed_;
SnapshotListFetchCallback* snap_list_callback_;
// number of distinct keys processed
size_t num_keys_ = 0;
bool IsShuttingDown() {
// This is a best-effort facility, so memory_order_relaxed is sufficient.
......
......@@ -315,7 +315,7 @@ CompactionJob::CompactionJob(
const SnapshotChecker* snapshot_checker, std::shared_ptr<Cache> table_cache,
EventLogger* event_logger, bool paranoid_file_checks, bool measure_io_stats,
const std::string& dbname, CompactionJobStats* compaction_job_stats,
Env::Priority thread_pri)
Env::Priority thread_pri, SnapshotListFetchCallback* snap_list_callback)
: job_id_(job_id),
compact_(new CompactionState(compaction)),
compaction_job_stats_(compaction_job_stats),
......@@ -336,6 +336,7 @@ CompactionJob::CompactionJob(
db_mutex_(db_mutex),
db_error_handler_(db_error_handler),
existing_snapshots_(std::move(existing_snapshots)),
snap_list_callback_(snap_list_callback),
earliest_write_conflict_snapshot_(earliest_write_conflict_snapshot),
snapshot_checker_(snapshot_checker),
table_cache_(std::move(table_cache)),
......@@ -892,7 +893,9 @@ void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) {
&existing_snapshots_, earliest_write_conflict_snapshot_,
snapshot_checker_, env_, ShouldReportDetailedTime(env_, stats_), false,
&range_del_agg, sub_compact->compaction, compaction_filter,
shutting_down_, preserve_deletes_seqnum_));
shutting_down_, preserve_deletes_seqnum_,
// Currently range_del_agg is incompatible with snapshot refresh feature.
range_del_agg.IsEmpty() ? snap_list_callback_ : nullptr));
auto c_iter = sub_compact->c_iter.get();
c_iter->SeekToFirst();
if (c_iter->Valid() && sub_compact->compaction->output_level() != 0) {
......
......@@ -57,22 +57,20 @@ class VersionSet;
class CompactionJob {
public:
CompactionJob(int job_id, Compaction* compaction,
const ImmutableDBOptions& db_options,
const EnvOptions env_options, VersionSet* versions,
const std::atomic<bool>* shutting_down,
const SequenceNumber preserve_deletes_seqnum,
LogBuffer* log_buffer, Directory* db_directory,
Directory* output_directory, Statistics* stats,
InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler,
std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot,
const SnapshotChecker* snapshot_checker,
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
bool paranoid_file_checks, bool measure_io_stats,
const std::string& dbname,
CompactionJobStats* compaction_job_stats,
Env::Priority thread_pri);
CompactionJob(
int job_id, Compaction* compaction, const ImmutableDBOptions& db_options,
const EnvOptions env_options, VersionSet* versions,
const std::atomic<bool>* shutting_down,
const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer,
Directory* db_directory, Directory* output_directory, Statistics* stats,
InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler,
std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot,
const SnapshotChecker* snapshot_checker,
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
bool paranoid_file_checks, bool measure_io_stats,
const std::string& dbname, CompactionJobStats* compaction_job_stats,
Env::Priority thread_pri, SnapshotListFetchCallback* snap_list_callback);
~CompactionJob();
......@@ -152,6 +150,7 @@ class CompactionJob {
// entirely within s1 and s2, then the earlier version of k1 can be safely
// deleted because that version is not visible in any snapshot.
std::vector<SequenceNumber> existing_snapshots_;
SnapshotListFetchCallback* snap_list_callback_;
// This is the earliest snapshot that could be used for write-conflict
// checking by a transaction. For any user-key newer than this snapshot, we
......
......@@ -5,7 +5,13 @@
#ifndef ROCKSDB_LITE
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>
#include <algorithm>
#include <array>
#include <map>
#include <string>
#include <tuple>
......@@ -194,6 +200,13 @@ class CompactionJobTest : public testing::Test {
}
void NewDB() {
DestroyDB(dbname_, Options());
EXPECT_OK(env_->CreateDirIfMissing(dbname_));
versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
table_cache_.get(), &write_buffer_manager_,
&write_controller_));
compaction_job_stats_.Reset();
VersionEdit new_db;
new_db.SetLogNumber(0);
new_db.SetNextFile(2);
......@@ -230,7 +243,10 @@ class CompactionJobTest : public testing::Test {
const std::vector<std::vector<FileMetaData*>>& input_files,
const stl_wrappers::KVMap& expected_results,
const std::vector<SequenceNumber>& snapshots = {},
SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber) {
SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber,
int output_level = 1, bool verify = true,
SnapshotListFetchCallback* snapshot_fetcher =
SnapshotListFetchCallback::kDisabled) {
auto cfd = versions_->GetColumnFamilySet()->GetDefault();
size_t num_input_files = 0;
......@@ -247,7 +263,7 @@ class CompactionJobTest : public testing::Test {
Compaction compaction(cfd->current()->storage_info(), *cfd->ioptions(),
*cfd->GetLatestMutableCFOptions(),
compaction_input_files, 1, 1024 * 1024,
compaction_input_files, output_level, 1024 * 1024,
10 * 1024 * 1024, 0, kNoCompression,
cfd->ioptions()->compression_opts, 0, {}, true);
compaction.SetInputVersion(cfd->current());
......@@ -263,7 +279,7 @@ class CompactionJobTest : public testing::Test {
nullptr, nullptr, &mutex_, &error_handler_, snapshots,
earliest_write_conflict_snapshot, snapshot_checker, table_cache_,
&event_logger, false, false, dbname_, &compaction_job_stats_,
Env::Priority::USER);
Env::Priority::USER, snapshot_fetcher);
VerifyInitializationOfCompactionJobStats(compaction_job_stats_);
compaction_job.Prepare();
......@@ -275,15 +291,17 @@ class CompactionJobTest : public testing::Test {
ASSERT_OK(compaction_job.Install(*cfd->GetLatestMutableCFOptions()));
mutex_.Unlock();
if (expected_results.size() == 0) {
ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U);
ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files);
ASSERT_EQ(compaction_job_stats_.num_output_files, 0U);
} else {
ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U);
ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files);
ASSERT_EQ(compaction_job_stats_.num_output_files, 1U);
mock_table_factory_->AssertLatestFile(expected_results);
if (verify) {
if (expected_results.size() == 0) {
ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U);
ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files);
ASSERT_EQ(compaction_job_stats_.num_output_files, 0U);
} else {
ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U);
ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files);
ASSERT_EQ(compaction_job_stats_.num_output_files, 1U);
mock_table_factory_->AssertLatestFile(expected_results);
}
}
}
......@@ -938,6 +956,105 @@ TEST_F(CompactionJobTest, CorruptionAfterDeletion) {
RunCompaction({files}, expected_results);
}
// Test the snapshot fetcher in compaction
TEST_F(CompactionJobTest, SnapshotRefresh) {
uint64_t time_seed = env_->NowMicros();
printf("time_seed is %" PRIu64 "\n", time_seed); // would help to reproduce
Random64 rand(time_seed);
std::vector<SequenceNumber> db_snapshots;
class SnapshotListFetchCallbackTest : public SnapshotListFetchCallback {
public:
SnapshotListFetchCallbackTest(Env* env, Random64& rand,
std::vector<SequenceNumber>* snapshots)
: SnapshotListFetchCallback(env, 0 /*no time delay*/,
1 /*fetch after each key*/),
rand_(rand),
snapshots_(snapshots) {}
virtual void Refresh(std::vector<SequenceNumber>* snapshots,
SequenceNumber) override {
assert(snapshots->size());
assert(snapshots_->size());
assert(snapshots_->size() == snapshots->size());
if (rand_.OneIn(2)) {
uint64_t release_index = rand_.Uniform(snapshots_->size());
snapshots_->erase(snapshots_->begin() + release_index);
*snapshots = *snapshots_;
}
}
private:
Random64 rand_;
std::vector<SequenceNumber>* snapshots_;
} snapshot_fetcher(env_, rand, &db_snapshots);
std::vector<std::pair<const std::string, std::string>> file1_kvs, file2_kvs;
std::array<ValueType, 4> types = {kTypeValue, kTypeDeletion,
kTypeSingleDeletion};
SequenceNumber last_seq = 0;
for (int i = 1; i < 100; i++) {
SequenceNumber seq = last_seq + 1;
last_seq = seq;
if (rand.OneIn(2)) {
auto type = types[rand.Uniform(types.size())];
file1_kvs.push_back(
{test::KeyStr("k" + ToString(i), seq, type), "v" + ToString(i)});
}
}
auto file1 = mock::MakeMockFile(file1_kvs);
for (int i = 1; i < 100; i++) {
SequenceNumber seq = last_seq + 1;
last_seq++;
if (rand.OneIn(2)) {
auto type = types[rand.Uniform(types.size())];
file2_kvs.push_back(
{test::KeyStr("k" + ToString(i), seq, type), "v" + ToString(i)});
}
}
auto file2 = mock::MakeMockFile(file2_kvs);
for (SequenceNumber i = 1; i < last_seq + 1; i++) {
if (rand.OneIn(5)) {
db_snapshots.push_back(i);
}
}
const bool kVerify = true;
const int output_level_0 = 0;
NewDB();
AddMockFile(file1);
AddMockFile(file2);
SetLastSequence(last_seq);
auto files = cfd_->current()->storage_info()->LevelFiles(0);
// put the output on L0 since it is easier to feed them again to the 2nd
// compaction
RunCompaction({files}, file1, db_snapshots, kMaxSequenceNumber,
output_level_0, !kVerify, &snapshot_fetcher);
// Now db_snapshots are changed. Run the compaction again without snapshot
// fetcher but with the updated snapshot list.
compaction_job_stats_.Reset();
files = cfd_->current()->storage_info()->LevelFiles(0);
RunCompaction({files}, file1, db_snapshots, kMaxSequenceNumber,
output_level_0 + 1, !kVerify);
// The result should be what we get if we run compaction without snapshot
// fetcher on the updated list of snapshots
auto expected = mock_table_factory_->output();
NewDB();
AddMockFile(file1);
AddMockFile(file2);
SetLastSequence(last_seq);
files = cfd_->current()->storage_info()->LevelFiles(0);
RunCompaction({files}, expected, db_snapshots, kMaxSequenceNumber,
output_level_0, !kVerify);
// The 2nd compaction above would get rid of useless delete markers. To get
// the output here exactly as what we got above after two compactions, we also
// run the compaction for 2nd time.
compaction_job_stats_.Reset();
files = cfd_->current()->storage_info()->LevelFiles(0);
RunCompaction({files}, expected, db_snapshots, kMaxSequenceNumber,
output_level_0 + 1, !kVerify);
}
} // namespace rocksdb
int main(int argc, char** argv) {
......
......@@ -561,6 +561,13 @@ class DBImpl : public DB {
const SnapshotList& snapshots() const { return snapshots_; }
void LoadSnapshots(std::vector<SequenceNumber>* snap_vector,
SequenceNumber* oldest_write_conflict_snapshot,
const SequenceNumber& max_seq) const {
InstrumentedMutexLock l(mutex());
snapshots().GetAll(snap_vector, oldest_write_conflict_snapshot, max_seq);
}
const ImmutableDBOptions& immutable_db_options() const {
return immutable_db_options_;
}
......@@ -739,7 +746,7 @@ class DBImpl : public DB {
// Not thread-safe.
void SetRecoverableStatePreReleaseCallback(PreReleaseCallback* callback);
InstrumentedMutex* mutex() { return &mutex_; }
InstrumentedMutex* mutex() const { return &mutex_; }
Status NewDB();
......
......@@ -798,6 +798,31 @@ Status DBImpl::CompactRange(const CompactRangeOptions& options,
return s;
}
namespace {
class SnapshotListFetchCallbackImpl : public SnapshotListFetchCallback {
public:
SnapshotListFetchCallbackImpl(DBImpl* db_impl, Env* env,
uint64_t snap_refresh_nanos, Logger* info_log)
: SnapshotListFetchCallback(env, snap_refresh_nanos),
db_impl_(db_impl),
info_log_(info_log) {}
virtual void Refresh(std::vector<SequenceNumber>* snapshots,
SequenceNumber max) override {
size_t prev = snapshots->size();
snapshots->clear();
db_impl_->LoadSnapshots(snapshots, nullptr, max);
size_t now = snapshots->size();
ROCKS_LOG_DEBUG(info_log_,
"Compaction snapshot count refreshed from %zu to %zu", prev,
now);
}
private:
DBImpl* db_impl_;
Logger* info_log_;
};
} // namespace
Status DBImpl::CompactFiles(const CompactionOptions& compact_options,
ColumnFamilyHandle* column_family,
const std::vector<std::string>& input_file_names,
......@@ -969,6 +994,9 @@ Status DBImpl::CompactFilesImpl(
assert(is_snapshot_supported_ || snapshots_.empty());
CompactionJobStats compaction_job_stats;
SnapshotListFetchCallbackImpl fetch_callback(
this, env_, c->mutable_cf_options()->snap_refresh_nanos,
immutable_db_options_.info_log.get());
CompactionJob compaction_job(
job_context->job_id, c.get(), immutable_db_options_,
env_options_for_compaction_, versions_.get(), &shutting_down_,
......@@ -978,7 +1006,9 @@ Status DBImpl::CompactFilesImpl(
snapshot_checker, table_cache_, &event_logger_,
c->mutable_cf_options()->paranoid_file_checks,
c->mutable_cf_options()->report_bg_io_stats, dbname_,
&compaction_job_stats, Env::Priority::USER);
&compaction_job_stats, Env::Priority::USER,
immutable_db_options_.max_subcompactions <= 1 ? &fetch_callback
: nullptr);
// Creating a compaction influences the compaction score because the score
// takes running compactions into account (by skipping files that are already
......@@ -2622,6 +2652,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress,
GetSnapshotContext(job_context, &snapshot_seqs,
&earliest_write_conflict_snapshot, &snapshot_checker);
assert(is_snapshot_supported_ || snapshots_.empty());
SnapshotListFetchCallbackImpl fetch_callback(
this, env_, c->mutable_cf_options()->snap_refresh_nanos,
immutable_db_options_.info_log.get());
CompactionJob compaction_job(
job_context->job_id, c.get(), immutable_db_options_,
env_options_for_compaction_, versions_.get(), &shutting_down_,
......@@ -2631,7 +2664,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress,
earliest_write_conflict_snapshot, snapshot_checker, table_cache_,
&event_logger_, c->mutable_cf_options()->paranoid_file_checks,
c->mutable_cf_options()->report_bg_io_stats, dbname_,
&compaction_job_stats, thread_pri);
&compaction_job_stats, thread_pri,
immutable_db_options_.max_subcompactions <= 1 ? &fetch_callback
: nullptr);
compaction_job.Prepare();
NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
......
......@@ -91,13 +91,23 @@ class SnapshotList {
SequenceNumber* oldest_write_conflict_snapshot = nullptr,
const SequenceNumber& max_seq = kMaxSequenceNumber) const {
std::vector<SequenceNumber> ret;
GetAll(&ret, oldest_write_conflict_snapshot, max_seq);
return ret;
}
void GetAll(std::vector<SequenceNumber>* snap_vector,
SequenceNumber* oldest_write_conflict_snapshot = nullptr,
const SequenceNumber& max_seq = kMaxSequenceNumber) const {
std::vector<SequenceNumber>& ret = *snap_vector;
// So far we have no use case that would pass a non-empty vector
assert(ret.size() == 0);
if (oldest_write_conflict_snapshot != nullptr) {
*oldest_write_conflict_snapshot = kMaxSequenceNumber;
}
if (empty()) {
return ret;
return;
}
const SnapshotImpl* s = &list_;
while (s->next_ != &list_) {
......@@ -119,7 +129,7 @@ class SnapshotList {
s = s->next_;
}
return ret;
return;
}
// get the sequence number of the most recent snapshot
......
......@@ -816,6 +816,8 @@ extern ROCKSDB_LIBRARY_API void rocksdb_options_set_target_file_size_multiplier(
rocksdb_options_t*, int);
extern ROCKSDB_LIBRARY_API void rocksdb_options_set_max_bytes_for_level_base(
rocksdb_options_t*, uint64_t);
extern ROCKSDB_LIBRARY_API void rocksdb_options_set_snap_refresh_nanos(
rocksdb_options_t*, uint64_t);
extern ROCKSDB_LIBRARY_API void
rocksdb_options_set_level_compaction_dynamic_level_bytes(rocksdb_options_t*,
unsigned char);
......
......@@ -269,6 +269,17 @@ struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions {
// Dynamically changeable through SetOptions() API
uint64_t max_bytes_for_level_base = 256 * 1048576;
// If non-zero, compactions will periodically refresh the snapshot list. The
// delay for the first refresh is snap_refresh_nanos nano seconds and
// exponentially increases afterwards. When having many short-lived snapshots,
// this option helps reducing the cpu usage of long-running compactions. The
// feature is disabled when max_subcompactions is greater than one.
//
// Default: 0.1s
//
// Dynamically changeable through SetOptions() API
uint64_t snap_refresh_nanos = 100 * 1000 * 1000; // 0.1s
// Disable automatic compactions. Manual compactions can still
// be issued on this column family
//
......
......@@ -169,6 +169,8 @@ void MutableCFOptions::Dump(Logger* log) const {
target_file_size_multiplier);
ROCKS_LOG_INFO(log, " max_bytes_for_level_base: %" PRIu64,
max_bytes_for_level_base);
ROCKS_LOG_INFO(log, " snap_refresh_nanos: %" PRIu64,
snap_refresh_nanos);
ROCKS_LOG_INFO(log, " max_bytes_for_level_multiplier: %f",
max_bytes_for_level_multiplier);
ROCKS_LOG_INFO(log, " ttl: %" PRIu64,
......
......@@ -149,6 +149,7 @@ struct MutableCFOptions {
target_file_size_base(options.target_file_size_base),
target_file_size_multiplier(options.target_file_size_multiplier),
max_bytes_for_level_base(options.max_bytes_for_level_base),
snap_refresh_nanos(options.snap_refresh_nanos),
max_bytes_for_level_multiplier(options.max_bytes_for_level_multiplier),
ttl(options.ttl),
periodic_compaction_seconds(options.periodic_compaction_seconds),
......@@ -185,6 +186,7 @@ struct MutableCFOptions {
target_file_size_base(0),
target_file_size_multiplier(0),
max_bytes_for_level_base(0),
snap_refresh_nanos(0),
max_bytes_for_level_multiplier(0),
ttl(0),
periodic_compaction_seconds(0),
......@@ -236,6 +238,7 @@ struct MutableCFOptions {
uint64_t target_file_size_base;
int target_file_size_multiplier;
uint64_t max_bytes_for_level_base;
uint64_t snap_refresh_nanos;
double max_bytes_for_level_multiplier;
uint64_t ttl;
uint64_t periodic_compaction_seconds;
......
......@@ -215,6 +215,9 @@ void ColumnFamilyOptions::Dump(Logger* log) const {
ROCKS_LOG_HEADER(
log, " Options.max_bytes_for_level_base: %" PRIu64,
max_bytes_for_level_base);
ROCKS_LOG_HEADER(
log, " Options.snap_refresh_nanos: %" PRIu64,
snap_refresh_nanos);
ROCKS_LOG_HEADER(log, "Options.level_compaction_dynamic_level_bytes: %d",
level_compaction_dynamic_level_bytes);
ROCKS_LOG_HEADER(log, " Options.max_bytes_for_level_multiplier: %f",
......@@ -490,6 +493,7 @@ ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForSmallDb(
write_buffer_size = 2 << 20;
target_file_size_base = 2 * 1048576;
max_bytes_for_level_base = 10 * 1048576;
snap_refresh_nanos = 0;
soft_pending_compaction_bytes_limit = 256 * 1048576;
hard_pending_compaction_bytes_limit = 1073741824ul;
......
......@@ -177,6 +177,7 @@ ColumnFamilyOptions BuildColumnFamilyOptions(
mutable_cf_options.target_file_size_multiplier;
cf_opts.max_bytes_for_level_base =
mutable_cf_options.max_bytes_for_level_base;
cf_opts.snap_refresh_nanos = mutable_cf_options.snap_refresh_nanos;
cf_opts.max_bytes_for_level_multiplier =
mutable_cf_options.max_bytes_for_level_multiplier;
cf_opts.ttl = mutable_cf_options.ttl;
......@@ -526,9 +527,9 @@ bool ParseOptionHelper(char* opt_address, const OptionType& opt_type,
opt_address));
case OptionType::kBlockBasedTableIndexShorteningMode:
return ParseEnum<BlockBasedTableOptions::IndexShorteningMode>(
block_base_table_index_shortening_mode_string_map, value,
reinterpret_cast<BlockBasedTableOptions::IndexShorteningMode*>(
opt_address));
block_base_table_index_shortening_mode_string_map, value,
reinterpret_cast<BlockBasedTableOptions::IndexShorteningMode*>(
opt_address));
case OptionType::kEncodingType:
return ParseEnum<EncodingType>(
encoding_type_string_map, value,
......@@ -1666,13 +1667,13 @@ std::unordered_map<std::string, BlockBasedTableOptions::DataBlockIndexType>
std::unordered_map<std::string, BlockBasedTableOptions::IndexShorteningMode>
OptionsHelper::block_base_table_index_shortening_mode_string_map = {
{"kNoShortening",
BlockBasedTableOptions::IndexShorteningMode::kNoShortening},
{"kShortenSeparators",
BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators},
{"kShortenSeparatorsAndSuccessor",
BlockBasedTableOptions::IndexShorteningMode::
kShortenSeparatorsAndSuccessor}};
{"kNoShortening",
BlockBasedTableOptions::IndexShorteningMode::kNoShortening},
{"kShortenSeparators",
BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators},
{"kShortenSeparatorsAndSuccessor",
BlockBasedTableOptions::IndexShorteningMode::
kShortenSeparatorsAndSuccessor}};
std::unordered_map<std::string, EncodingType>
OptionsHelper::encoding_type_string_map = {{"kPlain", kPlain},
......@@ -1910,6 +1911,10 @@ std::unordered_map<std::string, OptionTypeInfo>
{offset_of(&ColumnFamilyOptions::max_bytes_for_level_base),
OptionType::kUInt64T, OptionVerificationType::kNormal, true,
offsetof(struct MutableCFOptions, max_bytes_for_level_base)}},
{"snap_refresh_nanos",
{offset_of(&ColumnFamilyOptions::snap_refresh_nanos),
OptionType::kUInt64T, OptionVerificationType::kNormal, true,
offsetof(struct MutableCFOptions, snap_refresh_nanos)}},
{"max_bytes_for_level_multiplier",
{offset_of(&ColumnFamilyOptions::max_bytes_for_level_multiplier),
OptionType::kDouble, OptionVerificationType::kNormal, true,
......
......@@ -415,6 +415,7 @@ TEST_F(OptionsSettableTest, ColumnFamilyOptionsAllFieldsSettable) {
"kBZip2Compression:kNoCompression:kZlibCompression:kBZip2Compression:"
"kSnappyCompression;"
"max_bytes_for_level_base=986;"
"snap_refresh_nanos=1000000000;"
"bloom_locality=8016;"
"target_file_size_base=4294976376;"
"memtable_huge_page_size=2557;"
......
......@@ -74,6 +74,7 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) {
{"target_file_size_base", "12"},
{"target_file_size_multiplier", "13"},
{"max_bytes_for_level_base", "14"},
{"snap_refresh_nanos", "1000000000"},
{"level_compaction_dynamic_level_bytes", "true"},
{"max_bytes_for_level_multiplier", "15.0"},
{"max_bytes_for_level_multiplier_additional", "16:17:18"},
......@@ -183,6 +184,7 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) {
ASSERT_EQ(new_cf_opt.target_file_size_base, static_cast<uint64_t>(12));
ASSERT_EQ(new_cf_opt.target_file_size_multiplier, 13);
ASSERT_EQ(new_cf_opt.max_bytes_for_level_base, 14U);
ASSERT_EQ(new_cf_opt.snap_refresh_nanos, 1000000000U);
ASSERT_EQ(new_cf_opt.level_compaction_dynamic_level_bytes, true);
ASSERT_EQ(new_cf_opt.max_bytes_for_level_multiplier, 15.0);
ASSERT_EQ(new_cf_opt.max_bytes_for_level_multiplier_additional.size(), 3U);
......
......@@ -21,6 +21,12 @@ const InternalKeyComparator icmp_(BytewiseComparator());
} // namespace
stl_wrappers::KVMap MakeMockFile(
std::vector<std::pair<const std::string, std::string>> l) {
return stl_wrappers::KVMap(l.begin(), l.end(),
stl_wrappers::LessOfComparator(&icmp_));
}
stl_wrappers::KVMap MakeMockFile(
std::initializer_list<std::pair<const std::string, std::string>> l) {
return stl_wrappers::KVMap(l, stl_wrappers::LessOfComparator(&icmp_));
......@@ -137,6 +143,14 @@ void MockTableFactory::AssertLatestFile(
ParseInternalKey(Slice(key), &ikey);
std::cout << ikey.DebugString(false) << " -> " << value << std::endl;
}
std::cout << "Expected:" << std::endl;
for (const auto& kv : file_contents) {
ParsedInternalKey ikey;
std::string key, value;
std::tie(key, value) = kv;
ParseInternalKey(Slice(key), &ikey);
std::cout << ikey.DebugString(false) << " -> " << value << std::endl;
}
FAIL();
}
}
......
......@@ -28,6 +28,8 @@ namespace mock {
stl_wrappers::KVMap MakeMockFile(
std::initializer_list<std::pair<const std::string, std::string>> l = {});
stl_wrappers::KVMap MakeMockFile(
std::vector<std::pair<const std::string, std::string>> l);
struct MockTableFileSystem {
port::Mutex mutex;
......@@ -184,6 +186,12 @@ class MockTableFactory : public TableFactory {
// contents are equal to file_contents
void AssertSingleFile(const stl_wrappers::KVMap& file_contents);
void AssertLatestFile(const stl_wrappers::KVMap& file_contents);
stl_wrappers::KVMap output() {
assert(!file_system_.files.empty());
auto latest = file_system_.files.end();
--latest;
return latest->second;
}
private:
uint32_t GetAndWriteNextID(WritableFileWriter* file) const;
......
......@@ -343,8 +343,9 @@ def whitebox_crash_main(args, unknown_args):
if additional_opts['kill_random_test'] is None and (retncode == 0):
# we expect zero retncode if no kill option
expected = True
elif additional_opts['kill_random_test'] is not None and retncode < 0:
# we expect negative retncode if kill option was given
elif additional_opts['kill_random_test'] is not None and retncode <= 0:
# When kill option is given, the test MIGHT kill itself.
# If it does, negative retncode is expected. Otherwise 0.
expected = True
if not expected:
......
......@@ -40,6 +40,9 @@ void CompactionJobStats::Reset() {
file_fsync_nanos = 0;
file_prepare_write_nanos = 0;
smallest_output_key_prefix.clear();
largest_output_key_prefix.clear();
num_single_del_fallthru = 0;
num_single_del_mismatch = 0;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册