From c9589a88564fae7da1ef05e10c7342abdbda3364 Mon Sep 17 00:00:00 2001 From: Little-Wallace Date: Mon, 19 Oct 2020 15:36:10 +0800 Subject: [PATCH] remove mutex in all write method Signed-off-by: Little-Wallace --- db/db_impl/db_impl.cc | 54 +++++++++------- db/db_impl/db_impl.h | 17 +++-- db/db_impl/db_impl_compaction_flush.cc | 6 +- db/db_impl/db_impl_files.cc | 13 +--- db/db_impl/db_impl_write.cc | 87 +++++++++++++------------- db/error_handler.cc | 11 ++++ db/error_handler.h | 20 +++--- db/perf_context_test.cc | 2 +- 8 files changed, 114 insertions(+), 96 deletions(-) diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 50e8d711a85..dce27851ed8 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -173,7 +173,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, log_dir_synced_(false), log_empty_(true), persist_stats_cf_handle_(nullptr), - log_sync_cv_(&mutex_), + log_sync_cv_(&log_write_mutex_), total_log_size_(0), is_snapshot_supported_(true), write_buffer_manager_(immutable_db_options_.write_buffer_manager.get()), @@ -267,6 +267,8 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname, // we won't drop any deletion markers until SetPreserveDeletesSequenceNumber() // is called by client and this seqnum is advanced. preserve_deletes_seqnum_.store(0); + max_total_wal_size_.store(mutable_db_options_.max_total_wal_size, + std::memory_order_relaxed); } Status DBImpl::Resume() { @@ -573,25 +575,28 @@ Status DBImpl::CloseHelper() { mutex_.Lock(); } - for (auto l : logs_to_free_) { - delete l; - } - for (auto& log : logs_) { - uint64_t log_number = log.writer->get_log_number(); - Status s = log.ClearWriter(); - if (!s.ok()) { - ROCKS_LOG_WARN( - immutable_db_options_.info_log, - "Unable to Sync WAL file %s with error -- %s", - LogFileName(immutable_db_options_.wal_dir, log_number).c_str(), - s.ToString().c_str()); - // Retain the first error - if (ret.ok()) { - ret = s; + { + InstrumentedMutexLock lock(&log_write_mutex_); + for (auto l : logs_to_free_) { + delete l; + } + for (auto& log : logs_) { + uint64_t log_number = log.writer->get_log_number(); + Status s = log.ClearWriter(); + if (!s.ok()) { + ROCKS_LOG_WARN( + immutable_db_options_.info_log, + "Unable to Sync WAL file %s with error -- %s", + LogFileName(immutable_db_options_.wal_dir, log_number).c_str(), + s.ToString().c_str()); + // Retain the first error + if (ret.ok()) { + ret = s; + } } } + logs_.clear(); } - logs_.clear(); // Table cache may have table handles holding blocks from the block cache. // We need to release them before the block cache is destroyed. The block @@ -1105,6 +1110,11 @@ Status DBImpl::SetDBOptions( new_options.stats_persist_period_sec); mutex_.Lock(); } + if (new_options.max_total_wal_size != + mutable_db_options_.max_total_wal_size) { + max_total_wal_size_.store(new_options.max_total_wal_size, + std::memory_order_release); + } write_controller_.set_max_delayed_write_rate( new_options.delayed_write_rate); table_cache_.get()->SetCapacity(new_options.max_open_files == -1 @@ -1224,7 +1234,7 @@ Status DBImpl::SyncWAL() { uint64_t current_log_number; { - InstrumentedMutexLock l(&mutex_); + InstrumentedMutexLock l(&log_write_mutex_); assert(!logs_.empty()); // This SyncWAL() call only cares about logs up to this number. @@ -1281,7 +1291,7 @@ Status DBImpl::SyncWAL() { TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1"); { - InstrumentedMutexLock l(&mutex_); + InstrumentedMutexLock l(&log_write_mutex_); MarkLogsSynced(current_log_number, need_log_dir_sync, status); } TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2"); @@ -1310,7 +1320,7 @@ Status DBImpl::UnlockWAL() { void DBImpl::MarkLogsSynced(uint64_t up_to, bool synced_dir, const Status& status) { - mutex_.AssertHeld(); + log_write_mutex_.AssertHeld(); if (synced_dir && logfile_number_ == up_to && status.ok()) { log_dir_synced_ = true; } @@ -1319,8 +1329,6 @@ void DBImpl::MarkLogsSynced(uint64_t up_to, bool synced_dir, assert(log.getting_synced); if (status.ok() && logs_.size() > 1) { logs_to_free_.push_back(log.ReleaseWriter()); - // To modify logs_ both mutex_ and log_write_mutex_ must be held - InstrumentedMutexLock l(&log_write_mutex_); it = logs_.erase(it); } else { log.getting_synced = false; @@ -2582,7 +2590,6 @@ Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options, s = cfd->AddDirectories(&dummy_created_dirs); } if (s.ok()) { - single_column_family_mode_ = false; auto* cfd = versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name); assert(cfd != nullptr); @@ -2599,6 +2606,7 @@ Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options, ROCKS_LOG_INFO(immutable_db_options_.info_log, "Created column family [%s] (ID %u)", column_family_name.c_str(), (unsigned)cfd->GetID()); + single_column_family_mode_.store(false, std::memory_order_release); } else { ROCKS_LOG_ERROR(immutable_db_options_.info_log, "Creating column family [%s] FAILED -- %s", diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index b843093db82..ef768df7380 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -1050,10 +1050,10 @@ class DBImpl : public DB { // only used for dynamically adjusting max_total_wal_size. it is a sum of // [write_buffer_size * max_write_buffer_number] over all column families - uint64_t max_total_in_memory_state_; + std::atomic max_total_in_memory_state_; // If true, we have only one (default) column family. We use this to optimize // some code-paths - bool single_column_family_mode_; + std::atomic single_column_family_mode_; // The options to access storage files const FileOptions file_options_; @@ -1257,7 +1257,13 @@ class DBImpl : public DB { } } }; - + struct LogContext { + explicit LogContext(bool need_sync = false) + : need_log_sync(need_sync), need_log_dir_sync(need_sync) {} + bool need_log_sync; + bool need_log_dir_sync; + log::Writer* writer; + }; struct LogFileNumberSize { explicit LogFileNumberSize(uint64_t _number) : number(_number) {} void AddSize(uint64_t new_size) { size += new_size; } @@ -1551,8 +1557,8 @@ class DBImpl : public DB { Status HandleWriteBufferFull(WriteContext* write_context); // REQUIRES: mutex locked - Status PreprocessWrite(const WriteOptions& write_options, bool* need_log_sync, - WriteContext* write_context); + Status PreprocessWrite(const WriteOptions& write_options, + LogContext* log_context, WriteContext* write_context); WriteBatch* MergeBatch(const WriteThread::WriteGroup& write_group, WriteBatch* tmp_batch, size_t* write_with_wal, @@ -2168,6 +2174,7 @@ class DBImpl : public DB { InstrumentedCondVar atomic_flush_install_cv_; bool wal_in_db_path_; + std::atomic max_total_wal_size_; }; extern Options SanitizeOptions(const std::string& db, const Options& src); diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index 0ea4d7d0dd7..dd6216d1f7c 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -80,7 +80,7 @@ bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force, IOStatus DBImpl::SyncClosedLogs(JobContext* job_context) { TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Start"); - mutex_.AssertHeld(); + InstrumentedMutexLock l(&log_write_mutex_); autovector logs_to_sync; uint64_t current_log_number = logfile_number_; while (logs_.front().number < current_log_number && @@ -97,7 +97,7 @@ IOStatus DBImpl::SyncClosedLogs(JobContext* job_context) { IOStatus io_s; if (!logs_to_sync.empty()) { - mutex_.Unlock(); + log_write_mutex_.Unlock(); for (log::Writer* log : logs_to_sync) { ROCKS_LOG_INFO(immutable_db_options_.info_log, @@ -119,7 +119,7 @@ IOStatus DBImpl::SyncClosedLogs(JobContext* job_context) { io_s = directories_.GetWalDir()->Fsync(IOOptions(), nullptr); } - mutex_.Lock(); + log_write_mutex_.Lock(); // "number <= current_log_number - 1" is equivalent to // "number < current_log_number". diff --git a/db/db_impl/db_impl_files.cc b/db/db_impl/db_impl_files.cc index 2d30f5857b9..25ee7b00e9d 100644 --- a/db/db_impl/db_impl_files.cc +++ b/db/db_impl/db_impl_files.cc @@ -238,6 +238,7 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, // logs_ is empty when called during recovery, in which case there can't yet // be any tracked obsolete logs + InstrumentedMutexLock l(&log_write_mutex_); if (!alive_log_files_.empty() && !logs_.empty()) { uint64_t min_log_number = job_context->log_number; size_t num_alive_log_files = alive_log_files_.size(); @@ -259,13 +260,8 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, } job_context->size_log_to_delete += earliest.size; total_log_size_ -= earliest.size; - if (two_write_queues_) { - log_write_mutex_.Lock(); - } alive_log_files_.pop_front(); - if (two_write_queues_) { - log_write_mutex_.Unlock(); - } + // Current log should always stay alive since it can't have // number < MinLogNumber(). assert(alive_log_files_.size()); @@ -278,10 +274,7 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, continue; } logs_to_free_.push_back(log.ReleaseWriter()); - { - InstrumentedMutexLock wl(&log_write_mutex_); - logs_.pop_front(); - } + logs_.pop_front(); } // Current log cannot be obsolete. assert(!logs_.empty()); diff --git a/db/db_impl/db_impl_write.cc b/db/db_impl/db_impl_write.cc index 5d4f88f0b88..80523509f65 100644 --- a/db/db_impl/db_impl_write.cc +++ b/db/db_impl/db_impl_write.cc @@ -216,14 +216,12 @@ 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_; + // The writer will only be used when two_write_queues_ is false. if (!two_write_queues_ || !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 @@ -232,7 +230,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 @@ -241,9 +239,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, PERF_TIMER_START(write_pre_and_post_process_time); } - log::Writer* log_writer = logs_.back().writer; - mutex_.Unlock(); // Add to log and apply to memtable. We can release the lock // during this phase since &w is currently responsible for logging @@ -327,8 +323,9 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, if (!two_write_queues_) { if (status.ok() && !write_options.disableWAL) { 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); + io_s = WriteToWAL(write_group, log_context.writer, log_used, + log_context.need_log_sync, + log_context.need_log_dir_sync, last_sequence + 1); } } else { if (status.ok() && !write_options.disableWAL) { @@ -424,10 +421,10 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, } } - if (need_log_sync) { - mutex_.Lock(); - MarkLogsSynced(logfile_number_, need_log_dir_sync, status); - mutex_.Unlock(); + if (log_context.need_log_sync) { + log_write_mutex_.Lock(); + MarkLogsSynced(logfile_number_, log_context.need_log_dir_sync, status); + log_write_mutex_.Unlock(); // 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_) { @@ -479,15 +476,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; - mutex_.Unlock(); // This can set non-OK status if callback fail. last_batch_group_size_ = @@ -536,8 +529,9 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, wal_write_group.size - 1); RecordTick(stats_, WRITE_DONE_BY_OTHER, wal_write_group.size - 1); } - io_s = WriteToWAL(wal_write_group, log_writer, log_used, need_log_sync, - need_log_dir_sync, current_sequence); + io_s = WriteToWAL(wal_write_group, log_context.writer, log_used, + log_context.need_log_sync, + log_context.need_log_dir_sync, current_sequence); w.status = io_s; } @@ -549,10 +543,9 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, } } - if (need_log_sync) { - mutex_.Lock(); - MarkLogsSynced(logfile_number_, need_log_dir_sync, w.status); - mutex_.Unlock(); + if (log_context.need_log_sync) { + InstrumentedMutexLock l(&log_write_mutex_); + MarkLogsSynced(logfile_number_, log_context.need_log_dir_sync, w.status); } write_thread_.ExitAsBatchGroupLeader(wal_write_group, w.status); @@ -689,9 +682,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()) { @@ -836,9 +828,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? @@ -892,22 +883,22 @@ 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(); } PERF_TIMER_GUARD(write_scheduling_flushes_compactions_time); - assert(!single_column_family_mode_ || - versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1); - if (UNLIKELY(status.ok() && !single_column_family_mode_ && + if (UNLIKELY(status.ok() && + !single_column_family_mode_.load(std::memory_order_acquire) && total_log_size_ > GetMaxTotalWalSize())) { + InstrumentedMutexLock l(&mutex_); WaitForPendingWrites(); status = SwitchWAL(write_context); } @@ -918,15 +909,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 = HandleWriteBufferFull(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); } @@ -942,11 +936,13 @@ 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); } - 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 @@ -967,8 +963,11 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options, log.getting_synced = true; } } 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_; return status; } @@ -1416,10 +1415,11 @@ Status DBImpl::HandleWriteBufferFull(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; + auto 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_; } // REQUIRES: mutex_ is held @@ -1739,7 +1739,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 @@ -1763,7 +1763,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()) { diff --git a/db/error_handler.cc b/db/error_handler.cc index 7aa4aa82689..c5120534a15 100644 --- a/db/error_handler.cc +++ b/db/error_handler.cc @@ -321,6 +321,9 @@ Status ErrorHandler::SetBGError(const Status& bg_err, BackgroundErrorReason reas RecoverFromNoSpace(); } } + if (bg_error_.severity() >= Status::Severity::kHardError) { + stop_state_.store(true, std::memory_order_release); + } return bg_error_; } @@ -350,6 +353,10 @@ Status ErrorHandler::SetBGError(const IOStatus& bg_io_err, bool auto_recovery = false; Status bg_err(new_bg_io_err, Status::Severity::kUnrecoverableError); bg_error_ = bg_err; + if (bg_error_.severity() >= Status::Severity::kHardError) { + stop_state_.store(true, std::memory_order_release); + } + if (recovery_in_prog_ && recovery_error_.ok()) { recovery_error_ = bg_err; } @@ -401,6 +408,9 @@ Status ErrorHandler::SetBGError(const IOStatus& bg_io_err, if (bg_err.severity() > bg_error_.severity()) { bg_error_ = bg_err; } + if (bg_error_.severity() >= Status::Severity::kHardError) { + stop_state_.store(true, std::memory_order_release); + } recover_context_ = context; return StartRecoverFromRetryableBGIOError(bg_io_err); } @@ -618,6 +628,7 @@ void ErrorHandler::RecoverFromRetryableBGIOError() { // the bg_error and notify user. TEST_SYNC_POINT("RecoverFromRetryableBGIOError:RecoverSuccess"); Status old_bg_error = bg_error_; + stop_state_.store(false, std::memory_order_release); bg_error_ = Status::OK(); EventHelpers::NotifyOnErrorRecoveryCompleted(db_options_.listeners, old_bg_error, db_mutex_); diff --git a/db/error_handler.h b/db/error_handler.h index 084434101aa..0b061007288 100644 --- a/db/error_handler.h +++ b/db/error_handler.h @@ -40,7 +40,8 @@ class ErrorHandler { db_mutex_(db_mutex), auto_recovery_(false), recovery_in_prog_(false), - soft_error_no_bg_work_(false) {} + soft_error_no_bg_work_(false), + stop_state_(false) {} ~ErrorHandler() { bg_error_.PermitUncheckedError(); recovery_error_.PermitUncheckedError(); @@ -63,16 +64,14 @@ class ErrorHandler { Status ClearBGError(); - bool IsDBStopped() { - return !bg_error_.ok() && - bg_error_.severity() >= Status::Severity::kHardError; - } + bool IsDBStopped() { return stop_state_.load(std::memory_order_acquire); } - bool IsBGWorkStopped() { - return !bg_error_.ok() && - (bg_error_.severity() >= Status::Severity::kHardError || - !auto_recovery_ || soft_error_no_bg_work_); - } + // this method must be protect in mutex. + bool IsBGWorkStopped() { + return !bg_error_.ok() && + (bg_error_.severity() >= Status::Severity::kHardError || + !auto_recovery_ || soft_error_no_bg_work_); + } bool IsSoftErrorNoBGWork() { return soft_error_no_bg_work_; } @@ -109,6 +108,7 @@ class ErrorHandler { // Used to store the context for recover, such as flush reason. DBRecoverContext recover_context_; + std::atomic stop_state_; Status OverrideNoSpaceError(Status bg_error, bool* auto_recovery); void RecoverFromNoSpace(); diff --git a/db/perf_context_test.cc b/db/perf_context_test.cc index 5a714b9b85a..5eb600c3bf9 100644 --- a/db/perf_context_test.cc +++ b/db/perf_context_test.cc @@ -385,7 +385,7 @@ void ProfileQueries(bool enabled_time = false) { EXPECT_GT(hist_write_scheduling_time.Average(), 0); #ifndef NDEBUG - ASSERT_GT(total_db_mutex_nanos, 2000U); + ASSERT_LT(total_db_mutex_nanos, 100U); #endif }