Refactor PerfStepTimer to stop on destruct

This eliminates the need to remember to call PERF_TIMER_STOP when a section has
been timed. This allows more useful design with the perf timers and enables
possible return value optimizations. Simplistic example:

class Foo {
  public:
    Foo(int v) : m_v(v);
  private:
    int m_v;
}

Foo makeFrobbedFoo(int *errno)
{
  *errno = 0;
  return Foo();
}

Foo bar(int *errno)
{
  PERF_TIMER_GUARD(some_timer);

  return makeFrobbedFoo(errno);
}

int main(int argc, char[] argv)
{
  Foo f;
  int errno;

  f = bar(&errno);

  if (errno)
    return -1;
  return 0;
}

After bar() is called, perf_context.some_timer would be incremented as if
Stop(&perf_context.some_timer) was called at the end, and the compiler is still
able to produce optimizations on the return value from makeFrobbedFoo() through
to main().
main
Torrie Fischer 10 years ago
parent 2a8faf797c
commit 6614a48418
  1. 26
      db/db_impl.cc
  2. 23
      db/db_iter.cc
  3. 3
      db/memtable.cc
  4. 14
      table/format.cc
  5. 15
      table/merger.cc
  6. 44
      util/perf_context_imp.h

@ -3375,7 +3375,7 @@ Status DBImpl::GetImpl(const ReadOptions& options,
ColumnFamilyHandle* column_family, const Slice& key, ColumnFamilyHandle* column_family, const Slice& key,
std::string* value, bool* value_found) { std::string* value, bool* value_found) {
StopWatch sw(env_, stats_, DB_GET); StopWatch sw(env_, stats_, DB_GET);
PERF_TIMER_AUTO(get_snapshot_time); PERF_TIMER_GUARD(get_snapshot_time);
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family); auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
auto cfd = cfh->cfd(); auto cfd = cfh->cfd();
@ -3399,6 +3399,7 @@ Status DBImpl::GetImpl(const ReadOptions& options,
// merge_operands will contain the sequence of merges in the latter case. // merge_operands will contain the sequence of merges in the latter case.
LookupKey lkey(key, snapshot); LookupKey lkey(key, snapshot);
PERF_TIMER_STOP(get_snapshot_time); PERF_TIMER_STOP(get_snapshot_time);
if (sv->mem->Get(lkey, value, &s, merge_context, *cfd->options())) { if (sv->mem->Get(lkey, value, &s, merge_context, *cfd->options())) {
// Done // Done
RecordTick(stats_, MEMTABLE_HIT); RecordTick(stats_, MEMTABLE_HIT);
@ -3406,20 +3407,19 @@ Status DBImpl::GetImpl(const ReadOptions& options,
// Done // Done
RecordTick(stats_, MEMTABLE_HIT); RecordTick(stats_, MEMTABLE_HIT);
} else { } else {
PERF_TIMER_START(get_from_output_files_time); PERF_TIMER_GUARD(get_from_output_files_time);
sv->current->Get(options, lkey, value, &s, &merge_context, value_found); sv->current->Get(options, lkey, value, &s, &merge_context, value_found);
PERF_TIMER_STOP(get_from_output_files_time);
RecordTick(stats_, MEMTABLE_MISS); RecordTick(stats_, MEMTABLE_MISS);
} }
PERF_TIMER_START(get_post_process_time); {
PERF_TIMER_GUARD(get_post_process_time);
ReturnAndCleanupSuperVersion(cfd, sv); ReturnAndCleanupSuperVersion(cfd, sv);
RecordTick(stats_, NUMBER_KEYS_READ); RecordTick(stats_, NUMBER_KEYS_READ);
RecordTick(stats_, BYTES_READ, value->size()); RecordTick(stats_, BYTES_READ, value->size());
PERF_TIMER_STOP(get_post_process_time); }
return s; return s;
} }
@ -3429,7 +3429,7 @@ std::vector<Status> DBImpl::MultiGet(
const std::vector<Slice>& keys, std::vector<std::string>* values) { const std::vector<Slice>& keys, std::vector<std::string>* values) {
StopWatch sw(env_, stats_, DB_MULTIGET); StopWatch sw(env_, stats_, DB_MULTIGET);
PERF_TIMER_AUTO(get_snapshot_time); PERF_TIMER_GUARD(get_snapshot_time);
SequenceNumber snapshot; SequenceNumber snapshot;
@ -3505,7 +3505,7 @@ std::vector<Status> DBImpl::MultiGet(
} }
// Post processing (decrement reference counts and record statistics) // Post processing (decrement reference counts and record statistics)
PERF_TIMER_START(get_post_process_time); PERF_TIMER_GUARD(get_post_process_time);
autovector<SuperVersion*> superversions_to_delete; autovector<SuperVersion*> superversions_to_delete;
// TODO(icanadi) do we need lock here or just around Cleanup()? // TODO(icanadi) do we need lock here or just around Cleanup()?
@ -3878,7 +3878,7 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
if (my_batch == nullptr) { if (my_batch == nullptr) {
return Status::Corruption("Batch is nullptr!"); return Status::Corruption("Batch is nullptr!");
} }
PERF_TIMER_AUTO(write_pre_and_post_process_time); PERF_TIMER_GUARD(write_pre_and_post_process_time);
Writer w(&mutex_); Writer w(&mutex_);
w.batch = my_batch; w.batch = my_batch;
w.sync = options.sync; w.sync = options.sync;
@ -4011,7 +4011,7 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
uint64_t log_size = 0; uint64_t log_size = 0;
if (!options.disableWAL) { if (!options.disableWAL) {
PERF_TIMER_START(write_wal_time); PERF_TIMER_GUARD(write_wal_time);
Slice log_entry = WriteBatchInternal::Contents(updates); Slice log_entry = WriteBatchInternal::Contents(updates);
status = log_->AddRecord(log_entry); status = log_->AddRecord(log_entry);
total_log_size_ += log_entry.size(); total_log_size_ += log_entry.size();
@ -4029,10 +4029,9 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
status = log_->file()->Sync(); status = log_->file()->Sync();
} }
} }
PERF_TIMER_STOP(write_wal_time);
} }
if (status.ok()) { if (status.ok()) {
PERF_TIMER_START(write_memtable_time); PERF_TIMER_GUARD(write_memtable_time);
status = WriteBatchInternal::InsertInto( status = WriteBatchInternal::InsertInto(
updates, column_family_memtables_.get(), false, 0, this, false); updates, column_family_memtables_.get(), false, 0, this, false);
@ -4044,8 +4043,6 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
// into the memtable would result in a state that some write ops might // into the memtable would result in a state that some write ops might
// have succeeded in memtable but Status reports error for all writes. // have succeeded in memtable but Status reports error for all writes.
PERF_TIMER_STOP(write_memtable_time);
SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence); SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
} }
PERF_TIMER_START(write_pre_and_post_process_time); PERF_TIMER_START(write_pre_and_post_process_time);
@ -4079,7 +4076,6 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
RecordTick(stats_, WRITE_TIMEDOUT); RecordTick(stats_, WRITE_TIMEDOUT);
} }
PERF_TIMER_STOP(write_pre_and_post_process_time);
return status; return status;
} }

