Set logs as getting flushed before releasing lock, race condition fix
Summary: Relating to #1903: In MaybeFlushColumnFamilies() we want to modify the 'getting_flushed' flag before releasing the db mutex when SwitchMemtable() is called. The following 2 actions need to be atomic in MaybeFlushColumnFamilies() - getting_flushed is false on oldest log - we determine that all CFs can be flushed to successfully release oldest log - we set getting_flushed = true on the oldest log. ------- - getting_flushed is false on oldest log - we determine that all CFs can NOT be flushed to successfully release oldest log - we set unable_to_flush_oldest_log_ = true on the oldest log. #### In the 2pc case: T1 enters function but is unable to flush all CFs to release log T1 sets unable_to_flush_oldest_log_ = true T1 begins flushing all CFs possible T2 enters function but is unable to flush all CFs to release log T2 sees unable_to_flush_oldes_log_ has been set so exits T3 enters function and will be able to flush all CFs to release oldest log T3 sets getting_flushed = true on oldes Closes https://github.com/facebook/rocksdb/pull/1909 Differential Revision: D4646235 Pulled By: reidHoruff fbshipit-source-id: c8d0447
This commit is contained in:
parent
f8a4ea0206
commit
58b12dfe37
@ -5053,14 +5053,25 @@ void DBImpl::MaybeFlushColumnFamilies() {
|
|||||||
auto oldest_log_with_uncommited_prep = FindMinLogContainingOutstandingPrep();
|
auto oldest_log_with_uncommited_prep = FindMinLogContainingOutstandingPrep();
|
||||||
|
|
||||||
if (allow_2pc() &&
|
if (allow_2pc() &&
|
||||||
unable_to_flush_oldest_log_ &&
|
|
||||||
oldest_log_with_uncommited_prep > 0 &&
|
oldest_log_with_uncommited_prep > 0 &&
|
||||||
oldest_log_with_uncommited_prep <= oldest_alive_log) {
|
oldest_log_with_uncommited_prep <= oldest_alive_log) {
|
||||||
|
if (unable_to_flush_oldest_log_) {
|
||||||
// we already attempted to flush all column families dependent on
|
// we already attempted to flush all column families dependent on
|
||||||
// the oldest alive log but the log still contained uncommited transactions.
|
// the oldest alive log but the log still contained uncommited transactions.
|
||||||
// the oldest alive log STILL contains uncommited transaction so there
|
// the oldest alive log STILL contains uncommited transaction so there
|
||||||
// is still nothing that we can do.
|
// is still nothing that we can do.
|
||||||
return;
|
return;
|
||||||
|
} else {
|
||||||
|
Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log,
|
||||||
|
"Unable to release oldest log due to uncommited transaction");
|
||||||
|
unable_to_flush_oldest_log_ = true;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// we only mark this log as getting flushed if we have successfully
|
||||||
|
// flushed all data in this log. If this log contains outstanding prepared
|
||||||
|
// transactions then we cannot flush this log until those transactions are commited.
|
||||||
|
unable_to_flush_oldest_log_ = false;
|
||||||
|
alive_log_files_.begin()->getting_flushed = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
WriteContext context;
|
WriteContext context;
|
||||||
@ -5086,25 +5097,6 @@ void DBImpl::MaybeFlushColumnFamilies() {
|
|||||||
}
|
}
|
||||||
MaybeScheduleFlushOrCompaction();
|
MaybeScheduleFlushOrCompaction();
|
||||||
|
|
||||||
// we only mark this log as getting flushed if we have successfully
|
|
||||||
// flushed all data in this log. If this log contains outstanding prepred
|
|
||||||
// transactions then we cannot flush this log until those transactions are commited.
|
|
||||||
|
|
||||||
unable_to_flush_oldest_log_ = false;
|
|
||||||
|
|
||||||
if (allow_2pc()) {
|
|
||||||
if (oldest_log_with_uncommited_prep == 0 ||
|
|
||||||
oldest_log_with_uncommited_prep > oldest_alive_log) {
|
|
||||||
// this log contains no outstanding prepared transactions
|
|
||||||
alive_log_files_.begin()->getting_flushed = true;
|
|
||||||
} else {
|
|
||||||
Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log,
|
|
||||||
"Unable to release oldest log due to uncommited transaction");
|
|
||||||
unable_to_flush_oldest_log_ = true;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
alive_log_files_.begin()->getting_flushed = true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
uint64_t DBImpl::GetMaxTotalWalSize() const {
|
uint64_t DBImpl::GetMaxTotalWalSize() const {
|
||||||
|
Loading…
x
Reference in New Issue
Block a user