mirror of https://github.com/facebook/rocksdb.git
Improve write time breakdown stats
Summary: There's a group of stats in PerfContext for profiling the write path. They break down the write time into WAL write, memtable insert, throttling, and everything else. We use these stats a lot for figuring out the cause of slow writes. These stats got a bit out of date and are now categorizing some interesting things as "everything else", and also do some double counting. This PR fixes it and adds two new stats: time spent waiting for other threads of the batch group, and time spent waiting for scheduling flushes/compactions. Probably these will be enough to explain all the occasional abnormally slow (multiple seconds) writes that we're seeing. Closes https://github.com/facebook/rocksdb/pull/3602 Differential Revision: D7251562 Pulled By: al13n321 fbshipit-source-id: 0a2d0f5a4fa5677455e1f566da931cb46efe2a0d
This commit is contained in:
parent
d5afa73789
commit
affe01b0d5
|
@ -4,6 +4,7 @@
|
|||
* Add a BlockBasedTableOption to align uncompressed data blocks on the smaller of block size or page size boundary, to reduce flash reads by avoiding reads spanning 4K pages.
|
||||
* The background thread naming convention changed (on supporting platforms) to "rocksdb:<thread pool priority><thread number>", e.g., "rocksdb:low0".
|
||||
* Add a new ticker stat rocksdb.number.multiget.keys.found to count number of keys successfully read in MultiGet calls
|
||||
* Touch-up to write-related counters in PerfContext. New counters added: write_scheduling_flushes_compactions_time, write_thread_wait_nanos. Counters whose behavior was fixed or modified: write_memtable_time, write_pre_and_post_process_time, write_delay_time.
|
||||
|
||||
### New Features
|
||||
* Introduce TTL for level compaction so that all files older than ttl go through the compaction process to get rid of old data.
|
||||
|
|
|
@ -122,15 +122,19 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
|||
write_thread_.JoinBatchGroup(&w);
|
||||
if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) {
|
||||
// we are a non-leader in a parallel group
|
||||
PERF_TIMER_GUARD(write_memtable_time);
|
||||
|
||||
if (w.ShouldWriteToMemtable()) {
|
||||
PERF_TIMER_STOP(write_pre_and_post_process_time);
|
||||
PERF_TIMER_GUARD(write_memtable_time);
|
||||
|
||||
ColumnFamilyMemTablesImpl column_family_memtables(
|
||||
versions_->GetColumnFamilySet());
|
||||
w.status = WriteBatchInternal::InsertInto(
|
||||
&w, w.sequence, &column_family_memtables, &flush_scheduler_,
|
||||
write_options.ignore_missing_column_families, 0 /*log_number*/, this,
|
||||
true /*concurrent_memtable_writes*/, seq_per_batch_, w.batch_cnt);
|
||||
|
||||
PERF_TIMER_START(write_pre_and_post_process_time);
|
||||
}
|
||||
|
||||
if (write_thread_.CompleteParallelMemTableWriter(&w)) {
|
||||
|
@ -190,7 +194,13 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
|||
// 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
|
||||
|
||||
// PreprocessWrite does its own perf timing.
|
||||
PERF_TIMER_STOP(write_pre_and_post_process_time);
|
||||
|
||||
status = PreprocessWrite(write_options, &need_log_sync, &write_context);
|
||||
|
||||
PERF_TIMER_START(write_pre_and_post_process_time);
|
||||
}
|
||||
log::Writer* log_writer = logs_.back().writer;
|
||||
|
||||
|
@ -353,7 +363,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
|
|||
MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
|
||||
mutex_.Unlock();
|
||||
// Requesting sync with two_write_queues_ is expected to be very rare. We
|
||||
// hance provide a simple implementation that is not necessarily efficient.
|
||||
// hence provide a simple implementation that is not necessarily efficient.
|
||||
if (two_write_queues_) {
|
||||
if (manual_wal_flush_) {
|
||||
status = FlushWAL(true);
|
||||
|
@ -414,7 +424,10 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options,
|
|||
mutex_.Lock();
|
||||
bool need_log_sync = !write_options.disableWAL && write_options.sync;
|
||||
bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
|
||||
// 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);
|
||||
PERF_TIMER_START(write_pre_and_post_process_time);
|
||||
log::Writer* log_writer = logs_.back().writer;
|
||||
mutex_.Unlock();
|
||||
|
||||
|
@ -704,6 +717,8 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
|
|||
assert(write_context != nullptr && need_log_sync != nullptr);
|
||||
Status status;
|
||||
|
||||
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_ &&
|
||||
|
@ -728,14 +743,19 @@ Status DBImpl::PreprocessWrite(const WriteOptions& write_options,
|
|||
status = ScheduleFlushes(write_context);
|
||||
}
|
||||
|
||||
PERF_TIMER_STOP(write_scheduling_flushes_compactions_time);
|
||||
PERF_TIMER_GUARD(write_pre_and_post_process_time);
|
||||
|
||||
if (UNLIKELY(status.ok() && (write_controller_.IsStopped() ||
|
||||
write_controller_.NeedsDelay()))) {
|
||||
PERF_TIMER_STOP(write_pre_and_post_process_time);
|
||||
PERF_TIMER_GUARD(write_delay_time);
|
||||
// We don't know size of curent batch so that we always use the size
|
||||
// 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.
|
||||
status = DelayWrite(last_batch_group_size_, write_options);
|
||||
PERF_TIMER_START(write_pre_and_post_process_time);
|
||||
}
|
||||
|
||||
if (status.ok() && *need_log_sync) {
|
||||
|
@ -1189,6 +1209,7 @@ Status DBImpl::ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options,
|
|||
// is that in case the write is heavy, low pri writes may never have
|
||||
// a chance to run. Now we guarantee we are still slowly making
|
||||
// progress.
|
||||
PERF_TIMER_GUARD(write_delay_time);
|
||||
write_controller_.low_pri_rate_limiter()->Request(
|
||||
my_batch->GetDataSize(), Env::IO_HIGH, nullptr /* stats */,
|
||||
RateLimiter::OpType::kWrite);
|
||||
|
|
|
@ -227,6 +227,9 @@ void ProfileQueries(bool enabled_time = false) {
|
|||
HistogramImpl hist_write_pre_post;
|
||||
HistogramImpl hist_write_wal_time;
|
||||
HistogramImpl hist_write_memtable_time;
|
||||
HistogramImpl hist_write_delay_time;
|
||||
HistogramImpl hist_write_thread_wait_nanos;
|
||||
HistogramImpl hist_write_scheduling_time;
|
||||
|
||||
uint64_t total_db_mutex_nanos = 0;
|
||||
|
||||
|
@ -270,9 +273,15 @@ void ProfileQueries(bool enabled_time = false) {
|
|||
ThreadStatusUtil::TEST_SetStateDelay(ThreadStatus::STATE_MUTEX_WAIT, 0U);
|
||||
#endif
|
||||
}
|
||||
hist_write_pre_post.Add(get_perf_context()->write_pre_and_post_process_time);
|
||||
hist_write_pre_post.Add(
|
||||
get_perf_context()->write_pre_and_post_process_time);
|
||||
hist_write_wal_time.Add(get_perf_context()->write_wal_time);
|
||||
hist_write_memtable_time.Add(get_perf_context()->write_memtable_time);
|
||||
hist_write_delay_time.Add(get_perf_context()->write_delay_time);
|
||||
hist_write_thread_wait_nanos.Add(
|
||||
get_perf_context()->write_thread_wait_nanos);
|
||||
hist_write_scheduling_time.Add(
|
||||
get_perf_context()->write_scheduling_flushes_compactions_time);
|
||||
hist_put.Add(get_perf_context()->user_key_comparison_count);
|
||||
total_db_mutex_nanos += get_perf_context()->db_mutex_lock_nanos;
|
||||
}
|
||||
|
@ -320,6 +329,11 @@ void ProfileQueries(bool enabled_time = false) {
|
|||
<< hist_write_wal_time.ToString() << "\n"
|
||||
<< " Writing Mem Table time: \n"
|
||||
<< hist_write_memtable_time.ToString() << "\n"
|
||||
<< " Write Delay: \n" << hist_write_delay_time.ToString() << "\n"
|
||||
<< " Waiting for Batch time: \n"
|
||||
<< hist_write_thread_wait_nanos.ToString() << "\n"
|
||||
<< " Scheduling Flushes and Compactions Time: \n"
|
||||
<< hist_write_scheduling_time.ToString() << "\n"
|
||||
<< " Total DB mutex nanos: \n" << total_db_mutex_nanos << "\n";
|
||||
|
||||
std::cout << "Get(): Time to get snapshot: \n"
|
||||
|
@ -359,6 +373,14 @@ void ProfileQueries(bool enabled_time = false) {
|
|||
ASSERT_GT(hist_mget_files.Average(), 0);
|
||||
ASSERT_GT(hist_mget_post_process.Average(), 0);
|
||||
ASSERT_GT(hist_mget_num_memtable_checked.Average(), 0);
|
||||
|
||||
EXPECT_GT(hist_write_pre_post.Average(), 0);
|
||||
EXPECT_GT(hist_write_wal_time.Average(), 0);
|
||||
EXPECT_GT(hist_write_memtable_time.Average(), 0);
|
||||
EXPECT_EQ(hist_write_delay_time.Average(), 0);
|
||||
EXPECT_EQ(hist_write_thread_wait_nanos.Average(), 0);
|
||||
EXPECT_GT(hist_write_scheduling_time.Average(), 0);
|
||||
|
||||
#ifndef NDEBUG
|
||||
ASSERT_GT(total_db_mutex_nanos, 2000U);
|
||||
#endif
|
||||
|
|
|
@ -7,6 +7,7 @@
|
|||
#include <chrono>
|
||||
#include <thread>
|
||||
#include "db/column_family.h"
|
||||
#include "monitoring/perf_context_imp.h"
|
||||
#include "port/port.h"
|
||||
#include "util/random.h"
|
||||
#include "util/sync_point.h"
|
||||
|
@ -73,6 +74,10 @@ uint8_t WriteThread::AwaitState(Writer* w, uint8_t goal_mask,
|
|||
port::AsmVolatilePause();
|
||||
}
|
||||
|
||||
// This is below the fast path, so that the stat is zero when all writes are
|
||||
// from the same thread.
|
||||
PERF_TIMER_GUARD(write_thread_wait_nanos);
|
||||
|
||||
// If we're only going to end up waiting a short period of time,
|
||||
// it can be a lot more efficient to call std::this_thread::yield()
|
||||
// in a loop than to block in StateMutex(). For reference, on my 4.0
|
||||
|
|
|
@ -95,16 +95,27 @@ struct PerfContext {
|
|||
// total nanos spent on iterating internal entries to find the next user entry
|
||||
uint64_t find_next_user_entry_time;
|
||||
|
||||
// This group of stats provide a breakdown of time spent by Write().
|
||||
// May be inaccurate when 2PC, two_write_queues or enable_pipelined_write
|
||||
// are enabled.
|
||||
//
|
||||
// total nanos spent on writing to WAL
|
||||
uint64_t write_wal_time;
|
||||
// total nanos spent on writing to mem tables
|
||||
uint64_t write_memtable_time;
|
||||
// total nanos spent on delaying write
|
||||
// total nanos spent on delaying or throttling write
|
||||
uint64_t write_delay_time;
|
||||
// total nanos spent on writing a record, excluding the above three times
|
||||
// total nanos spent on switching memtable/wal and scheduling
|
||||
// flushes/compactions.
|
||||
uint64_t write_scheduling_flushes_compactions_time;
|
||||
// total nanos spent on writing a record, excluding the above four things
|
||||
uint64_t write_pre_and_post_process_time;
|
||||
|
||||
uint64_t db_mutex_lock_nanos; // time spent on acquiring DB mutex.
|
||||
// time spent waiting for other threads of the batch group
|
||||
uint64_t write_thread_wait_nanos;
|
||||
|
||||
// time spent on acquiring DB mutex.
|
||||
uint64_t db_mutex_lock_nanos;
|
||||
// Time spent on waiting with a condition variable created with DB mutex.
|
||||
uint64_t db_condition_wait_nanos;
|
||||
// Time spent on merge operator.
|
||||
|
|
|
@ -66,6 +66,8 @@ void PerfContext::Reset() {
|
|||
write_pre_and_post_process_time = 0;
|
||||
write_memtable_time = 0;
|
||||
write_delay_time = 0;
|
||||
write_thread_wait_nanos = 0;
|
||||
write_scheduling_flushes_compactions_time = 0;
|
||||
db_mutex_lock_nanos = 0;
|
||||
db_condition_wait_nanos = 0;
|
||||
merge_operator_time_nanos = 0;
|
||||
|
@ -146,6 +148,8 @@ std::string PerfContext::ToString(bool exclude_zero_counters) const {
|
|||
PERF_CONTEXT_OUTPUT(find_next_user_entry_time);
|
||||
PERF_CONTEXT_OUTPUT(write_pre_and_post_process_time);
|
||||
PERF_CONTEXT_OUTPUT(write_memtable_time);
|
||||
PERF_CONTEXT_OUTPUT(write_thread_wait_nanos);
|
||||
PERF_CONTEXT_OUTPUT(write_scheduling_flushes_compactions_time);
|
||||
PERF_CONTEXT_OUTPUT(db_mutex_lock_nanos);
|
||||
PERF_CONTEXT_OUTPUT(db_condition_wait_nanos);
|
||||
PERF_CONTEXT_OUTPUT(merge_operator_time_nanos);
|
||||
|
|
Loading…
Reference in New Issue