From 3e35aa6412f2fcaa75dc607131caa773d315761b Mon Sep 17 00:00:00 2001 From: Siying Dong Date: Thu, 21 Nov 2013 17:40:39 -0800 Subject: [PATCH] Revert "Allow users to profile a query and see bottleneck of the query" This reverts commit 3d8ac31d7168c916d6f2f0729eb627b07d8f082b. --- db/db_bench.cc | 8 +------ db/db_impl.cc | 34 +++------------------------- db/db_iter.cc | 22 ++---------------- db/db_test.cc | 41 ---------------------------------- db/memtable.cc | 20 +++++------------ db/perf_context_test.cc | 34 +++------------------------- db/version_set.cc | 2 +- include/rocksdb/perf_context.h | 22 +----------------- table/merger.cc | 23 +++---------------- table/merger.h | 3 +-- util/perf_context.cc | 15 +------------ 11 files changed, 21 insertions(+), 203 deletions(-) diff --git a/db/db_bench.cc b/db/db_bench.cc index 3ab130093..63cc906e7 100644 --- a/db/db_bench.cc +++ b/db/db_bench.cc @@ -22,7 +22,6 @@ #include "rocksdb/memtablerep.h" #include "rocksdb/write_batch.h" #include "rocksdb/statistics.h" -#include "rocksdb/perf_context.h" #include "port/port.h" #include "util/bit_set.h" #include "util/crc32c.h" @@ -351,8 +350,6 @@ DEFINE_int64(stats_interval, 0, "Stats are reported every N operations when " DEFINE_int32(stats_per_interval, 0, "Reports additional stats per interval when" " this is greater than 0."); -DEFINE_int32(perf_level, 0, "Level of perf collection"); - static bool ValidateRateLimit(const char* flagname, double value) { static constexpr double EPSILON = 1e-10; if ( value < -EPSILON ) { @@ -692,7 +689,6 @@ struct SharedState { port::Mutex mu; port::CondVar cv; int total; - int perf_level; // Each thread goes through the following states: // (1) initializing @@ -704,7 +700,7 @@ struct SharedState { long num_done; bool start; - SharedState() : cv(&mu), perf_level(FLAGS_perf_level) { } + SharedState() : cv(&mu) { } }; // Per-thread state for concurrent executions of the same benchmark. @@ -814,7 +810,6 @@ class Benchmark { fprintf(stdout, "Memtablerep: vector\n"); break; } - fprintf(stdout, "Perf Level: %d\n", FLAGS_perf_level); PrintWarnings(); fprintf(stdout, "------------------------------------------------\n"); @@ -1155,7 +1150,6 @@ class Benchmark { } } - SetPerfLevel(static_cast (shared->perf_level)); thread->stats.Start(thread->tid); (arg->bm->*(arg->method))(thread); thread->stats.Stop(); diff --git a/db/db_impl.cc b/db/db_impl.cc index a4e28b032..5a2f0de4a 100644 --- a/db/db_impl.cc +++ b/db/db_impl.cc @@ -1035,7 +1035,7 @@ Status DBImpl::WriteLevel0Table(std::vector &mems, VersionEdit* edit, (unsigned long)m->GetLogNumber()); list.push_back(m->NewIterator()); } - Iterator* iter = NewMergingIterator(env_, &internal_comparator_, &list[0], + Iterator* iter = NewMergingIterator(&internal_comparator_, &list[0], list.size()); const SequenceNumber newest_snapshot = snapshots_.GetNewest(); const SequenceNumber earliest_seqno_in_memtable = @@ -2519,7 +2519,7 @@ Iterator* DBImpl::NewInternalIterator(const ReadOptions& options, // Collect iterators for files in L0 - Ln versions_->current()->AddIterators(options, storage_options_, &list); Iterator* internal_iter = - NewMergingIterator(env_, &internal_comparator_, &list[0], list.size()); + NewMergingIterator(&internal_comparator_, &list[0], list.size()); versions_->current()->Ref(); cleanup->mu = &mutex_; @@ -2555,8 +2555,6 @@ Status DBImpl::GetImpl(const ReadOptions& options, Status s; StopWatch sw(env_, options_.statistics, DB_GET); - StopWatchNano snapshot_timer(env_, false); - StartPerfTimer(&snapshot_timer); SequenceNumber snapshot; mutex_.Lock(); if (options.snapshot != nullptr) { @@ -2585,23 +2583,15 @@ Status DBImpl::GetImpl(const ReadOptions& options, // s is both in/out. When in, s could either be OK or MergeInProgress. // merge_operands will contain the sequence of merges in the latter case. LookupKey lkey(key, snapshot); - BumpPerfTime(&perf_context.get_snapshot_time, &snapshot_timer); if (mem->Get(lkey, value, &s, &merge_operands, options_)) { // Done } else if (imm.Get(lkey, value, &s, &merge_operands, options_)) { // Done } else { - StopWatchNano from_files_timer(env_, false); - StartPerfTimer(&from_files_timer); - current->Get(options, lkey, value, &s, &merge_operands, &stats, options_, value_found); have_stat_update = true; - BumpPerfTime(&perf_context.get_from_output_files_time, &from_files_timer); } - - StopWatchNano post_process_timer(env_, false); - StartPerfTimer(&post_process_timer); mutex_.Lock(); if (!options_.disable_seek_compaction && @@ -2617,8 +2607,6 @@ Status DBImpl::GetImpl(const ReadOptions& options, // Note, tickers are atomic now - no lock protection needed any more. RecordTick(options_.statistics, NUMBER_KEYS_READ); RecordTick(options_.statistics, BYTES_READ, value->size()); - BumpPerfTime(&perf_context.get_post_process_time, &post_process_timer); - return s; } @@ -2627,8 +2615,6 @@ std::vector DBImpl::MultiGet(const ReadOptions& options, std::vector* values) { StopWatch sw(env_, options_.statistics, DB_MULTIGET); - StopWatchNano snapshot_timer(env_, false); - StartPerfTimer(&snapshot_timer); SequenceNumber snapshot; mutex_.Lock(); if (options.snapshot != nullptr) { @@ -2660,7 +2646,6 @@ std::vector DBImpl::MultiGet(const ReadOptions& options, // Keep track of bytes that we read for statistics-recording later uint64_t bytesRead = 0; - BumpPerfTime(&perf_context.get_snapshot_time, &snapshot_timer); // For each of the given keys, apply the entire "get" process as follows: // First look in the memtable, then in the immutable memtable (if any). @@ -2687,8 +2672,6 @@ std::vector DBImpl::MultiGet(const ReadOptions& options, } // Post processing (decrement reference counts and record statistics) - StopWatchNano post_process_timer(env_, false); - StartPerfTimer(&post_process_timer); mutex_.Lock(); if (!options_.disable_seek_compaction && have_stat_update && current->UpdateStats(stats)) { @@ -2703,7 +2686,6 @@ std::vector DBImpl::MultiGet(const ReadOptions& options, RecordTick(options_.statistics, NUMBER_MULTIGET_CALLS); RecordTick(options_.statistics, NUMBER_MULTIGET_KEYS_READ, numKeys); RecordTick(options_.statistics, NUMBER_MULTIGET_BYTES_READ, bytesRead); - BumpPerfTime(&perf_context.get_post_process_time, &post_process_timer); return statList; } @@ -2772,8 +2754,6 @@ Status DBImpl::Delete(const WriteOptions& options, const Slice& key) { } Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) { - StopWatchNano pre_post_process_timer(env_, false); - StartPerfTimer(&pre_post_process_timer); Writer w(&mutex_); w.batch = my_batch; w.sync = options.sync; @@ -2820,13 +2800,12 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) { if (options.disableWAL) { flush_on_destroy_ = true; } - BumpPerfTime(&perf_context.write_pre_and_post_process_time, - &pre_post_process_timer); if (!options.disableWAL) { StopWatchNano timer(env_); StartPerfTimer(&timer); status = log_->AddRecord(WriteBatchInternal::Contents(updates)); + BumpPerfTime(&perf_context.wal_write_time, &timer); if (status.ok() && options.sync) { if (options_.use_fsync) { StopWatch(env_, options_.statistics, WAL_FILE_SYNC_MICROS); @@ -2836,14 +2815,10 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) { status = log_->file()->Sync(); } } - BumpPerfTime(&perf_context.write_wal_time, &timer); } if (status.ok()) { - StopWatchNano write_memtable_timer(env_, false); - StartPerfTimer(&write_memtable_timer); status = WriteBatchInternal::InsertInto(updates, mem_, &options_, this, options_.filter_deletes); - BumpPerfTime(&perf_context.write_memtable_time, &write_memtable_timer); if (!status.ok()) { // Panic for in-memory corruptions // Note that existing logic was not sound. Any partial failure writing @@ -2853,7 +2828,6 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) { } SetTickerCount(options_.statistics, SEQUENCE_NUMBER, last_sequence); } - StartPerfTimer(&pre_post_process_timer); LogFlush(options_.info_log); mutex_.Lock(); if (status.ok()) { @@ -2881,8 +2855,6 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) { if (!writers_.empty()) { writers_.front()->cv.Signal(); } - BumpPerfTime(&perf_context.write_pre_and_post_process_time, - &pre_post_process_timer); return status; } diff --git a/db/db_iter.cc b/db/db_iter.cc index 1dc44b93c..4e3c52c6e 100644 --- a/db/db_iter.cc +++ b/db/db_iter.cc @@ -102,8 +102,7 @@ class DBIter: public Iterator { virtual void SeekToLast(); private: - inline void FindNextUserEntry(bool skipping); - void FindNextUserEntryInternal(bool skipping); + void FindNextUserEntry(bool skipping); void FindPrevUserEntry(); bool ParseKey(ParsedInternalKey* key); void MergeValuesNewToOld(); @@ -192,15 +191,7 @@ void DBIter::Next() { // // NOTE: In between, saved_key_ can point to a user key that has // a delete marker -inline void DBIter::FindNextUserEntry(bool skipping) { - StopWatchNano timer(env_, false); - StartPerfTimer(&timer); - FindNextUserEntryInternal(skipping); - BumpPerfTime(&perf_context.find_next_user_entry_time, &timer); -} - -// Actual implementation of DBIter::FindNextUserEntry() -void DBIter::FindNextUserEntryInternal(bool skipping) { +void DBIter::FindNextUserEntry(bool skipping) { // Loop until we hit an acceptable entry to yield assert(iter_->Valid()); assert(direction_ == kForward); @@ -440,10 +431,7 @@ void DBIter::Seek(const Slice& target) { saved_key_.clear(); AppendInternalKey( &saved_key_, ParsedInternalKey(target, sequence_, kValueTypeForSeek)); - StopWatchNano internal_seek_timer(env_, false); - StartPerfTimer(&internal_seek_timer); iter_->Seek(saved_key_); - BumpPerfTime(&perf_context.seek_internal_seek_time, &internal_seek_timer); if (iter_->Valid()) { FindNextUserEntry(false /*not skipping */); } else { @@ -454,10 +442,7 @@ void DBIter::Seek(const Slice& target) { void DBIter::SeekToFirst() { direction_ = kForward; ClearSavedValue(); - StopWatchNano internal_seek_timer(env_, false); - StartPerfTimer(&internal_seek_timer); iter_->SeekToFirst(); - BumpPerfTime(&perf_context.seek_internal_seek_time, &internal_seek_timer); if (iter_->Valid()) { FindNextUserEntry(false /* not skipping */); } else { @@ -476,10 +461,7 @@ void DBIter::SeekToLast() { direction_ = kReverse; ClearSavedValue(); - StopWatchNano internal_seek_timer(env_, false); - StartPerfTimer(&internal_seek_timer); iter_->SeekToLast(); - BumpPerfTime(&perf_context.seek_internal_seek_time, &internal_seek_timer); FindPrevUserEntry(); } diff --git a/db/db_test.cc b/db/db_test.cc index ed7425521..aca07bcff 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -22,7 +22,6 @@ #include "rocksdb/compaction_filter.h" #include "rocksdb/env.h" #include "rocksdb/table.h" -#include "rocksdb/perf_context.h" #include "util/hash.h" #include "util/logging.h" #include "util/mutexlock.h" @@ -1216,13 +1215,7 @@ TEST(DBTest, IterMulti) { ASSERT_EQ(IterStatus(iter), "a->va"); iter->Seek("ax"); ASSERT_EQ(IterStatus(iter), "b->vb"); - - SetPerfLevel(kEnableTime); - perf_context.Reset(); iter->Seek("b"); - ASSERT_TRUE((int) perf_context.seek_internal_seek_time > 0); - ASSERT_TRUE((int) perf_context.find_next_user_entry_time > 0); - SetPerfLevel(kDisable); ASSERT_EQ(IterStatus(iter), "b->vb"); iter->Seek("z"); ASSERT_EQ(IterStatus(iter), "(invalid)"); @@ -1237,12 +1230,7 @@ TEST(DBTest, IterMulti) { // Switch from forward to reverse iter->SeekToFirst(); iter->Next(); - SetPerfLevel(kEnableTime); - perf_context.Reset(); iter->Next(); - ASSERT_EQ(0, (int) perf_context.seek_internal_seek_time); - ASSERT_TRUE((int) perf_context.find_next_user_entry_time > 0); - SetPerfLevel(kDisable); iter->Prev(); ASSERT_EQ(IterStatus(iter), "b->vb"); @@ -1602,42 +1590,22 @@ TEST(DBTest, NumImmutableMemTable) { std::string big_value(1000000, 'x'); std::string num; - SetPerfLevel(kEnableTime);; ASSERT_OK(dbfull()->Put(writeOpt, "k1", big_value)); ASSERT_TRUE(dbfull()->GetProperty("rocksdb.num-immutable-mem-table", &num)); ASSERT_EQ(num, "0"); - perf_context.Reset(); - Get("k1"); - ASSERT_EQ(1, (int) perf_context.get_from_memtable_count); ASSERT_OK(dbfull()->Put(writeOpt, "k2", big_value)); ASSERT_TRUE(dbfull()->GetProperty("rocksdb.num-immutable-mem-table", &num)); ASSERT_EQ(num, "1"); - perf_context.Reset(); - Get("k1"); - ASSERT_EQ(2, (int) perf_context.get_from_memtable_count); - perf_context.Reset(); - Get("k2"); - ASSERT_EQ(1, (int) perf_context.get_from_memtable_count); ASSERT_OK(dbfull()->Put(writeOpt, "k3", big_value)); ASSERT_TRUE(dbfull()->GetProperty("rocksdb.num-immutable-mem-table", &num)); ASSERT_EQ(num, "2"); - perf_context.Reset(); - Get("k2"); - ASSERT_EQ(2, (int) perf_context.get_from_memtable_count); - perf_context.Reset(); - Get("k3"); - ASSERT_EQ(1, (int) perf_context.get_from_memtable_count); - perf_context.Reset(); - Get("k1"); - ASSERT_EQ(3, (int) perf_context.get_from_memtable_count); dbfull()->Flush(FlushOptions()); ASSERT_TRUE(dbfull()->GetProperty("rocksdb.num-immutable-mem-table", &num)); ASSERT_EQ(num, "0"); - SetPerfLevel(kDisable); } while (ChangeCompactOptions()); } @@ -1646,16 +1614,11 @@ TEST(DBTest, FLUSH) { Options options = CurrentOptions(); WriteOptions writeOpt = WriteOptions(); writeOpt.disableWAL = true; - SetPerfLevel(kEnableTime);; ASSERT_OK(dbfull()->Put(writeOpt, "foo", "v1")); // this will now also flush the last 2 writes dbfull()->Flush(FlushOptions()); ASSERT_OK(dbfull()->Put(writeOpt, "bar", "v1")); - perf_context.Reset(); - Get("foo"); - ASSERT_TRUE((int) perf_context.get_from_output_files_time > 0); - Reopen(); ASSERT_EQ("v1", Get("foo")); ASSERT_EQ("v1", Get("bar")); @@ -1667,9 +1630,7 @@ TEST(DBTest, FLUSH) { Reopen(); ASSERT_EQ("v2", Get("bar")); - perf_context.Reset(); ASSERT_EQ("v2", Get("foo")); - ASSERT_TRUE((int) perf_context.get_from_output_files_time > 0); writeOpt.disableWAL = false; ASSERT_OK(dbfull()->Put(writeOpt, "bar", "v3")); @@ -1681,8 +1642,6 @@ TEST(DBTest, FLUSH) { // has WAL enabled. ASSERT_EQ("v3", Get("foo")); ASSERT_EQ("v3", Get("bar")); - - SetPerfLevel(kDisable); } while (ChangeCompactOptions()); } diff --git a/db/memtable.cc b/db/memtable.cc index 1df7d6af2..291899c21 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -19,8 +19,6 @@ #include "util/coding.h" #include "util/mutexlock.h" #include "util/murmurhash.h" -#include "util/perf_context_imp.h" -#include "util/stop_watch.h" namespace std { template <> @@ -164,9 +162,6 @@ void MemTable::Add(SequenceNumber s, ValueType type, bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, std::deque* operands, const Options& options) { - StopWatchNano memtable_get_timer(options.env, false); - StartPerfTimer(&memtable_get_timer); - Slice memkey = key.memtable_key(); std::shared_ptr iter( table_->GetIterator(key.user_key())); @@ -180,8 +175,7 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, auto logger = options.info_log; std::string merge_result; - bool found_final_value = false; - for (; !found_final_value && iter->Valid(); iter->Next()) { + for (; iter->Valid(); iter->Next()) { // entry format is: // klength varint32 // userkey char[klength-8] @@ -218,8 +212,7 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, if (options.inplace_update_support) { GetLock(key.user_key())->Unlock(); } - found_final_value = true; - break; + return true; } case kTypeDeletion: { if (merge_in_progress) { @@ -233,8 +226,7 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, } else { *s = Status::NotFound(Slice()); } - found_final_value = true; - break; + return true; } case kTypeMerge: { Slice v = GetLengthPrefixedSlice(key_ptr + key_length); @@ -268,12 +260,10 @@ bool MemTable::Get(const LookupKey& key, std::string* value, Status* s, // No change to value, since we have not yet found a Put/Delete - if (!found_final_value && merge_in_progress) { + if (merge_in_progress) { *s = Status::MergeInProgress(""); } - BumpPerfTime(&perf_context.get_from_memtable_time, &memtable_get_timer); - BumpPerfCount(&perf_context.get_from_memtable_count); - return found_final_value; + return false; } bool MemTable::Update(SequenceNumber seq, ValueType type, diff --git a/db/perf_context_test.cc b/db/perf_context_test.cc index 2a6e6b7e4..05416748d 100644 --- a/db/perf_context_test.cc +++ b/db/perf_context_test.cc @@ -174,13 +174,6 @@ void ProfileKeyComparison() { HistogramImpl hist_put; HistogramImpl hist_get; - HistogramImpl hist_get_snapshot; - HistogramImpl hist_get_memtable; - HistogramImpl hist_get_post_process; - HistogramImpl hist_num_memtable_checked; - HistogramImpl hist_write_pre_post; - HistogramImpl hist_write_wal_time; - HistogramImpl hist_write_memtable_time; std::cout << "Inserting " << FLAGS_total_keys << " key/value pairs\n...\n"; @@ -199,37 +192,16 @@ void ProfileKeyComparison() { perf_context.Reset(); db->Put(write_options, key, value); - hist_write_pre_post.Add(perf_context.write_pre_and_post_process_time); - hist_write_wal_time.Add(perf_context.write_wal_time); - hist_write_memtable_time.Add(perf_context.write_memtable_time); hist_put.Add(perf_context.user_key_comparison_count); perf_context.Reset(); db->Get(read_options, key, &value); - hist_get_snapshot.Add(perf_context.get_snapshot_time); - hist_get_memtable.Add(perf_context.get_from_memtable_time); - hist_num_memtable_checked.Add(perf_context.get_from_memtable_count); - hist_get_post_process.Add(perf_context.get_post_process_time); hist_get.Add(perf_context.user_key_comparison_count); } std::cout << "Put uesr key comparison: \n" << hist_put.ToString() << "Get uesr key comparison: \n" << hist_get.ToString(); - std::cout << "Put(): Pre and Post Process Time: \n" - << hist_write_pre_post.ToString() - << " Writing WAL time: \n" - << hist_write_wal_time.ToString() << "\n" - << " Writing Mem Table time: \n" - << hist_write_memtable_time.ToString() << "\n"; - - std::cout << "Get(): Time to get snapshot: \n" - << hist_get_snapshot.ToString() - << " Time to get value from memtables: \n" - << hist_get_memtable.ToString() << "\n" - << " Number of memtables checked: \n" - << hist_num_memtable_checked.ToString() << "\n" - << " Time to post process: \n" - << hist_get_post_process.ToString() << "\n"; + } TEST(PerfContextTest, KeyComparisonCount) { @@ -287,8 +259,8 @@ TEST(PerfContextTest, SeekKeyComparison) { db->Put(write_options, key, value); auto put_time = timer.ElapsedNanos(); hist_put_time.Add(put_time); - hist_wal_time.Add(perf_context.write_wal_time); - hist_time_diff.Add(put_time - perf_context.write_wal_time); + hist_wal_time.Add(perf_context.wal_write_time); + hist_time_diff.Add(put_time - perf_context.wal_write_time); } std::cout << "Put time:\n" << hist_put_time.ToString() diff --git a/db/version_set.cc b/db/version_set.cc index 349abfbaa..d554657b4 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -2091,7 +2091,7 @@ Iterator* VersionSet::MakeInputIterator(Compaction* c) { } } assert(num <= space); - Iterator* result = NewMergingIterator(env_, &icmp_, list, num); + Iterator* result = NewMergingIterator(&icmp_, list, num); delete[] list; return result; } diff --git a/include/rocksdb/perf_context.h b/include/rocksdb/perf_context.h index 551ca8fe6..9e900e050 100644 --- a/include/rocksdb/perf_context.h +++ b/include/rocksdb/perf_context.h @@ -38,27 +38,7 @@ struct PerfContext { uint64_t internal_key_skipped_count; // total number of deletes skipped over during iteration uint64_t internal_delete_skipped_count; - - uint64_t get_snapshot_time; // total time spent on getting snapshot - uint64_t get_from_memtable_time; // total time spent on querying memtables - uint64_t get_from_memtable_count; // number of mem tables queried - // total time spent after Get() finds a key - uint64_t get_post_process_time; - uint64_t get_from_output_files_time; // total time reading from output files - // total time spent on seeking child iters - uint64_t seek_child_seek_time; - // number of seek issued in child iterators - uint64_t seek_child_seek_count; - uint64_t seek_min_heap_time; // total time spent on the merge heap - // total time spent on seeking the internal entries - uint64_t seek_internal_seek_time; - // total time spent on iterating internal entries to find the next user entry - uint64_t find_next_user_entry_time; - // total time spent on pre or post processing when writing a record - uint64_t write_pre_and_post_process_time; - uint64_t write_wal_time; // total time spent on writing to WAL - // total time spent on writing to mem tables - uint64_t write_memtable_time; + uint64_t wal_write_time; // total time spent on writing to WAL }; extern __thread PerfContext perf_context; diff --git a/table/merger.cc b/table/merger.cc index f66aa74c3..f5ce7440c 100644 --- a/table/merger.cc +++ b/table/merger.cc @@ -11,11 +11,8 @@ #include "rocksdb/comparator.h" #include "rocksdb/iterator.h" -#include "rocksdb/options.h" #include "table/iter_heap.h" #include "table/iterator_wrapper.h" -#include "util/stop_watch.h" -#include "util/perf_context_imp.h" #include @@ -25,12 +22,10 @@ namespace { class MergingIterator : public Iterator { public: - MergingIterator(Env* const env, const Comparator* comparator, - Iterator** children, int n) + MergingIterator(const Comparator* comparator, Iterator** children, int n) : comparator_(comparator), children_(n), current_(nullptr), - env_(env), direction_(kForward), maxHeap_(NewMaxIterHeap(comparator_)), minHeap_ (NewMinIterHeap(comparator_)) { @@ -76,24 +71,14 @@ class MergingIterator : public Iterator { virtual void Seek(const Slice& target) { ClearHeaps(); - StopWatchNano child_seek_timer(env_, false); - StopWatchNano min_heap_timer(env_, false); for (auto& child : children_) { - StartPerfTimer(&child_seek_timer); child.Seek(target); - BumpPerfTime(&perf_context.seek_child_seek_time, &child_seek_timer); - BumpPerfCount(&perf_context.seek_child_seek_count); - if (child.Valid()) { - StartPerfTimer(&min_heap_timer); minHeap_.push(&child); - BumpPerfTime(&perf_context.seek_min_heap_time, &child_seek_timer); } } - StartPerfTimer(&min_heap_timer); FindSmallest(); direction_ = kForward; - BumpPerfTime(&perf_context.seek_min_heap_time, &child_seek_timer); } virtual void Next() { @@ -193,7 +178,6 @@ class MergingIterator : public Iterator { const Comparator* comparator_; std::vector children_; IteratorWrapper* current_; - Env* const env_; // Which direction is the iterator moving? enum Direction { kForward, @@ -230,15 +214,14 @@ void MergingIterator::ClearHeaps() { } } // namespace -Iterator* NewMergingIterator(Env* const env, const Comparator* cmp, - Iterator** list, int n) { +Iterator* NewMergingIterator(const Comparator* cmp, Iterator** list, int n) { assert(n >= 0); if (n == 0) { return NewEmptyIterator(); } else if (n == 1) { return list[0]; } else { - return new MergingIterator(env, cmp, list, n); + return new MergingIterator(cmp, list, n); } } diff --git a/table/merger.h b/table/merger.h index 74f46ac9b..dbc1f69eb 100644 --- a/table/merger.h +++ b/table/merger.h @@ -13,7 +13,6 @@ namespace rocksdb { class Comparator; class Iterator; -class Env; // Return an iterator that provided the union of the data in // children[0,n-1]. Takes ownership of the child iterators and @@ -24,6 +23,6 @@ class Env; // // REQUIRES: n >= 0 extern Iterator* NewMergingIterator( - Env* const env, const Comparator* comparator, Iterator** children, int n); + const Comparator* comparator, Iterator** children, int n); } // namespace rocksdb diff --git a/util/perf_context.cc b/util/perf_context.cc index 6833f6836..1e8ddfb5e 100644 --- a/util/perf_context.cc +++ b/util/perf_context.cc @@ -22,20 +22,7 @@ void PerfContext::Reset() { block_decompress_time = 0; internal_key_skipped_count = 0; internal_delete_skipped_count = 0; - write_wal_time = 0; - - get_snapshot_time = 0; - get_from_memtable_time = 0; - get_from_memtable_count = 0; - get_post_process_time = 0; - get_from_output_files_time = 0; - seek_child_seek_time = 0; - seek_child_seek_count = 0; - seek_min_heap_time = 0; - seek_internal_seek_time = 0; - find_next_user_entry_time = 0; - write_pre_and_post_process_time = 0; - write_memtable_time = 0; + wal_write_time = 0; } __thread PerfContext perf_context;