@ -21,6 +21,7 @@
# include "monitoring/thread_status_util.h"
# include "monitoring/thread_status_util.h"
# include "test_util/sync_point.h"
# include "test_util/sync_point.h"
# include "util/cast_util.h"
# include "util/cast_util.h"
# include "util/coding.h"
# include "util/concurrent_task_limiter_impl.h"
# include "util/concurrent_task_limiter_impl.h"
namespace ROCKSDB_NAMESPACE {
namespace ROCKSDB_NAMESPACE {
@ -76,6 +77,40 @@ bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force,
return false ;
return false ;
}
}
bool DBImpl : : ShouldRescheduleFlushRequestToRetainUDT (
const FlushRequest & flush_req ) {
mutex_ . AssertHeld ( ) ;
assert ( flush_req . cfd_to_max_mem_id_to_persist . size ( ) = = 1 ) ;
ColumnFamilyData * cfd = flush_req . cfd_to_max_mem_id_to_persist . begin ( ) - > first ;
uint64_t max_memtable_id =
flush_req . cfd_to_max_mem_id_to_persist . begin ( ) - > second ;
if ( cfd - > IsDropped ( ) | |
! cfd - > ShouldPostponeFlushToRetainUDT ( max_memtable_id ) ) {
return false ;
}
// Check if holding on the flush will cause entering write stall mode.
// Write stall entered because of the accumulation of write buffers can be
// alleviated if we continue with the flush instead of postponing it.
const auto & mutable_cf_options = * cfd - > GetLatestMutableCFOptions ( ) ;
// Taking the status of the active Memtable into consideration so that we are
// not just checking if DB is currently already in write stall mode.
int mem_to_flush = cfd - > mem ( ) - > ApproximateMemoryUsageFast ( ) > =
cfd - > mem ( ) - > write_buffer_size ( ) / 2
? 1
: 0 ;
WriteStallCondition write_stall =
ColumnFamilyData : : GetWriteStallConditionAndCause (
cfd - > imm ( ) - > NumNotFlushed ( ) + mem_to_flush , /*num_l0_files=*/ 0 ,
/*num_compaction_needed_bytes=*/ 0 , mutable_cf_options ,
* cfd - > ioptions ( ) )
. first ;
if ( write_stall ! = WriteStallCondition : : kNormal ) {
return false ;
}
return true ;
}
IOStatus DBImpl : : SyncClosedLogs ( JobContext * job_context ,
IOStatus DBImpl : : SyncClosedLogs ( JobContext * job_context ,
VersionEdit * synced_wals ) {
VersionEdit * synced_wals ) {
TEST_SYNC_POINT ( " DBImpl::SyncClosedLogs:Start " ) ;
TEST_SYNC_POINT ( " DBImpl::SyncClosedLogs:Start " ) ;
@ -2506,8 +2541,11 @@ Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd,
// check whether one extra immutable memtable or an extra L0 file would
// check whether one extra immutable memtable or an extra L0 file would
// cause write stalling mode to be entered. It could still enter stall
// cause write stalling mode to be entered. It could still enter stall
// mode due to pending compaction bytes, but that's less common
// mode due to pending compaction bytes, but that's less common
// No extra immutable Memtable will be created if the current Memtable is
// empty.
int mem_to_flush = cfd - > mem ( ) - > IsEmpty ( ) ? 0 : 1 ;
write_stall_condition = ColumnFamilyData : : GetWriteStallConditionAndCause (
write_stall_condition = ColumnFamilyData : : GetWriteStallConditionAndCause (
cfd - > imm ( ) - > NumNotFlushed ( ) + 1 ,
cfd - > imm ( ) - > NumNotFlushed ( ) + mem_to_flush ,
vstorage - > l0_delay_trigger_count ( ) + 1 ,
vstorage - > l0_delay_trigger_count ( ) + 1 ,
vstorage - > estimated_compaction_needed_bytes ( ) ,
vstorage - > estimated_compaction_needed_bytes ( ) ,
mutable_cf_options , * cfd - > ioptions ( ) )
mutable_cf_options , * cfd - > ioptions ( ) )
@ -2945,6 +2983,7 @@ void DBImpl::UnscheduleFlushCallback(void* arg) {
Status DBImpl : : BackgroundFlush ( bool * made_progress , JobContext * job_context ,
Status DBImpl : : BackgroundFlush ( bool * made_progress , JobContext * job_context ,
LogBuffer * log_buffer , FlushReason * reason ,
LogBuffer * log_buffer , FlushReason * reason ,
bool * flush_rescheduled_to_retain_udt ,
Env : : Priority thread_pri ) {
Env : : Priority thread_pri ) {
mutex_ . AssertHeld ( ) ;
mutex_ . AssertHeld ( ) ;
@ -2970,12 +3009,43 @@ Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
autovector < ColumnFamilyData * > column_families_not_to_flush ;
autovector < ColumnFamilyData * > column_families_not_to_flush ;
while ( ! flush_queue_ . empty ( ) ) {
while ( ! flush_queue_ . empty ( ) ) {
// This cfd is already referenced
// This cfd is already referenced
auto [ flush_reason , cfd_to_max_mem_id_to_persist ] =
FlushRequest flush_req = PopFirstFromFlushQueue ( ) ;
PopFirstFromFlushQueue ( ) ;
FlushReason flush_reason = flush_req . flush_reason ;
if ( ! immutable_db_options_ . atomic_flush & &
ShouldRescheduleFlushRequestToRetainUDT ( flush_req ) ) {
assert ( flush_req . cfd_to_max_mem_id_to_persist . size ( ) = = 1 ) ;
ColumnFamilyData * cfd =
flush_req . cfd_to_max_mem_id_to_persist . begin ( ) - > first ;
if ( cfd - > UnrefAndTryDelete ( ) ) {
return Status : : OK ( ) ;
}
ROCKS_LOG_BUFFER ( log_buffer ,
" FlushRequest for column family %s is re-scheduled to "
" retain user-defined timestamps. " ,
cfd - > GetName ( ) . c_str ( ) ) ;
// Reschedule the `FlushRequest` as is without checking dropped column
// family etc. The follow-up job will do the check anyways, so save the
// duplication. Column family is deduplicated by `SchdulePendingFlush` and
// `PopFirstFromFlushQueue` contains at flush request enqueueing and
// dequeueing time.
// This flush request is rescheduled right after it's popped from the
// queue while the db mutex is held, so there should be no other
// FlushRequest for the same column family with higher `max_memtable_id`
// in the queue to block the reschedule from succeeding.
# ifndef NDEBUG
flush_req . reschedule_count + = 1 ;
# endif /* !NDEBUG */
SchedulePendingFlush ( flush_req ) ;
* reason = flush_reason ;
* flush_rescheduled_to_retain_udt = true ;
return Status : : TryAgain ( ) ;
}
superversion_contexts . clear ( ) ;
superversion_contexts . clear ( ) ;
superversion_contexts . reserve ( cfd_to_max_mem_id_to_persist . size ( ) ) ;
superversion_contexts . reserve (
flush_req . cfd_to_max_mem_id_to_persist . size ( ) ) ;
for ( const auto & [ cfd , max_memtable_id ] : cfd_to_max_mem_id_to_persist ) {
for ( const auto & [ cfd , max_memtable_id ] :
flush_req . cfd_to_max_mem_id_to_persist ) {
if ( cfd - > GetMempurgeUsed ( ) ) {
if ( cfd - > GetMempurgeUsed ( ) ) {
// If imm() contains silent memtables (e.g.: because
// If imm() contains silent memtables (e.g.: because
// MemPurge was activated), requesting a flush will
// MemPurge was activated), requesting a flush will
@ -2992,7 +3062,13 @@ Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
bg_flush_args . emplace_back ( cfd , max_memtable_id ,
bg_flush_args . emplace_back ( cfd , max_memtable_id ,
& ( superversion_contexts . back ( ) ) , flush_reason ) ;
& ( superversion_contexts . back ( ) ) , flush_reason ) ;
}
}
if ( ! bg_flush_args . empty ( ) ) {
// `MaybeScheduleFlushOrCompaction` schedules as many `BackgroundCallFlush`
// jobs as the number of `FlushRequest` in the `flush_queue_`, a.k.a
// `unscheduled_flushes_`. So it's sufficient to make each `BackgroundFlush`
// handle one `FlushRequest` and each have a Status returned.
if ( ! bg_flush_args . empty ( ) | | ! column_families_not_to_flush . empty ( ) ) {
TEST_SYNC_POINT_CALLBACK ( " DBImpl::BackgroundFlush:CheckFlushRequest:cb " ,
const_cast < int * > ( & flush_req . reschedule_count ) ) ;
break ;
break ;
}
}
}
}
@ -3054,11 +3130,20 @@ void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) {
pending_outputs_inserted_elem ( new std : : list < uint64_t > : : iterator (
pending_outputs_inserted_elem ( new std : : list < uint64_t > : : iterator (
CaptureCurrentFileNumberInPendingOutputs ( ) ) ) ;
CaptureCurrentFileNumberInPendingOutputs ( ) ) ) ;
FlushReason reason ;
FlushReason reason ;
bool flush_rescheduled_to_retain_udt = false ;
Status s = BackgroundFlush ( & made_progress , & job_context , & log_buffer ,
Status s =
& reason , thread_pri ) ;
BackgroundFlush ( & made_progress , & job_context , & log_buffer , & reason ,
if ( ! s . ok ( ) & & ! s . IsShutdownInProgress ( ) & & ! s . IsColumnFamilyDropped ( ) & &
& flush_rescheduled_to_retain_udt , thread_pri ) ;
reason ! = FlushReason : : kErrorRecovery ) {
if ( s . IsTryAgain ( ) & & flush_rescheduled_to_retain_udt ) {
bg_cv_ . SignalAll ( ) ; // In case a waiter can proceed despite the error
mutex_ . Unlock ( ) ;
TEST_SYNC_POINT_CALLBACK ( " DBImpl::AfterRetainUDTReschedule:cb " , nullptr ) ;
immutable_db_options_ . clock - > SleepForMicroseconds (
100000 ) ; // prevent hot loop
mutex_ . Lock ( ) ;
} else if ( ! s . ok ( ) & & ! s . IsShutdownInProgress ( ) & &
! s . IsColumnFamilyDropped ( ) & &
reason ! = FlushReason : : kErrorRecovery ) {
// Wait a little bit before retrying background flush in
// Wait a little bit before retrying background flush in
// case this is an environmental problem and we do not want to
// case this is an environmental problem and we do not want to
// chew up resources for failed flushes for the duration of
// chew up resources for failed flushes for the duration of
@ -3079,29 +3164,33 @@ void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) {
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:FlushFinish:0 " ) ;
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:FlushFinish:0 " ) ;
ReleaseFileNumberFromPendingOutputs ( pending_outputs_inserted_elem ) ;
ReleaseFileNumberFromPendingOutputs ( pending_outputs_inserted_elem ) ;
// There is no need to do these clean up if the flush job is rescheduled
// If flush failed, we want to delete all temporary files that we might have
// to retain user-defined timestamps because the job doesn't get to the
// created. Thus, we force full scan in FindObsoleteFiles()
// stage of actually flushing the MemTables.
FindObsoleteFiles ( & job_context , ! s . ok ( ) & & ! s . IsShutdownInProgress ( ) & &
if ( ! flush_rescheduled_to_retain_udt ) {
! s . IsColumnFamilyDropped ( ) ) ;
// If flush failed, we want to delete all temporary files that we might
// delete unnecessary files if any, this is done outside the mutex
// have created. Thus, we force full scan in FindObsoleteFiles()
if ( job_context . HaveSomethingToClean ( ) | |
FindObsoleteFiles ( & job_context , ! s . ok ( ) & & ! s . IsShutdownInProgress ( ) & &
job_context . HaveSomethingToDelete ( ) | | ! log_buffer . IsEmpty ( ) ) {
! s . IsColumnFamilyDropped ( ) ) ;
mutex_ . Unlock ( ) ;
// delete unnecessary files if any, this is done outside the mutex
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:FilesFound " ) ;
if ( job_context . HaveSomethingToClean ( ) | |
// Have to flush the info logs before bg_flush_scheduled_--
job_context . HaveSomethingToDelete ( ) | | ! log_buffer . IsEmpty ( ) ) {
// because if bg_flush_scheduled_ becomes 0 and the lock is
mutex_ . Unlock ( ) ;
// released, the deconstructor of DB can kick in and destroy all the
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:FilesFound " ) ;
// states of DB so info_log might not be available after that point.
// Have to flush the info logs before bg_flush_scheduled_--
// It also applies to access other states that DB owns.
// because if bg_flush_scheduled_ becomes 0 and the lock is
log_buffer . FlushBufferToLog ( ) ;
// released, the deconstructor of DB can kick in and destroy all the
if ( job_context . HaveSomethingToDelete ( ) ) {
// states of DB so info_log might not be available after that point.
PurgeObsoleteFiles ( job_context ) ;
// 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 ( ) ;
}
}
job_context . Clean ( ) ;
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:ContextCleanedUp " ) ;
mutex_ . Lock ( ) ;
}
}
TEST_SYNC_POINT ( " DBImpl::BackgroundCallFlush:ContextCleanedUp " ) ;
assert ( num_running_flushes_ > 0 ) ;
assert ( num_running_flushes_ > 0 ) ;
num_running_flushes_ - - ;
num_running_flushes_ - - ;