You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2018/02/21 23:55:30 UTC

[2/2] kudu git commit: KUDU-2291 (part 6) stacks: use libunwind directly for stack tracing

KUDU-2291 (part 6) stacks: use libunwind directly for stack tracing

Previously we used glog's wrapper around libunwind for stack tracing.
However that has a deficiency that it assumes that, process wide, only
one thread can be inside libunwind at a time[1]

It appears that this is left over from some very old versions of
libunwind, or was already unnecessarily conservative. libunwind is meant
to be thread safe, and we have tests that will trigger if it is not.

This just extracts the function body of the glog function we were using
and does the same work manually.

Without this fix, the "collect from all the threads at the same time"
code path resulted in most of the threads collecting an empty trace
since they tried to call libunwind at the same time.

[1] https://github.com/google/glog/issues/298
Change-Id: I3a53e55d7c4e7ee50bcac5b1e81267df56383634
Reviewed-on: http://gerrit.cloudera.org:8080/9319
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 0ee40b6618df235fa7d31eeba3caede93b2a3852
Parents: c20b84b
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 13 18:37:29 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 21 23:54:55 2018 +0000

----------------------------------------------------------------------
 build-support/iwyu/iwyu.sh                | 16 +++-----
 build-support/iwyu/mappings/libunwind.imp | 23 ++++++++++++
 src/kudu/util/debug-util-test.cc          |  2 +-
 src/kudu/util/debug-util.cc               | 51 +++++++++++++++++++++++++-
 src/kudu/util/debug-util.h                |  6 +--
 5 files changed, 81 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0ee40b66/build-support/iwyu/iwyu.sh
----------------------------------------------------------------------
diff --git a/build-support/iwyu/iwyu.sh b/build-support/iwyu/iwyu.sh
index 6ee6287..417cdf3 100755
--- a/build-support/iwyu/iwyu.sh
+++ b/build-support/iwyu/iwyu.sh
@@ -44,18 +44,12 @@ for p in $file_list_tmp; do
   IWYU_FILE_LIST="$IWYU_FILE_LIST $ROOT/$p"
 done
 
+IWYU_ARGS="--max_line_length=256"
+
 IWYU_MAPPINGS_PATH="$ROOT/build-support/iwyu/mappings"
-IWYU_ARGS="\
-    --max_line_length=256 \
-    --mapping_file=$IWYU_MAPPINGS_PATH/boost-all.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/boost-all-private.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/boost-extra.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/gflags.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/glog.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/gtest.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/kudu.imp \
-    --mapping_file=$IWYU_MAPPINGS_PATH/libstdcpp.imp\
-    --mapping_file=$IWYU_MAPPINGS_PATH/system-linux.imp"
+for path in $IWYU_MAPPINGS_PATH/*.imp ; do
+    IWYU_ARGS="$IWYU_ARGS --mapping_file=$path"
+done
 
 if ! PATH="$PATH:$PWD/../../thirdparty/clang-toolchain/bin" \
     python $ROOT/build-support/iwyu/iwyu_tool.py -p . $IWYU_FILE_LIST -- \

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ee40b66/build-support/iwyu/mappings/libunwind.imp
----------------------------------------------------------------------
diff --git a/build-support/iwyu/mappings/libunwind.imp b/build-support/iwyu/mappings/libunwind.imp
new file mode 100644
index 0000000..590e6c2
--- /dev/null
+++ b/build-support/iwyu/mappings/libunwind.imp
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+#
+# This file contains mappings for libunwind
+[
+  { include: ["<libunwind-x86_64.h>", "private", "<libunwind.h>", "public"] },
+  { include: ["\"libunwind-x86_64.h\"", "private", "<libunwind.h>", "public"] }
+]

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ee40b66/src/kudu/util/debug-util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/debug-util-test.cc b/src/kudu/util/debug-util-test.cc
index abd7c4f..c51bf0c 100644
--- a/src/kudu/util/debug-util-test.cc
+++ b/src/kudu/util/debug-util-test.cc
@@ -51,7 +51,7 @@ class DebugUtilTest : public KuduTest {
 
 TEST_F(DebugUtilTest, TestStackTrace) {
   StackTrace t;
-  t.Collect(1);
+  t.Collect(0);
   string trace = t.Symbolize();
   ASSERT_STR_CONTAINS(trace, "kudu::DebugUtilTest_TestStackTrace_Test::TestBody");
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ee40b66/src/kudu/util/debug-util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/debug-util.cc b/src/kudu/util/debug-util.cc
index 0e1130b..8fd1575 100644
--- a/src/kudu/util/debug-util.cc
+++ b/src/kudu/util/debug-util.cc
@@ -38,6 +38,11 @@
 #include <string>
 
 #include <glog/logging.h>
+#include <glog/raw_logging.h>
+#ifdef __linux__
+#define UNW_LOCAL_ONLY
+#include <libunwind.h>
+#endif
 
 #include "kudu/gutil/dynamic_annotations.h"
 #include "kudu/gutil/hash/city.h"
@@ -48,10 +53,13 @@
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug/leak_annotations.h"
+#ifndef __linux__
 #include "kudu/util/debug/sanitizer_scopes.h"
+#endif
 #include "kudu/util/debug/unwind_safeness.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/thread.h"
 
 using std::string;
@@ -214,6 +222,12 @@ namespace {
 // Signal handler for our stack trace signal.
 // We expect that the signal is only sent from DumpThreadStack() -- not by a user.
 void HandleStackTraceSignal(int /*signum*/, siginfo_t* info, void* /*ucontext*/) {
+  // Signal handlers may be invoked at any point, so it's important to preserve
+  // errno.
+  int save_errno = errno;
+  SCOPED_CLEANUP({
+      errno = save_errno;
+    });
   auto* sig_data = reinterpret_cast<SignalData*>(info->si_ptr);
   DCHECK(sig_data);
   if (!sig_data) {
@@ -231,7 +245,7 @@ void HandleStackTraceSignal(int /*signum*/, siginfo_t* info, void* /*ucontext*/)
   }
   // Marking it as kDumpInProgress ensures that the caller thread must now wait
   // for our response, since we are writing directly into their StackTrace object.
-  sig_data->stack->Collect(/*skip_frames=*/2);
+  sig_data->stack->Collect(/*skip_frames=*/1);
   sig_data->queued_to_tid = SignalData::kNotInUse;
   sig_data->result_ready.Signal();
 }
