Merge branch 'master' into performance
Conflicts: db/db_impl.cc db/db_test.cc db/memtable.cc db/version_set.cc include/rocksdb/statistics.h util/statistics_imp.h
This commit is contained in:
commit
054c5dda8c
@ -1,10 +1,11 @@
|
||||
{
|
||||
"project_id" : "leveldb",
|
||||
"project_id" : "rocksdb",
|
||||
"conduit_uri" : "https://reviews.facebook.net/",
|
||||
"copyright_holder" : "",
|
||||
"copyright_holder" : "Facebook",
|
||||
"load" : [
|
||||
"linters/src/"
|
||||
"linters"
|
||||
],
|
||||
"lint.engine" : "FacebookFbcodeLintEngine",
|
||||
"lint.engine.single.linter" : "FbcodeCppLinter"
|
||||
"lint.engine.single.linter" : "FbcodeCppLinter",
|
||||
"lint.cpplint.prefix" : "linters"
|
||||
}
|
||||
|
1
.gitignore
vendored
1
.gitignore
vendored
@ -20,3 +20,4 @@ util/build_version.cc
|
||||
build_tools/VALGRIND_LOGS/
|
||||
coverage/COVERAGE_REPORT
|
||||
.gdbhistory
|
||||
.phutil_module_cache
|
||||
|
@ -26,7 +26,7 @@ Compaction::Compaction(Version* input_version, int level, int out_level,
|
||||
: level_(level),
|
||||
out_level_(out_level),
|
||||
max_output_file_size_(target_file_size),
|
||||
maxGrandParentOverlapBytes_(max_grandparent_overlap_bytes),
|
||||
max_grandparent_overlap_bytes_(max_grandparent_overlap_bytes),
|
||||
input_version_(input_version),
|
||||
number_levels_(input_version_->NumberLevels()),
|
||||
seek_compaction_(seek_compaction),
|
||||
@ -64,7 +64,7 @@ bool Compaction::IsTrivialMove() const {
|
||||
return (level_ != out_level_ &&
|
||||
num_input_files(0) == 1 &&
|
||||
num_input_files(1) == 0 &&
|
||||
TotalFileSize(grandparents_) <= maxGrandParentOverlapBytes_);
|
||||
TotalFileSize(grandparents_) <= max_grandparent_overlap_bytes_);
|
||||
}
|
||||
|
||||
void Compaction::AddInputDeletions(VersionEdit* edit) {
|
||||
@ -117,7 +117,7 @@ bool Compaction::ShouldStopBefore(const Slice& internal_key) {
|
||||
}
|
||||
seen_key_ = true;
|
||||
|
||||
if (overlapped_bytes_ > maxGrandParentOverlapBytes_) {
|
||||
if (overlapped_bytes_ > max_grandparent_overlap_bytes_) {
|
||||
// Too much overlap for current output; start new output
|
||||
overlapped_bytes_ = 0;
|
||||
return true;
|
||||
|
@ -33,9 +33,14 @@ class Compaction {
|
||||
// "which" must be either 0 or 1
|
||||
int num_input_files(int which) const { return inputs_[which].size(); }
|
||||
|
||||
// Returns input version of the compaction
|
||||
Version* input_version() const { return input_version_; }
|
||||
|
||||
// Return the ith input file at "level()+which" ("which" must be 0 or 1).
|
||||
FileMetaData* input(int which, int i) const { return inputs_[which][i]; }
|
||||
|
||||
std::vector<FileMetaData*>* inputs(int which) { return &inputs_[which]; }
|
||||
|
||||
// Maximum size of files to build during this compaction.
|
||||
uint64_t MaxOutputFileSize() const { return max_output_file_size_; }
|
||||
|
||||
@ -74,8 +79,6 @@ class Compaction {
|
||||
bool IsFullCompaction() { return is_full_compaction_; }
|
||||
|
||||
private:
|
||||
friend class Version;
|
||||
friend class VersionSet;
|
||||
friend class CompactionPicker;
|
||||
friend class UniversalCompactionPicker;
|
||||
friend class LevelCompactionPicker;
|
||||
@ -87,7 +90,7 @@ class Compaction {
|
||||
int level_;
|
||||
int out_level_; // levels to which output files are stored
|
||||
uint64_t max_output_file_size_;
|
||||
uint64_t maxGrandParentOverlapBytes_;
|
||||
uint64_t max_grandparent_overlap_bytes_;
|
||||
Version* input_version_;
|
||||
VersionEdit* edit_;
|
||||
int number_levels_;
|
||||
|
@ -8,6 +8,7 @@
|
||||
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||
|
||||
#include "db/compaction_picker.h"
|
||||
#include "util/statistics.h"
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
@ -589,10 +590,8 @@ Compaction* UniversalCompactionPicker::PickCompaction(Version* version) {
|
||||
}
|
||||
|
||||
// update statistics
|
||||
if (options_->statistics != nullptr) {
|
||||
options_->statistics->measureTime(NUM_FILES_IN_SINGLE_COMPACTION,
|
||||
c->inputs_[0].size());
|
||||
}
|
||||
MeasureTime(options_->statistics.get(), NUM_FILES_IN_SINGLE_COMPACTION,
|
||||
c->inputs_[0].size());
|
||||
|
||||
// mark all the files that are being compacted
|
||||
c->MarkFilesBeingCompacted(true);
|
||||
|
@ -14,7 +14,7 @@
|
||||
#include <gflags/gflags.h>
|
||||
#include "db/db_impl.h"
|
||||
#include "db/version_set.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "rocksdb/options.h"
|
||||
#include "rocksdb/cache.h"
|
||||
#include "rocksdb/db.h"
|
||||
@ -31,6 +31,7 @@
|
||||
#include "util/random.h"
|
||||
#include "util/stack_trace.h"
|
||||
#include "util/string_util.h"
|
||||
#include "util/statistics.h"
|
||||
#include "util/testutil.h"
|
||||
#include "hdfs/env_hdfs.h"
|
||||
#include "utilities/merge_operators.h"
|
||||
@ -356,9 +357,9 @@ static bool ValidateCompressionLevel(const char* flagname, int32_t value) {
|
||||
return true;
|
||||
}
|
||||
|
||||
static const bool FLAGS_compression_level_dummy =
|
||||
google::RegisterFlagValidator(&FLAGS_compression_level,
|
||||
&ValidateCompressionLevel);
|
||||
static const bool FLAGS_compression_level_dummy __attribute__((unused)) =
|
||||
google::RegisterFlagValidator(&FLAGS_compression_level,
|
||||
&ValidateCompressionLevel);
|
||||
|
||||
DEFINE_int32(min_level_to_compress, -1, "If non-negative, compression starts"
|
||||
" from this level. Levels with number < min_level_to_compress are"
|
||||
|
@ -877,14 +877,11 @@ void DBImpl::PurgeObsoleteWALFiles() {
|
||||
}
|
||||
}
|
||||
|
||||
// If externalTable is set, then apply recovered transactions
|
||||
// to that table. This is used for readonly mode.
|
||||
Status DBImpl::Recover(VersionEdit* edit, MemTable* external_table,
|
||||
bool error_if_log_file_exist) {
|
||||
Status DBImpl::Recover(bool read_only, bool error_if_log_file_exist) {
|
||||
mutex_.AssertHeld();
|
||||
|
||||
assert(db_lock_ == nullptr);
|
||||
if (!external_table) {
|
||||
if (!read_only) {
|
||||
// We call CreateDirIfMissing() as the directory may already exist (if we
|
||||
// are reopening a DB), when this happens we don't want creating the
|
||||
// directory to cause an error. However, we need to check if creating the
|
||||
@ -967,11 +964,11 @@ Status DBImpl::Recover(VersionEdit* edit, MemTable* external_table,
|
||||
// Recover in the order in which the logs were generated
|
||||
std::sort(logs.begin(), logs.end());
|
||||
for (const auto& log : logs) {
|
||||
s = RecoverLogFile(log, edit, &max_sequence, external_table);
|
||||
// The previous incarnation may not have written any MANIFEST
|
||||
// records after allocating this log number. So we manually
|
||||
// update the file number allocation counter in VersionSet.
|
||||
versions_->MarkFileNumberUsed(log);
|
||||
s = RecoverLogFile(log, &max_sequence, read_only);
|
||||
}
|
||||
|
||||
if (s.ok()) {
|
||||
@ -986,10 +983,8 @@ Status DBImpl::Recover(VersionEdit* edit, MemTable* external_table,
|
||||
return s;
|
||||
}
|
||||
|
||||
Status DBImpl::RecoverLogFile(uint64_t log_number,
|
||||
VersionEdit* edit,
|
||||
SequenceNumber* max_sequence,
|
||||
MemTable* external_table) {
|
||||
Status DBImpl::RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
|
||||
bool read_only) {
|
||||
struct LogReporter : public log::Reader::Reporter {
|
||||
Env* env;
|
||||
Logger* info_log;
|
||||
@ -1006,6 +1001,8 @@ Status DBImpl::RecoverLogFile(uint64_t log_number,
|
||||
|
||||
mutex_.AssertHeld();
|
||||
|
||||
VersionEdit edit;
|
||||
|
||||
// Open the log file
|
||||
std::string fname = LogFileName(options_.wal_dir, log_number);
|
||||
unique_ptr<SequentialFile> file;
|
||||
@ -1035,11 +1032,8 @@ Status DBImpl::RecoverLogFile(uint64_t log_number,
|
||||
std::string scratch;
|
||||
Slice record;
|
||||
WriteBatch batch;
|
||||
MemTable* mem = nullptr;
|
||||
if (external_table) {
|
||||
mem = external_table;
|
||||
}
|
||||
while (reader.ReadRecord(&record, &scratch) && status.ok()) {
|
||||
bool memtable_empty = true;
|
||||
while (reader.ReadRecord(&record, &scratch)) {
|
||||
if (record.size() < 12) {
|
||||
reporter.Corruption(
|
||||
record.size(), Status::Corruption("log record too small"));
|
||||
@ -1047,14 +1041,11 @@ Status DBImpl::RecoverLogFile(uint64_t log_number,
|
||||
}
|
||||
WriteBatchInternal::SetContents(&batch, record);
|
||||
|
||||
if (mem == nullptr) {
|
||||
mem = new MemTable(internal_comparator_, options_);
|
||||
mem->Ref();
|
||||
}
|
||||
status = WriteBatchInternal::InsertInto(&batch, mem, &options_);
|
||||
status = WriteBatchInternal::InsertInto(&batch, mem_, &options_);
|
||||
memtable_empty = false;
|
||||
MaybeIgnoreError(&status);
|
||||
if (!status.ok()) {
|
||||
break;
|
||||
return status;
|
||||
}
|
||||
const SequenceNumber last_seq =
|
||||
WriteBatchInternal::Sequence(&batch) +
|
||||
@ -1063,28 +1054,44 @@ Status DBImpl::RecoverLogFile(uint64_t log_number,
|
||||
*max_sequence = last_seq;
|
||||
}
|
||||
|
||||
if (!external_table &&
|
||||
mem->ApproximateMemoryUsage() > options_.write_buffer_size) {
|
||||
status = WriteLevel0TableForRecovery(mem, edit);
|
||||
if (!read_only &&
|
||||
mem_->ApproximateMemoryUsage() > options_.write_buffer_size) {
|
||||
status = WriteLevel0TableForRecovery(mem_, &edit);
|
||||
// we still want to clear memtable, even if the recovery failed
|
||||
delete mem_->Unref();
|
||||
mem_ = new MemTable(internal_comparator_, options_);
|
||||
mem_->Ref();
|
||||
memtable_empty = true;
|
||||
if (!status.ok()) {
|
||||
// Reflect errors immediately so that conditions like full
|
||||
// file-systems cause the DB::Open() to fail.
|
||||
break;
|
||||
return status;
|
||||
}
|
||||
delete mem->Unref();
|
||||
mem = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
if (status.ok() && mem != nullptr && !external_table) {
|
||||
status = WriteLevel0TableForRecovery(mem, edit);
|
||||
// Reflect errors immediately so that conditions like full
|
||||
// file-systems cause the DB::Open() to fail.
|
||||
if (!memtable_empty && !read_only) {
|
||||
status = WriteLevel0TableForRecovery(mem_, &edit);
|
||||
delete mem_->Unref();
|
||||
mem_ = new MemTable(internal_comparator_, options_);
|
||||
mem_->Ref();
|
||||
if (!status.ok()) {
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
if (mem != nullptr && !external_table) {
|
||||
delete mem->Unref();
|
||||
if (edit.NumEntries() > 0) {
|
||||
// if read_only, NumEntries() will be 0
|
||||
assert(!read_only);
|
||||
// writing log number in the manifest means that any log file
|
||||
// with number strongly less than (log_number + 1) is already
|
||||
// recovered and should be ignored on next reincarnation.
|
||||
// Since we already recovered log_number, we want all logs
|
||||
// with numbers `<= log_number` (includes this one) to be ignored
|
||||
edit.SetLogNumber(log_number + 1);
|
||||
status = versions_->LogAndApply(&edit, &mutex_);
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
@ -2579,9 +2586,7 @@ Status DBImpl::DoCompactionWork(CompactionState* compact,
|
||||
|
||||
CompactionStats stats;
|
||||
stats.micros = env_->NowMicros() - start_micros - imm_micros;
|
||||
if (options_.statistics.get()) {
|
||||
options_.statistics.get()->measureTime(COMPACTION_TIME, stats.micros);
|
||||
}
|
||||
MeasureTime(options_.statistics.get(), COMPACTION_TIME, stats.micros);
|
||||
stats.files_in_leveln = compact->compaction->num_input_files(0);
|
||||
stats.files_in_levelnp1 = compact->compaction->num_input_files(1);
|
||||
|
||||
@ -3106,8 +3111,8 @@ Status DBImpl::Write(const WriteOptions& options, WriteBatch* my_batch) {
|
||||
// have succeeded in memtable but Status reports error for all writes.
|
||||
throw std::runtime_error("In memory WriteBatch corruption!");
|
||||
}
|
||||
SetTickerCount(options_.statistics.get(),
|
||||
SEQUENCE_NUMBER, last_sequence);
|
||||
SetTickerCount(options_.statistics.get(), SEQUENCE_NUMBER,
|
||||
last_sequence);
|
||||
}
|
||||
StartPerfTimer(&pre_post_process_timer);
|
||||
if (updates == &tmp_batch_) tmp_batch_.Clear();
|
||||
@ -3249,15 +3254,15 @@ Status DBImpl::MakeRoomForWrite(bool force,
|
||||
// individual write by 0-1ms to reduce latency variance. Also,
|
||||
// this delay hands over some CPU to the compaction thread in
|
||||
// case it is sharing the same core as the writer.
|
||||
uint64_t slowdown =
|
||||
SlowdownAmount(versions_->current()->NumLevelFiles(0),
|
||||
options_.level0_slowdown_writes_trigger,
|
||||
options_.level0_stop_writes_trigger);
|
||||
mutex_.Unlock();
|
||||
uint64_t delayed;
|
||||
{
|
||||
StopWatch sw(env_, options_.statistics.get(), STALL_L0_SLOWDOWN_COUNT);
|
||||
env_->SleepForMicroseconds(
|
||||
SlowdownAmount(versions_->current()->NumLevelFiles(0),
|
||||
options_.level0_slowdown_writes_trigger,
|
||||
options_.level0_stop_writes_trigger)
|
||||
);
|
||||
env_->SleepForMicroseconds(slowdown);
|
||||
delayed = sw.ElapsedMicros();
|
||||
}
|
||||
RecordTick(options_.statistics.get(), STALL_L0_SLOWDOWN_MICROS, delayed);
|
||||
@ -3874,9 +3879,8 @@ Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
|
||||
delete impl;
|
||||
return s;
|
||||
}
|
||||
impl->mutex_.Lock(); // DBImpl::Recover() requires lock being held
|
||||
VersionEdit edit;
|
||||
s = impl->Recover(&edit); // Handles create_if_missing, error_if_exists
|
||||
impl->mutex_.Lock();
|
||||
s = impl->Recover(); // Handles create_if_missing, error_if_exists
|
||||
if (s.ok()) {
|
||||
uint64_t new_log_number = impl->versions_->NewFileNumber();
|
||||
unique_ptr<WritableFile> lfile;
|
||||
@ -3888,6 +3892,7 @@ Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
|
||||
);
|
||||
if (s.ok()) {
|
||||
lfile->SetPreallocationBlockSize(1.1 * impl->options_.write_buffer_size);
|
||||
VersionEdit edit;
|
||||
edit.SetLogNumber(new_log_number);
|
||||
impl->logfile_number_ = new_log_number;
|
||||
impl->log_.reset(new log::Writer(std::move(lfile)));
|
||||
|
12
db/db_impl.h
12
db/db_impl.h
@ -262,10 +262,8 @@ class DBImpl : public DB {
|
||||
Status NewDB();
|
||||
|
||||
// Recover the descriptor from persistent storage. May do a significant
|
||||
// amount of work to recover recently logged updates. Any changes to
|
||||
// be made to the descriptor are added to *edit.
|
||||
Status Recover(VersionEdit* edit, MemTable* external_table = nullptr,
|
||||
bool error_if_log_file_exist = false);
|
||||
// amount of work to recover recently logged updates.
|
||||
Status Recover(bool read_only = false, bool error_if_log_file_exist = false);
|
||||
|
||||
void MaybeIgnoreError(Status* s) const;
|
||||
|
||||
@ -279,10 +277,8 @@ class DBImpl : public DB {
|
||||
Status FlushMemTableToOutputFile(bool* madeProgress,
|
||||
DeletionState& deletion_state);
|
||||
|
||||
Status RecoverLogFile(uint64_t log_number,
|
||||
VersionEdit* edit,
|
||||
SequenceNumber* max_sequence,
|
||||
MemTable* external_table);
|
||||
Status RecoverLogFile(uint64_t log_number, SequenceNumber* max_sequence,
|
||||
bool read_only);
|
||||
|
||||
// The following two methods are used to flush a memtable to
|
||||
// storage. The first one is used atdatabase RecoveryTime (when the
|
||||
|
@ -86,9 +86,7 @@ Status DB::OpenForReadOnly(const Options& options, const std::string& dbname,
|
||||
|
||||
DBImplReadOnly* impl = new DBImplReadOnly(options, dbname);
|
||||
impl->mutex_.Lock();
|
||||
VersionEdit edit;
|
||||
Status s = impl->Recover(&edit, impl->GetMemTable(),
|
||||
error_if_log_file_exist);
|
||||
Status s = impl->Recover(true /* read only */, error_if_log_file_exist);
|
||||
impl->mutex_.Unlock();
|
||||
if (s.ok()) {
|
||||
*dbptr = impl;
|
||||
|
@ -1,14 +0,0 @@
|
||||
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#include "db/db_statistics.h"
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
std::shared_ptr<Statistics> CreateDBStatistics() {
|
||||
return std::make_shared<DBStatistics>();
|
||||
}
|
||||
|
||||
} // namespace rocksdb
|
@ -1,63 +0,0 @@
|
||||
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
//
|
||||
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style license that can be
|
||||
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||
|
||||
#pragma once
|
||||
#include <cassert>
|
||||
#include <stdlib.h>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "util/histogram.h"
|
||||
#include "port/port.h"
|
||||
#include "util/mutexlock.h"
|
||||
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
class DBStatistics: public Statistics {
|
||||
public:
|
||||
DBStatistics() : allTickers_(TICKER_ENUM_MAX),
|
||||
allHistograms_(HISTOGRAM_ENUM_MAX) { }
|
||||
|
||||
virtual ~DBStatistics() {}
|
||||
|
||||
virtual long getTickerCount(Tickers tickerType) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
return allTickers_[tickerType].getCount();
|
||||
}
|
||||
|
||||
virtual void setTickerCount(Tickers tickerType, uint64_t count) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
allTickers_[tickerType].setTickerCount(count);
|
||||
}
|
||||
|
||||
virtual void recordTick(Tickers tickerType, uint64_t count) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
allTickers_[tickerType].recordTick(count);
|
||||
}
|
||||
|
||||
virtual void measureTime(Histograms histogramType, uint64_t value) {
|
||||
assert(histogramType < HISTOGRAM_ENUM_MAX);
|
||||
allHistograms_[histogramType].Add(value);
|
||||
}
|
||||
|
||||
virtual void histogramData(Histograms histogramType,
|
||||
HistogramData * const data) {
|
||||
assert(histogramType < HISTOGRAM_ENUM_MAX);
|
||||
allHistograms_[histogramType].Data(data);
|
||||
}
|
||||
|
||||
std::vector<Ticker> allTickers_;
|
||||
std::vector<HistogramImpl> allHistograms_;
|
||||
};
|
||||
|
||||
std::shared_ptr<Statistics> CreateDBStatistics();
|
||||
|
||||
} // namespace rocksdb
|
291
db/db_test.cc
291
db/db_test.cc
@ -11,26 +11,26 @@
|
||||
#include <set>
|
||||
#include <unistd.h>
|
||||
|
||||
#include "rocksdb/db.h"
|
||||
#include "rocksdb/filter_policy.h"
|
||||
#include "db/db_impl.h"
|
||||
#include "db/filename.h"
|
||||
#include "db/version_set.h"
|
||||
#include "db/write_batch_internal.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/cache.h"
|
||||
#include "rocksdb/compaction_filter.h"
|
||||
#include "rocksdb/db.h"
|
||||
#include "rocksdb/env.h"
|
||||
#include "rocksdb/table.h"
|
||||
#include "rocksdb/filter_policy.h"
|
||||
#include "rocksdb/perf_context.h"
|
||||
#include "rocksdb/plain_table_factory.h"
|
||||
#include "rocksdb/table.h"
|
||||
#include "util/hash.h"
|
||||
#include "util/logging.h"
|
||||
#include "util/mutexlock.h"
|
||||
#include "util/testharness.h"
|
||||
#include "util/testutil.h"
|
||||
#include "util/hash_linklist_rep.h"
|
||||
#include "utilities/merge_operators.h"
|
||||
#include "util/logging.h"
|
||||
#include "util/mutexlock.h"
|
||||
#include "util/statistics.h"
|
||||
#include "util/testharness.h"
|
||||
#include "util/testutil.h"
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
@ -706,7 +706,6 @@ class DBTest {
|
||||
delete iter;
|
||||
}
|
||||
|
||||
|
||||
// Used to test InplaceUpdate
|
||||
|
||||
// If previous value is nullptr or delta is > than previous value,
|
||||
@ -772,6 +771,31 @@ class DBTest {
|
||||
delete iter;
|
||||
ASSERT_EQ(0, seq);
|
||||
}
|
||||
|
||||
void CopyFile(const std::string& source, const std::string& destination,
|
||||
uint64_t size = 0) {
|
||||
const EnvOptions soptions;
|
||||
unique_ptr<SequentialFile> srcfile;
|
||||
ASSERT_OK(env_->NewSequentialFile(source, &srcfile, soptions));
|
||||
unique_ptr<WritableFile> destfile;
|
||||
ASSERT_OK(env_->NewWritableFile(destination, &destfile, soptions));
|
||||
|
||||
if (size == 0) {
|
||||
// default argument means copy everything
|
||||
ASSERT_OK(env_->GetFileSize(source, &size));
|
||||
}
|
||||
|
||||
char buffer[4096];
|
||||
Slice slice;
|
||||
while (size > 0) {
|
||||
uint64_t one = std::min(uint64_t(sizeof(buffer)), size);
|
||||
ASSERT_OK(srcfile->Read(one, &slice, buffer));
|
||||
ASSERT_OK(destfile->Append(slice));
|
||||
size -= slice.size();
|
||||
}
|
||||
ASSERT_OK(destfile->Close());
|
||||
}
|
||||
|
||||
};
|
||||
std::unique_ptr<const SliceTransform> DBTest::prefix_1_transform(
|
||||
NewFixedPrefixTransform(1));
|
||||
@ -784,8 +808,11 @@ static std::string Key(int i) {
|
||||
return std::string(buf);
|
||||
}
|
||||
|
||||
/*
|
||||
TEST(DBTest, GetFromImmutableLayer) {
|
||||
static long TestGetTickerCount(const Options& options, Tickers ticker_type) {
|
||||
return options.statistics->getTickerCount(ticker_type);
|
||||
}
|
||||
|
||||
TEST(DBTest, Empty) {
|
||||
do {
|
||||
Options options = CurrentOptions();
|
||||
options.env = env_;
|
||||
@ -802,7 +829,6 @@ TEST(DBTest, GetFromImmutableLayer) {
|
||||
env_->delay_sstable_sync_.Release_Store(nullptr); // Release sync calls
|
||||
} while (ChangeOptions());
|
||||
}
|
||||
*/
|
||||
|
||||
// Make sure that when options.block_cache is set, after a new table is
|
||||
// created its index/filter blocks are added to block cache.
|
||||
@ -819,14 +845,11 @@ TEST(DBTest, IndexAndFilterBlocksOfNewTableAddedToCache) {
|
||||
ASSERT_OK(dbfull()->Flush(FlushOptions()));
|
||||
|
||||
// index/filter blocks added to block cache right after table creation.
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_INDEX_MISS));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(2, /* only index/filter were added */
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(0,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_DATA_MISS));
|
||||
TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(0, TestGetTickerCount(options, BLOCK_CACHE_DATA_MISS));
|
||||
|
||||
// Make sure filter block is in cache.
|
||||
std::string value;
|
||||
@ -834,31 +857,24 @@ TEST(DBTest, IndexAndFilterBlocksOfNewTableAddedToCache) {
|
||||
db_->KeyMayExist(ReadOptions(), "key", &value);
|
||||
|
||||
// Miss count should remain the same.
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
|
||||
db_->KeyMayExist(ReadOptions(), "key", &value);
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(2,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_HIT));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(2, TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
|
||||
// Make sure index block is in cache.
|
||||
auto index_block_hit =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_HIT);
|
||||
auto index_block_hit = TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT);
|
||||
value = Get("key");
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(index_block_hit + 1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_HIT));
|
||||
TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
|
||||
value = Get("key");
|
||||
ASSERT_EQ(1,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(1, TestGetTickerCount(options, BLOCK_CACHE_FILTER_MISS));
|
||||
ASSERT_EQ(index_block_hit + 2,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_FILTER_HIT));
|
||||
TestGetTickerCount(options, BLOCK_CACHE_FILTER_HIT));
|
||||
}
|
||||
|
||||
TEST(DBTest, LevelLimitReopen) {
|
||||
@ -1073,47 +1089,39 @@ TEST(DBTest, KeyMayExist) {
|
||||
dbfull()->Flush(FlushOptions());
|
||||
value.clear();
|
||||
|
||||
long numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
long cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
long numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
long cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
ASSERT_TRUE(db_->KeyMayExist(ropts, "a", &value, &value_found));
|
||||
ASSERT_TRUE(!value_found);
|
||||
// assert that no new files were opened and no new blocks were
|
||||
// read into block cache.
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
|
||||
ASSERT_OK(db_->Delete(WriteOptions(), "a"));
|
||||
|
||||
numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
ASSERT_TRUE(!db_->KeyMayExist(ropts, "a", &value));
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
|
||||
dbfull()->Flush(FlushOptions());
|
||||
dbfull()->CompactRange(nullptr, nullptr);
|
||||
|
||||
numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
ASSERT_TRUE(!db_->KeyMayExist(ropts, "a", &value));
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
|
||||
ASSERT_OK(db_->Delete(WriteOptions(), "c"));
|
||||
|
||||
numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
ASSERT_TRUE(!db_->KeyMayExist(ropts, "c", &value));
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
|
||||
delete options.filter_policy;
|
||||
|
||||
@ -1149,9 +1157,8 @@ TEST(DBTest, NonBlockingIteration) {
|
||||
|
||||
// verify that a non-blocking iterator does not find any
|
||||
// kvs. Neither does it do any IOs to storage.
|
||||
long numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
long cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
long numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
long cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
iter = db_->NewIterator(non_blocking_opts);
|
||||
count = 0;
|
||||
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
|
||||
@ -1159,18 +1166,16 @@ TEST(DBTest, NonBlockingIteration) {
|
||||
}
|
||||
ASSERT_EQ(count, 0);
|
||||
ASSERT_TRUE(iter->status().IsIncomplete());
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
delete iter;
|
||||
|
||||
// read in the specified block via a regular get
|
||||
ASSERT_EQ(Get("a"), "b");
|
||||
|
||||
// verify that we can find it via a non-blocking scan
|
||||
numopen = options.statistics.get()->getTickerCount(NO_FILE_OPENS);
|
||||
cache_added =
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD);
|
||||
numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
iter = db_->NewIterator(non_blocking_opts);
|
||||
count = 0;
|
||||
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
|
||||
@ -1178,9 +1183,8 @@ TEST(DBTest, NonBlockingIteration) {
|
||||
count++;
|
||||
}
|
||||
ASSERT_EQ(count, 1);
|
||||
ASSERT_EQ(numopen, options.statistics.get()->getTickerCount(NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added,
|
||||
options.statistics.get()->getTickerCount(BLOCK_CACHE_ADD));
|
||||
ASSERT_EQ(numopen, TestGetTickerCount(options, NO_FILE_OPENS));
|
||||
ASSERT_EQ(cache_added, TestGetTickerCount(options, BLOCK_CACHE_ADD));
|
||||
delete iter;
|
||||
|
||||
// This test verifies block cache behaviors, which is not used by plain
|
||||
@ -1398,12 +1402,10 @@ TEST(DBTest, IterReseek) {
|
||||
ASSERT_OK(Put("b", "bone"));
|
||||
Iterator* iter = db_->NewIterator(ReadOptions());
|
||||
iter->SeekToFirst();
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(IterStatus(iter), "a->two");
|
||||
iter->Next();
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(IterStatus(iter), "b->bone");
|
||||
delete iter;
|
||||
|
||||
@ -1414,8 +1416,7 @@ TEST(DBTest, IterReseek) {
|
||||
iter->SeekToFirst();
|
||||
ASSERT_EQ(IterStatus(iter), "a->three");
|
||||
iter->Next();
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(IterStatus(iter), "b->bone");
|
||||
delete iter;
|
||||
|
||||
@ -1425,30 +1426,28 @@ TEST(DBTest, IterReseek) {
|
||||
iter = db_->NewIterator(ReadOptions());
|
||||
iter->SeekToFirst();
|
||||
ASSERT_EQ(IterStatus(iter), "a->four");
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION), 0);
|
||||
iter->Next();
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), 1);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION), 1);
|
||||
ASSERT_EQ(IterStatus(iter), "b->bone");
|
||||
delete iter;
|
||||
|
||||
// Testing reverse iterator
|
||||
// At this point, we have three versions of "a" and one version of "b".
|
||||
// The reseek statistics is already at 1.
|
||||
int num_reseeks = (int)options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION);
|
||||
int num_reseeks =
|
||||
(int)TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION);
|
||||
|
||||
// Insert another version of b and assert that reseek is not invoked
|
||||
ASSERT_OK(Put("b", "btwo"));
|
||||
iter = db_->NewIterator(ReadOptions());
|
||||
iter->SeekToLast();
|
||||
ASSERT_EQ(IterStatus(iter), "b->btwo");
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), num_reseeks);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION),
|
||||
num_reseeks);
|
||||
iter->Prev();
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), num_reseeks+1);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION),
|
||||
num_reseeks + 1);
|
||||
ASSERT_EQ(IterStatus(iter), "a->four");
|
||||
delete iter;
|
||||
|
||||
@ -1459,13 +1458,13 @@ TEST(DBTest, IterReseek) {
|
||||
iter = db_->NewIterator(ReadOptions());
|
||||
iter->SeekToLast();
|
||||
ASSERT_EQ(IterStatus(iter), "b->bfour");
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), num_reseeks + 2);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION),
|
||||
num_reseeks + 2);
|
||||
iter->Prev();
|
||||
|
||||
// the previous Prev call should have invoked reseek
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(
|
||||
NUMBER_OF_RESEEKS_IN_ITERATION), num_reseeks + 3);
|
||||
ASSERT_EQ(TestGetTickerCount(options, NUMBER_OF_RESEEKS_IN_ITERATION),
|
||||
num_reseeks + 3);
|
||||
ASSERT_EQ(IterStatus(iter), "a->four");
|
||||
delete iter;
|
||||
}
|
||||
@ -1619,6 +1618,82 @@ TEST(DBTest, Recover) {
|
||||
} while (ChangeOptions());
|
||||
}
|
||||
|
||||
TEST(DBTest, IgnoreRecoveredLog) {
|
||||
std::string backup_logs = dbname_ + "/backup_logs";
|
||||
|
||||
// delete old files in backup_logs directory
|
||||
env_->CreateDirIfMissing(backup_logs);
|
||||
std::vector<std::string> old_files;
|
||||
env_->GetChildren(backup_logs, &old_files);
|
||||
for (auto& file : old_files) {
|
||||
if (file != "." && file != "..") {
|
||||
env_->DeleteFile(backup_logs + "/" + file);
|
||||
}
|
||||
}
|
||||
|
||||
do {
|
||||
Options options = CurrentOptions();
|
||||
options.create_if_missing = true;
|
||||
options.merge_operator = MergeOperators::CreateUInt64AddOperator();
|
||||
options.wal_dir = dbname_ + "/logs";
|
||||
DestroyAndReopen(&options);
|
||||
|
||||
// fill up the DB
|
||||
std::string one, two;
|
||||
PutFixed64(&one, 1);
|
||||
PutFixed64(&two, 2);
|
||||
ASSERT_OK(db_->Merge(WriteOptions(), Slice("foo"), Slice(one)));
|
||||
ASSERT_OK(db_->Merge(WriteOptions(), Slice("foo"), Slice(one)));
|
||||
ASSERT_OK(db_->Merge(WriteOptions(), Slice("bar"), Slice(one)));
|
||||
|
||||
// copy the logs to backup
|
||||
std::vector<std::string> logs;
|
||||
env_->GetChildren(options.wal_dir, &logs);
|
||||
for (auto& log : logs) {
|
||||
if (log != ".." && log != ".") {
|
||||
CopyFile(options.wal_dir + "/" + log, backup_logs + "/" + log);
|
||||
}
|
||||
}
|
||||
|
||||
// recover the DB
|
||||
Reopen(&options);
|
||||
ASSERT_EQ(two, Get("foo"));
|
||||
ASSERT_EQ(one, Get("bar"));
|
||||
Close();
|
||||
|
||||
// copy the logs from backup back to wal dir
|
||||
for (auto& log : logs) {
|
||||
if (log != ".." && log != ".") {
|
||||
CopyFile(backup_logs + "/" + log, options.wal_dir + "/" + log);
|
||||
}
|
||||
}
|
||||
// this should ignore the log files, recovery should not happen again
|
||||
// if the recovery happens, the same merge operator would be called twice,
|
||||
// leading to incorrect results
|
||||
Reopen(&options);
|
||||
ASSERT_EQ(two, Get("foo"));
|
||||
ASSERT_EQ(one, Get("bar"));
|
||||
Close();
|
||||
Destroy(&options);
|
||||
|
||||
// copy the logs from backup back to wal dir
|
||||
env_->CreateDirIfMissing(options.wal_dir);
|
||||
for (auto& log : logs) {
|
||||
if (log != ".." && log != ".") {
|
||||
CopyFile(backup_logs + "/" + log, options.wal_dir + "/" + log);
|
||||
// we won't be needing this file no more
|
||||
env_->DeleteFile(backup_logs + "/" + log);
|
||||
}
|
||||
}
|
||||
// assert that we successfully recovered only from logs, even though we
|
||||
// destroyed the DB
|
||||
Reopen(&options);
|
||||
ASSERT_EQ(two, Get("foo"));
|
||||
ASSERT_EQ(one, Get("bar"));
|
||||
Close();
|
||||
} while (ChangeOptions());
|
||||
}
|
||||
|
||||
TEST(DBTest, RollLog) {
|
||||
do {
|
||||
ASSERT_OK(Put("foo", "v1"));
|
||||
@ -2257,24 +2332,18 @@ TEST(DBTest, CompressedCache) {
|
||||
switch (iter) {
|
||||
case 0:
|
||||
// only uncompressed block cache
|
||||
ASSERT_GT(options.statistics.get()->getTickerCount(BLOCK_CACHE_MISS),
|
||||
0);
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount
|
||||
(BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
ASSERT_GT(TestGetTickerCount(options, BLOCK_CACHE_MISS), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
break;
|
||||
case 1:
|
||||
// no block cache, only compressed cache
|
||||
ASSERT_EQ(options.statistics.get()->getTickerCount(BLOCK_CACHE_MISS),
|
||||
0);
|
||||
ASSERT_GT(options.statistics.get()->getTickerCount
|
||||
(BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
ASSERT_EQ(TestGetTickerCount(options, BLOCK_CACHE_MISS), 0);
|
||||
ASSERT_GT(TestGetTickerCount(options, BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
break;
|
||||
case 2:
|
||||
// both compressed and uncompressed block cache
|
||||
ASSERT_GT(options.statistics.get()->getTickerCount(BLOCK_CACHE_MISS),
|
||||
0);
|
||||
ASSERT_GT(options.statistics.get()->getTickerCount
|
||||
(BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
ASSERT_GT(TestGetTickerCount(options, BLOCK_CACHE_MISS), 0);
|
||||
ASSERT_GT(TestGetTickerCount(options, BLOCK_CACHE_COMPRESSED_MISS), 0);
|
||||
break;
|
||||
default:
|
||||
ASSERT_TRUE(false);
|
||||
@ -3878,7 +3947,6 @@ TEST(DBTest, BloomFilter) {
|
||||
TEST(DBTest, SnapshotFiles) {
|
||||
do {
|
||||
Options options = CurrentOptions();
|
||||
const EnvOptions soptions;
|
||||
options.write_buffer_size = 100000000; // Large write buffer
|
||||
Reopen(&options);
|
||||
|
||||
@ -3934,20 +4002,7 @@ TEST(DBTest, SnapshotFiles) {
|
||||
}
|
||||
}
|
||||
}
|
||||
unique_ptr<SequentialFile> srcfile;
|
||||
ASSERT_OK(env_->NewSequentialFile(src, &srcfile, soptions));
|
||||
unique_ptr<WritableFile> destfile;
|
||||
ASSERT_OK(env_->NewWritableFile(dest, &destfile, soptions));
|
||||
|
||||
char buffer[4096];
|
||||
Slice slice;
|
||||
while (size > 0) {
|
||||
uint64_t one = std::min(uint64_t(sizeof(buffer)), size);
|
||||
ASSERT_OK(srcfile->Read(one, &slice, buffer));
|
||||
ASSERT_OK(destfile->Append(slice));
|
||||
size -= slice.size();
|
||||
}
|
||||
ASSERT_OK(destfile->Close());
|
||||
CopyFile(src, dest, size);
|
||||
}
|
||||
|
||||
// release file snapshot
|
||||
|
@ -21,7 +21,7 @@
|
||||
#include "util/murmurhash.h"
|
||||
#include "util/mutexlock.h"
|
||||
#include "util/perf_context_imp.h"
|
||||
#include "util/statistics_imp.h"
|
||||
#include "util/statistics.h"
|
||||
#include "util/stop_watch.h"
|
||||
|
||||
namespace std {
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include "rocksdb/comparator.h"
|
||||
#include "rocksdb/db.h"
|
||||
#include "rocksdb/merge_operator.h"
|
||||
#include "util/statistics_imp.h"
|
||||
#include "util/statistics.h"
|
||||
#include <string>
|
||||
#include <stdio.h>
|
||||
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include "db/filename.h"
|
||||
#include "db/version_set.h"
|
||||
#include "db/write_batch_internal.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/cache.h"
|
||||
#include "rocksdb/compaction_filter.h"
|
||||
#include "rocksdb/env.h"
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include "db/filename.h"
|
||||
#include "db/version_set.h"
|
||||
#include "db/write_batch_internal.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "rocksdb/cache.h"
|
||||
#include "rocksdb/compaction_filter.h"
|
||||
#include "rocksdb/env.h"
|
||||
|
@ -2025,22 +2025,21 @@ Iterator* VersionSet::MakeInputIterator(Compaction* c) {
|
||||
// Level-0 files have to be merged together. For other levels,
|
||||
// we will make a concatenating iterator per level.
|
||||
// TODO(opt): use concatenating iterator for level-0 if there is no overlap
|
||||
const int space = (c->level() == 0 ? c->inputs_[0].size() + 1 : 2);
|
||||
const int space = (c->level() == 0 ? c->inputs(0)->size() + 1 : 2);
|
||||
Iterator** list = new Iterator*[space];
|
||||
int num = 0;
|
||||
for (int which = 0; which < 2; which++) {
|
||||
if (!c->inputs_[which].empty()) {
|
||||
if (!c->inputs(which)->empty()) {
|
||||
if (c->level() + which == 0) {
|
||||
const std::vector<FileMetaData*>& files = c->inputs_[which];
|
||||
for (size_t i = 0; i < files.size(); i++) {
|
||||
for (const auto& file : *c->inputs(which)) {
|
||||
list[num++] = table_cache_->NewIterator(
|
||||
options, storage_options_compactions_,
|
||||
*(files[i]), nullptr, true /* for compaction */);
|
||||
options, storage_options_compactions_, *file, nullptr,
|
||||
true /* for compaction */);
|
||||
}
|
||||
} else {
|
||||
// Create concatenating iterator for the files from this level
|
||||
list[num++] = NewTwoLevelIterator(
|
||||
new Version::LevelFileNumIterator(icmp_, &c->inputs_[which]),
|
||||
new Version::LevelFileNumIterator(icmp_, c->inputs(which)),
|
||||
&GetFileIterator, table_cache_, options, storage_options_,
|
||||
true /* for compaction */);
|
||||
}
|
||||
@ -2064,7 +2063,7 @@ uint64_t VersionSet::MaxFileSizeForLevel(int level) {
|
||||
// in the current version
|
||||
bool VersionSet::VerifyCompactionFileConsistency(Compaction* c) {
|
||||
#ifndef NDEBUG
|
||||
if (c->input_version_ != current_) {
|
||||
if (c->input_version() != current_) {
|
||||
Log(options_->info_log, "VerifyCompactionFileConsistency version mismatch");
|
||||
}
|
||||
|
||||
|
@ -28,7 +28,7 @@
|
||||
#include "db/snapshot.h"
|
||||
#include "db/write_batch_internal.h"
|
||||
#include "util/coding.h"
|
||||
#include "util/statistics_imp.h"
|
||||
#include "util/statistics.h"
|
||||
#include <stdexcept>
|
||||
|
||||
namespace rocksdb {
|
||||
|
@ -239,53 +239,11 @@ struct HistogramData {
|
||||
double standard_deviation;
|
||||
};
|
||||
|
||||
|
||||
class Histogram {
|
||||
public:
|
||||
// clear's the histogram
|
||||
virtual void Clear() = 0;
|
||||
virtual ~Histogram();
|
||||
// Add a value to be recorded in the histogram.
|
||||
virtual void Add(uint64_t value) = 0;
|
||||
|
||||
virtual std::string ToString() const = 0;
|
||||
|
||||
// Get statistics
|
||||
virtual double Median() const = 0;
|
||||
virtual double Percentile(double p) const = 0;
|
||||
virtual double Average() const = 0;
|
||||
virtual double StandardDeviation() const = 0;
|
||||
virtual void Data(HistogramData * const data) const = 0;
|
||||
|
||||
};
|
||||
|
||||
/**
|
||||
* A dumb ticker which keeps incrementing through its life time.
|
||||
* Thread safe. Locking managed by implementation of this interface.
|
||||
*/
|
||||
class Ticker {
|
||||
public:
|
||||
Ticker() : count_(0) { }
|
||||
|
||||
inline void setTickerCount(uint64_t count) {
|
||||
count_ = count;
|
||||
}
|
||||
|
||||
inline void recordTick(int64_t count = 1) {
|
||||
count_ += count;
|
||||
}
|
||||
|
||||
inline uint64_t getCount() {
|
||||
return count_;
|
||||
}
|
||||
|
||||
private:
|
||||
std::atomic_uint_fast64_t count_;
|
||||
};
|
||||
|
||||
// Analyze the performance of a db
|
||||
class Statistics {
|
||||
public:
|
||||
virtual ~Statistics() {}
|
||||
|
||||
virtual long getTickerCount(Tickers tickerType) = 0;
|
||||
virtual void recordTick(Tickers tickerType, uint64_t count = 0) = 0;
|
||||
virtual void setTickerCount(Tickers tickerType, uint64_t count) = 0;
|
||||
|
4759
linters/cpplint.py
vendored
Executable file
4759
linters/cpplint.py
vendored
Executable file
File diff suppressed because it is too large
Load Diff
@ -49,10 +49,15 @@ class FacebookFbcodeLintEngine extends ArcanistLintEngine {
|
||||
// Currently we can't run cpplint in commit hook mode, because it
|
||||
// depends on having access to the working directory.
|
||||
if (!$this->getCommitHookMode()) {
|
||||
$cpp_linter = new FbcodeCppLinter();
|
||||
$cpp_linter2 = new PfffCppLinter();
|
||||
$linters[] = $cpp_linter;
|
||||
$linters[] = $cpp_linter2;
|
||||
$cpp_linters = array();
|
||||
$google_linter = new ArcanistCpplintLinter();
|
||||
$google_linter->setConfig(array(
|
||||
'lint.cpplint.prefix' => '',
|
||||
'lint.cpplint.bin' => 'cpplint',
|
||||
));
|
||||
$cpp_linters[] = $linters[] = $google_linter;
|
||||
$cpp_linters[] = $linters[] = new FbcodeCppLinter();
|
||||
$cpp_linters[] = $linters[] = new PfffCppLinter();
|
||||
}
|
||||
|
||||
$spelling_linter = new ArcanistSpellingLinter();
|
||||
@ -93,13 +98,11 @@ class FacebookFbcodeLintEngine extends ArcanistLintEngine {
|
||||
$spelling_linter->addPath($path);
|
||||
$spelling_linter->addData($path, $this->loadData($path));
|
||||
}
|
||||
if (isset($cpp_linter) && isset($cpp_linter2) &&
|
||||
preg_match('/\.(cpp|c|cc|cxx|h|hh|hpp|hxx|tcc)$/', $path)) {
|
||||
$cpp_linter->addPath($path);
|
||||
$cpp_linter->addData($path, $this->loadData($path));
|
||||
$cpp_linter2->addPath($path);
|
||||
$cpp_linter2->addData($path, $this->loadData($path));
|
||||
|
||||
if (preg_match('/\.(cpp|c|cc|cxx|h|hh|hpp|hxx|tcc)$/', $path)) {
|
||||
foreach ($cpp_linters as &$linter) {
|
||||
$linter->addPath($path);
|
||||
$linter->addData($path, $this->loadData($path));
|
||||
}
|
||||
}
|
||||
|
||||
// Match *.py and contbuild config files
|
||||
@ -130,9 +133,6 @@ class FacebookFbcodeLintEngine extends ArcanistLintEngine {
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
$name_linter = new ArcanistFilenameLinter();
|
@ -1 +0,0 @@
|
||||
{"__symbol_cache_version__":8,"b937ad5f80a8bd1156038b730ff56ec5":{"have":{"class":{"FacebookFbcodeLintEngine":71}},"need":{"class":{"ArcanistLintEngine":104,"ArcanistGeneratedLinter":488,"ArcanistNoLintLinter":577,"ArcanistTextLinter":658,"ArcanistPEP8Linter":1227,"FbcodeCppLinter":1715,"PfffCppLinter":1759,"ArcanistSpellingLinter":1875,"ArcanistFilenameLinter":4207,"Filesystem":357,"ArcanistLintSeverity":778}},"xmap":{"FacebookFbcodeLintEngine":["ArcanistLintEngine"]}},"02e2a613e371424b2108d2d6cb849d39":{"have":{"class":{"PfffCppLinter":71}},"need":{"function":{"Futures":875},"class":{"ArcanistLinter":93,"ExecFuture":756,"ArcanistLintMessage":1270,"ArcanistLintSeverity":1607}},"xmap":{"PfffCppLinter":["ArcanistLinter"]}},"4443484928afb005f585843d07b04190":{"have":{"class":{"FbcodeCppLinter":13}},"need":{"function":{"Futures":1265},"class":{"ArcanistLinter":37,"ExecFuture":934,"ArcanistLintSeverity":1729}},"xmap":{"FbcodeCppLinter":["ArcanistLinter"]}}}
|
@ -12,7 +12,8 @@
|
||||
#include <vector>
|
||||
|
||||
#include "db/dbformat.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "util/statistics.h"
|
||||
#include "db/memtable.h"
|
||||
#include "db/write_batch_internal.h"
|
||||
|
||||
@ -1048,18 +1049,12 @@ TEST(TableTest, NumBlockStat) {
|
||||
class BlockCacheProperties {
|
||||
public:
|
||||
explicit BlockCacheProperties(Statistics* statistics) {
|
||||
block_cache_miss =
|
||||
statistics->getTickerCount(BLOCK_CACHE_MISS);
|
||||
block_cache_hit =
|
||||
statistics->getTickerCount(BLOCK_CACHE_HIT);
|
||||
index_block_cache_miss =
|
||||
statistics->getTickerCount(BLOCK_CACHE_INDEX_MISS);
|
||||
index_block_cache_hit =
|
||||
statistics->getTickerCount(BLOCK_CACHE_INDEX_HIT);
|
||||
data_block_cache_miss =
|
||||
statistics->getTickerCount(BLOCK_CACHE_DATA_MISS);
|
||||
data_block_cache_hit =
|
||||
statistics->getTickerCount(BLOCK_CACHE_DATA_HIT);
|
||||
block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_MISS);
|
||||
block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_HIT);
|
||||
index_block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_INDEX_MISS);
|
||||
index_block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_INDEX_HIT);
|
||||
data_block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_DATA_MISS);
|
||||
data_block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_DATA_HIT);
|
||||
}
|
||||
|
||||
// Check if the fetched props matches the expected ones.
|
||||
|
@ -26,7 +26,7 @@
|
||||
#include <gflags/gflags.h>
|
||||
#include "db/db_impl.h"
|
||||
#include "db/version_set.h"
|
||||
#include "db/db_statistics.h"
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "rocksdb/cache.h"
|
||||
#include "utilities/utility_db.h"
|
||||
#include "rocksdb/env.h"
|
||||
|
@ -16,27 +16,38 @@
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
HistogramBucketMapper::HistogramBucketMapper() :
|
||||
// Add newer bucket index here.
|
||||
// Should be alwyas added in sorted order.
|
||||
bucketValues_({
|
||||
1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12, 14, 16, 18, 20, 25, 30, 35, 40, 45,
|
||||
50, 60, 70, 80, 90, 100, 120, 140, 160, 180, 200, 250, 300, 350, 400, 450,
|
||||
500, 600, 700, 800, 900, 1000, 1200, 1400, 1600, 1800, 2000, 2500, 3000,
|
||||
3500, 4000, 4500, 5000, 6000, 7000, 8000, 9000, 10000, 12000, 14000,
|
||||
16000, 18000, 20000, 25000, 30000, 35000, 40000, 45000, 50000, 60000,
|
||||
70000, 80000, 90000, 100000, 120000, 140000, 160000, 180000, 200000,
|
||||
250000, 300000, 350000, 400000, 450000, 500000, 600000, 700000, 800000,
|
||||
900000, 1000000, 1200000, 1400000, 1600000, 1800000, 2000000, 2500000,
|
||||
3000000, 3500000, 4000000, 4500000, 5000000, 6000000, 7000000, 8000000,
|
||||
9000000, 10000000, 12000000, 14000000, 16000000, 18000000, 20000000,
|
||||
25000000, 30000000, 35000000, 40000000, 45000000, 50000000, 60000000,
|
||||
70000000, 80000000, 90000000, 100000000, 120000000, 140000000, 160000000,
|
||||
180000000, 200000000, 250000000, 300000000, 350000000, 400000000,
|
||||
450000000, 500000000, 600000000, 700000000, 800000000, 900000000,
|
||||
1000000000}),
|
||||
maxBucketValue_(bucketValues_.back()),
|
||||
minBucketValue_(bucketValues_.front()) {
|
||||
HistogramBucketMapper::HistogramBucketMapper()
|
||||
:
|
||||
// Add newer bucket index here.
|
||||
// Should be alwyas added in sorted order.
|
||||
// If you change this, you also need to change
|
||||
// size of array buckets_ in HistogramImpl
|
||||
bucketValues_(
|
||||
{1, 2, 3, 4, 5, 6,
|
||||
7, 8, 9, 10, 12, 14,
|
||||
16, 18, 20, 25, 30, 35,
|
||||
40, 45, 50, 60, 70, 80,
|
||||
90, 100, 120, 140, 160, 180,
|
||||
200, 250, 300, 350, 400, 450,
|
||||
500, 600, 700, 800, 900, 1000,
|
||||
1200, 1400, 1600, 1800, 2000, 2500,
|
||||
3000, 3500, 4000, 4500, 5000, 6000,
|
||||
7000, 8000, 9000, 10000, 12000, 14000,
|
||||
16000, 18000, 20000, 25000, 30000, 35000,
|
||||
40000, 45000, 50000, 60000, 70000, 80000,
|
||||
90000, 100000, 120000, 140000, 160000, 180000,
|
||||
200000, 250000, 300000, 350000, 400000, 450000,
|
||||
500000, 600000, 700000, 800000, 900000, 1000000,
|
||||
1200000, 1400000, 1600000, 1800000, 2000000, 2500000,
|
||||
3000000, 3500000, 4000000, 4500000, 5000000, 6000000,
|
||||
7000000, 8000000, 9000000, 10000000, 12000000, 14000000,
|
||||
16000000, 18000000, 20000000, 25000000, 30000000, 35000000,
|
||||
40000000, 45000000, 50000000, 60000000, 70000000, 80000000,
|
||||
90000000, 100000000, 120000000, 140000000, 160000000, 180000000,
|
||||
200000000, 250000000, 300000000, 350000000, 400000000, 450000000,
|
||||
500000000, 600000000, 700000000, 800000000, 900000000, 1000000000}),
|
||||
maxBucketValue_(bucketValues_.back()),
|
||||
minBucketValue_(bucketValues_.front()) {
|
||||
for (size_t i =0; i < bucketValues_.size(); ++i) {
|
||||
valueIndexMap_[bucketValues_[i]] = i;
|
||||
}
|
||||
@ -62,24 +73,17 @@ namespace {
|
||||
const HistogramBucketMapper bucketMapper;
|
||||
}
|
||||
|
||||
|
||||
HistogramImpl::HistogramImpl() :
|
||||
min_(bucketMapper.LastValue()),
|
||||
max_(0),
|
||||
num_(0),
|
||||
sum_(0),
|
||||
sum_squares_(0),
|
||||
buckets_(std::vector<uint64_t>(bucketMapper.BucketCount(), 0)) {}
|
||||
|
||||
void HistogramImpl::Clear() {
|
||||
min_ = bucketMapper.LastValue();
|
||||
max_ = 0;
|
||||
num_ = 0;
|
||||
sum_ = 0;
|
||||
sum_squares_ = 0;
|
||||
buckets_.resize(bucketMapper.BucketCount(), 0);
|
||||
memset(buckets_, 0, sizeof buckets_);
|
||||
}
|
||||
|
||||
bool HistogramImpl::Empty() { return sum_squares_ == 0; }
|
||||
|
||||
void HistogramImpl::Add(uint64_t value) {
|
||||
const size_t index = bucketMapper.IndexForValue(value);
|
||||
buckets_[index] += 1;
|
||||
|
@ -52,9 +52,8 @@ class HistogramBucketMapper {
|
||||
|
||||
class HistogramImpl {
|
||||
public:
|
||||
HistogramImpl();
|
||||
virtual ~HistogramImpl() {}
|
||||
virtual void Clear();
|
||||
virtual bool Empty();
|
||||
virtual void Add(uint64_t value);
|
||||
void Merge(const HistogramImpl& other);
|
||||
|
||||
@ -67,13 +66,14 @@ class HistogramImpl {
|
||||
virtual void Data(HistogramData * const data) const;
|
||||
|
||||
private:
|
||||
double min_;
|
||||
double max_;
|
||||
double num_;
|
||||
double sum_;
|
||||
double sum_squares_;
|
||||
std::vector<uint64_t> buckets_;
|
||||
|
||||
// To be able to use HistogramImpl as thread local variable, its constructor
|
||||
// has to be static. That's why we're using manually values from BucketMapper
|
||||
double min_ = 1000000000; // this is BucketMapper:LastValue()
|
||||
double max_ = 0;
|
||||
double num_ = 0;
|
||||
double sum_ = 0;
|
||||
double sum_squares_ = 0;
|
||||
uint64_t buckets_[138] = {0}; // this is BucketMapper::BucketCount()
|
||||
};
|
||||
|
||||
} // namespace rocksdb
|
||||
|
@ -3,12 +3,51 @@
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
//
|
||||
#include "util/statistics.h"
|
||||
#include "rocksdb/statistics.h"
|
||||
#include <algorithm>
|
||||
#include <cstdio>
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
std::shared_ptr<Statistics> CreateDBStatistics() {
|
||||
return std::make_shared<StatisticsImpl>();
|
||||
}
|
||||
|
||||
StatisticsImpl::StatisticsImpl()
|
||||
: tickers_(TICKER_ENUM_MAX),
|
||||
histograms_(HISTOGRAM_ENUM_MAX) {}
|
||||
|
||||
StatisticsImpl::~StatisticsImpl() {}
|
||||
|
||||
long StatisticsImpl::getTickerCount(Tickers tickerType) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
return tickers_[tickerType];
|
||||
}
|
||||
|
||||
void StatisticsImpl::setTickerCount(Tickers tickerType, uint64_t count) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
tickers_[tickerType] = count;
|
||||
}
|
||||
|
||||
void StatisticsImpl::recordTick(Tickers tickerType, uint64_t count) {
|
||||
assert(tickerType < TICKER_ENUM_MAX);
|
||||
tickers_[tickerType] += count;
|
||||
}
|
||||
|
||||
void StatisticsImpl::measureTime(Histograms histogramType, uint64_t value) {
|
||||
assert(histogramType < HISTOGRAM_ENUM_MAX);
|
||||
histograms_[histogramType].Add(value);
|
||||
}
|
||||
|
||||
void StatisticsImpl::histogramData(Histograms histogramType,
|
||||
HistogramData* const data) {
|
||||
assert(histogramType < HISTOGRAM_ENUM_MAX);
|
||||
histograms_[histogramType].Data(data);
|
||||
}
|
||||
|
||||
namespace {
|
||||
|
||||
// a buffer size used for temp string buffers
|
||||
const int kBufferSize = 200;
|
||||
|
||||
@ -32,11 +71,8 @@ std::string HistogramToString (
|
||||
return std::string(buffer);
|
||||
};
|
||||
|
||||
std::string TickerToString (
|
||||
Statistics* dbstats,
|
||||
const Tickers& ticker,
|
||||
const std::string& name) {
|
||||
|
||||
std::string TickerToString(Statistics* dbstats, const Tickers& ticker,
|
||||
const std::string& name) {
|
||||
char buffer[kBufferSize];
|
||||
snprintf(buffer, kBufferSize, "%s COUNT : %ld\n",
|
||||
name.c_str(), dbstats->getTickerCount(ticker));
|
||||
|
56
util/statistics.h
Normal file
56
util/statistics.h
Normal file
@ -0,0 +1,56 @@
|
||||
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
//
|
||||
#pragma once
|
||||
#include "rocksdb/statistics.h"
|
||||
#include "util/histogram.h"
|
||||
#include "util/mutexlock.h"
|
||||
|
||||
#include <vector>
|
||||
#include <atomic>
|
||||
|
||||
#define UNLIKELY(val) (__builtin_expect((val), 0))
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
class StatisticsImpl : public Statistics {
|
||||
public:
|
||||
StatisticsImpl();
|
||||
virtual ~StatisticsImpl();
|
||||
|
||||
virtual long getTickerCount(Tickers tickerType);
|
||||
virtual void setTickerCount(Tickers tickerType, uint64_t count);
|
||||
virtual void recordTick(Tickers tickerType, uint64_t count);
|
||||
virtual void measureTime(Histograms histogramType, uint64_t value);
|
||||
virtual void histogramData(Histograms histogramType,
|
||||
HistogramData* const data);
|
||||
|
||||
private:
|
||||
std::vector<std::atomic_uint_fast64_t> tickers_;
|
||||
std::vector<HistogramImpl> histograms_;
|
||||
};
|
||||
|
||||
// Utility functions
|
||||
inline void MeasureTime(Statistics* statistics, Histograms histogramType,
|
||||
uint64_t value) {
|
||||
if (statistics) {
|
||||
statistics->measureTime(histogramType, value);
|
||||
}
|
||||
}
|
||||
|
||||
inline void RecordTick(Statistics* statistics, Tickers ticker,
|
||||
uint64_t count = 1) {
|
||||
if (statistics) {
|
||||
statistics->recordTick(ticker, count);
|
||||
}
|
||||
}
|
||||
|
||||
inline void SetTickerCount(Statistics* statistics, Tickers ticker,
|
||||
uint64_t count) {
|
||||
if (statistics) {
|
||||
statistics->setTickerCount(ticker, count);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,32 +0,0 @@
|
||||
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
//
|
||||
#pragma once
|
||||
#include "rocksdb/statistics.h"
|
||||
|
||||
namespace rocksdb {
|
||||
|
||||
// Utility functions
|
||||
inline void RecordTick(Statistics* statistics,
|
||||
Tickers ticker,
|
||||
int64_t count = 1) {
|
||||
assert(HistogramsNameMap.size() == HISTOGRAM_ENUM_MAX);
|
||||
assert(TickersNameMap.size() == TICKER_ENUM_MAX);
|
||||
if (statistics) {
|
||||
statistics->recordTick(ticker, count);
|
||||
}
|
||||
}
|
||||
|
||||
inline void SetTickerCount(Statistics* statistics,
|
||||
Tickers ticker,
|
||||
uint64_t count) {
|
||||
assert(HistogramsNameMap.size() == HISTOGRAM_ENUM_MAX);
|
||||
assert(TickersNameMap.size() == TICKER_ENUM_MAX);
|
||||
if (statistics) {
|
||||
statistics->setTickerCount(ticker, count);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -5,7 +5,7 @@
|
||||
//
|
||||
#pragma once
|
||||
#include "rocksdb/env.h"
|
||||
#include "util/statistics_imp.h"
|
||||
#include "util/statistics.h"
|
||||
|
||||
namespace rocksdb {
|
||||
// Auto-scoped.
|
||||
@ -28,11 +28,7 @@ class StopWatch {
|
||||
return env_->NowMicros() - start_time_;
|
||||
}
|
||||
|
||||
~StopWatch() {
|
||||
if (statistics_) {
|
||||
statistics_->measureTime(histogram_name_, ElapsedMicros());
|
||||
}
|
||||
}
|
||||
~StopWatch() { MeasureTime(statistics_, histogram_name_, ElapsedMicros()); }
|
||||
|
||||
private:
|
||||
Env* const env_;
|
||||
|
Loading…
Reference in New Issue
Block a user