Add a counter for collecting the wait time on db mutex.
Summary: Add a counter for collecting the wait time on db mutex. Also add MutexWrapper and CondVarWrapper for measuring wait time. Test Plan: ./db_test export ROCKSDB_TESTS=MutexWaitStats ./db_test verify stats output using db_bench make clean make release ./db_bench --statistics=1 --benchmarks=fillseq,readwhilewriting --num=10000 --threads=10 Sample output: rocksdb.db.mutex.wait.micros COUNT : 7546866 Reviewers: MarkCallaghan, rven, sdong, igor Reviewed By: igor Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D32787
This commit is contained in:
parent
f36d394aed
commit
181191a1e4
@ -66,7 +66,7 @@ uint64_t SlowdownAmount(int n, double bottom, double top) {
|
|||||||
} // namespace
|
} // namespace
|
||||||
|
|
||||||
ColumnFamilyHandleImpl::ColumnFamilyHandleImpl(
|
ColumnFamilyHandleImpl::ColumnFamilyHandleImpl(
|
||||||
ColumnFamilyData* column_family_data, DBImpl* db, port::Mutex* mutex)
|
ColumnFamilyData* column_family_data, DBImpl* db, InstrumentedMutex* mutex)
|
||||||
: cfd_(column_family_data), db_(db), mutex_(mutex) {
|
: cfd_(column_family_data), db_(db), mutex_(mutex) {
|
||||||
if (cfd_ != nullptr) {
|
if (cfd_ != nullptr) {
|
||||||
cfd_->Ref();
|
cfd_->Ref();
|
||||||
@ -482,7 +482,7 @@ Compaction* ColumnFamilyData::CompactRange(
|
|||||||
}
|
}
|
||||||
|
|
||||||
SuperVersion* ColumnFamilyData::GetReferencedSuperVersion(
|
SuperVersion* ColumnFamilyData::GetReferencedSuperVersion(
|
||||||
port::Mutex* db_mutex) {
|
InstrumentedMutex* db_mutex) {
|
||||||
SuperVersion* sv = nullptr;
|
SuperVersion* sv = nullptr;
|
||||||
sv = GetThreadLocalSuperVersion(db_mutex);
|
sv = GetThreadLocalSuperVersion(db_mutex);
|
||||||
sv->Ref();
|
sv->Ref();
|
||||||
@ -493,7 +493,7 @@ SuperVersion* ColumnFamilyData::GetReferencedSuperVersion(
|
|||||||
}
|
}
|
||||||
|
|
||||||
SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(
|
SuperVersion* ColumnFamilyData::GetThreadLocalSuperVersion(
|
||||||
port::Mutex* db_mutex) {
|
InstrumentedMutex* db_mutex) {
|
||||||
SuperVersion* sv = nullptr;
|
SuperVersion* sv = nullptr;
|
||||||
// The SuperVersion is cached in thread local storage to avoid acquiring
|
// The SuperVersion is cached in thread local storage to avoid acquiring
|
||||||
// mutex when SuperVersion does not change since the last use. When a new
|
// mutex when SuperVersion does not change since the last use. When a new
|
||||||
@ -599,13 +599,13 @@ void ColumnFamilyData::NotifyOnFlushCompleted(
|
|||||||
}
|
}
|
||||||
|
|
||||||
SuperVersion* ColumnFamilyData::InstallSuperVersion(
|
SuperVersion* ColumnFamilyData::InstallSuperVersion(
|
||||||
SuperVersion* new_superversion, port::Mutex* db_mutex) {
|
SuperVersion* new_superversion, InstrumentedMutex* db_mutex) {
|
||||||
db_mutex->AssertHeld();
|
db_mutex->AssertHeld();
|
||||||
return InstallSuperVersion(new_superversion, db_mutex, mutable_cf_options_);
|
return InstallSuperVersion(new_superversion, db_mutex, mutable_cf_options_);
|
||||||
}
|
}
|
||||||
|
|
||||||
SuperVersion* ColumnFamilyData::InstallSuperVersion(
|
SuperVersion* ColumnFamilyData::InstallSuperVersion(
|
||||||
SuperVersion* new_superversion, port::Mutex* db_mutex,
|
SuperVersion* new_superversion, InstrumentedMutex* db_mutex,
|
||||||
const MutableCFOptions& mutable_cf_options) {
|
const MutableCFOptions& mutable_cf_options) {
|
||||||
new_superversion->db_mutex = db_mutex;
|
new_superversion->db_mutex = db_mutex;
|
||||||
new_superversion->mutable_cf_options = mutable_cf_options;
|
new_superversion->mutable_cf_options = mutable_cf_options;
|
||||||
|
@ -21,9 +21,10 @@
|
|||||||
#include "db/write_batch_internal.h"
|
#include "db/write_batch_internal.h"
|
||||||
#include "db/write_controller.h"
|
#include "db/write_controller.h"
|
||||||
#include "db/table_cache.h"
|
#include "db/table_cache.h"
|
||||||
#include "util/thread_local.h"
|
|
||||||
#include "db/flush_scheduler.h"
|
#include "db/flush_scheduler.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
#include "util/mutable_cf_options.h"
|
#include "util/mutable_cf_options.h"
|
||||||
|
#include "util/thread_local.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
@ -38,6 +39,8 @@ class InternalStats;
|
|||||||
class ColumnFamilyData;
|
class ColumnFamilyData;
|
||||||
class DBImpl;
|
class DBImpl;
|
||||||
class LogBuffer;
|
class LogBuffer;
|
||||||
|
class InstrumentedMutex;
|
||||||
|
class InstrumentedMutexLock;
|
||||||
|
|
||||||
// ColumnFamilyHandleImpl is the class that clients use to access different
|
// ColumnFamilyHandleImpl is the class that clients use to access different
|
||||||
// column families. It has non-trivial destructor, which gets called when client
|
// column families. It has non-trivial destructor, which gets called when client
|
||||||
@ -45,7 +48,8 @@ class LogBuffer;
|
|||||||
class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
|
class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
|
||||||
public:
|
public:
|
||||||
// create while holding the mutex
|
// create while holding the mutex
|
||||||
ColumnFamilyHandleImpl(ColumnFamilyData* cfd, DBImpl* db, port::Mutex* mutex);
|
ColumnFamilyHandleImpl(
|
||||||
|
ColumnFamilyData* cfd, DBImpl* db, InstrumentedMutex* mutex);
|
||||||
// destroy without mutex
|
// destroy without mutex
|
||||||
virtual ~ColumnFamilyHandleImpl();
|
virtual ~ColumnFamilyHandleImpl();
|
||||||
virtual ColumnFamilyData* cfd() const { return cfd_; }
|
virtual ColumnFamilyData* cfd() const { return cfd_; }
|
||||||
@ -57,7 +61,7 @@ class ColumnFamilyHandleImpl : public ColumnFamilyHandle {
|
|||||||
private:
|
private:
|
||||||
ColumnFamilyData* cfd_;
|
ColumnFamilyData* cfd_;
|
||||||
DBImpl* db_;
|
DBImpl* db_;
|
||||||
port::Mutex* mutex_;
|
InstrumentedMutex* mutex_;
|
||||||
};
|
};
|
||||||
|
|
||||||
// Does not ref-count ColumnFamilyData
|
// Does not ref-count ColumnFamilyData
|
||||||
@ -91,7 +95,7 @@ struct SuperVersion {
|
|||||||
autovector<MemTable*> to_delete;
|
autovector<MemTable*> to_delete;
|
||||||
// Version number of the current SuperVersion
|
// Version number of the current SuperVersion
|
||||||
uint64_t version_number;
|
uint64_t version_number;
|
||||||
port::Mutex* db_mutex;
|
InstrumentedMutex* db_mutex;
|
||||||
|
|
||||||
// should be called outside the mutex
|
// should be called outside the mutex
|
||||||
SuperVersion() = default;
|
SuperVersion() = default;
|
||||||
@ -235,11 +239,11 @@ class ColumnFamilyData {
|
|||||||
SuperVersion* GetSuperVersion() { return super_version_; }
|
SuperVersion* GetSuperVersion() { return super_version_; }
|
||||||
// thread-safe
|
// thread-safe
|
||||||
// Return a already referenced SuperVersion to be used safely.
|
// Return a already referenced SuperVersion to be used safely.
|
||||||
SuperVersion* GetReferencedSuperVersion(port::Mutex* db_mutex);
|
SuperVersion* GetReferencedSuperVersion(InstrumentedMutex* db_mutex);
|
||||||
// thread-safe
|
// thread-safe
|
||||||
// Get SuperVersion stored in thread local storage. If it does not exist,
|
// Get SuperVersion stored in thread local storage. If it does not exist,
|
||||||
// get a reference from a current SuperVersion.
|
// get a reference from a current SuperVersion.
|
||||||
SuperVersion* GetThreadLocalSuperVersion(port::Mutex* db_mutex);
|
SuperVersion* GetThreadLocalSuperVersion(InstrumentedMutex* db_mutex);
|
||||||
// Try to return SuperVersion back to thread local storage. Retrun true on
|
// Try to return SuperVersion back to thread local storage. Retrun true on
|
||||||
// success and false on failure. It fails when the thread local storage
|
// success and false on failure. It fails when the thread local storage
|
||||||
// contains anything other than SuperVersion::kSVInUse flag.
|
// contains anything other than SuperVersion::kSVInUse flag.
|
||||||
@ -254,10 +258,10 @@ class ColumnFamilyData {
|
|||||||
// the clients to allocate SuperVersion outside of mutex.
|
// the clients to allocate SuperVersion outside of mutex.
|
||||||
// IMPORTANT: Only call this from DBImpl::InstallSuperVersion()
|
// IMPORTANT: Only call this from DBImpl::InstallSuperVersion()
|
||||||
SuperVersion* InstallSuperVersion(SuperVersion* new_superversion,
|
SuperVersion* InstallSuperVersion(SuperVersion* new_superversion,
|
||||||
port::Mutex* db_mutex,
|
InstrumentedMutex* db_mutex,
|
||||||
const MutableCFOptions& mutable_cf_options);
|
const MutableCFOptions& mutable_cf_options);
|
||||||
SuperVersion* InstallSuperVersion(SuperVersion* new_superversion,
|
SuperVersion* InstallSuperVersion(SuperVersion* new_superversion,
|
||||||
port::Mutex* db_mutex);
|
InstrumentedMutex* db_mutex);
|
||||||
|
|
||||||
void ResetThreadLocalSuperVersions();
|
void ResetThreadLocalSuperVersions();
|
||||||
|
|
||||||
|
@ -470,7 +470,7 @@ Status CompactionJob::Run() {
|
|||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
void CompactionJob::Install(Status* status, port::Mutex* db_mutex) {
|
void CompactionJob::Install(Status* status, InstrumentedMutex* db_mutex) {
|
||||||
db_mutex->AssertHeld();
|
db_mutex->AssertHeld();
|
||||||
ColumnFamilyData* cfd = compact_->compaction->column_family_data();
|
ColumnFamilyData* cfd = compact_->compaction->column_family_data();
|
||||||
cfd->internal_stats()->AddCompactionStats(
|
cfd->internal_stats()->AddCompactionStats(
|
||||||
@ -955,7 +955,7 @@ Status CompactionJob::FinishCompactionOutputFile(Iterator* input) {
|
|||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
Status CompactionJob::InstallCompactionResults(port::Mutex* db_mutex) {
|
Status CompactionJob::InstallCompactionResults(InstrumentedMutex* db_mutex) {
|
||||||
db_mutex->AssertHeld();
|
db_mutex->AssertHeld();
|
||||||
|
|
||||||
// paranoia: verify that the files that we started with
|
// paranoia: verify that the files that we started with
|
||||||
|
@ -75,7 +75,7 @@ class CompactionJob {
|
|||||||
Status Run();
|
Status Run();
|
||||||
// REQUIRED: mutex held
|
// REQUIRED: mutex held
|
||||||
// status is the return of Run()
|
// status is the return of Run()
|
||||||
void Install(Status* status, port::Mutex* db_mutex);
|
void Install(Status* status, InstrumentedMutex* db_mutex);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void AllocateCompactionOutputFileNumbers();
|
void AllocateCompactionOutputFileNumbers();
|
||||||
@ -86,7 +86,7 @@ class CompactionJob {
|
|||||||
// Call compaction_filter_v2->Filter() on kv-pairs in compact
|
// Call compaction_filter_v2->Filter() on kv-pairs in compact
|
||||||
void CallCompactionFilterV2(CompactionFilterV2* compaction_filter_v2);
|
void CallCompactionFilterV2(CompactionFilterV2* compaction_filter_v2);
|
||||||
Status FinishCompactionOutputFile(Iterator* input);
|
Status FinishCompactionOutputFile(Iterator* input);
|
||||||
Status InstallCompactionResults(port::Mutex* db_mutex);
|
Status InstallCompactionResults(InstrumentedMutex* db_mutex);
|
||||||
SequenceNumber findEarliestVisibleSnapshot(
|
SequenceNumber findEarliestVisibleSnapshot(
|
||||||
SequenceNumber in, const std::vector<SequenceNumber>& snapshots,
|
SequenceNumber in, const std::vector<SequenceNumber>& snapshots,
|
||||||
SequenceNumber* prev_snapshot);
|
SequenceNumber* prev_snapshot);
|
||||||
|
@ -130,7 +130,7 @@ class CompactionJobTest {
|
|||||||
ColumnFamilyOptions cf_options_;
|
ColumnFamilyOptions cf_options_;
|
||||||
WriteBuffer write_buffer_;
|
WriteBuffer write_buffer_;
|
||||||
std::unique_ptr<VersionSet> versions_;
|
std::unique_ptr<VersionSet> versions_;
|
||||||
port::Mutex mutex_;
|
InstrumentedMutex mutex_;
|
||||||
std::atomic<bool> shutting_down_;
|
std::atomic<bool> shutting_down_;
|
||||||
std::shared_ptr<mock::MockTableFactory> mock_table_factory_;
|
std::shared_ptr<mock::MockTableFactory> mock_table_factory_;
|
||||||
};
|
};
|
||||||
|
@ -31,7 +31,7 @@
|
|||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
Status DBImpl::DisableFileDeletions() {
|
Status DBImpl::DisableFileDeletions() {
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
++disable_delete_obsolete_files_;
|
++disable_delete_obsolete_files_;
|
||||||
if (disable_delete_obsolete_files_ == 1) {
|
if (disable_delete_obsolete_files_ == 1) {
|
||||||
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
||||||
@ -48,7 +48,7 @@ Status DBImpl::EnableFileDeletions(bool force) {
|
|||||||
JobContext job_context;
|
JobContext job_context;
|
||||||
bool should_purge_files = false;
|
bool should_purge_files = false;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
if (force) {
|
if (force) {
|
||||||
// if force, we need to enable file deletions right away
|
// if force, we need to enable file deletions right away
|
||||||
disable_delete_obsolete_files_ = 0;
|
disable_delete_obsolete_files_ = 0;
|
||||||
|
@ -197,7 +197,9 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
|
|||||||
db_options_(SanitizeOptions(dbname, options)),
|
db_options_(SanitizeOptions(dbname, options)),
|
||||||
stats_(db_options_.statistics.get()),
|
stats_(db_options_.statistics.get()),
|
||||||
db_lock_(nullptr),
|
db_lock_(nullptr),
|
||||||
mutex_(options.use_adaptive_mutex),
|
mutex_(stats_, env_,
|
||||||
|
DB_MUTEX_WAIT_MICROS,
|
||||||
|
options.use_adaptive_mutex),
|
||||||
shutting_down_(false),
|
shutting_down_(false),
|
||||||
bg_cv_(&mutex_),
|
bg_cv_(&mutex_),
|
||||||
logfile_number_(0),
|
logfile_number_(0),
|
||||||
@ -411,7 +413,7 @@ void DBImpl::MaybeDumpStats() {
|
|||||||
GetPropertyType("rocksdb.dbstats", &tmp1, &tmp2);
|
GetPropertyType("rocksdb.dbstats", &tmp1, &tmp2);
|
||||||
std::string stats;
|
std::string stats;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
||||||
cfd->internal_stats()->GetStringProperty(cf_property_type,
|
cfd->internal_stats()->GetStringProperty(cf_property_type,
|
||||||
"rocksdb.cfstats", &stats);
|
"rocksdb.cfstats", &stats);
|
||||||
@ -1225,7 +1227,7 @@ Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
|
|||||||
|
|
||||||
int max_level_with_files = 0;
|
int max_level_with_files = 0;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
Version* base = cfd->current();
|
Version* base = cfd->current();
|
||||||
for (int level = 1; level < cfd->NumberLevels(); level++) {
|
for (int level = 1; level < cfd->NumberLevels(); level++) {
|
||||||
if (base->storage_info()->OverlapInLevel(level, begin, end)) {
|
if (base->storage_info()->OverlapInLevel(level, begin, end)) {
|
||||||
@ -1258,7 +1260,7 @@ Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
|
|||||||
LogFlush(db_options_.info_log);
|
LogFlush(db_options_.info_log);
|
||||||
|
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
// an automatic compaction that has been scheduled might have been
|
// an automatic compaction that has been scheduled might have been
|
||||||
// preempted by the manual compactions. Need to schedule it back.
|
// preempted by the manual compactions. Need to schedule it back.
|
||||||
MaybeScheduleFlushOrCompaction();
|
MaybeScheduleFlushOrCompaction();
|
||||||
@ -1276,7 +1278,7 @@ Status DBImpl::CompactFiles(
|
|||||||
// not supported in lite version
|
// not supported in lite version
|
||||||
return Status::NotSupported("Not supported in ROCKSDB LITE");
|
return Status::NotSupported("Not supported in ROCKSDB LITE");
|
||||||
#else
|
#else
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
if (column_family == nullptr) {
|
if (column_family == nullptr) {
|
||||||
return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
|
return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
|
||||||
}
|
}
|
||||||
@ -1471,7 +1473,7 @@ Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
|
|||||||
MutableCFOptions new_options;
|
MutableCFOptions new_options;
|
||||||
Status s;
|
Status s;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
s = cfd->SetOptions(options_map);
|
s = cfd->SetOptions(options_map);
|
||||||
if (s.ok()) {
|
if (s.ok()) {
|
||||||
new_options = *cfd->GetLatestMutableCFOptions();
|
new_options = *cfd->GetLatestMutableCFOptions();
|
||||||
@ -1607,14 +1609,14 @@ int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
|
|||||||
|
|
||||||
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
|
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
|
||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return cfh->cfd()->GetSuperVersion()->
|
return cfh->cfd()->GetSuperVersion()->
|
||||||
mutable_cf_options.max_mem_compaction_level;
|
mutable_cf_options.max_mem_compaction_level;
|
||||||
}
|
}
|
||||||
|
|
||||||
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
|
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
|
||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return cfh->cfd()->GetSuperVersion()->
|
return cfh->cfd()->GetSuperVersion()->
|
||||||
mutable_cf_options.level0_stop_writes_trigger;
|
mutable_cf_options.level0_stop_writes_trigger;
|
||||||
}
|
}
|
||||||
@ -1662,7 +1664,7 @@ Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
|
|||||||
manual.end = &end_storage;
|
manual.end = &end_storage;
|
||||||
}
|
}
|
||||||
|
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
|
|
||||||
// When a manual compaction arrives, temporarily disable scheduling of
|
// When a manual compaction arrives, temporarily disable scheduling of
|
||||||
// non-manual compactions and wait until the number of scheduled compaction
|
// non-manual compactions and wait until the number of scheduled compaction
|
||||||
@ -1717,7 +1719,7 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
|
|||||||
Status s;
|
Status s;
|
||||||
{
|
{
|
||||||
WriteContext context;
|
WriteContext context;
|
||||||
MutexLock guard_lock(&mutex_);
|
InstrumentedMutexLock guard_lock(&mutex_);
|
||||||
|
|
||||||
if (cfd->imm()->size() == 0 && cfd->mem()->IsEmpty()) {
|
if (cfd->imm()->size() == 0 && cfd->mem()->IsEmpty()) {
|
||||||
// Nothing to flush
|
// Nothing to flush
|
||||||
@ -1750,7 +1752,7 @@ Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
|
|||||||
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
|
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
|
||||||
Status s;
|
Status s;
|
||||||
// Wait until the compaction completes
|
// Wait until the compaction completes
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
while (cfd->imm()->size() > 0 && bg_error_.ok()) {
|
while (cfd->imm()->size() > 0 && bg_error_.ok()) {
|
||||||
bg_cv_.Wait();
|
bg_cv_.Wait();
|
||||||
}
|
}
|
||||||
@ -1917,7 +1919,7 @@ void DBImpl::BackgroundCallFlush() {
|
|||||||
|
|
||||||
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
|
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
|
|
||||||
auto pending_outputs_inserted_elem =
|
auto pending_outputs_inserted_elem =
|
||||||
CaptureCurrentFileNumberInPendingOutputs();
|
CaptureCurrentFileNumberInPendingOutputs();
|
||||||
@ -1985,7 +1987,7 @@ void DBImpl::BackgroundCallCompaction() {
|
|||||||
MaybeDumpStats();
|
MaybeDumpStats();
|
||||||
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
|
LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
|
|
||||||
auto pending_outputs_inserted_elem =
|
auto pending_outputs_inserted_elem =
|
||||||
CaptureCurrentFileNumberInPendingOutputs();
|
CaptureCurrentFileNumberInPendingOutputs();
|
||||||
@ -2352,11 +2354,11 @@ uint64_t DBImpl::CallFlushDuringCompaction(
|
|||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
struct IterState {
|
struct IterState {
|
||||||
IterState(DBImpl* _db, port::Mutex* _mu, SuperVersion* _super_version)
|
IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
|
||||||
: db(_db), mu(_mu), super_version(_super_version) {}
|
: db(_db), mu(_mu), super_version(_super_version) {}
|
||||||
|
|
||||||
DBImpl* db;
|
DBImpl* db;
|
||||||
port::Mutex* mu;
|
InstrumentedMutex* mu;
|
||||||
SuperVersion* super_version;
|
SuperVersion* super_version;
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -2643,7 +2645,7 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
|
|||||||
Status s;
|
Status s;
|
||||||
*handle = nullptr;
|
*handle = nullptr;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
|
|
||||||
if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
|
if (versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name) !=
|
||||||
nullptr) {
|
nullptr) {
|
||||||
@ -2691,7 +2693,7 @@ Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
|
|||||||
"Creating column family [%s] FAILED -- %s",
|
"Creating column family [%s] FAILED -- %s",
|
||||||
column_family_name.c_str(), s.ToString().c_str());
|
column_family_name.c_str(), s.ToString().c_str());
|
||||||
}
|
}
|
||||||
} // MutexLock l(&mutex_)
|
} // InstrumentedMutexLock l(&mutex_)
|
||||||
|
|
||||||
// this is outside the mutex
|
// this is outside the mutex
|
||||||
if (s.ok()) {
|
if (s.ok()) {
|
||||||
@ -2716,7 +2718,7 @@ Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
|
|||||||
|
|
||||||
Status s;
|
Status s;
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
if (cfd->IsDropped()) {
|
if (cfd->IsDropped()) {
|
||||||
s = Status::InvalidArgument("Column family already dropped!\n");
|
s = Status::InvalidArgument("Column family already dropped!\n");
|
||||||
}
|
}
|
||||||
@ -2919,14 +2921,14 @@ const Snapshot* DBImpl::GetSnapshot() {
|
|||||||
int64_t unix_time = 0;
|
int64_t unix_time = 0;
|
||||||
env_->GetCurrentTime(&unix_time); // Ignore error
|
env_->GetCurrentTime(&unix_time); // Ignore error
|
||||||
|
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
// returns null if the underlying memtable does not support snapshot.
|
// returns null if the underlying memtable does not support snapshot.
|
||||||
if (!is_snapshot_supported_) return nullptr;
|
if (!is_snapshot_supported_) return nullptr;
|
||||||
return snapshots_.New(versions_->LastSequence(), unix_time);
|
return snapshots_.New(versions_->LastSequence(), unix_time);
|
||||||
}
|
}
|
||||||
|
|
||||||
void DBImpl::ReleaseSnapshot(const Snapshot* s) {
|
void DBImpl::ReleaseSnapshot(const Snapshot* s) {
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
|
snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -3377,7 +3379,7 @@ bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
|
|||||||
} else {
|
} else {
|
||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
auto cfd = cfh->cfd();
|
auto cfd = cfh->cfd();
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return cfd->internal_stats()->GetStringProperty(property_type, property,
|
return cfd->internal_stats()->GetStringProperty(property_type, property,
|
||||||
value);
|
value);
|
||||||
}
|
}
|
||||||
@ -3403,7 +3405,7 @@ bool DBImpl::GetIntPropertyInternal(ColumnFamilyHandle* column_family,
|
|||||||
auto cfd = cfh->cfd();
|
auto cfd = cfh->cfd();
|
||||||
|
|
||||||
if (!need_out_of_mutex) {
|
if (!need_out_of_mutex) {
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return cfd->internal_stats()->GetIntProperty(property_type, value, this);
|
return cfd->internal_stats()->GetIntProperty(property_type, value, this);
|
||||||
} else {
|
} else {
|
||||||
SuperVersion* sv = GetAndRefSuperVersion(cfd);
|
SuperVersion* sv = GetAndRefSuperVersion(cfd);
|
||||||
@ -3430,7 +3432,7 @@ void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
|
|||||||
// Release SuperVersion
|
// Release SuperVersion
|
||||||
if (sv->Unref()) {
|
if (sv->Unref()) {
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
sv->Cleanup();
|
sv->Cleanup();
|
||||||
}
|
}
|
||||||
delete sv;
|
delete sv;
|
||||||
@ -3447,7 +3449,7 @@ void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
|
|||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
auto cfd = cfh->cfd();
|
auto cfd = cfh->cfd();
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
v = cfd->current();
|
v = cfd->current();
|
||||||
v->Ref();
|
v->Ref();
|
||||||
}
|
}
|
||||||
@ -3462,7 +3464,7 @@ void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
|
|||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
v->Unref();
|
v->Unref();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -3530,7 +3532,7 @@ Status DBImpl::DeleteFile(std::string name) {
|
|||||||
VersionEdit edit;
|
VersionEdit edit;
|
||||||
JobContext job_context(true);
|
JobContext job_context(true);
|
||||||
{
|
{
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
|
status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
|
Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
|
||||||
@ -3589,7 +3591,7 @@ Status DBImpl::DeleteFile(std::string name) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
|
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
versions_->GetLiveFilesMetaData(metadata);
|
versions_->GetLiveFilesMetaData(metadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -36,6 +36,7 @@
|
|||||||
#include "util/thread_local.h"
|
#include "util/thread_local.h"
|
||||||
#include "util/scoped_arena_iterator.h"
|
#include "util/scoped_arena_iterator.h"
|
||||||
#include "util/hash.h"
|
#include "util/hash.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
#include "db/internal_stats.h"
|
#include "db/internal_stats.h"
|
||||||
#include "db/write_controller.h"
|
#include "db/write_controller.h"
|
||||||
#include "db/flush_scheduler.h"
|
#include "db/flush_scheduler.h"
|
||||||
@ -412,7 +413,7 @@ class DBImpl : public DB {
|
|||||||
FileLock* db_lock_;
|
FileLock* db_lock_;
|
||||||
|
|
||||||
// State below is protected by mutex_
|
// State below is protected by mutex_
|
||||||
port::Mutex mutex_;
|
InstrumentedMutex mutex_;
|
||||||
std::atomic<bool> shutting_down_;
|
std::atomic<bool> shutting_down_;
|
||||||
// This condition variable is signaled on these conditions:
|
// This condition variable is signaled on these conditions:
|
||||||
// * whenever bg_compaction_scheduled_ goes down to 0
|
// * whenever bg_compaction_scheduled_ goes down to 0
|
||||||
@ -422,7 +423,7 @@ class DBImpl : public DB {
|
|||||||
// * whenever bg_flush_scheduled_ value decreases (i.e. whenever a flush is
|
// * whenever bg_flush_scheduled_ value decreases (i.e. whenever a flush is
|
||||||
// done, even if it didn't make any progress)
|
// done, even if it didn't make any progress)
|
||||||
// * whenever there is an error in background flush or compaction
|
// * whenever there is an error in background flush or compaction
|
||||||
port::CondVar bg_cv_;
|
InstrumentedCondVar bg_cv_;
|
||||||
uint64_t logfile_number_;
|
uint64_t logfile_number_;
|
||||||
unique_ptr<log::Writer> log_;
|
unique_ptr<log::Writer> log_;
|
||||||
bool log_dir_synced_;
|
bool log_dir_synced_;
|
||||||
|
@ -15,7 +15,7 @@
|
|||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
uint64_t DBImpl::TEST_GetLevel0TotalSize() {
|
uint64_t DBImpl::TEST_GetLevel0TotalSize() {
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return default_cf_handle_->cfd()->current()->storage_info()->NumLevelBytes(0);
|
return default_cf_handle_->cfd()->current()->storage_info()->NumLevelBytes(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -45,7 +45,7 @@ int64_t DBImpl::TEST_MaxNextLevelOverlappingBytes(
|
|||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
cfd = cfh->cfd();
|
cfd = cfh->cfd();
|
||||||
}
|
}
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
return cfd->current()->storage_info()->MaxNextLevelOverlappingBytes();
|
return cfd->current()->storage_info()->MaxNextLevelOverlappingBytes();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -54,7 +54,7 @@ void DBImpl::TEST_GetFilesMetaData(
|
|||||||
std::vector<std::vector<FileMetaData>>* metadata) {
|
std::vector<std::vector<FileMetaData>>* metadata) {
|
||||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
|
||||||
auto cfd = cfh->cfd();
|
auto cfd = cfh->cfd();
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
metadata->resize(NumberLevels());
|
metadata->resize(NumberLevels());
|
||||||
for (int level = 0; level < NumberLevels(); level++) {
|
for (int level = 0; level < NumberLevels(); level++) {
|
||||||
const std::vector<FileMetaData*>& files =
|
const std::vector<FileMetaData*>& files =
|
||||||
@ -113,7 +113,7 @@ Status DBImpl::TEST_WaitForCompact() {
|
|||||||
// wait for compact. It actually waits for scheduled compaction
|
// wait for compact. It actually waits for scheduled compaction
|
||||||
// OR flush to finish.
|
// OR flush to finish.
|
||||||
|
|
||||||
MutexLock l(&mutex_);
|
InstrumentedMutexLock l(&mutex_);
|
||||||
while ((bg_compaction_scheduled_ || bg_flush_scheduled_) && bg_error_.ok()) {
|
while ((bg_compaction_scheduled_ || bg_flush_scheduled_) && bg_error_.ok()) {
|
||||||
bg_cv_.Wait();
|
bg_cv_.Wait();
|
||||||
}
|
}
|
||||||
|
@ -10298,6 +10298,21 @@ TEST(DBTest, EncodeDecompressedBlockSizeTest) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TEST(DBTest, MutexWaitStats) {
|
||||||
|
Options options = CurrentOptions();
|
||||||
|
options.create_if_missing = true;
|
||||||
|
options.statistics = rocksdb::CreateDBStatistics();
|
||||||
|
CreateAndReopenWithCF({"pikachu"}, options);
|
||||||
|
const int64_t kMutexWaitDelay = 100;
|
||||||
|
ThreadStatusUtil::TEST_SetStateDelay(
|
||||||
|
ThreadStatus::STATE_MUTEX_WAIT, kMutexWaitDelay);
|
||||||
|
ASSERT_OK(Put("hello", "rocksdb"));
|
||||||
|
ASSERT_GE(TestGetTickerCount(
|
||||||
|
options, DB_MUTEX_WAIT_MICROS), kMutexWaitDelay);
|
||||||
|
ThreadStatusUtil::TEST_SetStateDelay(
|
||||||
|
ThreadStatus::STATE_MUTEX_WAIT, 0);
|
||||||
|
}
|
||||||
|
|
||||||
} // namespace rocksdb
|
} // namespace rocksdb
|
||||||
|
|
||||||
int main(int argc, char** argv) {
|
int main(int argc, char** argv) {
|
||||||
|
@ -55,7 +55,8 @@ FlushJob::FlushJob(const std::string& dbname, ColumnFamilyData* cfd,
|
|||||||
const DBOptions& db_options,
|
const DBOptions& db_options,
|
||||||
const MutableCFOptions& mutable_cf_options,
|
const MutableCFOptions& mutable_cf_options,
|
||||||
const EnvOptions& env_options, VersionSet* versions,
|
const EnvOptions& env_options, VersionSet* versions,
|
||||||
port::Mutex* db_mutex, std::atomic<bool>* shutting_down,
|
InstrumentedMutex* db_mutex,
|
||||||
|
std::atomic<bool>* shutting_down,
|
||||||
SequenceNumber newest_snapshot, JobContext* job_context,
|
SequenceNumber newest_snapshot, JobContext* job_context,
|
||||||
LogBuffer* log_buffer, Directory* db_directory,
|
LogBuffer* log_buffer, Directory* db_directory,
|
||||||
Directory* output_file_directory,
|
Directory* output_file_directory,
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
#include "rocksdb/memtablerep.h"
|
#include "rocksdb/memtablerep.h"
|
||||||
#include "rocksdb/transaction_log.h"
|
#include "rocksdb/transaction_log.h"
|
||||||
#include "util/autovector.h"
|
#include "util/autovector.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
#include "util/stop_watch.h"
|
#include "util/stop_watch.h"
|
||||||
#include "util/thread_local.h"
|
#include "util/thread_local.h"
|
||||||
#include "util/scoped_arena_iterator.h"
|
#include "util/scoped_arena_iterator.h"
|
||||||
@ -54,7 +55,7 @@ class FlushJob {
|
|||||||
const DBOptions& db_options,
|
const DBOptions& db_options,
|
||||||
const MutableCFOptions& mutable_cf_options,
|
const MutableCFOptions& mutable_cf_options,
|
||||||
const EnvOptions& env_options, VersionSet* versions,
|
const EnvOptions& env_options, VersionSet* versions,
|
||||||
port::Mutex* db_mutex, std::atomic<bool>* shutting_down,
|
InstrumentedMutex* db_mutex, std::atomic<bool>* shutting_down,
|
||||||
SequenceNumber newest_snapshot, JobContext* job_context,
|
SequenceNumber newest_snapshot, JobContext* job_context,
|
||||||
LogBuffer* log_buffer, Directory* db_directory,
|
LogBuffer* log_buffer, Directory* db_directory,
|
||||||
Directory* output_file_directory, CompressionType output_compression,
|
Directory* output_file_directory, CompressionType output_compression,
|
||||||
@ -72,7 +73,7 @@ class FlushJob {
|
|||||||
const MutableCFOptions& mutable_cf_options_;
|
const MutableCFOptions& mutable_cf_options_;
|
||||||
const EnvOptions& env_options_;
|
const EnvOptions& env_options_;
|
||||||
VersionSet* versions_;
|
VersionSet* versions_;
|
||||||
port::Mutex* db_mutex_;
|
InstrumentedMutex* db_mutex_;
|
||||||
std::atomic<bool>* shutting_down_;
|
std::atomic<bool>* shutting_down_;
|
||||||
SequenceNumber newest_snapshot_;
|
SequenceNumber newest_snapshot_;
|
||||||
JobContext* job_context_;
|
JobContext* job_context_;
|
||||||
|
@ -75,7 +75,7 @@ class FlushJobTest {
|
|||||||
WriteBuffer write_buffer_;
|
WriteBuffer write_buffer_;
|
||||||
ColumnFamilyOptions cf_options_;
|
ColumnFamilyOptions cf_options_;
|
||||||
std::unique_ptr<VersionSet> versions_;
|
std::unique_ptr<VersionSet> versions_;
|
||||||
port::Mutex mutex_;
|
InstrumentedMutex mutex_;
|
||||||
std::atomic<bool> shutting_down_;
|
std::atomic<bool> shutting_down_;
|
||||||
std::shared_ptr<mock::MockTableFactory> mock_table_factory_;
|
std::shared_ptr<mock::MockTableFactory> mock_table_factory_;
|
||||||
};
|
};
|
||||||
|
@ -31,8 +31,8 @@ void BM_LogAndApply(int iters, int num_base_files) {
|
|||||||
WriteController wc;
|
WriteController wc;
|
||||||
ColumnFamilyData* default_cfd;
|
ColumnFamilyData* default_cfd;
|
||||||
uint64_t fnum = 1;
|
uint64_t fnum = 1;
|
||||||
port::Mutex mu;
|
InstrumentedMutex mu;
|
||||||
MutexLock l(&mu);
|
InstrumentedMutexLock l(&mu);
|
||||||
|
|
||||||
BENCHMARK_SUSPEND {
|
BENCHMARK_SUSPEND {
|
||||||
std::string dbname = test::TmpDir() + "/rocksdb_test_benchmark";
|
std::string dbname = test::TmpDir() + "/rocksdb_test_benchmark";
|
||||||
|
@ -164,7 +164,7 @@ void MemTableList::RollbackMemtableFlush(const autovector<MemTable*>& mems,
|
|||||||
// Record a successful flush in the manifest file
|
// Record a successful flush in the manifest file
|
||||||
Status MemTableList::InstallMemtableFlushResults(
|
Status MemTableList::InstallMemtableFlushResults(
|
||||||
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
||||||
const autovector<MemTable*>& mems, VersionSet* vset, port::Mutex* mu,
|
const autovector<MemTable*>& mems, VersionSet* vset, InstrumentedMutex* mu,
|
||||||
uint64_t file_number, autovector<MemTable*>* to_delete,
|
uint64_t file_number, autovector<MemTable*>* to_delete,
|
||||||
Directory* db_directory, LogBuffer* log_buffer) {
|
Directory* db_directory, LogBuffer* log_buffer) {
|
||||||
mu->AssertHeld();
|
mu->AssertHeld();
|
||||||
|
@ -22,13 +22,14 @@
|
|||||||
#include "rocksdb/iterator.h"
|
#include "rocksdb/iterator.h"
|
||||||
#include "rocksdb/options.h"
|
#include "rocksdb/options.h"
|
||||||
#include "util/autovector.h"
|
#include "util/autovector.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
#include "util/log_buffer.h"
|
#include "util/log_buffer.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
class ColumnFamilyData;
|
class ColumnFamilyData;
|
||||||
class InternalKeyComparator;
|
class InternalKeyComparator;
|
||||||
class Mutex;
|
class InstrumentedMutex;
|
||||||
class MergeIteratorBuilder;
|
class MergeIteratorBuilder;
|
||||||
|
|
||||||
// keeps a list of immutable memtables in a vector. the list is immutable
|
// keeps a list of immutable memtables in a vector. the list is immutable
|
||||||
@ -113,7 +114,7 @@ class MemTableList {
|
|||||||
// Commit a successful flush in the manifest file
|
// Commit a successful flush in the manifest file
|
||||||
Status InstallMemtableFlushResults(
|
Status InstallMemtableFlushResults(
|
||||||
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
||||||
const autovector<MemTable*>& m, VersionSet* vset, port::Mutex* mu,
|
const autovector<MemTable*>& m, VersionSet* vset, InstrumentedMutex* mu,
|
||||||
uint64_t file_number, autovector<MemTable*>* to_delete,
|
uint64_t file_number, autovector<MemTable*>* to_delete,
|
||||||
Directory* db_directory, LogBuffer* log_buffer);
|
Directory* db_directory, LogBuffer* log_buffer);
|
||||||
|
|
||||||
|
@ -1487,11 +1487,11 @@ std::string Version::DebugString(bool hex) const {
|
|||||||
struct VersionSet::ManifestWriter {
|
struct VersionSet::ManifestWriter {
|
||||||
Status status;
|
Status status;
|
||||||
bool done;
|
bool done;
|
||||||
port::CondVar cv;
|
InstrumentedCondVar cv;
|
||||||
ColumnFamilyData* cfd;
|
ColumnFamilyData* cfd;
|
||||||
VersionEdit* edit;
|
VersionEdit* edit;
|
||||||
|
|
||||||
explicit ManifestWriter(port::Mutex* mu, ColumnFamilyData* _cfd,
|
explicit ManifestWriter(InstrumentedMutex* mu, ColumnFamilyData* _cfd,
|
||||||
VersionEdit* e)
|
VersionEdit* e)
|
||||||
: done(false), cv(mu), cfd(_cfd), edit(e) {}
|
: done(false), cv(mu), cfd(_cfd), edit(e) {}
|
||||||
};
|
};
|
||||||
@ -1556,7 +1556,7 @@ void VersionSet::AppendVersion(ColumnFamilyData* column_family_data,
|
|||||||
|
|
||||||
Status VersionSet::LogAndApply(ColumnFamilyData* column_family_data,
|
Status VersionSet::LogAndApply(ColumnFamilyData* column_family_data,
|
||||||
const MutableCFOptions& mutable_cf_options,
|
const MutableCFOptions& mutable_cf_options,
|
||||||
VersionEdit* edit, port::Mutex* mu,
|
VersionEdit* edit, InstrumentedMutex* mu,
|
||||||
Directory* db_directory, bool new_descriptor_log,
|
Directory* db_directory, bool new_descriptor_log,
|
||||||
const ColumnFamilyOptions* new_cf_options) {
|
const ColumnFamilyOptions* new_cf_options) {
|
||||||
mu->AssertHeld();
|
mu->AssertHeld();
|
||||||
@ -1824,7 +1824,7 @@ void VersionSet::LogAndApplyCFHelper(VersionEdit* edit) {
|
|||||||
|
|
||||||
void VersionSet::LogAndApplyHelper(ColumnFamilyData* cfd,
|
void VersionSet::LogAndApplyHelper(ColumnFamilyData* cfd,
|
||||||
VersionBuilder* builder, Version* v,
|
VersionBuilder* builder, Version* v,
|
||||||
VersionEdit* edit, port::Mutex* mu) {
|
VersionEdit* edit, InstrumentedMutex* mu) {
|
||||||
mu->AssertHeld();
|
mu->AssertHeld();
|
||||||
assert(!edit->IsColumnFamilyManipulation());
|
assert(!edit->IsColumnFamilyManipulation());
|
||||||
|
|
||||||
@ -2275,8 +2275,8 @@ Status VersionSet::ReduceNumberOfLevels(const std::string& dbname,
|
|||||||
|
|
||||||
MutableCFOptions mutable_cf_options(*options, ImmutableCFOptions(*options));
|
MutableCFOptions mutable_cf_options(*options, ImmutableCFOptions(*options));
|
||||||
VersionEdit ve;
|
VersionEdit ve;
|
||||||
port::Mutex dummy_mutex;
|
InstrumentedMutex dummy_mutex;
|
||||||
MutexLock l(&dummy_mutex);
|
InstrumentedMutexLock l(&dummy_mutex);
|
||||||
return versions.LogAndApply(
|
return versions.LogAndApply(
|
||||||
versions.GetColumnFamilySet()->GetDefault(),
|
versions.GetColumnFamilySet()->GetDefault(),
|
||||||
mutable_cf_options, &ve, &dummy_mutex, nullptr, true);
|
mutable_cf_options, &ve, &dummy_mutex, nullptr, true);
|
||||||
|
@ -36,6 +36,7 @@
|
|||||||
#include "db/log_reader.h"
|
#include "db/log_reader.h"
|
||||||
#include "db/file_indexer.h"
|
#include "db/file_indexer.h"
|
||||||
#include "db/write_controller.h"
|
#include "db/write_controller.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
@ -485,7 +486,7 @@ class VersionSet {
|
|||||||
Status LogAndApply(
|
Status LogAndApply(
|
||||||
ColumnFamilyData* column_family_data,
|
ColumnFamilyData* column_family_data,
|
||||||
const MutableCFOptions& mutable_cf_options, VersionEdit* edit,
|
const MutableCFOptions& mutable_cf_options, VersionEdit* edit,
|
||||||
port::Mutex* mu, Directory* db_directory = nullptr,
|
InstrumentedMutex* mu, Directory* db_directory = nullptr,
|
||||||
bool new_descriptor_log = false,
|
bool new_descriptor_log = false,
|
||||||
const ColumnFamilyOptions* column_family_options = nullptr);
|
const ColumnFamilyOptions* column_family_options = nullptr);
|
||||||
|
|
||||||
@ -656,7 +657,7 @@ class VersionSet {
|
|||||||
|
|
||||||
void LogAndApplyCFHelper(VersionEdit* edit);
|
void LogAndApplyCFHelper(VersionEdit* edit);
|
||||||
void LogAndApplyHelper(ColumnFamilyData* cfd, VersionBuilder* b, Version* v,
|
void LogAndApplyHelper(ColumnFamilyData* cfd, VersionBuilder* b, Version* v,
|
||||||
VersionEdit* edit, port::Mutex* mu);
|
VersionEdit* edit, InstrumentedMutex* mu);
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace rocksdb
|
} // namespace rocksdb
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include "db/write_batch_internal.h"
|
#include "db/write_batch_internal.h"
|
||||||
#include "util/autovector.h"
|
#include "util/autovector.h"
|
||||||
#include "port/port.h"
|
#include "port/port.h"
|
||||||
|
#include "util/instrumented_mutex.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
|
||||||
@ -27,9 +28,9 @@ class WriteThread {
|
|||||||
bool in_batch_group;
|
bool in_batch_group;
|
||||||
bool done;
|
bool done;
|
||||||
uint64_t timeout_hint_us;
|
uint64_t timeout_hint_us;
|
||||||
port::CondVar cv;
|
InstrumentedCondVar cv;
|
||||||
|
|
||||||
explicit Writer(port::Mutex* mu)
|
explicit Writer(InstrumentedMutex* mu)
|
||||||
: batch(nullptr),
|
: batch(nullptr),
|
||||||
sync(false),
|
sync(false),
|
||||||
disableWAL(false),
|
disableWAL(false),
|
||||||
|
@ -81,6 +81,8 @@ enum Tickers : uint32_t {
|
|||||||
STALL_L0_NUM_FILES_MICROS,
|
STALL_L0_NUM_FILES_MICROS,
|
||||||
// Writer has to wait for compaction or flush to finish.
|
// Writer has to wait for compaction or flush to finish.
|
||||||
STALL_MICROS,
|
STALL_MICROS,
|
||||||
|
// The wait time for db mutex.
|
||||||
|
DB_MUTEX_WAIT_MICROS,
|
||||||
RATE_LIMIT_DELAY_MILLIS,
|
RATE_LIMIT_DELAY_MILLIS,
|
||||||
NO_ITERATORS, // number of iterators currently open
|
NO_ITERATORS, // number of iterators currently open
|
||||||
|
|
||||||
@ -163,6 +165,7 @@ const std::vector<std::pair<Tickers, std::string>> TickersNameMap = {
|
|||||||
{STALL_MEMTABLE_COMPACTION_MICROS, "rocksdb.memtable.compaction.micros"},
|
{STALL_MEMTABLE_COMPACTION_MICROS, "rocksdb.memtable.compaction.micros"},
|
||||||
{STALL_L0_NUM_FILES_MICROS, "rocksdb.l0.num.files.stall.micros"},
|
{STALL_L0_NUM_FILES_MICROS, "rocksdb.l0.num.files.stall.micros"},
|
||||||
{STALL_MICROS, "rocksdb.stall.micros"},
|
{STALL_MICROS, "rocksdb.stall.micros"},
|
||||||
|
{DB_MUTEX_WAIT_MICROS, "rocksdb.db.mutex.wait.micros"},
|
||||||
{RATE_LIMIT_DELAY_MILLIS, "rocksdb.rate.limit.delay.millis"},
|
{RATE_LIMIT_DELAY_MILLIS, "rocksdb.rate.limit.delay.millis"},
|
||||||
{NO_ITERATORS, "rocksdb.num.iterators"},
|
{NO_ITERATORS, "rocksdb.num.iterators"},
|
||||||
{NUMBER_MULTIGET_CALLS, "rocksdb.number.multiget.get"},
|
{NUMBER_MULTIGET_CALLS, "rocksdb.number.multiget.get"},
|
||||||
|
@ -53,6 +53,7 @@ struct ThreadStatus {
|
|||||||
// such as reading / writing a file or waiting for a mutex.
|
// such as reading / writing a file or waiting for a mutex.
|
||||||
enum StateType : int {
|
enum StateType : int {
|
||||||
STATE_UNKNOWN = 0,
|
STATE_UNKNOWN = 0,
|
||||||
|
STATE_MUTEX_WAIT = 1,
|
||||||
NUM_STATE_TYPES
|
NUM_STATE_TYPES
|
||||||
};
|
};
|
||||||
|
|
||||||
|
72
util/instrumented_mutex.cc
Normal file
72
util/instrumented_mutex.cc
Normal file
@ -0,0 +1,72 @@
|
|||||||
|
// Copyright (c) 2015, 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 "util/instrumented_mutex.h"
|
||||||
|
#include "util/thread_status_util.h"
|
||||||
|
|
||||||
|
namespace rocksdb {
|
||||||
|
void InstrumentedMutex::Lock() {
|
||||||
|
uint64_t wait_time_micros = 0;
|
||||||
|
if (env_ != nullptr && stats_ != nullptr) {
|
||||||
|
{
|
||||||
|
StopWatch sw(env_, nullptr, 0, &wait_time_micros);
|
||||||
|
LockInternal();
|
||||||
|
}
|
||||||
|
RecordTick(stats_, stats_code_, wait_time_micros);
|
||||||
|
} else {
|
||||||
|
LockInternal();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void InstrumentedMutex::LockInternal() {
|
||||||
|
#ifndef NDEBUG
|
||||||
|
ThreadStatusUtil::TEST_StateDelay(ThreadStatus::STATE_MUTEX_WAIT);
|
||||||
|
#endif
|
||||||
|
mutex_.Lock();
|
||||||
|
}
|
||||||
|
|
||||||
|
void InstrumentedCondVar::Wait() {
|
||||||
|
uint64_t wait_time_micros = 0;
|
||||||
|
if (env_ != nullptr && stats_ != nullptr) {
|
||||||
|
{
|
||||||
|
StopWatch sw(env_, nullptr, 0, &wait_time_micros);
|
||||||
|
WaitInternal();
|
||||||
|
}
|
||||||
|
RecordTick(stats_, stats_code_, wait_time_micros);
|
||||||
|
} else {
|
||||||
|
WaitInternal();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void InstrumentedCondVar::WaitInternal() {
|
||||||
|
#ifndef NDEBUG
|
||||||
|
ThreadStatusUtil::TEST_StateDelay(ThreadStatus::STATE_MUTEX_WAIT);
|
||||||
|
#endif
|
||||||
|
cond_.Wait();
|
||||||
|
}
|
||||||
|
|
||||||
|
bool InstrumentedCondVar::TimedWait(uint64_t abs_time_us) {
|
||||||
|
uint64_t wait_time_micros = 0;
|
||||||
|
bool result = false;
|
||||||
|
if (env_ != nullptr && stats_ != nullptr) {
|
||||||
|
{
|
||||||
|
StopWatch sw(env_, nullptr, 0, &wait_time_micros);
|
||||||
|
result = TimedWaitInternal(abs_time_us);
|
||||||
|
}
|
||||||
|
RecordTick(stats_, stats_code_, wait_time_micros);
|
||||||
|
} else {
|
||||||
|
result = TimedWaitInternal(abs_time_us);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool InstrumentedCondVar::TimedWaitInternal(uint64_t abs_time_us) {
|
||||||
|
#ifndef NDEBUG
|
||||||
|
ThreadStatusUtil::TEST_StateDelay(ThreadStatus::STATE_MUTEX_WAIT);
|
||||||
|
#endif
|
||||||
|
return cond_.TimedWait(abs_time_us);
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace rocksdb
|
98
util/instrumented_mutex.h
Normal file
98
util/instrumented_mutex.h
Normal file
@ -0,0 +1,98 @@
|
|||||||
|
// Copyright (c) 2015, 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 "port/port.h"
|
||||||
|
#include "rocksdb/env.h"
|
||||||
|
#include "rocksdb/statistics.h"
|
||||||
|
#include "rocksdb/thread_status.h"
|
||||||
|
#include "util/statistics.h"
|
||||||
|
#include "util/stop_watch.h"
|
||||||
|
|
||||||
|
namespace rocksdb {
|
||||||
|
class InstrumentedCondVar;
|
||||||
|
|
||||||
|
// A wrapper class for port::Mutex that provides additional layer
|
||||||
|
// for collecting stats and instrumentation.
|
||||||
|
class InstrumentedMutex {
|
||||||
|
public:
|
||||||
|
explicit InstrumentedMutex(bool adaptive = false)
|
||||||
|
: mutex_(adaptive), stats_(nullptr), env_(nullptr),
|
||||||
|
stats_code_(0) {}
|
||||||
|
|
||||||
|
InstrumentedMutex(
|
||||||
|
Statistics* stats, Env* env,
|
||||||
|
int stats_code, bool adaptive = false)
|
||||||
|
: mutex_(adaptive), stats_(stats), env_(env),
|
||||||
|
stats_code_(stats_code) {}
|
||||||
|
|
||||||
|
void Lock();
|
||||||
|
|
||||||
|
void Unlock() {
|
||||||
|
mutex_.Unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
void AssertHeld() {
|
||||||
|
mutex_.AssertHeld();
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
void LockInternal();
|
||||||
|
friend class InstrumentedCondVar;
|
||||||
|
port::Mutex mutex_;
|
||||||
|
Statistics* stats_;
|
||||||
|
Env* env_;
|
||||||
|
int stats_code_;
|
||||||
|
};
|
||||||
|
|
||||||
|
// A wrapper class for port::Mutex that provides additional layer
|
||||||
|
// for collecting stats and instrumentation.
|
||||||
|
class InstrumentedMutexLock {
|
||||||
|
public:
|
||||||
|
explicit InstrumentedMutexLock(InstrumentedMutex* mutex) : mutex_(mutex) {
|
||||||
|
mutex_->Lock();
|
||||||
|
}
|
||||||
|
|
||||||
|
~InstrumentedMutexLock() {
|
||||||
|
mutex_->Unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
InstrumentedMutex* const mutex_;
|
||||||
|
InstrumentedMutexLock(const InstrumentedMutexLock&) = delete;
|
||||||
|
void operator=(const InstrumentedMutexLock&) = delete;
|
||||||
|
};
|
||||||
|
|
||||||
|
class InstrumentedCondVar {
|
||||||
|
public:
|
||||||
|
explicit InstrumentedCondVar(InstrumentedMutex* instrumented_mutex)
|
||||||
|
: cond_(&(instrumented_mutex->mutex_)),
|
||||||
|
stats_(instrumented_mutex->stats_),
|
||||||
|
env_(instrumented_mutex->env_),
|
||||||
|
stats_code_(instrumented_mutex->stats_code_) {}
|
||||||
|
|
||||||
|
void Wait();
|
||||||
|
|
||||||
|
bool TimedWait(uint64_t abs_time_us);
|
||||||
|
|
||||||
|
void Signal() {
|
||||||
|
cond_.Signal();
|
||||||
|
}
|
||||||
|
|
||||||
|
void SignalAll() {
|
||||||
|
cond_.SignalAll();
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
void WaitInternal();
|
||||||
|
bool TimedWaitInternal(uint64_t abs_time_us);
|
||||||
|
port::CondVar cond_;
|
||||||
|
Statistics* stats_;
|
||||||
|
Env* env_;
|
||||||
|
int stats_code_;
|
||||||
|
};
|
||||||
|
|
||||||
|
} // namespace rocksdb
|
@ -11,6 +11,7 @@
|
|||||||
#include "util/thread_status_updater.h"
|
#include "util/thread_status_updater.h"
|
||||||
|
|
||||||
namespace rocksdb {
|
namespace rocksdb {
|
||||||
|
class ColumnFamilyData;
|
||||||
|
|
||||||
|
|
||||||
// The static utility class for updating thread-local status.
|
// The static utility class for updating thread-local status.
|
||||||
|
Loading…
Reference in New Issue
Block a user