@ -194,9 +194,8 @@ void DBIter::Next() {
// NOTE: In between, saved_key_ can point to a user key that has // NOTE: In between, saved_key_ can point to a user key that has
// a delete marker // a delete marker
inline void DBIter::FindNextUserEntry(bool skipping) { inline void DBIter::FindNextUserEntry(bool skipping) {
PERF_TIMER_AUTO(find_next_user_entry_time); PERF_TIMER_GUARD(find_next_user_entry_time);
FindNextUserEntryInternal(skipping); FindNextUserEntryInternal(skipping);
PERF_TIMER_STOP(find_next_user_entry_time);
} }
// Actual implementation of DBIter::FindNextUserEntry() // Actual implementation of DBIter::FindNextUserEntry()
@ -557,9 +556,12 @@ void DBIter::Seek(const Slice& target) {
saved_key_.Clear(); saved_key_.Clear();
// now savved_key is used to store internal key. // now savved_key is used to store internal key.
saved_key_.SetInternalKey(target, sequence_); saved_key_.SetInternalKey(target, sequence_);
PERF_TIMER_AUTO(seek_internal_seek_time);
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->Seek(saved_key_.GetKey()); iter_->Seek(saved_key_.GetKey());
PERF_TIMER_STOP(seek_internal_seek_time); }
if (iter_->Valid()) { if (iter_->Valid()) {
direction_ = kForward; direction_ = kForward;
ClearSavedValue(); ClearSavedValue();
@ -577,9 +579,12 @@ void DBIter::SeekToFirst() {
} }
direction_ = kForward; direction_ = kForward;
ClearSavedValue(); ClearSavedValue();
PERF_TIMER_AUTO(seek_internal_seek_time);
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->SeekToFirst(); iter_->SeekToFirst();
PERF_TIMER_STOP(seek_internal_seek_time); }
if (iter_->Valid()) { if (iter_->Valid()) {
FindNextUserEntry(false /* not skipping */); FindNextUserEntry(false /* not skipping */);
} else { } else {
@ -595,9 +600,11 @@ void DBIter::SeekToLast() {
} }
direction_ = kReverse; direction_ = kReverse;
ClearSavedValue(); ClearSavedValue();
PERF_TIMER_AUTO(seek_internal_seek_time);
{
PERF_TIMER_GUARD(seek_internal_seek_time);
iter_->SeekToLast(); iter_->SeekToLast();
PERF_TIMER_STOP(seek_internal_seek_time); }
PrevInternal(); PrevInternal();
} }

