2017-04-06 02:14:05 +02:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// 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).
|
2017-04-06 02:14:05 +02:00
|
|
|
//
|
|
|
|
// 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_impl.h"
|
|
|
|
|
|
|
|
#ifndef __STDC_FORMAT_MACROS
|
|
|
|
#define __STDC_FORMAT_MACROS
|
|
|
|
#endif
|
|
|
|
#include <inttypes.h>
|
|
|
|
|
|
|
|
#include "db/builder.h"
|
2017-06-23 04:30:39 +02:00
|
|
|
#include "db/event_helpers.h"
|
2017-04-06 04:02:00 +02:00
|
|
|
#include "monitoring/iostats_context_imp.h"
|
|
|
|
#include "monitoring/perf_context_imp.h"
|
|
|
|
#include "monitoring/thread_status_updater.h"
|
|
|
|
#include "monitoring/thread_status_util.h"
|
2017-04-07 05:06:34 +02:00
|
|
|
#include "util/sst_file_manager_impl.h"
|
|
|
|
#include "util/sync_point.h"
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
namespace rocksdb {
|
2018-02-09 21:09:55 +01:00
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
Status DBImpl::SyncClosedLogs(JobContext* job_context) {
|
|
|
|
TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Start");
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
autovector<log::Writer*, 1> logs_to_sync;
|
|
|
|
uint64_t current_log_number = logfile_number_;
|
|
|
|
while (logs_.front().number < current_log_number &&
|
|
|
|
logs_.front().getting_synced) {
|
|
|
|
log_sync_cv_.Wait();
|
|
|
|
}
|
|
|
|
for (auto it = logs_.begin();
|
|
|
|
it != logs_.end() && it->number < current_log_number; ++it) {
|
|
|
|
auto& log = *it;
|
|
|
|
assert(!log.getting_synced);
|
|
|
|
log.getting_synced = true;
|
|
|
|
logs_to_sync.push_back(log.writer);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status s;
|
|
|
|
if (!logs_to_sync.empty()) {
|
|
|
|
mutex_.Unlock();
|
|
|
|
|
|
|
|
for (log::Writer* log : logs_to_sync) {
|
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[JOB %d] Syncing log #%" PRIu64, job_context->job_id,
|
|
|
|
log->get_log_number());
|
|
|
|
s = log->file()->Sync(immutable_db_options_.use_fsync);
|
|
|
|
}
|
|
|
|
if (s.ok()) {
|
|
|
|
s = directories_.GetWalDir()->Fsync();
|
|
|
|
}
|
|
|
|
|
|
|
|
mutex_.Lock();
|
|
|
|
|
|
|
|
// "number <= current_log_number - 1" is equivalent to
|
|
|
|
// "number < current_log_number".
|
|
|
|
MarkLogsSynced(current_log_number - 1, true, s);
|
|
|
|
if (!s.ok()) {
|
2017-06-23 04:30:39 +02:00
|
|
|
Status new_bg_error = s;
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
EventHelpers::NotifyOnBackgroundError(immutable_db_options_.listeners,
|
|
|
|
BackgroundErrorReason::kFlush,
|
|
|
|
&new_bg_error, &mutex_);
|
|
|
|
if (!new_bg_error.ok()) {
|
|
|
|
bg_error_ = new_bg_error;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Failed");
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::FlushMemTableToOutputFile(
|
|
|
|
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
|
|
|
bool* made_progress, JobContext* job_context, LogBuffer* log_buffer) {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
assert(cfd->imm()->NumNotFlushed() != 0);
|
|
|
|
assert(cfd->imm()->IsFlushPending());
|
|
|
|
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot;
|
|
|
|
std::vector<SequenceNumber> snapshot_seqs =
|
|
|
|
snapshots_.GetAll(&earliest_write_conflict_snapshot);
|
|
|
|
|
2017-10-18 18:09:31 +02:00
|
|
|
auto snapshot_checker = snapshot_checker_.get();
|
|
|
|
if (use_custom_gc_ && snapshot_checker == nullptr) {
|
|
|
|
snapshot_checker = DisableGCSnapshotChecker::Instance();
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
FlushJob flush_job(
|
2017-09-28 02:37:08 +02:00
|
|
|
dbname_, cfd, immutable_db_options_, mutable_cf_options,
|
2017-10-06 19:26:38 +02:00
|
|
|
env_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_,
|
2017-10-18 18:09:31 +02:00
|
|
|
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
|
2017-09-28 02:37:08 +02:00
|
|
|
job_context, log_buffer, directories_.GetDbDir(),
|
|
|
|
directories_.GetDataDir(0U),
|
2017-04-06 02:14:05 +02:00
|
|
|
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
|
|
|
|
&event_logger_, mutable_cf_options.report_bg_io_stats);
|
|
|
|
|
|
|
|
FileMetaData file_meta;
|
|
|
|
|
|
|
|
flush_job.PickMemTable();
|
|
|
|
|
2017-04-18 21:00:36 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
NotifyOnFlushBegin(cfd, &file_meta, mutable_cf_options, job_context->job_id,
|
|
|
|
flush_job.GetTableProperties());
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
Status s;
|
|
|
|
if (logfile_number_ > 0 &&
|
|
|
|
versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 0) {
|
|
|
|
// If there are more than one column families, we need to make sure that
|
|
|
|
// all the log files except the most recent one are synced. Otherwise if
|
|
|
|
// the host crashes after flushing and before WAL is persistent, the
|
|
|
|
// flushed SST may contain data from write batches whose updates to
|
|
|
|
// other column families are missing.
|
|
|
|
// SyncClosedLogs() may unlock and re-lock the db_mutex.
|
|
|
|
s = SyncClosedLogs(job_context);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Within flush_job.Run, rocksdb may call event listener to notify
|
|
|
|
// file creation and deletion.
|
|
|
|
//
|
|
|
|
// Note that flush_job.Run will unlock and lock the db_mutex,
|
|
|
|
// and EventListener callback will be called when the db_mutex
|
|
|
|
// is unlocked by the current thread.
|
|
|
|
if (s.ok()) {
|
|
|
|
s = flush_job.Run(&file_meta);
|
|
|
|
} else {
|
|
|
|
flush_job.Cancel();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (s.ok()) {
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(cfd, &job_context->superversion_context,
|
|
|
|
mutable_cf_options);
|
2017-04-06 02:14:05 +02:00
|
|
|
if (made_progress) {
|
|
|
|
*made_progress = 1;
|
|
|
|
}
|
|
|
|
VersionStorageInfo::LevelSummaryStorage tmp;
|
|
|
|
ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
|
|
|
|
cfd->GetName().c_str(),
|
|
|
|
cfd->current()->storage_info()->LevelSummary(&tmp));
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!s.ok() && !s.IsShutdownInProgress() &&
|
|
|
|
immutable_db_options_.paranoid_checks && bg_error_.ok()) {
|
2017-06-23 04:30:39 +02:00
|
|
|
Status new_bg_error = s;
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
EventHelpers::NotifyOnBackgroundError(immutable_db_options_.listeners,
|
|
|
|
BackgroundErrorReason::kFlush,
|
|
|
|
&new_bg_error, &mutex_);
|
|
|
|
if (!new_bg_error.ok()) {
|
|
|
|
// if a bad error happened (not ShutdownInProgress), paranoid_checks is
|
|
|
|
// true, and the error isn't handled by callback, mark DB read-only
|
|
|
|
bg_error_ = new_bg_error;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
if (s.ok()) {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
|
|
|
|
job_context->job_id, flush_job.GetTableProperties());
|
|
|
|
auto sfm = static_cast<SstFileManagerImpl*>(
|
|
|
|
immutable_db_options_.sst_file_manager.get());
|
|
|
|
if (sfm) {
|
|
|
|
// Notify sst_file_manager that a new file was added
|
|
|
|
std::string file_path = MakeTableFileName(
|
|
|
|
immutable_db_options_.db_paths[0].path, file_meta.fd.GetNumber());
|
|
|
|
sfm->OnAddFile(file_path);
|
|
|
|
if (sfm->IsMaxAllowedSpaceReached() && bg_error_.ok()) {
|
2018-03-31 00:17:13 +02:00
|
|
|
Status new_bg_error = Status::NoSpace("Max allowed space was reached");
|
2017-04-06 02:14:05 +02:00
|
|
|
TEST_SYNC_POINT_CALLBACK(
|
|
|
|
"DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached",
|
2017-06-23 04:30:39 +02:00
|
|
|
&new_bg_error);
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
EventHelpers::NotifyOnBackgroundError(immutable_db_options_.listeners,
|
|
|
|
BackgroundErrorReason::kFlush,
|
|
|
|
&new_bg_error, &mutex_);
|
|
|
|
if (!new_bg_error.ok()) {
|
|
|
|
bg_error_ = new_bg_error;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2017-04-18 21:00:36 +02:00
|
|
|
void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
|
|
|
|
const MutableCFOptions& mutable_cf_options,
|
|
|
|
int job_id, TableProperties prop) {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
if (immutable_db_options_.listeners.size() == 0U) {
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
bool triggered_writes_slowdown =
|
|
|
|
(cfd->current()->storage_info()->NumLevelFiles(0) >=
|
|
|
|
mutable_cf_options.level0_slowdown_writes_trigger);
|
|
|
|
bool triggered_writes_stop =
|
|
|
|
(cfd->current()->storage_info()->NumLevelFiles(0) >=
|
|
|
|
mutable_cf_options.level0_stop_writes_trigger);
|
|
|
|
// release lock while notifying events
|
|
|
|
mutex_.Unlock();
|
|
|
|
{
|
|
|
|
FlushJobInfo info;
|
|
|
|
info.cf_name = cfd->GetName();
|
|
|
|
// TODO(yhchiang): make db_paths dynamic in case flush does not
|
|
|
|
// go to L0 in the future.
|
|
|
|
info.file_path = MakeTableFileName(immutable_db_options_.db_paths[0].path,
|
|
|
|
file_meta->fd.GetNumber());
|
|
|
|
info.thread_id = env_->GetThreadID();
|
|
|
|
info.job_id = job_id;
|
|
|
|
info.triggered_writes_slowdown = triggered_writes_slowdown;
|
|
|
|
info.triggered_writes_stop = triggered_writes_stop;
|
|
|
|
info.smallest_seqno = file_meta->smallest_seqno;
|
|
|
|
info.largest_seqno = file_meta->largest_seqno;
|
|
|
|
info.table_properties = prop;
|
2018-02-09 21:09:55 +01:00
|
|
|
info.flush_reason = cfd->GetFlushReason();
|
2017-04-18 21:00:36 +02:00
|
|
|
for (auto listener : immutable_db_options_.listeners) {
|
|
|
|
listener->OnFlushBegin(this, info);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
mutex_.Lock();
|
|
|
|
// no need to signal bg_cv_ as it will be signaled at the end of the
|
|
|
|
// flush process.
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
|
|
|
|
FileMetaData* file_meta,
|
|
|
|
const MutableCFOptions& mutable_cf_options,
|
|
|
|
int job_id, TableProperties prop) {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
if (immutable_db_options_.listeners.size() == 0U) {
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
bool triggered_writes_slowdown =
|
|
|
|
(cfd->current()->storage_info()->NumLevelFiles(0) >=
|
|
|
|
mutable_cf_options.level0_slowdown_writes_trigger);
|
|
|
|
bool triggered_writes_stop =
|
|
|
|
(cfd->current()->storage_info()->NumLevelFiles(0) >=
|
|
|
|
mutable_cf_options.level0_stop_writes_trigger);
|
|
|
|
// release lock while notifying events
|
|
|
|
mutex_.Unlock();
|
|
|
|
{
|
|
|
|
FlushJobInfo info;
|
|
|
|
info.cf_name = cfd->GetName();
|
|
|
|
// TODO(yhchiang): make db_paths dynamic in case flush does not
|
|
|
|
// go to L0 in the future.
|
|
|
|
info.file_path = MakeTableFileName(immutable_db_options_.db_paths[0].path,
|
|
|
|
file_meta->fd.GetNumber());
|
|
|
|
info.thread_id = env_->GetThreadID();
|
|
|
|
info.job_id = job_id;
|
|
|
|
info.triggered_writes_slowdown = triggered_writes_slowdown;
|
|
|
|
info.triggered_writes_stop = triggered_writes_stop;
|
|
|
|
info.smallest_seqno = file_meta->smallest_seqno;
|
|
|
|
info.largest_seqno = file_meta->largest_seqno;
|
|
|
|
info.table_properties = prop;
|
2018-02-09 21:09:55 +01:00
|
|
|
info.flush_reason = cfd->GetFlushReason();
|
2017-04-06 02:14:05 +02:00
|
|
|
for (auto listener : immutable_db_options_.listeners) {
|
|
|
|
listener->OnFlushCompleted(this, info);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
mutex_.Lock();
|
|
|
|
// no need to signal bg_cv_ as it will be signaled at the end of the
|
|
|
|
// flush process.
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::CompactRange(const CompactRangeOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice* begin, const Slice* end) {
|
|
|
|
if (options.target_path_id >= immutable_db_options_.db_paths.size()) {
|
|
|
|
return Status::InvalidArgument("Invalid target path ID");
|
|
|
|
}
|
|
|
|
|
|
|
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
|
|
|
auto cfd = cfh->cfd();
|
|
|
|
bool exclusive = options.exclusive_manual_compaction;
|
|
|
|
|
2018-02-13 00:34:39 +01:00
|
|
|
bool flush_needed = true;
|
2018-02-28 02:08:34 +01:00
|
|
|
if (begin != nullptr && end != nullptr) {
|
|
|
|
// TODO(ajkr): We could also optimize away the flush in certain cases where
|
|
|
|
// one/both sides of the interval are unbounded. But it requires more
|
|
|
|
// changes to RangesOverlapWithMemtables.
|
|
|
|
Range range(*begin, *end);
|
|
|
|
SuperVersion* super_version = cfd->GetReferencedSuperVersion(&mutex_);
|
|
|
|
cfd->RangesOverlapWithMemtables({range}, super_version, &flush_needed);
|
|
|
|
CleanupSuperVersion(super_version);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!options.allow_write_stall && flush_needed) {
|
2018-02-13 00:34:39 +01:00
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
uint64_t orig_active_memtable_id = cfd->mem()->GetID();
|
|
|
|
WriteStallCondition write_stall_condition = WriteStallCondition::kNormal;
|
|
|
|
do {
|
|
|
|
if (write_stall_condition != WriteStallCondition::kNormal) {
|
|
|
|
TEST_SYNC_POINT("DBImpl::CompactRange:StallWait");
|
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[%s] CompactRange waiting on stall conditions to clear",
|
|
|
|
cfd->GetName().c_str());
|
|
|
|
bg_cv_.Wait();
|
|
|
|
}
|
|
|
|
if (cfd->IsDropped() || shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return Status::ShutdownInProgress();
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t earliest_memtable_id =
|
|
|
|
std::min(cfd->mem()->GetID(), cfd->imm()->GetEarliestMemTableID());
|
|
|
|
if (earliest_memtable_id > orig_active_memtable_id) {
|
|
|
|
// We waited so long that the memtable we were originally waiting on was
|
|
|
|
// flushed.
|
|
|
|
flush_needed = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions();
|
|
|
|
const auto* vstorage = cfd->current()->storage_info();
|
|
|
|
|
|
|
|
// Skip stalling check if we're below auto-flush and auto-compaction
|
|
|
|
// triggers. If it stalled in these conditions, that'd mean the stall
|
|
|
|
// triggers are so low that stalling is needed for any background work. In
|
|
|
|
// that case we shouldn't wait since background work won't be scheduled.
|
|
|
|
if (cfd->imm()->NumNotFlushed() <
|
|
|
|
cfd->ioptions()->min_write_buffer_number_to_merge &&
|
|
|
|
vstorage->l0_delay_trigger_count() <
|
|
|
|
mutable_cf_options.level0_file_num_compaction_trigger) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
// check whether one extra immutable memtable or an extra L0 file would
|
|
|
|
// cause write stalling mode to be entered. It could still enter stall
|
|
|
|
// mode due to pending compaction bytes, but that's less common
|
|
|
|
write_stall_condition =
|
|
|
|
ColumnFamilyData::GetWriteStallConditionAndCause(
|
|
|
|
cfd->imm()->NumNotFlushed() + 1,
|
|
|
|
vstorage->l0_delay_trigger_count() + 1,
|
|
|
|
vstorage->estimated_compaction_needed_bytes(), mutable_cf_options)
|
|
|
|
.first;
|
|
|
|
} while (write_stall_condition != WriteStallCondition::kNormal);
|
|
|
|
}
|
|
|
|
TEST_SYNC_POINT("DBImpl::CompactRange:StallWaitDone");
|
|
|
|
Status s;
|
|
|
|
if (flush_needed) {
|
|
|
|
s = FlushMemTable(cfd, FlushOptions(), FlushReason::kManualCompaction);
|
|
|
|
if (!s.ok()) {
|
|
|
|
LogFlush(immutable_db_options_.info_log);
|
|
|
|
return s;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
int max_level_with_files = 0;
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
Version* base = cfd->current();
|
|
|
|
for (int level = 1; level < base->storage_info()->num_non_empty_levels();
|
|
|
|
level++) {
|
|
|
|
if (base->storage_info()->OverlapInLevel(level, begin, end)) {
|
|
|
|
max_level_with_files = level;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
int final_output_level = 0;
|
|
|
|
if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
|
|
|
|
cfd->NumberLevels() > 1) {
|
|
|
|
// Always compact all files together.
|
2017-05-17 20:32:26 +02:00
|
|
|
final_output_level = cfd->NumberLevels() - 1;
|
|
|
|
// if bottom most level is reserved
|
|
|
|
if (immutable_db_options_.allow_ingest_behind) {
|
|
|
|
final_output_level--;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
|
2017-05-17 20:32:26 +02:00
|
|
|
final_output_level, options.target_path_id,
|
2017-04-06 02:14:05 +02:00
|
|
|
begin, end, exclusive);
|
|
|
|
} else {
|
|
|
|
for (int level = 0; level <= max_level_with_files; level++) {
|
|
|
|
int output_level;
|
|
|
|
// in case the compaction is universal or if we're compacting the
|
|
|
|
// bottom-most level, the output level will be the same as input one.
|
|
|
|
// level 0 can never be the bottommost level (i.e. if all files are in
|
|
|
|
// level 0, we will compact to level 1)
|
|
|
|
if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
|
|
|
|
cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
|
|
|
|
output_level = level;
|
|
|
|
} else if (level == max_level_with_files && level > 0) {
|
|
|
|
if (options.bottommost_level_compaction ==
|
|
|
|
BottommostLevelCompaction::kSkip) {
|
|
|
|
// Skip bottommost level compaction
|
|
|
|
continue;
|
|
|
|
} else if (options.bottommost_level_compaction ==
|
|
|
|
BottommostLevelCompaction::kIfHaveCompactionFilter &&
|
|
|
|
cfd->ioptions()->compaction_filter == nullptr &&
|
|
|
|
cfd->ioptions()->compaction_filter_factory == nullptr) {
|
|
|
|
// Skip bottommost level compaction since we don't have a compaction
|
|
|
|
// filter
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
output_level = level;
|
|
|
|
} else {
|
|
|
|
output_level = level + 1;
|
|
|
|
if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
|
|
|
|
cfd->ioptions()->level_compaction_dynamic_level_bytes &&
|
|
|
|
level == 0) {
|
|
|
|
output_level = ColumnFamilyData::kCompactToBaseLevel;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
s = RunManualCompaction(cfd, level, output_level, options.target_path_id,
|
|
|
|
begin, end, exclusive);
|
|
|
|
if (!s.ok()) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
|
|
|
|
final_output_level = cfd->NumberLevels() - 1;
|
|
|
|
} else if (output_level > final_output_level) {
|
|
|
|
final_output_level = output_level;
|
|
|
|
}
|
|
|
|
TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
|
|
|
|
TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (!s.ok()) {
|
|
|
|
LogFlush(immutable_db_options_.info_log);
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (options.change_level) {
|
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[RefitLevel] waiting for background threads to stop");
|
|
|
|
s = PauseBackgroundWork();
|
|
|
|
if (s.ok()) {
|
|
|
|
s = ReFitLevel(cfd, final_output_level, options.target_level);
|
|
|
|
}
|
|
|
|
ContinueBackgroundWork();
|
|
|
|
}
|
|
|
|
LogFlush(immutable_db_options_.info_log);
|
|
|
|
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
// an automatic compaction that has been scheduled might have been
|
|
|
|
// preempted by the manual compactions. Need to schedule it back.
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
}
|
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::CompactFiles(
|
|
|
|
const CompactionOptions& compact_options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const std::vector<std::string>& input_file_names,
|
2018-03-15 19:46:16 +01:00
|
|
|
const int output_level, const int output_path_id,
|
|
|
|
std::vector<std::string>* const output_file_names) {
|
2017-04-06 02:14:05 +02:00
|
|
|
#ifdef ROCKSDB_LITE
|
|
|
|
// not supported in lite version
|
|
|
|
return Status::NotSupported("Not supported in ROCKSDB LITE");
|
|
|
|
#else
|
|
|
|
if (column_family == nullptr) {
|
|
|
|
return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
|
|
|
|
}
|
|
|
|
|
|
|
|
auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
|
|
|
|
assert(cfd);
|
|
|
|
|
|
|
|
Status s;
|
|
|
|
JobContext job_context(0, true);
|
|
|
|
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
|
|
|
|
immutable_db_options_.info_log.get());
|
|
|
|
|
|
|
|
// Perform CompactFiles
|
|
|
|
SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
|
|
|
|
// This call will unlock/lock the mutex to wait for current running
|
|
|
|
// IngestExternalFile() calls to finish.
|
|
|
|
WaitForIngestFile();
|
|
|
|
|
|
|
|
s = CompactFilesImpl(compact_options, cfd, sv->current,
|
2018-03-15 19:46:16 +01:00
|
|
|
input_file_names, output_file_names, output_level,
|
2017-04-06 02:14:05 +02:00
|
|
|
output_path_id, &job_context, &log_buffer);
|
|
|
|
}
|
|
|
|
if (sv->Unref()) {
|
|
|
|
mutex_.Lock();
|
|
|
|
sv->Cleanup();
|
|
|
|
mutex_.Unlock();
|
|
|
|
delete sv;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Find and delete obsolete files
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
// If !s.ok(), this means that Compaction failed. In that case, we want
|
|
|
|
// to delete all obsolete files we might have created and we force
|
|
|
|
// FindObsoleteFiles(). This is because job_context does not
|
|
|
|
// catch all created files if compaction failed.
|
|
|
|
FindObsoleteFiles(&job_context, !s.ok());
|
|
|
|
} // release the mutex
|
|
|
|
|
|
|
|
// delete unnecessary files if any, this is done outside the mutex
|
2018-01-12 22:16:39 +01:00
|
|
|
if (job_context.HaveSomethingToClean() ||
|
|
|
|
job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// Have to flush the info logs before bg_compaction_scheduled_--
|
|
|
|
// because if bg_flush_scheduled_ becomes 0 and the lock is
|
|
|
|
// released, the deconstructor of DB can kick in and destroy all the
|
|
|
|
// states of DB so info_log might not be available after that point.
|
|
|
|
// It also applies to access other states that DB owns.
|
|
|
|
log_buffer.FlushBufferToLog();
|
|
|
|
if (job_context.HaveSomethingToDelete()) {
|
|
|
|
// no mutex is locked here. No need to Unlock() and Lock() here.
|
|
|
|
PurgeObsoleteFiles(job_context);
|
|
|
|
}
|
|
|
|
job_context.Clean();
|
|
|
|
}
|
|
|
|
|
|
|
|
return s;
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
Status DBImpl::CompactFilesImpl(
|
|
|
|
const CompactionOptions& compact_options, ColumnFamilyData* cfd,
|
|
|
|
Version* version, const std::vector<std::string>& input_file_names,
|
2018-03-15 19:46:16 +01:00
|
|
|
std::vector<std::string>* const output_file_names,
|
2017-04-06 02:14:05 +02:00
|
|
|
const int output_level, int output_path_id, JobContext* job_context,
|
|
|
|
LogBuffer* log_buffer) {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
|
|
|
|
if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return Status::ShutdownInProgress();
|
|
|
|
}
|
|
|
|
|
|
|
|
std::unordered_set<uint64_t> input_set;
|
|
|
|
for (auto file_name : input_file_names) {
|
|
|
|
input_set.insert(TableFileNameToNumber(file_name));
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyMetaData cf_meta;
|
|
|
|
// TODO(yhchiang): can directly use version here if none of the
|
|
|
|
// following functions call is pluggable to external developers.
|
|
|
|
version->GetColumnFamilyMetaData(&cf_meta);
|
|
|
|
|
|
|
|
if (output_path_id < 0) {
|
|
|
|
if (immutable_db_options_.db_paths.size() == 1U) {
|
|
|
|
output_path_id = 0;
|
|
|
|
} else {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"Automatic output path selection is not "
|
|
|
|
"yet supported in CompactFiles()");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Status s = cfd->compaction_picker()->SanitizeCompactionInputFiles(
|
|
|
|
&input_set, cf_meta, output_level);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::vector<CompactionInputFiles> input_files;
|
|
|
|
s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers(
|
|
|
|
&input_files, &input_set, version->storage_info(), compact_options);
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (auto inputs : input_files) {
|
2017-04-07 05:06:34 +02:00
|
|
|
if (cfd->compaction_picker()->AreFilesInCompaction(inputs.files)) {
|
2017-04-06 02:14:05 +02:00
|
|
|
return Status::Aborted(
|
|
|
|
"Some of the necessary compaction input "
|
|
|
|
"files are already being compacted");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// At this point, CompactFiles will be run.
|
|
|
|
bg_compaction_scheduled_++;
|
|
|
|
|
|
|
|
unique_ptr<Compaction> c;
|
|
|
|
assert(cfd->compaction_picker());
|
2017-04-07 05:06:34 +02:00
|
|
|
c.reset(cfd->compaction_picker()->CompactFiles(
|
2017-04-06 02:14:05 +02:00
|
|
|
compact_options, input_files, output_level, version->storage_info(),
|
|
|
|
*cfd->GetLatestMutableCFOptions(), output_path_id));
|
2017-09-14 00:31:34 +02:00
|
|
|
// we already sanitized the set of input files and checked for conflicts
|
|
|
|
// without releasing the lock, so we're guaranteed a compaction can be formed.
|
|
|
|
assert(c != nullptr);
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
c->SetInputVersion(version);
|
|
|
|
// deletion compaction currently not allowed in CompactFiles.
|
|
|
|
assert(!c->deletion_compaction());
|
|
|
|
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot;
|
|
|
|
std::vector<SequenceNumber> snapshot_seqs =
|
|
|
|
snapshots_.GetAll(&earliest_write_conflict_snapshot);
|
|
|
|
|
|
|
|
auto pending_outputs_inserted_elem =
|
|
|
|
CaptureCurrentFileNumberInPendingOutputs();
|
|
|
|
|
2017-10-18 18:09:31 +02:00
|
|
|
auto snapshot_checker = snapshot_checker_.get();
|
|
|
|
if (use_custom_gc_ && snapshot_checker == nullptr) {
|
|
|
|
snapshot_checker = DisableGCSnapshotChecker::Instance();
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
assert(is_snapshot_supported_ || snapshots_.empty());
|
|
|
|
CompactionJob compaction_job(
|
2017-09-28 02:37:08 +02:00
|
|
|
job_context->job_id, c.get(), immutable_db_options_,
|
Added support for differential snapshots
Summary:
The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2).
This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages.
From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff".
This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR.
For now, what's done here according to initial discussions:
Preserving deletes:
- We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion.
- I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum.
- Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum.
Iterator changes:
- couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum.
TableCache changes:
- I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span.
What's left:
- Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type.
Closes https://github.com/facebook/rocksdb/pull/2999
Differential Revision: D6175602
Pulled By: mikhail-antonov
fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 02:43:29 +01:00
|
|
|
env_options_for_compaction_, versions_.get(), &shutting_down_,
|
|
|
|
preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
|
2017-11-17 02:46:43 +01:00
|
|
|
directories_.GetDataDir(c->output_path_id()), stats_, &mutex_, &bg_error_,
|
|
|
|
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
|
|
|
|
table_cache_, &event_logger_,
|
|
|
|
c->mutable_cf_options()->paranoid_file_checks,
|
2017-04-06 02:14:05 +02:00
|
|
|
c->mutable_cf_options()->report_bg_io_stats, dbname_,
|
|
|
|
nullptr); // Here we pass a nullptr for CompactionJobStats because
|
|
|
|
// CompactFiles does not trigger OnCompactionCompleted(),
|
|
|
|
// which is the only place where CompactionJobStats is
|
|
|
|
// returned. The idea of not triggering OnCompationCompleted()
|
|
|
|
// is that CompactFiles runs in the caller thread, so the user
|
|
|
|
// should always know when it completes. As a result, it makes
|
|
|
|
// less sense to notify the users something they should already
|
|
|
|
// know.
|
|
|
|
//
|
|
|
|
// In the future, if we would like to add CompactionJobStats
|
|
|
|
// support for CompactFiles, we should have CompactFiles API
|
|
|
|
// pass a pointer of CompactionJobStats as the out-value
|
|
|
|
// instead of using EventListener.
|
|
|
|
|
|
|
|
// Creating a compaction influences the compaction score because the score
|
|
|
|
// takes running compactions into account (by skipping files that are already
|
|
|
|
// being compacted). Since we just changed compaction score, we recalculate it
|
|
|
|
// here.
|
|
|
|
version->storage_info()->ComputeCompactionScore(*cfd->ioptions(),
|
|
|
|
*c->mutable_cf_options());
|
|
|
|
|
|
|
|
compaction_job.Prepare();
|
|
|
|
|
|
|
|
mutex_.Unlock();
|
|
|
|
TEST_SYNC_POINT("CompactFilesImpl:0");
|
|
|
|
TEST_SYNC_POINT("CompactFilesImpl:1");
|
|
|
|
compaction_job.Run();
|
|
|
|
TEST_SYNC_POINT("CompactFilesImpl:2");
|
|
|
|
TEST_SYNC_POINT("CompactFilesImpl:3");
|
|
|
|
mutex_.Lock();
|
|
|
|
|
|
|
|
Status status = compaction_job.Install(*c->mutable_cf_options());
|
|
|
|
if (status.ok()) {
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(
|
|
|
|
c->column_family_data(), &job_context->superversion_context,
|
2018-03-24 01:21:47 +01:00
|
|
|
*c->mutable_cf_options(), FlushReason::kManualCompaction);
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
c->ReleaseCompactionFiles(s);
|
|
|
|
|
|
|
|
ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
|
|
|
|
|
|
|
|
if (status.ok()) {
|
|
|
|
// Done
|
|
|
|
} else if (status.IsShutdownInProgress()) {
|
|
|
|
// Ignore compaction errors found during shutting down
|
|
|
|
} else {
|
|
|
|
ROCKS_LOG_WARN(immutable_db_options_.info_log,
|
|
|
|
"[%s] [JOB %d] Compaction error: %s",
|
|
|
|
c->column_family_data()->GetName().c_str(),
|
|
|
|
job_context->job_id, status.ToString().c_str());
|
|
|
|
if (immutable_db_options_.paranoid_checks && bg_error_.ok()) {
|
2017-06-23 04:30:39 +02:00
|
|
|
Status new_bg_error = status;
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
EventHelpers::NotifyOnBackgroundError(immutable_db_options_.listeners,
|
|
|
|
BackgroundErrorReason::kCompaction,
|
|
|
|
&new_bg_error, &mutex_);
|
|
|
|
if (!new_bg_error.ok()) {
|
|
|
|
bg_error_ = new_bg_error;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-03-15 19:46:16 +01:00
|
|
|
if (output_file_names != nullptr) {
|
|
|
|
for (const auto newf : c->edit()->GetNewFiles()) {
|
|
|
|
(*output_file_names).push_back(TableFileName(
|
|
|
|
immutable_db_options_.db_paths, newf.second.fd.GetNumber(),
|
|
|
|
newf.second.fd.GetPathId()) );
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
c.reset();
|
|
|
|
|
|
|
|
bg_compaction_scheduled_--;
|
|
|
|
if (bg_compaction_scheduled_ == 0) {
|
|
|
|
bg_cv_.SignalAll();
|
|
|
|
}
|
|
|
|
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
|
|
|
Status DBImpl::PauseBackgroundWork() {
|
|
|
|
InstrumentedMutexLock guard_lock(&mutex_);
|
|
|
|
bg_compaction_paused_++;
|
2017-08-04 00:36:28 +02:00
|
|
|
while (bg_bottom_compaction_scheduled_ > 0 || bg_compaction_scheduled_ > 0 ||
|
|
|
|
bg_flush_scheduled_ > 0) {
|
2017-04-06 02:14:05 +02:00
|
|
|
bg_cv_.Wait();
|
|
|
|
}
|
|
|
|
bg_work_paused_++;
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::ContinueBackgroundWork() {
|
|
|
|
InstrumentedMutexLock guard_lock(&mutex_);
|
|
|
|
if (bg_work_paused_ == 0) {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
|
|
|
assert(bg_work_paused_ > 0);
|
|
|
|
assert(bg_compaction_paused_ > 0);
|
|
|
|
bg_compaction_paused_--;
|
|
|
|
bg_work_paused_--;
|
|
|
|
// It's sufficient to check just bg_work_paused_ here since
|
|
|
|
// bg_work_paused_ is always no greater than bg_compaction_paused_
|
|
|
|
if (bg_work_paused_ == 0) {
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::NotifyOnCompactionCompleted(
|
|
|
|
ColumnFamilyData* cfd, Compaction *c, const Status &st,
|
|
|
|
const CompactionJobStats& compaction_job_stats,
|
|
|
|
const int job_id) {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
if (immutable_db_options_.listeners.size() == 0U) {
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return;
|
|
|
|
}
|
2017-09-15 20:45:33 +02:00
|
|
|
Version* current = cfd->current();
|
|
|
|
current->Ref();
|
2017-04-06 02:14:05 +02:00
|
|
|
// release lock while notifying events
|
|
|
|
mutex_.Unlock();
|
|
|
|
TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex");
|
|
|
|
{
|
|
|
|
CompactionJobInfo info;
|
|
|
|
info.cf_name = cfd->GetName();
|
|
|
|
info.status = st;
|
|
|
|
info.thread_id = env_->GetThreadID();
|
|
|
|
info.job_id = job_id;
|
|
|
|
info.base_input_level = c->start_level();
|
|
|
|
info.output_level = c->output_level();
|
|
|
|
info.stats = compaction_job_stats;
|
|
|
|
info.table_properties = c->GetOutputTableProperties();
|
|
|
|
info.compaction_reason = c->compaction_reason();
|
|
|
|
info.compression = c->output_compression();
|
|
|
|
for (size_t i = 0; i < c->num_input_levels(); ++i) {
|
|
|
|
for (const auto fmd : *c->inputs(i)) {
|
|
|
|
auto fn = TableFileName(immutable_db_options_.db_paths,
|
|
|
|
fmd->fd.GetNumber(), fmd->fd.GetPathId());
|
|
|
|
info.input_files.push_back(fn);
|
|
|
|
if (info.table_properties.count(fn) == 0) {
|
|
|
|
std::shared_ptr<const TableProperties> tp;
|
2017-09-15 20:45:33 +02:00
|
|
|
auto s = current->GetTableProperties(&tp, fmd, &fn);
|
2017-04-06 02:14:05 +02:00
|
|
|
if (s.ok()) {
|
|
|
|
info.table_properties[fn] = tp;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
for (const auto newf : c->edit()->GetNewFiles()) {
|
|
|
|
info.output_files.push_back(TableFileName(immutable_db_options_.db_paths,
|
|
|
|
newf.second.fd.GetNumber(),
|
|
|
|
newf.second.fd.GetPathId()));
|
|
|
|
}
|
|
|
|
for (auto listener : immutable_db_options_.listeners) {
|
|
|
|
listener->OnCompactionCompleted(this, info);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
mutex_.Lock();
|
2017-09-15 20:45:33 +02:00
|
|
|
current->Unref();
|
2017-04-06 02:14:05 +02:00
|
|
|
// no need to signal bg_cv_ as it will be signaled at the end of the
|
|
|
|
// flush process.
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
|
|
|
// REQUIREMENT: block all background work by calling PauseBackgroundWork()
|
|
|
|
// before calling this function
|
|
|
|
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
|
|
|
|
assert(level < cfd->NumberLevels());
|
|
|
|
if (target_level >= cfd->NumberLevels()) {
|
|
|
|
return Status::InvalidArgument("Target level exceeds number of levels");
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
SuperVersionContext sv_context(/* create_superversion */ true);
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
Status status;
|
|
|
|
|
|
|
|
InstrumentedMutexLock guard_lock(&mutex_);
|
|
|
|
|
|
|
|
// only allow one thread refitting
|
|
|
|
if (refitting_level_) {
|
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[ReFitLevel] another thread is refitting");
|
|
|
|
return Status::NotSupported("another thread is refitting");
|
|
|
|
}
|
|
|
|
refitting_level_ = true;
|
|
|
|
|
|
|
|
const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
|
|
|
|
// move to a smaller level
|
|
|
|
int to_level = target_level;
|
|
|
|
if (target_level < 0) {
|
|
|
|
to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
|
|
|
|
}
|
|
|
|
|
|
|
|
auto* vstorage = cfd->current()->storage_info();
|
|
|
|
if (to_level > level) {
|
|
|
|
if (level == 0) {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"Cannot change from level 0 to other levels.");
|
|
|
|
}
|
|
|
|
// Check levels are empty for a trivial move
|
|
|
|
for (int l = level + 1; l <= to_level; l++) {
|
|
|
|
if (vstorage->NumLevelFiles(l) > 0) {
|
|
|
|
return Status::NotSupported(
|
|
|
|
"Levels between source and target are not empty for a move.");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (to_level != level) {
|
|
|
|
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
|
|
|
|
"[%s] Before refitting:\n%s", cfd->GetName().c_str(),
|
|
|
|
cfd->current()->DebugString().data());
|
|
|
|
|
|
|
|
VersionEdit edit;
|
|
|
|
edit.SetColumnFamily(cfd->GetID());
|
|
|
|
for (const auto& f : vstorage->LevelFiles(level)) {
|
|
|
|
edit.DeleteFile(level, f->fd.GetNumber());
|
|
|
|
edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
|
|
|
|
f->fd.GetFileSize(), f->smallest, f->largest,
|
|
|
|
f->smallest_seqno, f->largest_seqno,
|
|
|
|
f->marked_for_compaction);
|
|
|
|
}
|
|
|
|
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
|
|
|
|
"[%s] Apply version edit:\n%s", cfd->GetName().c_str(),
|
|
|
|
edit.DebugString().data());
|
|
|
|
|
|
|
|
status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
|
|
|
|
directories_.GetDbDir());
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(cfd, &sv_context, mutable_cf_options);
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] LogAndApply: %s\n",
|
|
|
|
cfd->GetName().c_str(), status.ToString().data());
|
|
|
|
|
|
|
|
if (status.ok()) {
|
|
|
|
ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
|
|
|
|
"[%s] After refitting:\n%s", cfd->GetName().c_str(),
|
|
|
|
cfd->current()->DebugString().data());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
sv_context.Clean();
|
2017-04-06 02:14:05 +02:00
|
|
|
refitting_level_ = false;
|
|
|
|
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
|
|
|
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
|
|
|
return cfh->cfd()->NumberLevels();
|
|
|
|
}
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* /*column_family*/) {
|
2017-04-06 02:14:05 +02:00
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
|
|
|
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
return cfh->cfd()->GetSuperVersion()->
|
|
|
|
mutable_cf_options.level0_stop_writes_trigger;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::Flush(const FlushOptions& flush_options,
|
|
|
|
ColumnFamilyHandle* column_family) {
|
|
|
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
2018-01-19 02:32:50 +01:00
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log, "[%s] Manual flush start.",
|
|
|
|
cfh->GetName().c_str());
|
2018-02-09 21:09:55 +01:00
|
|
|
Status s =
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
FlushMemTable(cfh->cfd(), flush_options, FlushReason::kManualFlush);
|
2018-01-19 02:32:50 +01:00
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[%s] Manual flush finished, status: %s\n",
|
|
|
|
cfh->GetName().c_str(), s.ToString().c_str());
|
|
|
|
return s;
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
|
|
|
|
int output_level, uint32_t output_path_id,
|
|
|
|
const Slice* begin, const Slice* end,
|
|
|
|
bool exclusive, bool disallow_trivial_move) {
|
|
|
|
assert(input_level == ColumnFamilyData::kCompactAllLevels ||
|
|
|
|
input_level >= 0);
|
|
|
|
|
|
|
|
InternalKey begin_storage, end_storage;
|
|
|
|
CompactionArg* ca;
|
|
|
|
|
|
|
|
bool scheduled = false;
|
|
|
|
bool manual_conflict = false;
|
2017-08-04 00:36:28 +02:00
|
|
|
ManualCompactionState manual;
|
2017-04-06 02:14:05 +02:00
|
|
|
manual.cfd = cfd;
|
|
|
|
manual.input_level = input_level;
|
|
|
|
manual.output_level = output_level;
|
|
|
|
manual.output_path_id = output_path_id;
|
|
|
|
manual.done = false;
|
|
|
|
manual.in_progress = false;
|
|
|
|
manual.incomplete = false;
|
|
|
|
manual.exclusive = exclusive;
|
|
|
|
manual.disallow_trivial_move = disallow_trivial_move;
|
|
|
|
// For universal compaction, we enforce every manual compaction to compact
|
|
|
|
// all files.
|
|
|
|
if (begin == nullptr ||
|
|
|
|
cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
|
|
|
|
cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
|
|
|
|
manual.begin = nullptr;
|
|
|
|
} else {
|
2017-09-13 02:16:44 +02:00
|
|
|
begin_storage.SetMinPossibleForUserKey(*begin);
|
2017-04-06 02:14:05 +02:00
|
|
|
manual.begin = &begin_storage;
|
|
|
|
}
|
|
|
|
if (end == nullptr ||
|
|
|
|
cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
|
|
|
|
cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
|
|
|
|
manual.end = nullptr;
|
|
|
|
} else {
|
2017-09-13 02:16:44 +02:00
|
|
|
end_storage.SetMaxPossibleForUserKey(*end);
|
2017-04-06 02:14:05 +02:00
|
|
|
manual.end = &end_storage;
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_SYNC_POINT("DBImpl::RunManualCompaction:0");
|
|
|
|
TEST_SYNC_POINT("DBImpl::RunManualCompaction:1");
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
|
|
|
|
// When a manual compaction arrives, temporarily disable scheduling of
|
|
|
|
// non-manual compactions and wait until the number of scheduled compaction
|
|
|
|
// jobs drops to zero. This is needed to ensure that this manual compaction
|
|
|
|
// can compact any range of keys/files.
|
|
|
|
//
|
|
|
|
// HasPendingManualCompaction() is true when at least one thread is inside
|
|
|
|
// RunManualCompaction(), i.e. during that time no other compaction will
|
|
|
|
// get scheduled (see MaybeScheduleFlushOrCompaction).
|
|
|
|
//
|
|
|
|
// Note that the following loop doesn't stop more that one thread calling
|
|
|
|
// RunManualCompaction() from getting to the second while loop below.
|
|
|
|
// However, only one of them will actually schedule compaction, while
|
|
|
|
// others will wait on a condition variable until it completes.
|
|
|
|
|
|
|
|
AddManualCompaction(&manual);
|
|
|
|
TEST_SYNC_POINT_CALLBACK("DBImpl::RunManualCompaction:NotScheduled", &mutex_);
|
|
|
|
if (exclusive) {
|
2017-08-04 00:36:28 +02:00
|
|
|
while (bg_bottom_compaction_scheduled_ > 0 ||
|
|
|
|
bg_compaction_scheduled_ > 0) {
|
2017-05-03 00:01:07 +02:00
|
|
|
TEST_SYNC_POINT("DBImpl::RunManualCompaction:WaitScheduled");
|
2017-04-06 02:14:05 +02:00
|
|
|
ROCKS_LOG_INFO(
|
|
|
|
immutable_db_options_.info_log,
|
|
|
|
"[%s] Manual compaction waiting for all other scheduled background "
|
|
|
|
"compactions to finish",
|
|
|
|
cfd->GetName().c_str());
|
|
|
|
bg_cv_.Wait();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ROCKS_LOG_INFO(immutable_db_options_.info_log,
|
|
|
|
"[%s] Manual compaction starting", cfd->GetName().c_str());
|
|
|
|
|
|
|
|
// We don't check bg_error_ here, because if we get the error in compaction,
|
|
|
|
// the compaction will set manual.status to bg_error_ and set manual.done to
|
|
|
|
// true.
|
|
|
|
while (!manual.done) {
|
|
|
|
assert(HasPendingManualCompaction());
|
|
|
|
manual_conflict = false;
|
2017-10-19 19:48:47 +02:00
|
|
|
Compaction* compaction = nullptr;
|
2017-04-06 02:14:05 +02:00
|
|
|
if (ShouldntRunManualCompaction(&manual) || (manual.in_progress == true) ||
|
|
|
|
scheduled ||
|
2017-10-19 19:48:47 +02:00
|
|
|
(((manual.manual_end = &manual.tmp_storage1) != nullptr) &&
|
|
|
|
((compaction = manual.cfd->CompactRange(
|
|
|
|
*manual.cfd->GetLatestMutableCFOptions(), manual.input_level,
|
|
|
|
manual.output_level, manual.output_path_id, manual.begin,
|
|
|
|
manual.end, &manual.manual_end, &manual_conflict)) == nullptr &&
|
|
|
|
manual_conflict))) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// exclusive manual compactions should not see a conflict during
|
|
|
|
// CompactRange
|
|
|
|
assert(!exclusive || !manual_conflict);
|
|
|
|
// Running either this or some other manual compaction
|
|
|
|
bg_cv_.Wait();
|
|
|
|
if (scheduled && manual.incomplete == true) {
|
|
|
|
assert(!manual.in_progress);
|
|
|
|
scheduled = false;
|
|
|
|
manual.incomplete = false;
|
|
|
|
}
|
|
|
|
} else if (!scheduled) {
|
2017-08-04 00:36:28 +02:00
|
|
|
if (compaction == nullptr) {
|
2017-04-06 02:14:05 +02:00
|
|
|
manual.done = true;
|
|
|
|
bg_cv_.SignalAll();
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
ca = new CompactionArg;
|
|
|
|
ca->db = this;
|
2017-08-04 00:36:28 +02:00
|
|
|
ca->prepicked_compaction = new PrepickedCompaction;
|
|
|
|
ca->prepicked_compaction->manual_compaction_state = &manual;
|
|
|
|
ca->prepicked_compaction->compaction = compaction;
|
2017-04-06 02:14:05 +02:00
|
|
|
manual.incomplete = false;
|
|
|
|
bg_compaction_scheduled_++;
|
|
|
|
env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
|
|
|
|
&DBImpl::UnscheduleCallback);
|
|
|
|
scheduled = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
assert(!manual.in_progress);
|
|
|
|
assert(HasPendingManualCompaction());
|
|
|
|
RemoveManualCompaction(&manual);
|
|
|
|
bg_cv_.SignalAll();
|
|
|
|
return manual.status;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
|
|
|
|
const FlushOptions& flush_options,
|
2018-02-09 21:09:55 +01:00
|
|
|
FlushReason flush_reason, bool writes_stopped) {
|
2017-04-06 02:14:05 +02:00
|
|
|
Status s;
|
2018-01-19 02:32:50 +01:00
|
|
|
uint64_t flush_memtable_id = 0;
|
2017-04-06 02:14:05 +02:00
|
|
|
{
|
|
|
|
WriteContext context;
|
|
|
|
InstrumentedMutexLock guard_lock(&mutex_);
|
|
|
|
|
2017-11-02 01:23:52 +01:00
|
|
|
if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty() &&
|
|
|
|
cached_recoverable_state_empty_.load()) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// Nothing to flush
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
WriteThread::Writer w;
|
|
|
|
if (!writes_stopped) {
|
|
|
|
write_thread_.EnterUnbatched(&w, &mutex_);
|
|
|
|
}
|
|
|
|
|
2017-11-02 01:23:52 +01:00
|
|
|
// SwitchMemtable() will release and reacquire mutex during execution
|
2017-04-06 02:14:05 +02:00
|
|
|
s = SwitchMemtable(cfd, &context);
|
2018-01-19 02:32:50 +01:00
|
|
|
flush_memtable_id = cfd->imm()->GetLatestMemTableID();
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
if (!writes_stopped) {
|
|
|
|
write_thread_.ExitUnbatched(&w);
|
|
|
|
}
|
|
|
|
|
|
|
|
cfd->imm()->FlushRequested();
|
|
|
|
|
|
|
|
// schedule flush
|
2018-02-09 21:09:55 +01:00
|
|
|
SchedulePendingFlush(cfd, flush_reason);
|
2017-04-06 02:14:05 +02:00
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (s.ok() && flush_options.wait) {
|
|
|
|
// Wait until the compaction completes
|
2018-01-19 02:32:50 +01:00
|
|
|
s = WaitForFlushMemTable(cfd, &flush_memtable_id);
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2018-01-19 02:32:50 +01:00
|
|
|
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd,
|
|
|
|
const uint64_t* flush_memtable_id) {
|
2017-04-06 02:14:05 +02:00
|
|
|
Status s;
|
|
|
|
// Wait until the compaction completes
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
2018-01-19 02:32:50 +01:00
|
|
|
while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok() &&
|
|
|
|
(flush_memtable_id == nullptr ||
|
|
|
|
cfd->imm()->GetEarliestMemTableID() <= *flush_memtable_id)) {
|
2017-04-06 02:14:05 +02:00
|
|
|
if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
return Status::ShutdownInProgress();
|
|
|
|
}
|
|
|
|
if (cfd->IsDropped()) {
|
|
|
|
// FlushJob cannot flush a dropped CF, if we did not break here
|
|
|
|
// we will loop forever since cfd->imm()->NumNotFlushed() will never
|
|
|
|
// drop to zero
|
|
|
|
return Status::InvalidArgument("Cannot flush a dropped CF");
|
|
|
|
}
|
|
|
|
bg_cv_.Wait();
|
|
|
|
}
|
|
|
|
if (!bg_error_.ok()) {
|
|
|
|
s = bg_error_;
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::EnableAutoCompaction(
|
|
|
|
const std::vector<ColumnFamilyHandle*>& column_family_handles) {
|
|
|
|
Status s;
|
|
|
|
for (auto cf_ptr : column_family_handles) {
|
|
|
|
Status status =
|
|
|
|
this->SetOptions(cf_ptr, {{"disable_auto_compactions", "false"}});
|
|
|
|
if (!status.ok()) {
|
|
|
|
s = status;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::MaybeScheduleFlushOrCompaction() {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
if (!opened_successfully_) {
|
|
|
|
// Compaction may introduce data race to DB open
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
if (bg_work_paused_ > 0) {
|
|
|
|
// we paused the background work
|
|
|
|
return;
|
|
|
|
} else if (shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
// DB is being deleted; no more background compactions
|
|
|
|
return;
|
|
|
|
}
|
2017-05-24 20:25:38 +02:00
|
|
|
auto bg_job_limits = GetBGJobLimits();
|
2017-05-23 20:04:25 +02:00
|
|
|
bool is_flush_pool_empty =
|
|
|
|
env_->GetBackgroundThreads(Env::Priority::HIGH) == 0;
|
|
|
|
while (!is_flush_pool_empty && unscheduled_flushes_ > 0 &&
|
2017-05-24 20:25:38 +02:00
|
|
|
bg_flush_scheduled_ < bg_job_limits.max_flushes) {
|
2017-04-06 02:14:05 +02:00
|
|
|
unscheduled_flushes_--;
|
|
|
|
bg_flush_scheduled_++;
|
|
|
|
env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
|
|
|
|
}
|
|
|
|
|
2017-05-23 20:04:25 +02:00
|
|
|
// special case -- if high-pri (flush) thread pool is empty, then schedule
|
|
|
|
// flushes in low-pri (compaction) thread pool.
|
|
|
|
if (is_flush_pool_empty) {
|
2017-04-06 02:14:05 +02:00
|
|
|
while (unscheduled_flushes_ > 0 &&
|
|
|
|
bg_flush_scheduled_ + bg_compaction_scheduled_ <
|
2017-05-24 20:25:38 +02:00
|
|
|
bg_job_limits.max_flushes) {
|
2017-04-06 02:14:05 +02:00
|
|
|
unscheduled_flushes_--;
|
|
|
|
bg_flush_scheduled_++;
|
|
|
|
env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::LOW, this);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (bg_compaction_paused_ > 0) {
|
|
|
|
// we paused the background compaction
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (HasExclusiveManualCompaction()) {
|
|
|
|
// only manual compactions are allowed to run. don't schedule automatic
|
|
|
|
// compactions
|
2018-01-17 07:56:47 +01:00
|
|
|
TEST_SYNC_POINT("DBImpl::MaybeScheduleFlushOrCompaction:Conflict");
|
2017-04-06 02:14:05 +02:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2017-05-24 20:25:38 +02:00
|
|
|
while (bg_compaction_scheduled_ < bg_job_limits.max_compactions &&
|
2017-04-06 02:14:05 +02:00
|
|
|
unscheduled_compactions_ > 0) {
|
|
|
|
CompactionArg* ca = new CompactionArg;
|
|
|
|
ca->db = this;
|
2017-08-04 00:36:28 +02:00
|
|
|
ca->prepicked_compaction = nullptr;
|
2017-04-06 02:14:05 +02:00
|
|
|
bg_compaction_scheduled_++;
|
|
|
|
unscheduled_compactions_--;
|
|
|
|
env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
|
|
|
|
&DBImpl::UnscheduleCallback);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-24 20:25:38 +02:00
|
|
|
DBImpl::BGJobLimits DBImpl::GetBGJobLimits() const {
|
2017-04-06 02:14:05 +02:00
|
|
|
mutex_.AssertHeld();
|
2017-05-24 20:25:38 +02:00
|
|
|
return GetBGJobLimits(immutable_db_options_.max_background_flushes,
|
|
|
|
mutable_db_options_.max_background_compactions,
|
|
|
|
mutable_db_options_.max_background_jobs,
|
|
|
|
write_controller_.NeedSpeedupCompaction());
|
|
|
|
}
|
|
|
|
|
|
|
|
DBImpl::BGJobLimits DBImpl::GetBGJobLimits(int max_background_flushes,
|
|
|
|
int max_background_compactions,
|
|
|
|
int max_background_jobs,
|
|
|
|
bool parallelize_compactions) {
|
|
|
|
BGJobLimits res;
|
|
|
|
if (max_background_flushes == -1 && max_background_compactions == -1) {
|
|
|
|
// for our first stab implementing max_background_jobs, simply allocate a
|
|
|
|
// quarter of the threads to flushes.
|
|
|
|
res.max_flushes = std::max(1, max_background_jobs / 4);
|
|
|
|
res.max_compactions = std::max(1, max_background_jobs - res.max_flushes);
|
2017-04-06 02:14:05 +02:00
|
|
|
} else {
|
2017-05-24 20:25:38 +02:00
|
|
|
// compatibility code in case users haven't migrated to max_background_jobs,
|
|
|
|
// which automatically computes flush/compaction limits
|
|
|
|
res.max_flushes = std::max(1, max_background_flushes);
|
|
|
|
res.max_compactions = std::max(1, max_background_compactions);
|
|
|
|
}
|
|
|
|
if (!parallelize_compactions) {
|
|
|
|
// throttle background compactions until we deem necessary
|
|
|
|
res.max_compactions = 1;
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
2017-05-24 20:25:38 +02:00
|
|
|
return res;
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::AddToCompactionQueue(ColumnFamilyData* cfd) {
|
|
|
|
assert(!cfd->pending_compaction());
|
|
|
|
cfd->Ref();
|
|
|
|
compaction_queue_.push_back(cfd);
|
|
|
|
cfd->set_pending_compaction(true);
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* DBImpl::PopFirstFromCompactionQueue() {
|
|
|
|
assert(!compaction_queue_.empty());
|
|
|
|
auto cfd = *compaction_queue_.begin();
|
|
|
|
compaction_queue_.pop_front();
|
|
|
|
assert(cfd->pending_compaction());
|
|
|
|
cfd->set_pending_compaction(false);
|
|
|
|
return cfd;
|
|
|
|
}
|
|
|
|
|
2018-02-09 21:09:55 +01:00
|
|
|
void DBImpl::AddToFlushQueue(ColumnFamilyData* cfd, FlushReason flush_reason) {
|
2017-04-06 02:14:05 +02:00
|
|
|
assert(!cfd->pending_flush());
|
|
|
|
cfd->Ref();
|
|
|
|
flush_queue_.push_back(cfd);
|
|
|
|
cfd->set_pending_flush(true);
|
2018-02-09 21:09:55 +01:00
|
|
|
cfd->SetFlushReason(flush_reason);
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* DBImpl::PopFirstFromFlushQueue() {
|
|
|
|
assert(!flush_queue_.empty());
|
|
|
|
auto cfd = *flush_queue_.begin();
|
|
|
|
flush_queue_.pop_front();
|
|
|
|
assert(cfd->pending_flush());
|
|
|
|
cfd->set_pending_flush(false);
|
2018-02-09 21:09:55 +01:00
|
|
|
// TODO: need to unset flush reason?
|
2017-04-06 02:14:05 +02:00
|
|
|
return cfd;
|
|
|
|
}
|
|
|
|
|
2018-02-09 21:09:55 +01:00
|
|
|
void DBImpl::SchedulePendingFlush(ColumnFamilyData* cfd,
|
|
|
|
FlushReason flush_reason) {
|
2017-04-06 02:14:05 +02:00
|
|
|
if (!cfd->pending_flush() && cfd->imm()->IsFlushPending()) {
|
2018-02-09 21:09:55 +01:00
|
|
|
AddToFlushQueue(cfd, flush_reason);
|
2017-04-06 02:14:05 +02:00
|
|
|
++unscheduled_flushes_;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
|
|
|
|
if (!cfd->pending_compaction() && cfd->NeedsCompaction()) {
|
|
|
|
AddToCompactionQueue(cfd);
|
|
|
|
++unscheduled_compactions_;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::SchedulePendingPurge(std::string fname, FileType type,
|
|
|
|
uint64_t number, uint32_t path_id,
|
|
|
|
int job_id) {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
PurgeFileInfo file_info(fname, type, number, path_id, job_id);
|
|
|
|
purge_queue_.push_back(std::move(file_info));
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::BGWorkFlush(void* db) {
|
|
|
|
IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkFlush");
|
|
|
|
reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::BGWorkCompaction(void* arg) {
|
|
|
|
CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
|
|
|
|
delete reinterpret_cast<CompactionArg*>(arg);
|
|
|
|
IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
|
2017-08-04 00:36:28 +02:00
|
|
|
auto prepicked_compaction =
|
|
|
|
static_cast<PrepickedCompaction*>(ca.prepicked_compaction);
|
|
|
|
reinterpret_cast<DBImpl*>(ca.db)->BackgroundCallCompaction(
|
|
|
|
prepicked_compaction, Env::Priority::LOW);
|
|
|
|
delete prepicked_compaction;
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::BGWorkBottomCompaction(void* arg) {
|
|
|
|
CompactionArg ca = *(static_cast<CompactionArg*>(arg));
|
|
|
|
delete static_cast<CompactionArg*>(arg);
|
|
|
|
IOSTATS_SET_THREAD_POOL_ID(Env::Priority::BOTTOM);
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkBottomCompaction");
|
|
|
|
auto* prepicked_compaction = ca.prepicked_compaction;
|
|
|
|
assert(prepicked_compaction && prepicked_compaction->compaction &&
|
|
|
|
!prepicked_compaction->manual_compaction_state);
|
|
|
|
ca.db->BackgroundCallCompaction(prepicked_compaction, Env::Priority::BOTTOM);
|
|
|
|
delete prepicked_compaction;
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::BGWorkPurge(void* db) {
|
|
|
|
IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkPurge:start");
|
|
|
|
reinterpret_cast<DBImpl*>(db)->BackgroundCallPurge();
|
|
|
|
TEST_SYNC_POINT("DBImpl::BGWorkPurge:end");
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::UnscheduleCallback(void* arg) {
|
|
|
|
CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
|
|
|
|
delete reinterpret_cast<CompactionArg*>(arg);
|
2017-08-04 00:36:28 +02:00
|
|
|
if (ca.prepicked_compaction != nullptr) {
|
|
|
|
if (ca.prepicked_compaction->compaction != nullptr) {
|
|
|
|
delete ca.prepicked_compaction->compaction;
|
|
|
|
}
|
|
|
|
delete ca.prepicked_compaction;
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
TEST_SYNC_POINT("DBImpl::UnscheduleCallback");
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
|
|
|
|
LogBuffer* log_buffer) {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
|
|
|
|
Status status = bg_error_;
|
|
|
|
if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
status = Status::ShutdownInProgress();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!status.ok()) {
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* cfd = nullptr;
|
|
|
|
while (!flush_queue_.empty()) {
|
|
|
|
// This cfd is already referenced
|
|
|
|
auto first_cfd = PopFirstFromFlushQueue();
|
|
|
|
|
|
|
|
if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
|
|
|
|
// can't flush this CF, try next one
|
|
|
|
if (first_cfd->Unref()) {
|
|
|
|
delete first_cfd;
|
|
|
|
}
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
// found a flush!
|
|
|
|
cfd = first_cfd;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (cfd != nullptr) {
|
|
|
|
const MutableCFOptions mutable_cf_options =
|
|
|
|
*cfd->GetLatestMutableCFOptions();
|
2017-05-24 20:25:38 +02:00
|
|
|
auto bg_job_limits = GetBGJobLimits();
|
2017-04-06 02:14:05 +02:00
|
|
|
ROCKS_LOG_BUFFER(
|
|
|
|
log_buffer,
|
|
|
|
"Calling FlushMemTableToOutputFile with column "
|
2017-05-24 20:25:38 +02:00
|
|
|
"family [%s], flush slots available %d, compaction slots available %d, "
|
|
|
|
"flush slots scheduled %d, compaction slots scheduled %d",
|
|
|
|
cfd->GetName().c_str(), bg_job_limits.max_flushes,
|
|
|
|
bg_job_limits.max_compactions, bg_flush_scheduled_,
|
|
|
|
bg_compaction_scheduled_);
|
2017-04-06 02:14:05 +02:00
|
|
|
status = FlushMemTableToOutputFile(cfd, mutable_cf_options, made_progress,
|
|
|
|
job_context, log_buffer);
|
|
|
|
if (cfd->Unref()) {
|
|
|
|
delete cfd;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
void DBImpl::BackgroundCallFlush() {
|
|
|
|
bool made_progress = false;
|
|
|
|
JobContext job_context(next_job_id_.fetch_add(1), true);
|
|
|
|
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:start");
|
|
|
|
|
|
|
|
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
|
|
|
|
immutable_db_options_.info_log.get());
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
2017-06-21 01:43:09 +02:00
|
|
|
assert(bg_flush_scheduled_);
|
2017-04-06 02:14:05 +02:00
|
|
|
num_running_flushes_++;
|
|
|
|
|
|
|
|
auto pending_outputs_inserted_elem =
|
|
|
|
CaptureCurrentFileNumberInPendingOutputs();
|
|
|
|
|
|
|
|
Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer);
|
|
|
|
if (!s.ok() && !s.IsShutdownInProgress()) {
|
|
|
|
// Wait a little bit before retrying background flush in
|
|
|
|
// case this is an environmental problem and we do not want to
|
|
|
|
// chew up resources for failed flushes for the duration of
|
|
|
|
// the problem.
|
|
|
|
uint64_t error_cnt =
|
|
|
|
default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
|
|
|
|
bg_cv_.SignalAll(); // In case a waiter can proceed despite the error
|
|
|
|
mutex_.Unlock();
|
|
|
|
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
|
|
|
|
"Waiting after background flush error: %s"
|
|
|
|
"Accumulated background error counts: %" PRIu64,
|
|
|
|
s.ToString().c_str(), error_cnt);
|
|
|
|
log_buffer.FlushBufferToLog();
|
|
|
|
LogFlush(immutable_db_options_.info_log);
|
|
|
|
env_->SleepForMicroseconds(1000000);
|
|
|
|
mutex_.Lock();
|
|
|
|
}
|
|
|
|
|
|
|
|
ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
|
|
|
|
|
|
|
|
// If flush failed, we want to delete all temporary files that we might have
|
|
|
|
// created. Thus, we force full scan in FindObsoleteFiles()
|
|
|
|
FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
|
|
|
|
// delete unnecessary files if any, this is done outside the mutex
|
2018-01-12 22:16:39 +01:00
|
|
|
if (job_context.HaveSomethingToClean() ||
|
|
|
|
job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
|
2017-04-06 02:14:05 +02:00
|
|
|
mutex_.Unlock();
|
2018-01-18 02:37:10 +01:00
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FilesFound");
|
2017-04-06 02:14:05 +02:00
|
|
|
// Have to flush the info logs before bg_flush_scheduled_--
|
|
|
|
// because if bg_flush_scheduled_ becomes 0 and the lock is
|
|
|
|
// released, the deconstructor of DB can kick in and destroy all the
|
|
|
|
// states of DB so info_log might not be available after that point.
|
|
|
|
// It also applies to access other states that DB owns.
|
|
|
|
log_buffer.FlushBufferToLog();
|
|
|
|
if (job_context.HaveSomethingToDelete()) {
|
|
|
|
PurgeObsoleteFiles(job_context);
|
|
|
|
}
|
|
|
|
job_context.Clean();
|
|
|
|
mutex_.Lock();
|
|
|
|
}
|
|
|
|
|
|
|
|
assert(num_running_flushes_ > 0);
|
|
|
|
num_running_flushes_--;
|
|
|
|
bg_flush_scheduled_--;
|
|
|
|
// See if there's more work to be done
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
bg_cv_.SignalAll();
|
|
|
|
// IMPORTANT: there should be no code after calling SignalAll. This call may
|
|
|
|
// signal the DB destructor that it's OK to proceed with destruction. In
|
|
|
|
// that case, all DB variables will be dealloacated and referencing them
|
|
|
|
// will cause trouble.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction,
|
|
|
|
Env::Priority bg_thread_pri) {
|
2017-04-06 02:14:05 +02:00
|
|
|
bool made_progress = false;
|
|
|
|
JobContext job_context(next_job_id_.fetch_add(1), true);
|
|
|
|
TEST_SYNC_POINT("BackgroundCallCompaction:0");
|
|
|
|
MaybeDumpStats();
|
|
|
|
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
|
|
|
|
immutable_db_options_.info_log.get());
|
|
|
|
{
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
|
|
|
|
// This call will unlock/lock the mutex to wait for current running
|
|
|
|
// IngestExternalFile() calls to finish.
|
|
|
|
WaitForIngestFile();
|
|
|
|
|
|
|
|
num_running_compactions_++;
|
|
|
|
|
|
|
|
auto pending_outputs_inserted_elem =
|
|
|
|
CaptureCurrentFileNumberInPendingOutputs();
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
assert((bg_thread_pri == Env::Priority::BOTTOM &&
|
|
|
|
bg_bottom_compaction_scheduled_) ||
|
|
|
|
(bg_thread_pri == Env::Priority::LOW && bg_compaction_scheduled_));
|
|
|
|
Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer,
|
|
|
|
prepicked_compaction);
|
2017-04-06 02:14:05 +02:00
|
|
|
TEST_SYNC_POINT("BackgroundCallCompaction:1");
|
|
|
|
if (!s.ok() && !s.IsShutdownInProgress()) {
|
|
|
|
// Wait a little bit before retrying background compaction in
|
|
|
|
// case this is an environmental problem and we do not want to
|
|
|
|
// chew up resources for failed compactions for the duration of
|
|
|
|
// the problem.
|
|
|
|
uint64_t error_cnt =
|
|
|
|
default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
|
|
|
|
bg_cv_.SignalAll(); // In case a waiter can proceed despite the error
|
|
|
|
mutex_.Unlock();
|
|
|
|
log_buffer.FlushBufferToLog();
|
|
|
|
ROCKS_LOG_ERROR(immutable_db_options_.info_log,
|
|
|
|
"Waiting after background compaction error: %s, "
|
|
|
|
"Accumulated background error counts: %" PRIu64,
|
|
|
|
s.ToString().c_str(), error_cnt);
|
|
|
|
LogFlush(immutable_db_options_.info_log);
|
|
|
|
env_->SleepForMicroseconds(1000000);
|
|
|
|
mutex_.Lock();
|
|
|
|
}
|
|
|
|
|
|
|
|
ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
|
|
|
|
|
|
|
|
// If compaction failed, we want to delete all temporary files that we might
|
|
|
|
// have created (they might not be all recorded in job_context in case of a
|
|
|
|
// failure). Thus, we force full scan in FindObsoleteFiles()
|
|
|
|
FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
|
2018-03-28 19:23:31 +02:00
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:FoundObsoleteFiles");
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
// delete unnecessary files if any, this is done outside the mutex
|
2018-01-12 22:16:39 +01:00
|
|
|
if (job_context.HaveSomethingToClean() ||
|
|
|
|
job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
|
2017-04-06 02:14:05 +02:00
|
|
|
mutex_.Unlock();
|
|
|
|
// Have to flush the info logs before bg_compaction_scheduled_--
|
|
|
|
// because if bg_flush_scheduled_ becomes 0 and the lock is
|
|
|
|
// released, the deconstructor of DB can kick in and destroy all the
|
|
|
|
// states of DB so info_log might not be available after that point.
|
|
|
|
// It also applies to access other states that DB owns.
|
|
|
|
log_buffer.FlushBufferToLog();
|
|
|
|
if (job_context.HaveSomethingToDelete()) {
|
|
|
|
PurgeObsoleteFiles(job_context);
|
2018-03-28 19:23:31 +02:00
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles");
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
job_context.Clean();
|
|
|
|
mutex_.Lock();
|
|
|
|
}
|
|
|
|
|
|
|
|
assert(num_running_compactions_ > 0);
|
|
|
|
num_running_compactions_--;
|
2017-08-04 00:36:28 +02:00
|
|
|
if (bg_thread_pri == Env::Priority::LOW) {
|
|
|
|
bg_compaction_scheduled_--;
|
|
|
|
} else {
|
|
|
|
assert(bg_thread_pri == Env::Priority::BOTTOM);
|
|
|
|
bg_bottom_compaction_scheduled_--;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
|
|
|
|
|
|
|
|
// See if there's more work to be done
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
2017-08-04 00:36:28 +02:00
|
|
|
if (made_progress ||
|
|
|
|
(bg_compaction_scheduled_ == 0 &&
|
|
|
|
bg_bottom_compaction_scheduled_ == 0) ||
|
2018-03-07 01:13:05 +01:00
|
|
|
HasPendingManualCompaction() || unscheduled_compactions_ == 0) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// signal if
|
|
|
|
// * made_progress -- need to wakeup DelayWrite
|
2017-08-04 00:36:28 +02:00
|
|
|
// * bg_{bottom,}_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
|
2017-04-06 02:14:05 +02:00
|
|
|
// * HasPendingManualCompaction -- need to wakeup RunManualCompaction
|
|
|
|
// If none of this is true, there is no need to signal since nobody is
|
|
|
|
// waiting for it
|
|
|
|
bg_cv_.SignalAll();
|
|
|
|
}
|
|
|
|
// IMPORTANT: there should be no code after calling SignalAll. This call may
|
|
|
|
// signal the DB destructor that it's OK to proceed with destruction. In
|
|
|
|
// that case, all DB variables will be dealloacated and referencing them
|
|
|
|
// will cause trouble.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DBImpl::BackgroundCompaction(bool* made_progress,
|
|
|
|
JobContext* job_context,
|
2017-08-04 00:36:28 +02:00
|
|
|
LogBuffer* log_buffer,
|
|
|
|
PrepickedCompaction* prepicked_compaction) {
|
|
|
|
ManualCompactionState* manual_compaction =
|
|
|
|
prepicked_compaction == nullptr
|
|
|
|
? nullptr
|
|
|
|
: prepicked_compaction->manual_compaction_state;
|
2017-04-06 02:14:05 +02:00
|
|
|
*made_progress = false;
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Start");
|
|
|
|
|
|
|
|
bool is_manual = (manual_compaction != nullptr);
|
2017-08-04 00:36:28 +02:00
|
|
|
unique_ptr<Compaction> c;
|
|
|
|
if (prepicked_compaction != nullptr &&
|
|
|
|
prepicked_compaction->compaction != nullptr) {
|
|
|
|
c.reset(prepicked_compaction->compaction);
|
|
|
|
}
|
|
|
|
bool is_prepicked = is_manual || c;
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
// (manual_compaction->in_progress == false);
|
|
|
|
bool trivial_move_disallowed =
|
|
|
|
is_manual && manual_compaction->disallow_trivial_move;
|
|
|
|
|
|
|
|
CompactionJobStats compaction_job_stats;
|
|
|
|
Status status = bg_error_;
|
|
|
|
if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
|
|
|
|
status = Status::ShutdownInProgress();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!status.ok()) {
|
|
|
|
if (is_manual) {
|
|
|
|
manual_compaction->status = status;
|
|
|
|
manual_compaction->done = true;
|
|
|
|
manual_compaction->in_progress = false;
|
|
|
|
manual_compaction = nullptr;
|
|
|
|
}
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (is_manual) {
|
|
|
|
// another thread cannot pick up the same work
|
|
|
|
manual_compaction->in_progress = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
// InternalKey manual_end_storage;
|
|
|
|
// InternalKey* manual_end = &manual_end_storage;
|
2018-03-07 01:13:05 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
bool sfm_bookkeeping = false;
|
|
|
|
#endif // ROCKSDB_LITE
|
2017-04-06 02:14:05 +02:00
|
|
|
if (is_manual) {
|
2017-08-04 00:36:28 +02:00
|
|
|
ManualCompactionState* m = manual_compaction;
|
2017-04-06 02:14:05 +02:00
|
|
|
assert(m->in_progress);
|
|
|
|
if (!c) {
|
|
|
|
m->done = true;
|
|
|
|
m->manual_end = nullptr;
|
|
|
|
ROCKS_LOG_BUFFER(log_buffer,
|
|
|
|
"[%s] Manual compaction from level-%d from %s .. "
|
|
|
|
"%s; nothing to do\n",
|
|
|
|
m->cfd->GetName().c_str(), m->input_level,
|
|
|
|
(m->begin ? m->begin->DebugString().c_str() : "(begin)"),
|
|
|
|
(m->end ? m->end->DebugString().c_str() : "(end)"));
|
|
|
|
} else {
|
|
|
|
ROCKS_LOG_BUFFER(
|
|
|
|
log_buffer,
|
|
|
|
"[%s] Manual compaction from level-%d to level-%d from %s .. "
|
|
|
|
"%s; will stop at %s\n",
|
|
|
|
m->cfd->GetName().c_str(), m->input_level, c->output_level(),
|
|
|
|
(m->begin ? m->begin->DebugString().c_str() : "(begin)"),
|
|
|
|
(m->end ? m->end->DebugString().c_str() : "(end)"),
|
|
|
|
((m->done || m->manual_end == nullptr)
|
|
|
|
? "(end)"
|
|
|
|
: m->manual_end->DebugString().c_str()));
|
|
|
|
}
|
2017-08-04 00:36:28 +02:00
|
|
|
} else if (!is_prepicked && !compaction_queue_.empty()) {
|
2018-01-16 22:10:34 +01:00
|
|
|
if (HasExclusiveManualCompaction()) {
|
2017-05-03 00:01:07 +02:00
|
|
|
// Can't compact right now, but try again later
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction()::Conflict");
|
|
|
|
|
2017-05-18 08:03:54 +02:00
|
|
|
// Stay in the compaction queue.
|
2017-05-03 00:01:07 +02:00
|
|
|
unscheduled_compactions_++;
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
// cfd is referenced here
|
|
|
|
auto cfd = PopFirstFromCompactionQueue();
|
|
|
|
// We unreference here because the following code will take a Ref() on
|
|
|
|
// this cfd if it is going to use it (Compaction class holds a
|
|
|
|
// reference).
|
|
|
|
// This will all happen under a mutex so we don't have to be afraid of
|
|
|
|
// somebody else deleting it.
|
|
|
|
if (cfd->Unref()) {
|
|
|
|
delete cfd;
|
|
|
|
// This was the last reference of the column family, so no need to
|
|
|
|
// compact.
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Pick up latest mutable CF Options and use it throughout the
|
|
|
|
// compaction job
|
|
|
|
// Compaction makes a copy of the latest MutableCFOptions. It should be used
|
|
|
|
// throughout the compaction procedure to make sure consistency. It will
|
|
|
|
// eventually be installed into SuperVersion
|
|
|
|
auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
|
|
|
|
if (!mutable_cf_options->disable_auto_compactions && !cfd->IsDropped()) {
|
|
|
|
// NOTE: try to avoid unnecessary copy of MutableCFOptions if
|
|
|
|
// compaction is not necessary. Need to make sure mutex is held
|
|
|
|
// until we make a copy in the following code
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction():BeforePickCompaction");
|
|
|
|
c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction():AfterPickCompaction");
|
2018-03-07 01:13:05 +01:00
|
|
|
|
|
|
|
bool enough_room = true;
|
2017-04-06 02:14:05 +02:00
|
|
|
if (c != nullptr) {
|
2018-03-07 01:13:05 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
auto sfm = static_cast<SstFileManagerImpl*>(
|
|
|
|
immutable_db_options_.sst_file_manager.get());
|
|
|
|
if (sfm) {
|
|
|
|
enough_room = sfm->EnoughRoomForCompaction(c.get());
|
|
|
|
if (enough_room) {
|
|
|
|
sfm_bookkeeping = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
if (!enough_room) {
|
|
|
|
// Just in case tests want to change the value of enough_room
|
|
|
|
TEST_SYNC_POINT_CALLBACK(
|
|
|
|
"DBImpl::BackgroundCompaction():CancelledCompaction",
|
|
|
|
&enough_room);
|
|
|
|
}
|
|
|
|
if (!enough_room) {
|
|
|
|
// Then don't do the compaction
|
|
|
|
c->ReleaseCompactionFiles(status);
|
|
|
|
c->column_family_data()
|
|
|
|
->current()
|
|
|
|
->storage_info()
|
|
|
|
->ComputeCompactionScore(*(c->immutable_cf_options()),
|
|
|
|
*(c->mutable_cf_options()));
|
|
|
|
|
|
|
|
ROCKS_LOG_BUFFER(log_buffer,
|
|
|
|
"Cancelled compaction because not enough room");
|
2017-04-06 02:14:05 +02:00
|
|
|
AddToCompactionQueue(cfd);
|
|
|
|
++unscheduled_compactions_;
|
2018-03-07 01:13:05 +01:00
|
|
|
|
|
|
|
c.reset();
|
|
|
|
// Don't need to sleep here, because BackgroundCallCompaction
|
|
|
|
// will sleep if !s.ok()
|
|
|
|
status = Status::CompactionTooLarge();
|
2018-03-08 19:39:15 +01:00
|
|
|
RecordTick(stats_, COMPACTION_CANCELLED, 1);
|
2018-03-07 01:13:05 +01:00
|
|
|
} else {
|
|
|
|
// update statistics
|
|
|
|
MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
|
|
|
|
c->inputs(0)->size());
|
|
|
|
// There are three things that can change compaction score:
|
|
|
|
// 1) When flush or compaction finish. This case is covered by
|
|
|
|
// InstallSuperVersionAndScheduleWork
|
|
|
|
// 2) When MutableCFOptions changes. This case is also covered by
|
|
|
|
// InstallSuperVersionAndScheduleWork, because this is when the new
|
|
|
|
// options take effect.
|
|
|
|
// 3) When we Pick a new compaction, we "remove" those files being
|
|
|
|
// compacted from the calculation, which then influences compaction
|
|
|
|
// score. Here we check if we need the new compaction even without the
|
|
|
|
// files that are currently being compacted. If we need another
|
|
|
|
// compaction, we might be able to execute it in parallel, so we add
|
|
|
|
// it to the queue and schedule a new thread.
|
|
|
|
if (cfd->NeedsCompaction()) {
|
|
|
|
// Yes, we need more compactions!
|
|
|
|
AddToCompactionQueue(cfd);
|
|
|
|
++unscheduled_compactions_;
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!c) {
|
|
|
|
// Nothing to do
|
|
|
|
ROCKS_LOG_BUFFER(log_buffer, "Compaction nothing to do");
|
|
|
|
} else if (c->deletion_compaction()) {
|
|
|
|
// TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
|
|
|
|
// file if there is alive snapshot pointing to it
|
|
|
|
assert(c->num_input_files(1) == 0);
|
|
|
|
assert(c->level() == 0);
|
|
|
|
assert(c->column_family_data()->ioptions()->compaction_style ==
|
|
|
|
kCompactionStyleFIFO);
|
|
|
|
|
|
|
|
compaction_job_stats.num_input_files = c->num_input_files(0);
|
|
|
|
|
|
|
|
for (const auto& f : *c->inputs(0)) {
|
|
|
|
c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
|
|
|
|
}
|
|
|
|
status = versions_->LogAndApply(c->column_family_data(),
|
|
|
|
*c->mutable_cf_options(), c->edit(),
|
|
|
|
&mutex_, directories_.GetDbDir());
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(
|
|
|
|
c->column_family_data(), &job_context->superversion_context,
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
*c->mutable_cf_options(), FlushReason::kAutoCompaction);
|
2017-04-06 02:14:05 +02:00
|
|
|
ROCKS_LOG_BUFFER(log_buffer, "[%s] Deleted %d files\n",
|
|
|
|
c->column_family_data()->GetName().c_str(),
|
|
|
|
c->num_input_files(0));
|
|
|
|
*made_progress = true;
|
|
|
|
} else if (!trivial_move_disallowed && c->IsTrivialMove()) {
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
|
|
|
|
// Instrument for event update
|
|
|
|
// TODO(yhchiang): add op details for showing trivial-move.
|
|
|
|
ThreadStatusUtil::SetColumnFamily(
|
|
|
|
c->column_family_data(), c->column_family_data()->ioptions()->env,
|
|
|
|
immutable_db_options_.enable_thread_tracking);
|
|
|
|
ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);
|
|
|
|
|
|
|
|
compaction_job_stats.num_input_files = c->num_input_files(0);
|
|
|
|
|
|
|
|
// Move files to next level
|
|
|
|
int32_t moved_files = 0;
|
|
|
|
int64_t moved_bytes = 0;
|
|
|
|
for (unsigned int l = 0; l < c->num_input_levels(); l++) {
|
|
|
|
if (c->level(l) == c->output_level()) {
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
for (size_t i = 0; i < c->num_input_files(l); i++) {
|
|
|
|
FileMetaData* f = c->input(l, i);
|
|
|
|
c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
|
|
|
|
c->edit()->AddFile(c->output_level(), f->fd.GetNumber(),
|
|
|
|
f->fd.GetPathId(), f->fd.GetFileSize(), f->smallest,
|
|
|
|
f->largest, f->smallest_seqno, f->largest_seqno,
|
|
|
|
f->marked_for_compaction);
|
|
|
|
|
|
|
|
ROCKS_LOG_BUFFER(log_buffer, "[%s] Moving #%" PRIu64
|
|
|
|
" to level-%d %" PRIu64 " bytes\n",
|
|
|
|
c->column_family_data()->GetName().c_str(),
|
|
|
|
f->fd.GetNumber(), c->output_level(),
|
|
|
|
f->fd.GetFileSize());
|
|
|
|
++moved_files;
|
|
|
|
moved_bytes += f->fd.GetFileSize();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
status = versions_->LogAndApply(c->column_family_data(),
|
|
|
|
*c->mutable_cf_options(), c->edit(),
|
|
|
|
&mutex_, directories_.GetDbDir());
|
|
|
|
// Use latest MutableCFOptions
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(
|
|
|
|
c->column_family_data(), &job_context->superversion_context,
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
*c->mutable_cf_options(), FlushReason::kAutoCompaction);
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
VersionStorageInfo::LevelSummaryStorage tmp;
|
|
|
|
c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
|
|
|
|
moved_bytes);
|
|
|
|
{
|
|
|
|
event_logger_.LogToBuffer(log_buffer)
|
|
|
|
<< "job" << job_context->job_id << "event"
|
|
|
|
<< "trivial_move"
|
|
|
|
<< "destination_level" << c->output_level() << "files" << moved_files
|
|
|
|
<< "total_files_size" << moved_bytes;
|
|
|
|
}
|
|
|
|
ROCKS_LOG_BUFFER(
|
|
|
|
log_buffer,
|
|
|
|
"[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
|
|
|
|
c->column_family_data()->GetName().c_str(), moved_files,
|
|
|
|
c->output_level(), moved_bytes, status.ToString().c_str(),
|
|
|
|
c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
|
|
|
|
*made_progress = true;
|
|
|
|
|
|
|
|
// Clear Instrument
|
|
|
|
ThreadStatusUtil::ResetThreadStatus();
|
2017-08-04 00:36:28 +02:00
|
|
|
} else if (c->column_family_data()->ioptions()->compaction_style ==
|
|
|
|
kCompactionStyleUniversal &&
|
|
|
|
!is_prepicked && c->output_level() > 0 &&
|
|
|
|
c->output_level() ==
|
|
|
|
c->column_family_data()
|
|
|
|
->current()
|
|
|
|
->storage_info()
|
|
|
|
->MaxOutputLevel(
|
|
|
|
immutable_db_options_.allow_ingest_behind) &&
|
|
|
|
env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) {
|
|
|
|
// Forward universal compactions involving last level to the bottom pool
|
|
|
|
// if it exists, such that long-running compactions can't block short-
|
|
|
|
// lived ones, like L0->L0s.
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction:ForwardToBottomPriPool");
|
|
|
|
CompactionArg* ca = new CompactionArg;
|
|
|
|
ca->db = this;
|
|
|
|
ca->prepicked_compaction = new PrepickedCompaction;
|
|
|
|
ca->prepicked_compaction->compaction = c.release();
|
|
|
|
ca->prepicked_compaction->manual_compaction_state = nullptr;
|
|
|
|
++bg_bottom_compaction_scheduled_;
|
|
|
|
env_->Schedule(&DBImpl::BGWorkBottomCompaction, ca, Env::Priority::BOTTOM,
|
|
|
|
this, &DBImpl::UnscheduleCallback);
|
2017-04-06 02:14:05 +02:00
|
|
|
} else {
|
2018-02-02 21:14:42 +01:00
|
|
|
int output_level __attribute__((__unused__));
|
2017-10-23 23:20:53 +02:00
|
|
|
output_level = c->output_level();
|
2017-04-06 02:14:05 +02:00
|
|
|
TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
|
|
|
|
&output_level);
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot;
|
|
|
|
std::vector<SequenceNumber> snapshot_seqs =
|
|
|
|
snapshots_.GetAll(&earliest_write_conflict_snapshot);
|
|
|
|
|
2017-10-18 18:09:31 +02:00
|
|
|
auto snapshot_checker = snapshot_checker_.get();
|
|
|
|
if (use_custom_gc_ && snapshot_checker == nullptr) {
|
|
|
|
snapshot_checker = DisableGCSnapshotChecker::Instance();
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
assert(is_snapshot_supported_ || snapshots_.empty());
|
|
|
|
CompactionJob compaction_job(
|
2017-09-28 02:37:08 +02:00
|
|
|
job_context->job_id, c.get(), immutable_db_options_,
|
|
|
|
env_options_for_compaction_, versions_.get(), &shutting_down_,
|
Added support for differential snapshots
Summary:
The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2).
This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages.
From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff".
This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR.
For now, what's done here according to initial discussions:
Preserving deletes:
- We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion.
- I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum.
- Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum.
Iterator changes:
- couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum.
TableCache changes:
- I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span.
What's left:
- Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type.
Closes https://github.com/facebook/rocksdb/pull/2999
Differential Revision: D6175602
Pulled By: mikhail-antonov
fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 02:43:29 +01:00
|
|
|
preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
|
2017-04-06 02:14:05 +02:00
|
|
|
directories_.GetDataDir(c->output_path_id()), stats_, &mutex_,
|
2017-10-06 19:26:38 +02:00
|
|
|
&bg_error_, snapshot_seqs, earliest_write_conflict_snapshot,
|
2017-10-18 18:09:31 +02:00
|
|
|
snapshot_checker, table_cache_, &event_logger_,
|
2017-04-06 02:14:05 +02:00
|
|
|
c->mutable_cf_options()->paranoid_file_checks,
|
|
|
|
c->mutable_cf_options()->report_bg_io_stats, dbname_,
|
|
|
|
&compaction_job_stats);
|
|
|
|
compaction_job.Prepare();
|
|
|
|
|
|
|
|
mutex_.Unlock();
|
|
|
|
compaction_job.Run();
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
|
|
|
|
mutex_.Lock();
|
|
|
|
|
|
|
|
status = compaction_job.Install(*c->mutable_cf_options());
|
|
|
|
if (status.ok()) {
|
2017-10-06 03:00:38 +02:00
|
|
|
InstallSuperVersionAndScheduleWork(
|
|
|
|
c->column_family_data(), &job_context->superversion_context,
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
*c->mutable_cf_options(), FlushReason::kAutoCompaction);
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
*made_progress = true;
|
|
|
|
}
|
|
|
|
if (c != nullptr) {
|
|
|
|
c->ReleaseCompactionFiles(status);
|
|
|
|
*made_progress = true;
|
2018-03-07 01:13:05 +01:00
|
|
|
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
// Need to make sure SstFileManager does its bookkeeping
|
|
|
|
auto sfm = static_cast<SstFileManagerImpl*>(
|
|
|
|
immutable_db_options_.sst_file_manager.get());
|
|
|
|
if (sfm && sfm_bookkeeping) {
|
|
|
|
sfm->OnCompactionCompletion(c.get());
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
NotifyOnCompactionCompleted(
|
|
|
|
c->column_family_data(), c.get(), status,
|
|
|
|
compaction_job_stats, job_context->job_id);
|
|
|
|
}
|
|
|
|
// this will unref its input_version and column_family_data
|
|
|
|
c.reset();
|
|
|
|
|
2018-03-07 01:13:05 +01:00
|
|
|
if (status.ok() || status.IsCompactionTooLarge()) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// Done
|
|
|
|
} else if (status.IsShutdownInProgress()) {
|
|
|
|
// Ignore compaction errors found during shutting down
|
|
|
|
} else {
|
|
|
|
ROCKS_LOG_WARN(immutable_db_options_.info_log, "Compaction error: %s",
|
|
|
|
status.ToString().c_str());
|
|
|
|
if (immutable_db_options_.paranoid_checks && bg_error_.ok()) {
|
2017-06-23 04:30:39 +02:00
|
|
|
Status new_bg_error = status;
|
|
|
|
// may temporarily unlock and lock the mutex.
|
|
|
|
EventHelpers::NotifyOnBackgroundError(immutable_db_options_.listeners,
|
|
|
|
BackgroundErrorReason::kCompaction,
|
|
|
|
&new_bg_error, &mutex_);
|
|
|
|
if (!new_bg_error.ok()) {
|
|
|
|
bg_error_ = new_bg_error;
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (is_manual) {
|
2017-08-04 00:36:28 +02:00
|
|
|
ManualCompactionState* m = manual_compaction;
|
2017-04-06 02:14:05 +02:00
|
|
|
if (!status.ok()) {
|
|
|
|
m->status = status;
|
|
|
|
m->done = true;
|
|
|
|
}
|
|
|
|
// For universal compaction:
|
|
|
|
// Because universal compaction always happens at level 0, so one
|
|
|
|
// compaction will pick up all overlapped files. No files will be
|
|
|
|
// filtered out due to size limit and left for a successive compaction.
|
|
|
|
// So we can safely conclude the current compaction.
|
|
|
|
//
|
|
|
|
// Also note that, if we don't stop here, then the current compaction
|
|
|
|
// writes a new file back to level 0, which will be used in successive
|
|
|
|
// compaction. Hence the manual compaction will never finish.
|
|
|
|
//
|
|
|
|
// Stop the compaction if manual_end points to nullptr -- this means
|
|
|
|
// that we compacted the whole range. manual_end should always point
|
|
|
|
// to nullptr in case of universal compaction
|
|
|
|
if (m->manual_end == nullptr) {
|
|
|
|
m->done = true;
|
|
|
|
}
|
|
|
|
if (!m->done) {
|
|
|
|
// We only compacted part of the requested range. Update *m
|
|
|
|
// to the range that is left to be compacted.
|
|
|
|
// Universal and FIFO compactions should always compact the whole range
|
|
|
|
assert(m->cfd->ioptions()->compaction_style !=
|
|
|
|
kCompactionStyleUniversal ||
|
|
|
|
m->cfd->ioptions()->num_levels > 1);
|
|
|
|
assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
|
|
|
|
m->tmp_storage = *m->manual_end;
|
|
|
|
m->begin = &m->tmp_storage;
|
|
|
|
m->incomplete = true;
|
|
|
|
}
|
|
|
|
m->in_progress = false; // not being processed anymore
|
|
|
|
}
|
|
|
|
TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Finish");
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool DBImpl::HasPendingManualCompaction() {
|
|
|
|
return (!manual_compaction_dequeue_.empty());
|
|
|
|
}
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
void DBImpl::AddManualCompaction(DBImpl::ManualCompactionState* m) {
|
2017-04-06 02:14:05 +02:00
|
|
|
manual_compaction_dequeue_.push_back(m);
|
|
|
|
}
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
void DBImpl::RemoveManualCompaction(DBImpl::ManualCompactionState* m) {
|
2017-04-06 02:14:05 +02:00
|
|
|
// Remove from queue
|
2017-08-04 00:36:28 +02:00
|
|
|
std::deque<ManualCompactionState*>::iterator it =
|
2017-04-06 02:14:05 +02:00
|
|
|
manual_compaction_dequeue_.begin();
|
|
|
|
while (it != manual_compaction_dequeue_.end()) {
|
|
|
|
if (m == (*it)) {
|
|
|
|
it = manual_compaction_dequeue_.erase(it);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
it++;
|
|
|
|
}
|
|
|
|
assert(false);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
bool DBImpl::ShouldntRunManualCompaction(ManualCompactionState* m) {
|
2017-04-06 02:14:05 +02:00
|
|
|
if (num_running_ingest_file_ > 0) {
|
|
|
|
// We need to wait for other IngestExternalFile() calls to finish
|
|
|
|
// before running a manual compaction.
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if (m->exclusive) {
|
2017-08-04 00:36:28 +02:00
|
|
|
return (bg_bottom_compaction_scheduled_ > 0 ||
|
|
|
|
bg_compaction_scheduled_ > 0);
|
2017-04-06 02:14:05 +02:00
|
|
|
}
|
2017-08-04 00:36:28 +02:00
|
|
|
std::deque<ManualCompactionState*>::iterator it =
|
2017-04-06 02:14:05 +02:00
|
|
|
manual_compaction_dequeue_.begin();
|
|
|
|
bool seen = false;
|
|
|
|
while (it != manual_compaction_dequeue_.end()) {
|
|
|
|
if (m == (*it)) {
|
|
|
|
it++;
|
|
|
|
seen = true;
|
|
|
|
continue;
|
|
|
|
} else if (MCOverlap(m, (*it)) && (!seen && !(*it)->in_progress)) {
|
|
|
|
// Consider the other manual compaction *it, conflicts if:
|
|
|
|
// overlaps with m
|
|
|
|
// and (*it) is ahead in the queue and is not yet in progress
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
it++;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool DBImpl::HaveManualCompaction(ColumnFamilyData* cfd) {
|
|
|
|
// Remove from priority queue
|
2017-08-04 00:36:28 +02:00
|
|
|
std::deque<ManualCompactionState*>::iterator it =
|
2017-04-06 02:14:05 +02:00
|
|
|
manual_compaction_dequeue_.begin();
|
|
|
|
while (it != manual_compaction_dequeue_.end()) {
|
|
|
|
if ((*it)->exclusive) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if ((cfd == (*it)->cfd) && (!((*it)->in_progress || (*it)->done))) {
|
|
|
|
// Allow automatic compaction if manual compaction is
|
2017-06-14 01:46:17 +02:00
|
|
|
// in progress
|
2017-04-06 02:14:05 +02:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
it++;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool DBImpl::HasExclusiveManualCompaction() {
|
|
|
|
// Remove from priority queue
|
2017-08-04 00:36:28 +02:00
|
|
|
std::deque<ManualCompactionState*>::iterator it =
|
2017-04-06 02:14:05 +02:00
|
|
|
manual_compaction_dequeue_.begin();
|
|
|
|
while (it != manual_compaction_dequeue_.end()) {
|
|
|
|
if ((*it)->exclusive) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
it++;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
bool DBImpl::MCOverlap(ManualCompactionState* m, ManualCompactionState* m1) {
|
2017-04-06 02:14:05 +02:00
|
|
|
if ((m->exclusive) || (m1->exclusive)) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if (m->cfd != m1->cfd) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
// SuperVersionContext gets created and destructed outside of the lock --
|
2017-04-06 02:14:05 +02:00
|
|
|
// we
|
|
|
|
// use this convinently to:
|
|
|
|
// * malloc one SuperVersion() outside of the lock -- new_superversion
|
|
|
|
// * delete SuperVersion()s outside of the lock -- superversions_to_free
|
|
|
|
//
|
|
|
|
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
|
2017-10-06 03:00:38 +02:00
|
|
|
// same sv_context, we can't reuse the SuperVersion() that got
|
2017-04-06 02:14:05 +02:00
|
|
|
// malloced because
|
|
|
|
// first call already used it. In that rare case, we take a hit and create a
|
|
|
|
// new SuperVersion() inside of the mutex. We do similar thing
|
|
|
|
// for superversion_to_free
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
void DBImpl::InstallSuperVersionAndScheduleWork(
|
|
|
|
ColumnFamilyData* cfd, SuperVersionContext* sv_context,
|
2018-03-24 01:21:47 +01:00
|
|
|
const MutableCFOptions& mutable_cf_options, FlushReason flush_reason) {
|
2017-04-06 02:14:05 +02:00
|
|
|
mutex_.AssertHeld();
|
|
|
|
|
|
|
|
// Update max_total_in_memory_state_
|
|
|
|
size_t old_memtable_size = 0;
|
|
|
|
auto* old_sv = cfd->GetSuperVersion();
|
|
|
|
if (old_sv) {
|
|
|
|
old_memtable_size = old_sv->mutable_cf_options.write_buffer_size *
|
|
|
|
old_sv->mutable_cf_options.max_write_buffer_number;
|
|
|
|
}
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
if (sv_context->new_superversion == nullptr) {
|
|
|
|
sv_context->NewSuperVersion();
|
|
|
|
}
|
|
|
|
cfd->InstallSuperVersion(sv_context, &mutex_,
|
|
|
|
mutable_cf_options);
|
2017-04-06 02:14:05 +02:00
|
|
|
|
|
|
|
// Whenever we install new SuperVersion, we might need to issue new flushes or
|
|
|
|
// compactions.
|
FlushReason improvement
Summary:
Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full"
> $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304
$ grep 'flush_reason' /dev/shm/dbbench/LOG
...
2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"}
2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"}
With the fix:
> 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"}
2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"}
Closes https://github.com/facebook/rocksdb/pull/3627
Differential Revision: D7328772
Pulled By: miasantreble
fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
|
|
|
SchedulePendingFlush(cfd, flush_reason);
|
2017-04-06 02:14:05 +02:00
|
|
|
SchedulePendingCompaction(cfd);
|
|
|
|
MaybeScheduleFlushOrCompaction();
|
|
|
|
|
|
|
|
// Update max_total_in_memory_state_
|
|
|
|
max_total_in_memory_state_ =
|
|
|
|
max_total_in_memory_state_ - old_memtable_size +
|
|
|
|
mutable_cf_options.write_buffer_size *
|
|
|
|
mutable_cf_options.max_write_buffer_number;
|
|
|
|
}
|
2017-10-06 19:26:38 +02:00
|
|
|
|
2018-03-28 19:23:31 +02:00
|
|
|
// ShouldPurge is called by FindObsoleteFiles when doing a full scan,
|
|
|
|
// and db mutex (mutex_) should already be held. This function performs a
|
|
|
|
// linear scan of an vector (files_grabbed_for_purge_) in search of a
|
|
|
|
// certain element. We expect FindObsoleteFiles with full scan to occur once
|
|
|
|
// every 10 hours by default, and the size of the vector is small.
|
|
|
|
// Therefore, the cost is affordable even if the mutex is held.
|
|
|
|
// Actually, the current implementation of FindObsoleteFiles with
|
|
|
|
// full_scan=true can issue I/O requests to obtain list of files in
|
|
|
|
// directories, e.g. env_->getChildren while holding db mutex.
|
|
|
|
// In the future, if we want to reduce the cost of search, we may try to keep
|
|
|
|
// the vector sorted.
|
|
|
|
bool DBImpl::ShouldPurge(uint64_t file_number) const {
|
|
|
|
for (auto fn : files_grabbed_for_purge_) {
|
|
|
|
if (file_number == fn) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
for (const auto& purge_file_info : purge_queue_) {
|
|
|
|
if (purge_file_info.number == file_number) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
// MarkAsGrabbedForPurge is called by FindObsoleteFiles, and db mutex
|
|
|
|
// (mutex_) should already be held.
|
|
|
|
void DBImpl::MarkAsGrabbedForPurge(uint64_t file_number) {
|
|
|
|
files_grabbed_for_purge_.emplace_back(file_number);
|
|
|
|
}
|
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
void DBImpl::SetSnapshotChecker(SnapshotChecker* snapshot_checker) {
|
|
|
|
InstrumentedMutexLock l(&mutex_);
|
|
|
|
// snapshot_checker_ should only set once. If we need to set it multiple
|
|
|
|
// times, we need to make sure the old one is not deleted while it is still
|
|
|
|
// using by a compaction job.
|
|
|
|
assert(!snapshot_checker_);
|
|
|
|
snapshot_checker_.reset(snapshot_checker);
|
|
|
|
}
|
2017-04-06 02:14:05 +02:00
|
|
|
} // namespace rocksdb
|