@ -89,6 +89,7 @@ void DumpRocksDBBuildVersion(Logger * log);
struct DBImpl : : WriteContext {
autovector < SuperVersion * > superversions_to_free_ ;
autovector < log : : Writer * > logs_to_free_ ;
bool schedule_bg_work_ = false ;
~ WriteContext ( ) {
for ( auto & sv : superversions_to_free_ ) {
@ -205,8 +206,8 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
max_total_in_memory_state_ ( 0 ) ,
is_snapshot_supported_ ( true ) ,
write_buffer_ ( options . db_write_buffer_size ) ,
tmp_batch_ ( ) ,
bg_schedule_needed_ ( false ) ,
unscheduled_flushes_ ( 0 ) ,
unscheduled_compactions_ ( 0 ) ,
bg_compaction_scheduled_ ( 0 ) ,
bg_manual_only_ ( 0 ) ,
bg_flush_scheduled_ ( 0 ) ,
@ -272,6 +273,19 @@ DBImpl::~DBImpl() {
listeners_ . clear ( ) ;
flush_scheduler_ . Clear ( ) ;
while ( ! flush_queue_ . empty ( ) ) {
auto cfd = PopFirstFromFlushQueue ( ) ;
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
}
while ( ! compaction_queue_ . empty ( ) ) {
auto cfd = PopFirstFromCompactionQueue ( ) ;
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
}
if ( default_cf_handle_ ! = nullptr ) {
// we need to delete handle outside of lock because it does its own locking
mutex_ . Unlock ( ) ;
@ -1643,10 +1657,13 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
// SetNewMemtableAndNewLogFile() will release and reacquire mutex
// during execution
s = SetNewMemtableAndNewLogFile ( cfd , & context ) ;
write_thread_ . ExitWriteThread ( & w , & w , s ) ;
cfd - > imm ( ) - > FlushRequested ( ) ;
MaybeScheduleFlushOrCompaction ( ) ;
write_thread_ . ExitWriteThread ( & w , & w , s ) ;
// schedule flush
SchedulePendingFlush ( cfd ) ;
MaybeScheduleFlushOrCompaction ( ) ;
}
if ( s . ok ( ) & & flush_options . wait ) {
@ -1671,52 +1688,90 @@ Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
void DBImpl : : MaybeScheduleFlushOrCompaction ( ) {
mutex_ . AssertHeld ( ) ;
bg_schedule_needed_ = false ;
if ( bg_work_gate_closed_ ) {
// gate closed for backgrond work
// gate closed for background work
return ;
} else if ( shutting_down_ . load ( std : : memory_order_acquire ) ) {
// DB is being deleted; no more background compactions
} else {
bool is_flush_pending = false ;
// no need to refcount since we're under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > imm ( ) - > IsFlushPending ( ) ) {
is_flush_pending = true ;
}
}
if ( is_flush_pending ) {
// memtable flush needed
if ( bg_flush_scheduled_ < db_options_ . max_background_flushes ) {
bg_flush_scheduled_ + + ;
env_ - > Schedule ( & DBImpl : : BGWorkFlush , this , Env : : Priority : : HIGH ) ;
} else if ( db_options_ . max_background_flushes > 0 ) {
bg_schedule_needed_ = true ;
}
}
bool is_compaction_needed = false ;
// no need to refcount since we're under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
if ( cfd - > compaction_picker ( ) - > NeedsCompaction (
cfd - > current ( ) - > storage_info ( ) ) ) {
is_compaction_needed = true ;
break ;
}
}
return ;
} else if ( bg_manual_only_ ) {
// manual only
return ;
}
// Schedule BGWorkCompaction if there's a compaction pending (or a memtable
// flush, but the HIGH pool is not enabled)
// Do it only if max_background_compactions hasn't been reached and
// bg_manual_only_ == 0
if ( ! bg_manual_only_ & &
( is_compaction_needed | |
( is_flush_pending & & db_options_ . max_background_flushes = = 0 ) ) ) {
if ( bg_compaction_scheduled_ < db_options_ . max_background_compactions ) {
bg_compaction_scheduled_ + + ;
env_ - > Schedule ( & DBImpl : : BGWorkCompaction , this , Env : : Priority : : LOW ) ;
} else {
bg_schedule_needed_ = true ;
}
while ( unscheduled_flushes_ > 0 & &
bg_flush_scheduled_ < db_options_ . max_background_flushes ) {
unscheduled_flushes_ - - ;
bg_flush_scheduled_ + + ;
env_ - > Schedule ( & DBImpl : : BGWorkFlush , this , Env : : Priority : : HIGH ) ;
}
if ( db_options_ . max_background_flushes = = 0 & &
bg_compaction_scheduled_ < db_options_ . max_background_compactions & &
unscheduled_flushes_ > 0 ) {
// special case where flush is executed by compaction thread
// (if max_background_flushes == 0).
// Compaction thread will execute all the flushes
unscheduled_flushes_ = 0 ;
if ( unscheduled_compactions_ > 0 ) {
// bg compaction will execute one compaction
unscheduled_compactions_ - - ;
}
bg_compaction_scheduled_ + + ;
env_ - > Schedule ( & DBImpl : : BGWorkCompaction , this , Env : : Priority : : LOW ) ;
}
while ( bg_compaction_scheduled_ < db_options_ . max_background_compactions & &
unscheduled_compactions_ > 0 ) {
bg_compaction_scheduled_ + + ;
unscheduled_compactions_ - - ;
env_ - > Schedule ( & DBImpl : : BGWorkCompaction , this , Env : : Priority : : LOW ) ;
}
}
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 ;
}
void DBImpl : : AddToFlushQueue ( ColumnFamilyData * cfd ) {
assert ( ! cfd - > pending_flush ( ) ) ;
cfd - > Ref ( ) ;
flush_queue_ . push_back ( cfd ) ;
cfd - > set_pending_flush ( true ) ;
}
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 ) ;
return cfd ;
}
void DBImpl : : SchedulePendingFlush ( ColumnFamilyData * cfd ) {
if ( ! cfd - > pending_flush ( ) & & cfd - > imm ( ) - > IsFlushPending ( ) ) {
AddToFlushQueue ( cfd ) ;
+ + unscheduled_flushes_ ;
}
}
void DBImpl : : SchedulePendingCompaction ( ColumnFamilyData * cfd ) {
if ( ! cfd - > pending_compaction ( ) & & cfd - > NeedsCompaction ( ) ) {
AddToCompactionQueue ( cfd ) ;
+ + unscheduled_compactions_ ;
}
}
@ -1743,33 +1798,41 @@ Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
return bg_error_ ;
}
// call_status is failure if at least one flush was a failure. even if
// flushing one column family reports a failure, we will continue flushing
// other column families. however, call_status will be a failure in that case.
Status call_status ;
// refcounting in iteration
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
cfd - > Ref ( ) ;
Status flush_status ;
const MutableCFOptions mutable_cf_options =
* cfd - > GetLatestMutableCFOptions ( ) ;
while ( flush_status . ok ( ) & & cfd - > imm ( ) - > IsFlushPending ( ) ) {
LogToBuffer (
log_buffer ,
" BackgroundCallFlush doing FlushMemTableToOutputFile with column "
" family [%s], flush slots available %d " ,
cfd - > GetName ( ) . c_str ( ) ,
db_options_ . max_background_flushes - bg_flush_scheduled_ ) ;
flush_status = FlushMemTableToOutputFile (
cfd , mutable_cf_options , madeProgress , job_context , log_buffer ) ;
ColumnFamilyData * cfd = nullptr ;
while ( ! flush_queue_ . empty ( ) ) {
// This cfd is already referenced
cfd = PopFirstFromFlushQueue ( ) ;
if ( cfd - > IsDropped ( ) | | ! cfd - > imm ( ) - > IsFlushPending ( ) ) {
// can't flush this CF, try next one
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
continue ;
}
if ( call_status . ok ( ) & & ! flush_status . ok ( ) ) {
call_status = flush_status ;
// found a flush!
break ;
}
Status status ;
if ( cfd ! = nullptr ) {
const MutableCFOptions mutable_cf_options =
* cfd - > GetLatestMutableCFOptions ( ) ;
LogToBuffer (
log_buffer ,
" Calling FlushMemTableToOutputFile with column "
" family [%s], flush slots available %d, compaction slots available %d " ,
cfd - > GetName ( ) . c_str ( ) ,
db_options_ . max_background_flushes - bg_flush_scheduled_ ,
db_options_ . max_background_compactions - bg_compaction_scheduled_ ) ;
status = FlushMemTableToOutputFile ( cfd , mutable_cf_options , madeProgress ,
job_context , log_buffer ) ;
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
cfd - > Unref ( ) ;
}
versions_ - > GetColumnFamilySet ( ) - > FreeDeadColumnFamilies ( ) ;
return call_status ;
return status ;
}
void DBImpl : : BackgroundCallFlush ( ) {
@ -1829,13 +1892,8 @@ void DBImpl::BackgroundCallFlush() {
}
bg_flush_scheduled_ - - ;
// Any time the mutex is released After finding the work to do, another
// thread might execute MaybeScheduleFlushOrCompaction(). It is possible
// that there is a pending job but it is not scheduled because of the
// max thread limit.
if ( madeProgress | | bg_schedule_needed_ ) {
MaybeScheduleFlushOrCompaction ( ) ;
}
// See if there's more work to be done
MaybeScheduleFlushOrCompaction ( ) ;
RecordFlushIOStats ( ) ;
bg_cv_ . SignalAll ( ) ;
// IMPORTANT: there should be no code after calling SignalAll. This call may
@ -1909,17 +1967,8 @@ void DBImpl::BackgroundCallCompaction() {
versions_ - > GetColumnFamilySet ( ) - > FreeDeadColumnFamilies ( ) ;
// Previous compaction may have produced too many files in a level,
// So reschedule another compaction if we made progress in the
// last compaction.
//
// Also, any time the mutex is released After finding the work to do,
// another thread might execute MaybeScheduleFlushOrCompaction(). It is
// possible that there is a pending job but it is not scheduled because of
// the max thread limit.
if ( madeProgress | | bg_schedule_needed_ ) {
MaybeScheduleFlushOrCompaction ( ) ;
}
// See if there's more work to be done
MaybeScheduleFlushOrCompaction ( ) ;
if ( madeProgress | | bg_compaction_scheduled_ = = 0 | | bg_manual_only_ > 0 ) {
// signal if
// * madeProgress -- need to wakeup DelayWrite
@ -1964,35 +2013,28 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
}
// FLUSH preempts compaction
Status flush_stat ;
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
const MutableCFOptions mutable_cf_options =
* cfd - > GetLatestMutableCFOptions ( ) ;
while ( cfd - > imm ( ) - > IsFlushPending ( ) ) {
LogToBuffer (
log_buffer ,
" BackgroundCompaction doing FlushMemTableToOutputFile, "
" compaction slots available %d " ,
db_options_ . max_background_compactions - bg_compaction_scheduled_ ) ;
cfd - > Ref ( ) ;
flush_stat = FlushMemTableToOutputFile (
cfd , mutable_cf_options , madeProgress , job_context , log_buffer ) ;
cfd - > Unref ( ) ;
if ( ! flush_stat . ok ( ) ) {
if ( is_manual ) {
manual_compaction_ - > status = flush_stat ;
manual_compaction_ - > done = true ;
manual_compaction_ - > in_progress = false ;
manual_compaction_ = nullptr ;
}
return flush_stat ;
// TODO(icanadi) we should only do this if max_background_flushes == 0
// BackgroundFlush() will only execute a single flush. We keep calling it as
// long as there's more flushes to be done
while ( ! flush_queue_ . empty ( ) ) {
LogToBuffer (
log_buffer ,
" BackgroundCompaction calling BackgroundFlush. flush slots available "
" %d, compaction slots available %d " ,
db_options_ . max_background_flushes - bg_flush_scheduled_ ,
db_options_ . max_background_compactions - bg_compaction_scheduled_ ) ;
auto flush_status = BackgroundFlush ( madeProgress , job_context , log_buffer ) ;
if ( ! flush_status . ok ( ) ) {
if ( is_manual ) {
manual_compaction_ - > status = flush_status ;
manual_compaction_ - > done = true ;
manual_compaction_ - > in_progress = false ;
manual_compaction_ = nullptr ;
}
return flush_status ;
}
}
// 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
unique_ptr < Compaction > c ;
InternalKey manual_end_storage ;
InternalKey * manual_end = & manual_end_storage ;
@ -2014,22 +2056,53 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
( ( m - > done | | manual_end = = nullptr )
? " (end) "
: manual_end - > DebugString ( ) . c_str ( ) ) ) ;
} else {
// no need to refcount in iteration since it's always under a mutex
for ( auto cfd : * versions_ - > GetColumnFamilySet ( ) ) {
// Pick up latest mutable CF Options and use it throughout the
// compaction job
auto * mutable_cf_options = cfd - > GetLatestMutableCFOptions ( ) ;
if ( ! mutable_cf_options - > disable_auto_compactions ) {
// 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
c . reset ( cfd - > PickCompaction ( * mutable_cf_options , log_buffer ) ) ;
if ( c ! = nullptr ) {
// update statistics
MeasureTime ( stats_ , NUM_FILES_IN_SINGLE_COMPACTION ,
c - > inputs ( 0 ) - > size ( ) ) ;
break ;
} else if ( ! compaction_queue_ . empty ( ) ) {
// 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
c . reset ( cfd - > PickCompaction ( * mutable_cf_options , log_buffer ) ) ;
if ( c ! = nullptr ) {
// 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
// InstallSuperVersion()
// 2) When MutableCFOptions changes. This case is also covered by
// InstallSuperVersion(), 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 ( ) ;
}
}
}
@ -2085,8 +2158,6 @@ Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
c - > ReleaseCompactionFiles ( status ) ;
* madeProgress = true ;
} else {
MaybeScheduleFlushOrCompaction ( ) ; // do more compaction work in parallel.
auto yield_callback = [ & ] ( ) {
return CallFlushDuringCompaction ( c - > column_family_data ( ) ,
* c - > mutable_cf_options ( ) , job_context ,
@ -2275,7 +2346,7 @@ void DBImpl::InstallSuperVersionBackground(
SuperVersion * DBImpl : : InstallSuperVersion (
ColumnFamilyData * cfd , SuperVersion * new_sv ,
const MutableCFOptions & mutable_cf_options ) {
const MutableCFOptions & mutable_cf_options , bool dont_schedule_bg_work ) {
mutex_ . AssertHeld ( ) ;
// Update max_total_in_memory_state_
@ -2289,10 +2360,15 @@ SuperVersion* DBImpl::InstallSuperVersion(
auto * old = cfd - > InstallSuperVersion (
new_sv ? new_sv : new SuperVersion ( ) , & mutex_ , mutable_cf_options ) ;
// We want to schedule potential flush or compactions since new options may
// have been picked up in this new version. New options may cause flush
// compaction trigger condition to change.
MaybeScheduleFlushOrCompaction ( ) ;
// Whenever we install new SuperVersion, we might need to issue new flushes or
// compactions. dont_schedule_bg_work is true when scheduling from write
// thread and we don't want to add additional overhead. Callers promise to
// call SchedulePendingFlush() and MaybeScheduleFlushOrCompaction() eventually
if ( ! dont_schedule_bg_work ) {
SchedulePendingFlush ( cfd ) ;
SchedulePendingCompaction ( cfd ) ;
MaybeScheduleFlushOrCompaction ( ) ;
}
// Update max_total_in_memory_state_
max_total_in_memory_state_ =
@ -2848,9 +2924,10 @@ Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
break ;
}
cfd - > imm ( ) - > FlushRequested ( ) ;
SchedulePendingFlush ( cfd ) ;
context . schedule_bg_work_ = true ;
}
}
MaybeScheduleFlushOrCompaction ( ) ;
} else if ( UNLIKELY ( write_buffer_ . ShouldFlush ( ) ) ) {
Log ( InfoLogLevel : : INFO_LEVEL , db_options_ . info_log ,
" Flushing all column families. Write buffer is using % " PRIu64
@ -2865,6 +2942,8 @@ Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
break ;
}
cfd - > imm ( ) - > FlushRequested ( ) ;
SchedulePendingFlush ( cfd ) ;
context . schedule_bg_work_ = true ;
}
}
MaybeScheduleFlushOrCompaction ( ) ;
@ -2986,6 +3065,10 @@ Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
}
write_thread_ . ExitWriteThread ( & w , last_writer , status ) ;
if ( context . schedule_bg_work_ ) {
MaybeScheduleFlushOrCompaction ( ) ;
}
mutex_ . Unlock ( ) ;
if ( status . IsTimedOut ( ) ) {
@ -3023,11 +3106,11 @@ Status DBImpl::DelayWrite(uint64_t expiration_time) {
}
Status DBImpl : : ScheduleFlushes ( WriteContext * context ) {
bool schedule_bg_work = false ;
ColumnFamilyData * cfd ;
while ( ( cfd = flush_scheduler_ . GetNextColumnFamily ( ) ) ! = nullptr ) {
schedule_bg_work = true ;
auto status = SetNewMemtableAndNewLogFile ( cfd , context ) ;
SchedulePendingFlush ( cfd ) ;
context - > schedule_bg_work_ = true ;
if ( cfd - > Unref ( ) ) {
delete cfd ;
}
@ -3035,9 +3118,6 @@ Status DBImpl::ScheduleFlushes(WriteContext* context) {
return status ;
}
}
if ( schedule_bg_work ) {
MaybeScheduleFlushOrCompaction ( ) ;
}
return Status : : OK ( ) ;
}
@ -3113,7 +3193,7 @@ Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd,
new_mem - > Ref ( ) ;
cfd - > SetMemtable ( new_mem ) ;
context - > superversions_to_free_ . push_back (
InstallSuperVersion ( cfd , new_superversion , mutable_cf_options ) ) ;
InstallSuperVersion ( cfd , new_superversion , mutable_cf_options , true ) ) ;
return s ;
}
@ -3380,12 +3460,6 @@ Status DBImpl::DeleteFile(std::string name) {
PurgeObsoleteFiles ( job_context ) ;
}
job_context . Clean ( ) ;
{
MutexLock l ( & mutex_ ) ;
// schedule flush if file deletion means we freed the space for flushes to
// continue
MaybeScheduleFlushOrCompaction ( ) ;
}
return status ;
}
@ -3620,7 +3694,6 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname,
impl - > alive_log_files_ . push_back (
DBImpl : : LogFileNumberSize ( impl - > logfile_number_ ) ) ;
impl - > DeleteObsoleteFiles ( ) ;
impl - > MaybeScheduleFlushOrCompaction ( ) ;
s = impl - > db_directory_ - > Fsync ( ) ;
}
}