@ -349,14 +349,11 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
// when it finds suitable, and finish them in the same write batch.
// This is how a write job could be done by the other writer.
WriteContext write_context ;
LogContext log_context ( write_options . sync ) ;
WriteThread : : WriteGroup write_group ;
bool in_parallel_group = false ;
uint64_t last_sequence = kMaxSequenceNumber ;
mutex_ . Lock ( ) ;
bool need_log_sync = write_options . sync ;
bool need_log_dir_sync = need_log_sync & & ! log_dir_synced_ ;
assert ( ! two_write_queues_ | | ! disable_memtable ) ;
{
// With concurrent writes we do preprocess only in the write thread that
@ -366,7 +363,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
// PreprocessWrite does its own perf timing.
PERF_TIMER_STOP ( write_pre_and_post_process_time ) ;
status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
status = PreprocessWrite ( write_options , & log_context , & write_context ) ;
if ( ! two_write_queues_ ) {
// Assign it after ::PreprocessWrite since the sequence might advance
// inside it by WriteRecoverableState
@ -376,13 +373,6 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
PERF_TIMER_START ( write_pre_and_post_process_time ) ;
}
log : : Writer * log_writer = logs_ . back ( ) . writer ;
LogFileNumberSize & log_file_number_size = alive_log_files_ . back ( ) ;
assert ( log_writer - > get_log_number ( ) = = log_file_number_size . number ) ;
mutex_ . Unlock ( ) ;
// Add to log and apply to memtable. We can release the lock
// during this phase since &w is currently responsible for logging
// and protects against concurrent loggers and concurrent writes
@ -477,10 +467,14 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
if ( ! two_write_queues_ ) {
if ( status . ok ( ) & & ! write_options . disableWAL ) {
assert ( log_context . log_file_number_size ) ;
LogFileNumberSize & log_file_number_size =
* ( log_context . log_file_number_size ) ;
PERF_TIMER_GUARD ( write_wal_time ) ;
io_s = WriteToWAL ( write_group , log_writer , log_used , need_log_sync ,
need_log_dir_sync , last_sequence + 1 ,
log_file_number_size ) ;
io_s =
WriteToWAL ( write_group , log_context . writer , log_used ,
log_context . need_log_sync , log_context . need_log_dir_sync ,
last_sequence + 1 , log_file_number_size ) ;
}
} else {
if ( status . ok ( ) & & ! write_options . disableWAL ) {
@ -582,14 +576,21 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
assert ( pre_release_cb_status . ok ( ) ) ;
}
if ( need_log_sync ) {
mutex_ . Lock ( ) ;
if ( log_context . need_log_sync ) {
VersionEdit synced_wals ;
log_write_mutex_ . Lock ( ) ;
if ( status . ok ( ) ) {
status = MarkLogsSynced ( logfile_number_ , need_log_dir_sync ) ;
MarkLogsSynced ( logfile_number_ , log_context . need_log_dir_sync ,
& synced_wals ) ;
} else {
MarkLogsNotSynced ( logfile_number_ ) ;
}
mutex_ . Unlock ( ) ;
log_write_mutex_ . Unlock ( ) ;
if ( status . ok ( ) & & synced_wals . IsWalAddition ( ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
status = ApplyWALToManifest ( & synced_wals ) ;
}
// Requesting sync with two_write_queues_ is expected to be very rare. We
// hence provide a simple implementation that is not necessarily efficient.
if ( two_write_queues_ ) {
@ -652,19 +653,11 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
if ( w . callback & & ! w . callback - > AllowWriteBatching ( ) ) {
write_thread_ . WaitForMemTableWriters ( ) ;
}
mutex_ . Lock ( ) ;
bool need_log_sync = ! write_options . disableWAL & & write_options . sync ;
bool need_log_dir_sync = need_log_sync & & ! log_dir_synced_ ;
LogContext log_context ( ! write_options . disableWAL & & write_options . sync ) ;
// PreprocessWrite does its own perf timing.
PERF_TIMER_STOP ( write_pre_and_post_process_time ) ;
w . status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
w . status = PreprocessWrite ( write_options , & log_context , & write_context ) ;
PERF_TIMER_START ( write_pre_and_post_process_time ) ;
log : : Writer * log_writer = logs_ . back ( ) . writer ;
LogFileNumberSize & log_file_number_size = alive_log_files_ . back ( ) ;
assert ( log_writer - > get_log_number ( ) = = log_file_number_size . number ) ;
mutex_ . Unlock ( ) ;
// This can set non-OK status if callback fail.
last_batch_group_size_ =
@ -727,9 +720,13 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
wal_write_group . size - 1 ) ;
RecordTick ( stats_ , WRITE_DONE_BY_OTHER , wal_write_group . size - 1 ) ;
}
assert ( log_context . log_file_number_size ) ;
LogFileNumberSize & log_file_number_size =
* ( log_context . log_file_number_size ) ;
io_s =
WriteToWAL ( wal_write_group , log_writer , log_used , need_log_sync ,
need_log_dir_sync , current_sequence , log_file_number_size ) ;
WriteToWAL ( wal_write_group , log_context . writer , log_used ,
log_context . need_log_sync , log_context . need_log_dir_sync ,
current_sequence , log_file_number_size ) ;
w . status = io_s ;
}
@ -740,16 +737,20 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
WriteStatusCheck ( w . status ) ;
}
if ( need_log_sync ) {
mutex_ . Lock ( ) ;
VersionEdit synced_wals ;
if ( log_context . need_log_sync ) {
InstrumentedMutexLock l ( & log_write_mutex_ ) ;
if ( w . status . ok ( ) ) {
w . status = MarkLogsSynced ( logfile_number_ , need_log_dir_sync ) ;
MarkLogsSynced ( logfile_number_ , log_context . need_log_dir_sync ,
& synced_wals ) ;
} else {
MarkLogsNotSynced ( logfile_number_ ) ;
}
mutex_ . Unlock ( ) ;
}
if ( w . status . ok ( ) & & synced_wals . IsWalAddition ( ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
w . status = ApplyWALToManifest ( & synced_wals ) ;
}
write_thread_ . ExitAsBatchGroupLeader ( wal_write_group , w . status ) ;
}
@ -893,9 +894,8 @@ Status DBImpl::WriteImplWALOnly(
// TODO(myabandeh): Make preliminary checks thread-safe so we could do them
// without paying the cost of obtaining the mutex.
if ( status . ok ( ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
bool need_log_sync = false ;
status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
LogContext log_context ;
status = PreprocessWrite ( write_options , & log_context , & write_context ) ;
WriteStatusCheckOnLocked ( status ) ;
}
if ( ! status . ok ( ) ) {
@ -1057,9 +1057,8 @@ Status DBImpl::WriteImplWALOnly(
void DBImpl : : WriteStatusCheckOnLocked ( const Status & status ) {
// Is setting bg_error_ enough here? This will at least stop
// compaction and fail any further writes.
// Caller must hold mutex_.
InstrumentedMutexLock l ( & mutex_ ) ;
assert ( ! status . IsIOFenced ( ) | | ! error_handler_ . GetBGError ( ) . ok ( ) ) ;
mutex_ . AssertHeld ( ) ;
if ( immutable_db_options_ . paranoid_checks & & ! status . ok ( ) & &
! status . IsBusy ( ) & & ! status . IsIncomplete ( ) ) {
// Maybe change the return status to void?
@ -1110,13 +1109,13 @@ void DBImpl::MemTableInsertStatusCheck(const Status& status) {
}
Status DBImpl : : PreprocessWrite ( const WriteOptions & write_options ,
bool * need_log_sync ,
LogContext * log_context ,
WriteContext * write_context ) {
mutex_ . AssertHeld ( ) ;
assert ( write_context ! = nullptr & & need_log_sync ! = nullptr ) ;
assert ( write_context ! = nullptr & & log_context ! = nullptr ) ;
Status status ;
if ( error_handler_ . IsDBStopped ( ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
status = error_handler_ . GetBGError ( ) ;
}
@ -1124,11 +1123,11 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
if ( UNLIKELY ( status . ok ( ) & & total_log_size_ > GetMaxTotalWalSize ( ) ) ) {
assert ( versions_ ) ;
InstrumentedMutexLock l ( & mutex_ ) ;
const ColumnFamilySet * const column_families =
versions_ - > GetColumnFamilySet ( ) ;
assert ( column_families ) ;
size_t num_cfs = column_families - > NumberOfColumnFamilies ( ) ;
assert ( num_cfs > = 1 ) ;
if ( num_cfs > 1 ) {
WaitForPendingWrites ( ) ;
@ -1142,15 +1141,18 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
// thread is writing to another DB with the same write buffer, they may also
// be flushed. We may end up with flushing much more DBs than needed. It's
// suboptimal but still correct.
InstrumentedMutexLock l ( & mutex_ ) ;
WaitForPendingWrites ( ) ;
status = HandleWriteBufferManagerFlush ( write_context ) ;
}
if ( UNLIKELY ( status . ok ( ) & & ! trim_history_scheduler_ . Empty ( ) ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
status = TrimMemtableHistory ( write_context ) ;
}
if ( UNLIKELY ( status . ok ( ) & & ! flush_scheduler_ . Empty ( ) ) ) {
InstrumentedMutexLock l ( & mutex_ ) ;
WaitForPendingWrites ( ) ;
status = ScheduleFlushes ( write_context ) ;
}
@ -1166,6 +1168,7 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
// for previous one. It might create a fairness issue that expiration
// might happen for smaller writes but larger writes can go through.
// Can optimize it if it is an issue.
InstrumentedMutexLock l ( & mutex_ ) ;
status = DelayWrite ( last_batch_group_size_ , write_options ) ;
PERF_TIMER_START ( write_pre_and_post_process_time ) ;
}
@ -1180,11 +1183,12 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
if ( write_options . no_slowdown ) {
status = Status : : Incomplete ( " Write stall " ) ;
} else {
InstrumentedMutexLock l ( & mutex_ ) ;
WriteBufferManagerStallWrites ( ) ;
}
}
if ( status . ok ( ) & & * need_log_sync ) {
InstrumentedMutexLock l ( & log_write_mutex_ ) ;
if ( status . ok ( ) & & log_context - > need_log_sync ) {
// Wait until the parallel syncs are finished. Any sync process has to sync
// the front log too so it is enough to check the status of front()
// We do a while loop since log_sync_cv_ is signalled when any sync is
@ -1204,8 +1208,12 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
log . PrepareForSync ( ) ;
}
} else {
* need_log_sync = false ;
log_context - > need_log_sync = false ;
}
log_context - > writer = logs_ . back ( ) . writer ;
log_context - > need_log_dir_sync =
log_context - > need_log_dir_sync & & ! log_dir_synced_ ;
log_context - > log_file_number_size = std : : addressof ( alive_log_files_ . back ( ) ) ;
return status ;
}
@ -1714,10 +1722,12 @@ Status DBImpl::HandleWriteBufferManagerFlush(WriteContext* write_context) {
}
uint64_t DBImpl : : GetMaxTotalWalSize ( ) const {
mutex_ . AssertHeld ( ) ;
return mutable_db_options_ . max_total_wal_size = = 0
? 4 * max_total_in_memory_state_
: mutable_db_options_ . max_total_wal_size ;
uint64_t max_total_wal_size =
max_total_wal_size_ . load ( std : : memory_order_acquire ) ;
if ( max_total_wal_size > 0 ) {
return max_total_wal_size ;
}
return 4 * max_total_in_memory_state_ . load ( std : : memory_order_acquire ) ;
}
// REQUIRES: mutex_ is held
@ -2065,7 +2075,7 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
log_recycle_files_ . pop_front ( ) ;
}
if ( s . ok ( ) & & creating_new_log ) {
log_write_mutex_ . Lock ( ) ;
InstrumentedMutexLock l ( & log_write_mutex_ ) ;
assert ( new_log ! = nullptr ) ;
if ( ! logs_ . empty ( ) ) {
// Alway flush the buffer of the last log before switching to a new one
@ -2089,7 +2099,6 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
logs_ . emplace_back ( logfile_number_ , new_log ) ;
alive_log_files_ . push_back ( LogFileNumberSize ( logfile_number_ ) ) ;
}
log_write_mutex_ . Unlock ( ) ;
}
if ( ! s . ok ( ) ) {