@ -422,7 +422,7 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s,
// Avoiding recording stats for speed. // Avoiding recording stats for speed.
return false; return false;
} }
PERF_TIMER_AUTO(get_from_memtable_time); PERF_TIMER_GUARD(get_from_memtable_time);
Slice user_key = key.user_key(); Slice user_key = key.user_key();
bool found_final_value = false; bool found_final_value = false;
@ -452,7 +452,6 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s,
if (!found_final_value && merge_in_progress) { if (!found_final_value && merge_in_progress) {
*s = Status::MergeInProgress(""); *s = Status::MergeInProgress("");
} }
PERF_TIMER_STOP(get_from_memtable_time);
PERF_COUNTER_ADD(get_from_memtable_count, 1); PERF_COUNTER_ADD(get_from_memtable_count, 1);
return found_final_value; return found_final_value;
} }

@ -211,10 +211,13 @@ Status ReadBlock(RandomAccessFile* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle, const ReadOptions& options, const BlockHandle& handle,
Slice* contents, /* result of reading */ char* buf) { Slice* contents, /* result of reading */ char* buf) {
size_t n = static_cast<size_t>(handle.size()); size_t n = static_cast<size_t>(handle.size());
Status s;
{
PERF_TIMER_GUARD(block_read_time);
s = file->Read(handle.offset(), n + kBlockTrailerSize, contents, buf);
}
PERF_TIMER_AUTO(block_read_time);
Status s = file->Read(handle.offset(), n + kBlockTrailerSize, contents, buf);
PERF_TIMER_MEASURE(block_read_time);
PERF_COUNTER_ADD(block_read_count, 1); PERF_COUNTER_ADD(block_read_count, 1);
PERF_COUNTER_ADD(block_read_byte, n + kBlockTrailerSize); PERF_COUNTER_ADD(block_read_byte, n + kBlockTrailerSize);
@ -228,6 +231,7 @@ Status ReadBlock(RandomAccessFile* file, const Footer& footer,
// Check the crc of the type and the block contents // Check the crc of the type and the block contents
const char* data = contents->data(); // Pointer to where Read put the data const char* data = contents->data(); // Pointer to where Read put the data
if (options.verify_checksums) { if (options.verify_checksums) {
PERF_TIMER_GUARD(block_checksum_time);
uint32_t value = DecodeFixed32(data + n + 1); uint32_t value = DecodeFixed32(data + n + 1);
uint32_t actual = 0; uint32_t actual = 0;
switch (footer.checksum()) { switch (footer.checksum()) {
@ -247,7 +251,6 @@ Status ReadBlock(RandomAccessFile* file, const Footer& footer,
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
PERF_TIMER_STOP(block_checksum_time);
} }
return s; return s;
} }
@ -265,7 +268,7 @@ Status DecompressBlock(BlockContents* result, size_t block_size,
result->cachable = false; result->cachable = false;
result->heap_allocated = false; result->heap_allocated = false;
PERF_TIMER_AUTO(block_decompress_time); PERF_TIMER_GUARD(block_decompress_time);
rocksdb::CompressionType compression_type = rocksdb::CompressionType compression_type =
static_cast<rocksdb::CompressionType>(data[n]); static_cast<rocksdb::CompressionType>(data[n]);
// If the caller has requested that the block not be uncompressed // If the caller has requested that the block not be uncompressed
@ -295,7 +298,6 @@ Status DecompressBlock(BlockContents* result, size_t block_size,
} else { } else {
s = UncompressBlockContents(data, n, result); s = UncompressBlockContents(data, n, result);
} }
PERF_TIMER_STOP(block_decompress_time);
return s; return s;
} }

@ -116,12 +116,12 @@ class MergingIterator : public Iterator {
// Invalidate the heap. // Invalidate the heap.
use_heap_ = false; use_heap_ = false;
IteratorWrapper* first_child = nullptr; IteratorWrapper* first_child = nullptr;
PERF_TIMER_DECLARE();
for (auto& child : children_) { for (auto& child : children_) {
PERF_TIMER_START(seek_child_seek_time); {
PERF_TIMER_GUARD(seek_child_seek_time);
child.Seek(target); child.Seek(target);
PERF_TIMER_STOP(seek_child_seek_time); }
PERF_COUNTER_ADD(seek_child_seek_count, 1); PERF_COUNTER_ADD(seek_child_seek_count, 1);
if (child.Valid()) { if (child.Valid()) {
@ -134,24 +134,21 @@ class MergingIterator : public Iterator {
} else { } else {
// We have more than one children with valid keys. Initialize // We have more than one children with valid keys. Initialize
// the heap and put the first child into the heap. // the heap and put the first child into the heap.
PERF_TIMER_START(seek_min_heap_time); PERF_TIMER_GUARD(seek_min_heap_time);
ClearHeaps(); ClearHeaps();
minHeap_.push(first_child); minHeap_.push(first_child);
PERF_TIMER_STOP(seek_min_heap_time);
} }
} }
if (use_heap_) { if (use_heap_) {
PERF_TIMER_START(seek_min_heap_time); PERF_TIMER_GUARD(seek_min_heap_time);
minHeap_.push(&child); minHeap_.push(&child);
PERF_TIMER_STOP(seek_min_heap_time);
} }
} }
} }
if (use_heap_) { if (use_heap_) {
// If heap is valid, need to put the smallest key to curent_. // If heap is valid, need to put the smallest key to curent_.
PERF_TIMER_START(seek_min_heap_time); PERF_TIMER_GUARD(seek_min_heap_time);
FindSmallest(); FindSmallest();
PERF_TIMER_STOP(seek_min_heap_time);
} else { } else {
// The heap is not valid, then the current_ iterator is the first // The heap is not valid, then the current_ iterator is the first
// one, or null if there is no first child. // one, or null if there is no first child.

@ -11,11 +11,10 @@ namespace rocksdb {
#if defined(NPERF_CONTEXT) || defined(IOS_CROSS_COMPILE) #if defined(NPERF_CONTEXT) || defined(IOS_CROSS_COMPILE)
#define PERF_TIMER_DECLARE() #define PERF_TIMER_GUARD(metric)
#define PERF_TIMER_START(metric)
#define PERF_TIMER_AUTO(metric)
#define PERF_TIMER_MEASURE(metric) #define PERF_TIMER_MEASURE(metric)
#define PERF_TIMER_STOP(metric) #define PERF_TIMER_STOP(metric)
#define PERF_TIMER_START(metric)
#define PERF_COUNTER_ADD(metric, value) #define PERF_COUNTER_ADD(metric, value)
#else #else
@ -24,10 +23,15 @@ extern __thread PerfLevel perf_level;
class PerfStepTimer { class PerfStepTimer {
public: public:
PerfStepTimer() PerfStepTimer(uint64_t* metric)
: enabled_(perf_level >= PerfLevel::kEnableTime), : enabled_(perf_level >= PerfLevel::kEnableTime),
env_(enabled_ ? Env::Default() : nullptr), env_(enabled_ ? Env::Default() : nullptr),
start_(0) { start_(0),
metric_(metric) {
}
~PerfStepTimer() {
Stop();
} }
void Start() { void Start() {
@ -36,17 +40,17 @@ class PerfStepTimer {
} }
} }
void Measure(uint64_t* metric) { void Measure() {
if (start_) { if (start_) {
uint64_t now = env_->NowNanos(); uint64_t now = env_->NowNanos();
*metric += now - start_; *metric_ += now - start_;
start_ = now; start_ = now;
} }
} }
void Stop(uint64_t* metric) { void Stop() {
if (start_) { if (start_) {
*metric += env_->NowNanos() - start_; *metric_ += env_->NowNanos() - start_;
start_ = 0; start_ = 0;
} }
} }
@ -55,29 +59,25 @@ class PerfStepTimer {
const bool enabled_; const bool enabled_;
Env* const env_; Env* const env_;
uint64_t start_; uint64_t start_;
uint64_t* metric_;
}; };
// Declare the local timer object to be used later on // Stop the timer and update the metric
#define PERF_TIMER_DECLARE() \ #define PERF_TIMER_STOP(metric) \
PerfStepTimer perf_step_timer; perf_step_timer_ ## metric.Stop();
// Set start time of the timer
#define PERF_TIMER_START(metric) \ #define PERF_TIMER_START(metric) \
perf_step_timer.Start(); perf_step_timer_ ## metric.Start();
// Declare and set start time of the timer // Declare and set start time of the timer
#define PERF_TIMER_AUTO(metric) \ #define PERF_TIMER_GUARD(metric) \
PerfStepTimer perf_step_timer; \ PerfStepTimer perf_step_timer_ ## metric(&(perf_context.metric)); \
perf_step_timer.Start(); perf_step_timer_ ## metric.Start();
// Update metric with time elapsed since last START. start time is reset // Update metric with time elapsed since last START. start time is reset
// to current timestamp. // to current timestamp.
#define PERF_TIMER_MEASURE(metric) \ #define PERF_TIMER_MEASURE(metric) \
perf_step_timer.Measure(&(perf_context.metric)); perf_step_timer_ ## metric.Measure();
// Update metric with time elapsed since last START. But start time is not set.
#define PERF_TIMER_STOP(metric) \
perf_step_timer.Stop(&(perf_context.metric));
// Increase metric value // Increase metric value
#define PERF_COUNTER_ADD(metric, value) \ #define PERF_COUNTER_ADD(metric, value) \

Loading…
Cancel
Save