@ -66,6 +66,10 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
if ( my_batch = = nullptr ) {
return Status : : Corruption ( " Batch is nullptr! " ) ;
}
if ( concurrent_prepare_ & & immutable_db_options_ . enable_pipelined_write ) {
return Status : : NotSupported (
" pipelined_writes is not compatible with concurrent prepares " ) ;
}
Status status ;
if ( write_options . low_pri ) {
@ -75,6 +79,11 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
}
}
if ( concurrent_prepare_ & & disable_memtable ) {
return WriteImplWALOnly ( write_options , my_batch , callback , log_used ,
log_ref ) ;
}
if ( immutable_db_options_ . enable_pipelined_write ) {
return PipelinedWriteImpl ( write_options , my_batch , callback , log_used ,
log_ref , disable_memtable ) ;
@ -133,14 +142,22 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
WriteContext write_context ;
WriteThread : : WriteGroup write_group ;
bool in_parallel_group = false ;
uint64_t last_sequence = versions_ - > LastSequence ( ) ;
uint64_t last_sequence ;
if ( ! concurrent_prepare_ ) {
last_sequence = versions_ - > LastSequence ( ) ;
}
mutex_ . Lock ( ) ;
bool need_log_sync = ! write_options . disableWAL & & write_options . sync ;
bool need_log_dir_sync = need_log_sync & & ! log_dir_synced_ ;
status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
log : : Writer * cur_log_writer = logs_ . back ( ) . writer ;
if ( ! concurrent_prepare_ | | ! disable_memtable ) {
// With concurrent writes we do preprocess only in the write thread that
// also does write to memtable to avoid sync issue on shared data structure
// with the other thread
status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
}
log : : Writer * log_writer = logs_ . back ( ) . writer ;
mutex_ . Unlock ( ) ;
@ -180,9 +197,9 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
}
}
const SequenceNumber current_sequence = last_sequence + 1 ;
last_sequence + = total_count ;
if ( concurrent_prepare_ ) {
stat_mutex_ . Lock ( ) ;
}
// Update stats while we are an exclusive group leader, so we know
// that nobody else can be writing to these particular stats.
// We're optimistic, updating the stats before we successfully
@ -201,6 +218,9 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
RecordTick ( stats_ , WRITE_DONE_BY_OTHER , write_done_by_other ) ;
}
MeasureTime ( stats_ , BYTES_PER_WRITE , total_byte_size ) ;
if ( concurrent_prepare_ ) {
stat_mutex_ . Unlock ( ) ;
}
if ( write_options . disableWAL ) {
has_unpersisted_data_ . store ( true , std : : memory_order_relaxed ) ;
@ -208,14 +228,27 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
PERF_TIMER_STOP ( write_pre_and_post_process_time ) ;
if ( status . ok ( ) & & ! write_options . disableWAL ) {
PERF_TIMER_GUARD ( write_wal_time ) ;
status = WriteToWAL ( write_group , cur_log_writer , need_log_sync ,
need_log_dir_sync , current_sequence ) ;
if ( log_used ! = nullptr ) {
* log_used = logfile_number_ ;
if ( ! concurrent_prepare_ ) {
if ( status . ok ( ) & & ! write_options . disableWAL ) {
PERF_TIMER_GUARD ( write_wal_time ) ;
status = WriteToWAL ( write_group , log_writer , log_used , need_log_sync ,
need_log_dir_sync , last_sequence + 1 ) ;
}
} else {
assert ( ! need_log_sync & & ! need_log_dir_sync ) ;
if ( status . ok ( ) & & ! write_options . disableWAL ) {
PERF_TIMER_GUARD ( write_wal_time ) ;
// LastToBeWrittenSequence is increased inside WriteToWAL under
// wal_write_mutex_ to ensure ordered events in WAL
status = ConcurrentWriteToWAL ( write_group , log_used , & last_sequence ,
total_count ) ;
} else {
// Otherwise we inc seq number for memtable writes
last_sequence = versions_ - > FetchAddLastToBeWrittenSequence ( total_count ) ;
}
}
const SequenceNumber current_sequence = last_sequence + 1 ;
last_sequence + = total_count ;
if ( status . ok ( ) ) {
PERF_TIMER_GUARD ( write_memtable_time ) ;
@ -263,6 +296,15 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
mutex_ . Lock ( ) ;
MarkLogsSynced ( logfile_number_ , need_log_dir_sync , status ) ;
mutex_ . Unlock ( ) ;
// Requesting sync with concurrent_prepare_ is expected to be very rare. We
// hance provide a simple implementation that is not necessarily efficient.
if ( concurrent_prepare_ ) {
if ( manual_wal_flush_ ) {
status = FlushWAL ( true ) ;
} else {
status = SyncWAL ( ) ;
}
}
}
bool should_exit_batch_group = true ;
@ -272,7 +314,9 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
should_exit_batch_group = write_thread_ . CompleteParallelMemTableWriter ( & w ) ;
}
if ( should_exit_batch_group ) {
versions_ - > SetLastSequence ( last_sequence ) ;
if ( status . ok ( ) ) {
versions_ - > SetLastSequence ( last_sequence ) ;
}
MemTableInsertStatusCheck ( w . status ) ;
write_thread_ . ExitAsBatchGroupLeader ( write_group , w . status ) ;
}
@ -304,7 +348,7 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
bool need_log_sync = ! write_options . disableWAL & & write_options . sync ;
bool need_log_dir_sync = need_log_sync & & ! log_dir_synced_ ;
w . status = PreprocessWrite ( write_options , & need_log_sync , & write_context ) ;
log : : Writer * cur_ log_writer = logs_ . back ( ) . writer ;
log : : Writer * log_writer = logs_ . back ( ) . writer ;
mutex_ . Unlock ( ) ;
// This can set non-OK status if callback fail.
@ -352,8 +396,8 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
wal_write_group . size - 1 ) ;
RecordTick ( stats_ , WRITE_DONE_BY_OTHER , wal_write_group . size - 1 ) ;
}
w . status = WriteToWAL ( wal_write_group , cur_ log_writer, need_log_sync ,
need_log_dir_sync , current_sequence ) ;
w . status = WriteToWAL ( wal_write_group , log_writer , log_used ,
need_log_sync , need_log_ dir_sync , current_sequence ) ;
}
if ( ! w . CallbackFailed ( ) ) {
@ -403,11 +447,91 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
}
assert ( w . state = = WriteThread : : STATE_COMPLETED ) ;
if ( log_used ! = nullptr ) {
* log_used = w . log_used ;
return w . FinalStatus ( ) ;
}
Status DBImpl : : WriteImplWALOnly ( const WriteOptions & write_options ,
WriteBatch * my_batch , WriteCallback * callback ,
uint64_t * log_used , uint64_t log_ref ) {
Status status ;
PERF_TIMER_GUARD ( write_pre_and_post_process_time ) ;
WriteThread : : Writer w ( write_options , my_batch , callback , log_ref ,
true /* disable_memtable */ ) ;
if ( write_options . disableWAL ) {
return status ;
}
RecordTick ( stats_ , WRITE_WITH_WAL ) ;
StopWatch write_sw ( env_ , immutable_db_options_ . statistics . get ( ) , DB_WRITE ) ;
return w . FinalStatus ( ) ;
nonmem_write_thread_ . JoinBatchGroup ( & w ) ;
assert ( w . state ! = WriteThread : : STATE_PARALLEL_MEMTABLE_WRITER ) ;
if ( w . state = = WriteThread : : STATE_COMPLETED ) {
if ( log_used ! = nullptr ) {
* log_used = w . log_used ;
}
return w . FinalStatus ( ) ;
}
// else we are the leader of the write batch group
assert ( w . state = = WriteThread : : STATE_GROUP_LEADER ) ;
WriteContext write_context ;
WriteThread : : WriteGroup write_group ;
uint64_t last_sequence ;
nonmem_write_thread_ . EnterAsBatchGroupLeader ( & w , & write_group ) ;
// Note: no need to update last_batch_group_size_ here since the batch writes
// to WAL only
uint64_t total_byte_size = 0 ;
for ( auto * writer : write_group ) {
if ( writer - > CheckCallback ( this ) ) {
total_byte_size = WriteBatchInternal : : AppendedByteSize (
total_byte_size , WriteBatchInternal : : ByteSize ( writer - > batch ) ) ;
}
}
stat_mutex_ . Lock ( ) ;
// Update stats while we are an exclusive group leader, so we know
// that nobody else can be writing to these particular stats.
// We're optimistic, updating the stats before we successfully
// commit. That lets us release our leader status early.
auto stats = default_cf_internal_stats_ ;
stats - > AddDBStats ( InternalStats : : BYTES_WRITTEN , total_byte_size ) ;
RecordTick ( stats_ , BYTES_WRITTEN , total_byte_size ) ;
stats - > AddDBStats ( InternalStats : : WRITE_DONE_BY_SELF , 1 ) ;
RecordTick ( stats_ , WRITE_DONE_BY_SELF ) ;
auto write_done_by_other = write_group . size - 1 ;
if ( write_done_by_other > 0 ) {
stats - > AddDBStats ( InternalStats : : WRITE_DONE_BY_OTHER , write_done_by_other ) ;
RecordTick ( stats_ , WRITE_DONE_BY_OTHER , write_done_by_other ) ;
}
MeasureTime ( stats_ , BYTES_PER_WRITE , total_byte_size ) ;
stat_mutex_ . Unlock ( ) ;
PERF_TIMER_STOP ( write_pre_and_post_process_time ) ;
PERF_TIMER_GUARD ( write_wal_time ) ;
// LastToBeWrittenSequence is increased inside WriteToWAL under
// wal_write_mutex_ to ensure ordered events in WAL
status = ConcurrentWriteToWAL ( write_group , log_used , & last_sequence ,
0 /*total_count*/ ) ;
if ( status . ok ( ) & & write_options . sync ) {
// Requesting sync with concurrent_prepare_ is expected to be very rare. We
// hance provide a simple implementation that is not necessarily efficient.
if ( manual_wal_flush_ ) {
status = FlushWAL ( true ) ;
} else {
status = SyncWAL ( ) ;
}
}
PERF_TIMER_START ( write_pre_and_post_process_time ) ;
if ( ! w . CallbackFailed ( ) ) {
ParanoidCheck ( status ) ;
}
nonmem_write_thread_ . ExitAsBatchGroupLeader ( write_group , w . status ) ;
if ( status . ok ( ) ) {
status = w . FinalStatus ( ) ;
}
return status ;
}
void DBImpl : : WriteCallbackStatusCheck ( const Status & status ) {
@ -519,13 +643,12 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
return status ;
}
Status DBImpl : : WriteToWAL ( const WriteThread : : WriteGroup & write_group ,
log : : Writer * log_writer , bool need_log_sync ,
bool need_log_dir_sync , SequenceNumber sequence ) {
Status status ;
WriteBatch * DBImpl : : MergeBatch ( const WriteThread : : WriteGroup & write_group ,
WriteBatch * tmp_batch , size_t * write_with_wal ) {
assert ( write_with_wal ! = nullptr ) ;
assert ( tmp_batch ! = nullptr ) ;
WriteBatch * merged_batch = nullptr ;
size_t write_with_wal = 0 ;
* write_with_wal = 0 ;
auto * leader = write_group . leader ;
if ( write_group . size = = 1 & & leader - > ShouldWriteToWAL ( ) & &
leader - > batch - > GetWalTerminationPoint ( ) . is_cleared ( ) ) {
@ -534,30 +657,54 @@ Status DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group,
// and the batch is not wanting to be truncated
merged_batch = leader - > batch ;
leader - > log_used = logfile_number_ ;
write_with_wal = 1 ;
* write_with_wal = 1 ;
} else {
// WAL needs all of the batches flattened into a single batch.
// We could avoid copying here with an iov-like AddRecord
// interface
merged_batch = & tmp_batch_ ;
merged_batch = tmp_batch ;
for ( auto writer : write_group ) {
if ( writer - > ShouldWriteToWAL ( ) ) {
WriteBatchInternal : : Append ( merged_batch , writer - > batch ,
/*WAL_only*/ true ) ;
write_with_wal + + ;
( * write_with_wal ) + + ;
}
writer - > log_used = logfile_number_ ;
}
}
return merged_batch ;
}
WriteBatchInternal : : SetSequence ( merged_batch , sequence ) ;
Slice log_entry = WriteBatchInternal : : Contents ( merged_batch ) ;
status = log_writer - > AddRecord ( log_entry ) ;
Status DBImpl : : WriteToWAL ( const WriteBatch & merged_batch ,
log : : Writer * log_writer , uint64_t * log_used ,
uint64_t * log_size ) {
assert ( log_size ! = nullptr ) ;
Slice log_entry = WriteBatchInternal : : Contents ( & merged_batch ) ;
* log_size = log_entry . size ( ) ;
Status status = log_writer - > AddRecord ( log_entry ) ;
if ( log_used ! = nullptr ) {
* log_used = logfile_number_ ;
}
total_log_size_ + = log_entry . size ( ) ;
alive_log_files_ . back ( ) . AddSize ( log_entry . size ( ) ) ;
log_empty_ = false ;
uint64_t log_size = log_entry . size ( ) ;
return status ;
}
Status DBImpl : : WriteToWAL ( const WriteThread : : WriteGroup & write_group ,
log : : Writer * log_writer , uint64_t * log_used ,
bool need_log_sync , bool need_log_dir_sync ,
SequenceNumber sequence ) {
Status status ;
size_t write_with_wal = 0 ;
WriteBatch * merged_batch =
MergeBatch ( write_group , & tmp_batch_ , & write_with_wal ) ;
WriteBatchInternal : : SetSequence ( merged_batch , sequence ) ;
uint64_t log_size ;
status = WriteToWAL ( * merged_batch , log_writer , log_used , & log_size ) ;
if ( status . ok ( ) & & need_log_sync ) {
StopWatch sw ( env_ , stats_ , WAL_FILE_SYNC_MICROS ) ;
@ -599,6 +746,41 @@ Status DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group,
return status ;
}
Status DBImpl : : ConcurrentWriteToWAL ( const WriteThread : : WriteGroup & write_group ,
uint64_t * log_used ,
SequenceNumber * last_sequence ,
int total_count ) {
Status status ;
WriteBatch tmp_batch ;
size_t write_with_wal = 0 ;
WriteBatch * merged_batch =
MergeBatch ( write_group , & tmp_batch , & write_with_wal ) ;
// We need to lock log_write_mutex_ since logs_ and alive_log_files might be
// pushed back concurrently
log_write_mutex_ . Lock ( ) ;
* last_sequence = versions_ - > FetchAddLastToBeWrittenSequence ( total_count ) ;
auto sequence = * last_sequence + 1 ;
WriteBatchInternal : : SetSequence ( merged_batch , sequence ) ;
log : : Writer * log_writer = logs_ . back ( ) . writer ;
uint64_t log_size ;
status = WriteToWAL ( * merged_batch , log_writer , log_used , & log_size ) ;
log_write_mutex_ . Unlock ( ) ;
if ( status . ok ( ) ) {
stat_mutex_ . Lock ( ) ;
auto stats = default_cf_internal_stats_ ;
stats - > AddDBStats ( InternalStats : : WAL_FILE_BYTES , log_size ) ;
RecordTick ( stats_ , WAL_FILE_BYTES , log_size ) ;
stats - > AddDBStats ( InternalStats : : WRITE_WITH_WAL , write_with_wal ) ;
RecordTick ( stats_ , WRITE_WITH_WAL , write_with_wal ) ;
stat_mutex_ . Unlock ( ) ;
}
return status ;
}
Status DBImpl : : HandleWALFull ( WriteContext * write_context ) {
mutex_ . AssertHeld ( ) ;
assert ( write_context ! = nullptr ) ;
@ -895,9 +1077,9 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
lfile - > SetPreallocationBlockSize ( preallocate_block_size ) ;
unique_ptr < WritableFileWriter > file_writer (
new WritableFileWriter ( std : : move ( lfile ) , opt_env_opt ) ) ;
new_log =
new log : : Writer ( std : : move ( file_writer ) , new_log_number ,
immutable_db_options_ . recycle_log_file_num > 0 ) ;
new_log = new log : : Writer (
std : : move ( file_writer ) , new_log_number ,
immutable_db_options_ . recycle_log_file_num > 0 , manual_wal_flush_ ) ;
}
}
@ -931,8 +1113,15 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
assert ( new_log ! = nullptr ) ;
log_empty_ = true ;
log_dir_synced_ = false ;
log_write_mutex_ . Lock ( ) ;
if ( ! logs_ . empty ( ) ) {
// Alway flush the buffer of the last log before switching to a new one
log : : Writer * cur_log_writer = logs_ . back ( ) . writer ;
cur_log_writer - > WriteBuffer ( ) ;
}
logs_ . emplace_back ( logfile_number_ , new_log ) ;
alive_log_files_ . push_back ( LogFileNumberSize ( logfile_number_ ) ) ;
log_write_mutex_ . Unlock ( ) ;
}
for ( auto loop_cfd : * versions_ - > GetColumnFamilySet ( ) ) {
// all this is just optimization to delete logs that