@@ -520,10 +534,43 @@ void StackTrace::Collect(int skip_frames) {
     num_frames_ = 1;
     return;
   }
+  const int kMaxDepth = arraysize(frames_);
+
+#ifdef __linux__
+  unw_cursor_t cursor;
+  unw_context_t uc;
+  unw_getcontext(&uc);
+  RAW_CHECK(unw_init_local(&cursor, &uc) >= 0, "unw_init_local failed");
+  skip_frames++;         // Do not include the "Collect" frame
+
+  num_frames_ = 0;
+  while (num_frames_ < kMaxDepth) {
+    void *ip;
+    int ret = unw_get_reg(&cursor, UNW_REG_IP, reinterpret_cast<unw_word_t *>(&ip));
+    if (ret < 0) {
+      break;
+    }
+    if (skip_frames > 0) {
+      skip_frames--;
+    } else {
+      frames_[num_frames_++] = ip;
+    }
+    ret = unw_step(&cursor);
+    if (ret <= 0) {
+      break;
+    }
+  }
+#else
+  // On OSX, use the unwinder from glog. However, that unwinder has an issue where
+  // concurrent invocations will return no frames. See:
+  // https://github.com/google/glog/issues/298
+  // The worst result here is an empty result.
+
   // google::GetStackTrace has a data race. This is called frequently, so better
   // to ignore it with an annotation rather than use a suppression.
   debug::ScopedTSANIgnoreReadsAndWrites ignore_tsan;
-  num_frames_ = google::GetStackTrace(frames_, arraysize(frames_), skip_frames);
+  num_frames_ = google::GetStackTrace(frames_, kMaxDepth, skip_frames + 1);
+#endif
 }
 
 void StackTrace::StringifyToHex(char* buf, size_t size, int flags) const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/0ee40b66/src/kudu/util/debug-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/debug-util.h b/src/kudu/util/debug-util.h
index efe5857..637e49f 100644
--- a/src/kudu/util/debug-util.h
+++ b/src/kudu/util/debug-util.h
@@ -132,11 +132,11 @@ class StackTrace {
   }
 
   // Collect and store the current stack trace. Skips the top 'skip_frames' frames
-  // from the stack. For example, a value of '1' will skip the 'Collect()' function
-  // call itself.
+  // from the stack. For example, a value of '1' will skip whichever function
+  // called the 'Collect()' function. The 'Collect' function itself is always skipped.
   //
   // This function is async-safe.
-  void Collect(int skip_frames = 1);
+  void Collect(int skip_frames = 0);
 
 
   enum Flags {