You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/01/31 20:44:03 UTC

[1/3] kudu git commit: consensus_peers-test: Call FsManager::Open() in SetUp()

Repository: kudu
Updated Branches:
  refs/heads/master defd6e831 -> f994ee96c


consensus_peers-test: Call FsManager::Open() in SetUp()

This test previously neglected to call FsManager::Open() after
FsManager::CreateInitialFileSystemLayout(), resulting in an invalid
state for the FsManager in this test.

Change-Id: I077fc2c95257c48a24e1e8fe87516293881ce5c5
Reviewed-on: http://gerrit.cloudera.org:8080/5831
Tested-by: Mike Percy <mp...@apache.org>
Reviewed-by: Adar Dembo <ad...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/99fcc31e
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/99fcc31e
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/99fcc31e

Branch: refs/heads/master
Commit: 99fcc31edd9e61b87a12c852c120179e9e771805
Parents: defd6e8
Author: Mike Percy <mp...@apache.org>
Authored: Mon Jan 30 18:51:44 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Jan 31 20:42:51 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_peers-test.cc | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/99fcc31e/src/kudu/consensus/consensus_peers-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers-test.cc b/src/kudu/consensus/consensus_peers-test.cc
index 40ba0d2..8f23cbe 100644
--- a/src/kudu/consensus/consensus_peers-test.cc
+++ b/src/kudu/consensus/consensus_peers-test.cc
@@ -57,8 +57,9 @@ class ConsensusPeersTest : public KuduTest {
   virtual void SetUp() OVERRIDE {
     KuduTest::SetUp();
     fs_manager_.reset(new FsManager(env_, GetTestPath("fs_root")));
-    CHECK_OK(fs_manager_->CreateInitialFileSystemLayout());
-    CHECK_OK(Log::Open(options_,
+    ASSERT_OK(fs_manager_->CreateInitialFileSystemLayout());
+    ASSERT_OK(fs_manager_->Open());
+    ASSERT_OK(Log::Open(options_,
                        fs_manager_.get(),
                        kTabletId,
                        schema_,
@@ -78,7 +79,7 @@ class ConsensusPeersTest : public KuduTest {
   }
 
   virtual void TearDown() OVERRIDE {
-    CHECK_OK(log_->WaitUntilAllFlushed());
+    ASSERT_OK(log_->WaitUntilAllFlushed());
   }
 
   DelayablePeerProxy<NoOpTestPeerProxy>* NewRemotePeer(


[2/3] kudu git commit: wal: Include standard prefix on glog lines

Posted by mp...@apache.org.
wal: Include standard prefix on glog lines

Change-Id: I724e85001beb68adbb806212ea3cb63292d0de56
Reviewed-on: http://gerrit.cloudera.org:8080/5827
Tested-by: Mike Percy <mp...@apache.org>
Reviewed-by: Mike Percy <mp...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/19e88d57
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/19e88d57
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/19e88d57

Branch: refs/heads/master
Commit: 19e88d57d5c686a88fbc063f96e2965bc74fe703
Parents: 99fcc31
Author: Mike Percy <mp...@apache.org>
Authored: Sun Jan 29 00:33:06 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Jan 31 20:43:28 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log.cc | 79 ++++++++++++++++++++++++------------------
 src/kudu/consensus/log.h  |  2 ++
 2 files changed, 47 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/19e88d57/src/kudu/consensus/log.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 2eb1c7d..716baa0 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -164,6 +164,8 @@ class Log::AppendThread {
  private:
   void RunThread();
 
+  string LogPrefix() const;
+
   Log* const log_;
 
   // Lock to protect access to thread_ during shutdown.
@@ -178,7 +180,7 @@ Log::AppendThread::AppendThread(Log *log)
 
 Status Log::AppendThread::Init() {
   DCHECK(!thread_) << "Already initialized";
-  VLOG(1) << "Starting log append thread for tablet " << log_->tablet_id();
+  VLOG_WITH_PREFIX(1) << "Starting log append thread";
   RETURN_NOT_OK(kudu::Thread::Create("log", "appender",
       &AppendThread::RunThread, this, &thread_));
   return Status::OK();
@@ -212,9 +214,9 @@ void Log::AppendThread::RunThread() {
       TRACE_EVENT_FLOW_END0("log", "Batch", entry_batch);
       Status s = log_->DoAppend(entry_batch);
       if (PREDICT_FALSE(!s.ok())) {
-        LOG(ERROR) << "Error appending to the log: " << s.ToString();
+        LOG_WITH_PREFIX(ERROR) << "Error appending to the log: " << s.ToString();
         entry_batch->set_failed_to_append();
-        // TODO If a single transaction fails to append, should we
+        // TODO(af): If a single transaction fails to append, should we
         // abort all subsequent transactions in this batch or allow
         // them to be appended? What about transactions in future
         // batches?
@@ -232,7 +234,7 @@ void Log::AppendThread::RunThread() {
       s = log_->Sync();
     }
     if (PREDICT_FALSE(!s.ok())) {
-      LOG(ERROR) << "Error syncing log" << s.ToString();
+      LOG_WITH_PREFIX(ERROR) << "Error syncing log: " << s.ToString();
       for (LogEntryBatch* entry_batch : entry_batches) {
         if (!entry_batch->callback().is_null()) {
           entry_batch->callback().Run(s);
@@ -240,7 +242,7 @@ void Log::AppendThread::RunThread() {
       }
     } else {
       TRACE_EVENT0("log", "Callbacks");
-      VLOG(2) << "Synchronized " << entry_batches.size() << " entry batches";
+      VLOG_WITH_PREFIX(2) << "Synchronized " << entry_batches.size() << " entry batches";
       SCOPED_WATCH_STACK(100);
       for (LogEntryBatch* entry_batch : entry_batches) {
         if (PREDICT_TRUE(!entry_batch->failed_to_append()
@@ -255,20 +257,24 @@ void Log::AppendThread::RunThread() {
       entry_batches.clear();
     }
   }
-  VLOG(1) << "Exiting AppendThread for tablet " << log_->tablet_id();
+  VLOG_WITH_PREFIX(1) << "Exiting AppendThread";
 }
 
 void Log::AppendThread::Shutdown() {
   log_->entry_queue()->Shutdown();
   std::lock_guard<std::mutex> lock_guard(lock_);
   if (thread_) {
-    VLOG(1) << "Shutting down log append thread for tablet " << log_->tablet_id();
+    VLOG_WITH_PREFIX(1) << "Shutting down log append thread";
     CHECK_OK(ThreadJoiner(thread_.get()).Join());
-    VLOG(1) << "Log append thread for tablet " << log_->tablet_id() << " is shut down";
+    VLOG_WITH_PREFIX(1) << "Log append thread is shut down";
     thread_.reset();
   }
 }
 
+string Log::AppendThread::LogPrefix() const {
+  return log_->LogPrefix();
+}
+
 // This task is submitted to allocation_pool_ in order to
 // asynchronously pre-allocate new log segments.
 void Log::SegmentAllocationTask() {
@@ -355,8 +361,8 @@ Status Log::Init() {
   // We must pick up where the previous WAL left off in terms of
   // sequence numbers.
   if (reader_->num_segments() != 0) {
-    VLOG(1) << "Using existing " << reader_->num_segments()
-            << " segments from path: " << fs_manager_->GetWalsRootDir();
+    VLOG_WITH_PREFIX(1) << "Using existing " << reader_->num_segments()
+                        << " segments from path: " << fs_manager_->GetWalsRootDir();
 
     vector<scoped_refptr<ReadableLogSegment> > segments;
     RETURN_NOT_OK(reader_->GetSegmentsSnapshot(&segments));
@@ -364,9 +370,10 @@ Status Log::Init() {
   }
 
   if (force_sync_all_) {
-    KLOG_FIRST_N(INFO, 1) << "Log is configured to fsync() on all Append() calls";
+    KLOG_FIRST_N(INFO, 1) << LogPrefix() << "Log is configured to fsync() on all Append() calls";
   } else {
-    KLOG_FIRST_N(INFO, 1) << "Log is configured to *not* fsync() on all Append() calls";
+    KLOG_FIRST_N(INFO, 1) << LogPrefix()
+                          << "Log is configured to *not* fsync() on all Append() calls";
   }
 
   // We always create a new segment when the log starts.
@@ -391,11 +398,11 @@ Status Log::AsyncAllocateSegment() {
 
 Status Log::CloseCurrentSegment() {
   if (!footer_builder_.has_min_replicate_index()) {
-    VLOG(1) << "Writing a segment without any REPLICATE message. "
-        "Segment: " << active_segment_->path();
+    VLOG_WITH_PREFIX(1) << "Writing a segment without any REPLICATE message. Segment: "
+                        << active_segment_->path();
   }
-  VLOG(2) << "Segment footer for " << active_segment_->path()
-          << ": " << SecureShortDebugString(footer_builder_);
+  VLOG_WITH_PREFIX(2) << "Segment footer for " << active_segment_->path()
+                      << ": " << SecureShortDebugString(footer_builder_);
 
   footer_builder_.set_close_timestamp_micros(GetCurrentTimeMicros());
   RETURN_NOT_OK(active_segment_->WriteFooterAndClose(footer_builder_));
@@ -416,7 +423,7 @@ Status Log::RollOver() {
 
   RETURN_NOT_OK(SwitchToAllocatedSegment());
 
-  LOG(INFO) << "Rolled over to a new segment: " << active_segment_->path();
+  LOG_WITH_PREFIX(INFO) << "Rolled over to a new log segment at " << active_segment_->path();
   return Status::OK();
 }
 
@@ -531,25 +538,25 @@ Status Log::DoAppend(LogEntryBatch* entry_batch) {
   // if the size of this entry overflows the current segment, get a new one
   if (allocation_state() == kAllocationNotStarted) {
     if ((active_segment_->Size() + entry_batch_bytes + 4) > max_segment_size_) {
-      LOG(INFO) << "Max segment size reached. Starting new segment allocation. ";
+      LOG_WITH_PREFIX(INFO) << "Max segment size reached. Starting new segment allocation";
       RETURN_NOT_OK(AsyncAllocateSegment());
       if (!options_.async_preallocate_segments) {
-        LOG_SLOW_EXECUTION(WARNING, 50, "Log roll took a long time") {
+        LOG_SLOW_EXECUTION(WARNING, 50, Substitute("$0Log roll took a long time", LogPrefix())) {
           RETURN_NOT_OK(RollOver());
         }
       }
     }
   } else if (allocation_state() == kAllocationFinished) {
-    LOG_SLOW_EXECUTION(WARNING, 50, "Log roll took a long time") {
+    LOG_SLOW_EXECUTION(WARNING, 50, Substitute("$0Log roll took a long time", LogPrefix())) {
       RETURN_NOT_OK(RollOver());
     }
   } else {
-    VLOG(1) << "Segment allocation already in progress...";
+    VLOG_WITH_PREFIX(1) << "Segment allocation already in progress...";
   }
 
   int64_t start_offset = active_segment_->written_offset();
 
-  LOG_SLOW_EXECUTION(WARNING, 50, "Append to log took a long time") {
+  LOG_SLOW_EXECUTION(WARNING, 50, Substitute("$0Append to log took a long time", LogPrefix())) {
     SCOPED_LATENCY_METRIC(metrics_, append_latency);
     SCOPED_WATCH_STACK(500);
 
@@ -624,14 +631,13 @@ Status Log::Sync() {
     int sleep_ms = r.Normal(FLAGS_log_inject_latency_ms_mean,
                             FLAGS_log_inject_latency_ms_stddev);
     if (sleep_ms > 0) {
-      LOG(INFO) << "T " << tablet_id_ << ": Injecting "
-                << sleep_ms << "ms of latency in Log::Sync()";
+      LOG_WITH_PREFIX(WARNING) << "Injecting " << sleep_ms << "ms of latency in Log::Sync()";
       SleepFor(MonoDelta::FromMilliseconds(sleep_ms));
     }
   }
 
   if (force_sync_all_ && !sync_disabled_) {
-    LOG_SLOW_EXECUTION(WARNING, 50, "Fsync log took a long time") {
+    LOG_SLOW_EXECUTION(WARNING, 50, Substitute("$0Fsync log took a long time", LogPrefix())) {
       RETURN_NOT_OK(active_segment_->Sync());
 
       if (log_hooks_) {
@@ -722,10 +728,10 @@ void Log::GetLatestEntryOpId(consensus::OpId* op_id) const {
 Status Log::GC(RetentionIndexes retention_indexes, int32_t* num_gced) {
   CHECK_GE(retention_indexes.for_durability, 0);
 
-  VLOG(1) << "Running Log GC on " << log_dir_ << ": retaining "
+  VLOG_WITH_PREFIX(1) << "Running Log GC on " << log_dir_ << ": retaining "
       "ops >= " << retention_indexes.for_durability << " for durability, "
       "ops >= " << retention_indexes.for_peers << " for peers";
-  VLOG_TIMING(1, "Log GC") {
+  VLOG_TIMING(1, Substitute("$0Log GC", LogPrefix())) {
     SegmentSequence segments_to_delete;
 
     {
@@ -735,7 +741,7 @@ Status Log::GC(RetentionIndexes retention_indexes, int32_t* num_gced) {
       RETURN_NOT_OK(GetSegmentsToGCUnlocked(retention_indexes, &segments_to_delete));
 
       if (segments_to_delete.empty()) {
-        VLOG(1) << "No segments to delete.";
+        VLOG_WITH_PREFIX(1) << "No segments to delete.";
         *num_gced = 0;
         return Status::OK();
       }
@@ -755,7 +761,7 @@ Status Log::GC(RetentionIndexes retention_indexes, int32_t* num_gced) {
                              segment->footer().min_replicate_index(),
                              segment->footer().max_replicate_index());
       }
-      LOG(INFO) << "Deleting log segment in path: " << segment->path() << ops_str;
+      LOG_WITH_PREFIX(INFO) << "Deleting log segment in path: " << segment->path() << ops_str;
       RETURN_NOT_OK(fs_manager_->env()->DeleteFile(segment->path()));
       (*num_gced)++;
     }
@@ -829,7 +835,7 @@ Status Log::Close() {
       RETURN_NOT_OK(CloseCurrentSegment());
       RETURN_NOT_OK(ReplaceSegmentInReaderUnlocked());
       log_state_ = kLogClosed;
-      VLOG(1) << "Log closed";
+      VLOG_WITH_PREFIX(1) << "Log closed";
 
       // Release FDs held by these objects.
       log_index_.reset();
@@ -842,7 +848,7 @@ Status Log::Close() {
       return Status::OK();
 
     case kLogClosed:
-      VLOG(1) << "Log already closed";
+      VLOG_WITH_PREFIX(1) << "Log already closed";
       return Status::OK();
 
     default:
@@ -861,7 +867,8 @@ Status Log::DeleteOnDiskData(FsManager* fs_manager, const string& tablet_id) {
   if (!env->FileExists(wal_dir)) {
     return Status::OK();
   }
-  LOG(INFO) << "T " << tablet_id << " Deleting WAL directory";
+  LOG(INFO) << Substitute("T $0 P $1: Deleting WAL directory at $2",
+                          tablet_id, fs_manager->uuid(), wal_dir);
   RETURN_NOT_OK_PREPEND(env->DeleteRecursively(wal_dir),
                         "Unable to recursively delete WAL dir for tablet " + tablet_id);
   return Status::OK();
@@ -989,17 +996,21 @@ Status Log::CreatePlaceholderSegment(const WritableFileOptions& opts,
                                      shared_ptr<WritableFile>* out) {
   string tmp_suffix = strings::Substitute("$0$1", kTmpInfix, ".newsegmentXXXXXX");
   string path_tmpl = JoinPathSegments(log_dir_, tmp_suffix);
-  VLOG(2) << "Creating temp. file for place holder segment, template: " << path_tmpl;
+  VLOG_WITH_PREFIX(2) << "Creating temp. file for place holder segment, template: " << path_tmpl;
   unique_ptr<WritableFile> segment_file;
   RETURN_NOT_OK(fs_manager_->env()->NewTempWritableFile(opts,
                                                         path_tmpl,
                                                         result_path,
                                                         &segment_file));
-  VLOG(1) << "Created next WAL segment, placeholder path: " << *result_path;
+  VLOG_WITH_PREFIX(1) << "Created next WAL segment, placeholder path: " << *result_path;
   out->reset(segment_file.release());
   return Status::OK();
 }
 
+std::string Log::LogPrefix() const {
+  return Substitute("T $0 P $1: ", tablet_id_, fs_manager_->uuid());
+}
+
 Log::~Log() {
   WARN_NOT_OK(Close(), "Error closing log");
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/19e88d57/src/kudu/consensus/log.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.h b/src/kudu/consensus/log.h
index 8a5767f..5caa347 100644
--- a/src/kudu/consensus/log.h
+++ b/src/kudu/consensus/log.h
@@ -333,6 +333,8 @@ class Log : public RefCountedThreadSafe<Log> {
     return allocation_state_;
   }
 
+  std::string LogPrefix() const;
+
   LogOptions options_;
   FsManager *fs_manager_;
   std::string log_dir_;


[3/3] kudu git commit: Control mutex stack walking in DEBUG mode with a gflag

Posted by mp...@apache.org.
Control mutex stack walking in DEBUG mode with a gflag

This patch disables the Mutex owner stack trace collection on DEBUG
builds by default, only enabling it when a certain gflag is set.

In DEBUG mode, our Mutex implementation collects a stack trace of the
owning thread each time the Mutex is acquired. It does this by calling
google::GetStackTrace() from glog, which in the context of the Kudu
build environment calls into libunwind to collect the stack trace.

At the time of writing, google::GetStackTrace() only allows access by
one thread at a time. If more than one thread attempts to invoke this
function simultaneously, there is a CAS that determines exclusivity. The
"loser" of this contest gets a short-circuit return along with an empty
stack trace, indicating a failure to collect the stack trace.

NB: I have filed a glog issue about that behavior upstream. For more
information, see https://github.com/google/glog/issues/160

This situation becomes a problem when there are one or more Mutexes
constantly being acquired. When that happens, there is always a thread
collecting a stack trace, and so the probability of being able to
successfully collect a stack trace at any given moment is greatly
reduced.

One important caller of google::GetStackTrace() is the glog failure
function and SIGABRT signal handler that is called when a CHECK() fails
or a LOG(FATAL) call is invoked. I have observed that this crash handler
will often print an empty stack trace in DEBUG mode. Investigating this
issue led me to discover that we had a thread (our AsyncLogger thread)
constantly acquiring a Mutex and racing on the above-mentioned CAS check
inside google::GetStackTrace(). Depriving our DEBUG builds of stack
traces on LOG(FATAL) or CHECK failures, especially on Jenkins runs, is
counterproductive. One simple solution to this problem is to disable
this behavior by default.

Change-Id: Ie4593cf7173867ce2f6151e03df0be94f97d95d2
Reviewed-on: http://gerrit.cloudera.org:8080/5741
Tested-by: Mike Percy <mp...@apache.org>
Reviewed-by: Adar Dembo <ad...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/f994ee96
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/f994ee96
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/f994ee96

Branch: refs/heads/master
Commit: f994ee96c73fd16352d046e2ee085688c5840c83
Parents: 19e88d5
Author: Mike Percy <mp...@apache.org>
Authored: Thu Jan 12 19:52:45 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Jan 31 20:43:49 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/mutex.cc | 39 ++++++++++++++++++++++++++++++++-------
 src/kudu/util/mutex.h  |  6 +++++-
 2 files changed, 37 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f994ee96/src/kudu/util/mutex.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/mutex.cc b/src/kudu/util/mutex.cc
index 842a354..bd9539b 100644
--- a/src/kudu/util/mutex.cc
+++ b/src/kudu/util/mutex.cc
@@ -19,13 +19,27 @@
 
 #include "kudu/util/mutex.h"
 
+#include <gflags/gflags.h>
 #include <glog/logging.h>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/env.h"
+#include "kudu/util/flag_tags.h"
 #include "kudu/util/trace.h"
 
+using std::string;
+using strings::Substitute;
+using strings::SubstituteAndAppend;
+
+#ifndef NDEBUG
+DEFINE_bool(debug_mutex_collect_stacktrace, false,
+            "Whether to collect a stacktrace on Mutex contention in a DEBUG build");
+TAG_FLAG(debug_mutex_collect_stacktrace, advanced);
+TAG_FLAG(debug_mutex_collect_stacktrace, hidden);
+#endif
+
 namespace kudu {
 
 Mutex::Mutex()
@@ -59,9 +73,7 @@ Mutex::~Mutex() {
 bool Mutex::TryAcquire() {
   int rv = pthread_mutex_trylock(&native_handle_);
 #ifndef NDEBUG
-  DCHECK(rv == 0 || rv == EBUSY) << ". " << strerror(rv)
-      << ". Owner tid: " << owning_tid_ << "; Self tid: " << Env::Default()->gettid()
-      << "; Owner stack: " << std::endl << stack_trace_->Symbolize();;
+  DCHECK(rv == 0 || rv == EBUSY) << ". " << strerror(rv) << ". " << GetOwnerThreadInfo();
   if (rv == 0) {
     CheckUnheldAndMark();
   }
@@ -86,8 +98,7 @@ void Mutex::Acquire() {
   int rv = pthread_mutex_lock(&native_handle_);
   DCHECK_EQ(0, rv) << ". " << strerror(rv)
 #ifndef NDEBUG
-      << ". Owner tid: " << owning_tid_ << "; Self tid: " << Env::Default()->gettid()
-      << "; Owner stack: " << std::endl << stack_trace_->Symbolize()
+                   << ". " << GetOwnerThreadInfo()
 #endif
   ; // NOLINT(whitespace/semicolon)
   MicrosecondsInt64 end_time = GetMonoTimeMicros();
@@ -118,13 +129,27 @@ void Mutex::AssertAcquired() const {
 void Mutex::CheckHeldAndUnmark() {
   AssertAcquired();
   owning_tid_ = 0;
-  stack_trace_->Reset();
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    stack_trace_->Reset();
+  }
 }
 
 void Mutex::CheckUnheldAndMark() {
   DCHECK_EQ(0, owning_tid_);
   owning_tid_ = Env::Default()->gettid();
-  stack_trace_->Collect();
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    stack_trace_->Collect();
+  }
+}
+
+string Mutex::GetOwnerThreadInfo() const {
+  string str = Substitute("Owner tid: $0; Self tid: $1; ", owning_tid_, Env::Default()->gettid());
+  if (FLAGS_debug_mutex_collect_stacktrace) {
+    SubstituteAndAppend(&str, "Owner stack:\n$0", stack_trace_->Symbolize());
+  } else {
+    str += "To collect the owner stack trace, enable the flag --debug_mutex_collect_stacktrace";
+  }
+  return str;
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/kudu/blob/f994ee96/src/kudu/util/mutex.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/mutex.h b/src/kudu/util/mutex.h
index fb6c53c..9277ac0 100644
--- a/src/kudu/util/mutex.h
+++ b/src/kudu/util/mutex.h
@@ -18,9 +18,12 @@
 #define KUDU_UTIL_MUTEX_H
 
 #include <pthread.h>
-#include <glog/logging.h>
 #include <sys/types.h>
 
+#include <string>
+
+#include <glog/logging.h>
+
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 
@@ -62,6 +65,7 @@ class Mutex {
   // Members and routines taking care of locks assertions.
   void CheckHeldAndUnmark();
   void CheckUnheldAndMark();
+  std::string GetOwnerThreadInfo() const;
 
   // All private data is implicitly protected by native_handle_.
   // Be VERY careful to only access members under that lock.