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 2016/01/07 00:46:13 UTC

[1/8] incubator-kudu git commit: Add ASAN disabler to twitter oauth test

Repository: incubator-kudu
Updated Branches:
  refs/heads/master bf1f244fb -> 2187df374


Add ASAN disabler to twitter oauth test

Change-Id: I7f0d98fa9b1cde9a0ebef2a5a0c3c346c0445ac7
Reviewed-on: http://gerrit.cloudera.org:8080/1704
Reviewed-by: Todd Lipcon <to...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 8ff679fb6aae253a82655a2029aa4dd8581bcb74
Parents: bf1f244
Author: Dan Burkert <da...@cloudera.com>
Authored: Tue Jan 5 10:35:27 2016 -0800
Committer: Dan Burkert <da...@danburkert.com>
Committed: Tue Jan 5 22:07:16 2016 +0000

----------------------------------------------------------------------
 src/kudu/twitter-demo/oauth-test.cc | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/8ff679fb/src/kudu/twitter-demo/oauth-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/twitter-demo/oauth-test.cc b/src/kudu/twitter-demo/oauth-test.cc
index 4f3f0b1..604e5c1 100644
--- a/src/kudu/twitter-demo/oauth-test.cc
+++ b/src/kudu/twitter-demo/oauth-test.cc
@@ -18,6 +18,8 @@
 #include <gtest/gtest.h>
 #include <string>
 
+#include "kudu/util/debug/leakcheck_disabler.h"
+
 using std::string;
 
 namespace kudu {
@@ -31,6 +33,9 @@ TEST(OAuthTest, TestSignature) {
   const string kTokenKey = "nnch734d00sl2jdk";
   const string kTokenSecret = "pfkkdhi9sl3r4s00";
 
+  // Necessary to squelch a leak originating in the NSS SSL library.
+  debug::ScopedLeakCheckDisabler disabler;
+
   OAuthRequest req("GET", "http://photos.example.net/photos");
 
   req.AddPair("oauth_consumer_key", kConsumerKey);


[8/8] incubator-kudu git commit: Fix faulty hybrid clock test case

Posted by to...@apache.org.
Fix faulty hybrid clock test case

Change-Id: I42c270f67cfb33ac6649c0e266784ebf4fd7ccdc
Reviewed-on: http://gerrit.cloudera.org:8080/1719
Reviewed-by: David Ribeiro Alves <da...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 2187df374c032e737c59fa70e32bdde58babfde5
Parents: abe9448
Author: Dan Burkert <da...@cloudera.com>
Authored: Wed Jan 6 11:47:45 2016 -0800
Committer: Dan Burkert <da...@cloudera.com>
Committed: Wed Jan 6 22:46:22 2016 +0000

----------------------------------------------------------------------
 src/kudu/server/hybrid_clock-test.cc | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/2187df37/src/kudu/server/hybrid_clock-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/hybrid_clock-test.cc b/src/kudu/server/hybrid_clock-test.cc
index 9a408ba..d87ba46 100644
--- a/src/kudu/server/hybrid_clock-test.cc
+++ b/src/kudu/server/hybrid_clock-test.cc
@@ -75,9 +75,7 @@ TEST(MockHybridClockTest, TestMockedSystemClock) {
 TEST_F(HybridClockTest, TestNow_ValuesIncreaseMonotonically) {
   const Timestamp now1 = clock_->Now();
   const Timestamp now2 = clock_->Now();
-  ASSERT_GE(HybridClock::GetLogicalValue(now1), HybridClock::GetLogicalValue(now2));
-  ASSERT_GE(HybridClock::GetPhysicalValueMicros(now1),
-            HybridClock::GetPhysicalValueMicros(now1));
+  ASSERT_LT(now1.value(), now2.value());
 }
 
 // Tests the clock updates with the incoming value if it is higher.


[5/8] incubator-kudu git commit: KUDU-1277 Remove kudu.split_keys example

Posted by to...@apache.org.
KUDU-1277 Remove kudu.split_keys example

Change-Id: Ibb8930b4a434c490b570279f74ed1381ab41a445
Reviewed-on: http://gerrit.cloudera.org:8080/1652
Reviewed-by: Todd Lipcon <to...@cloudera.com>
Tested-by: Todd Lipcon <to...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4b6aec3a06b1b8c94763d6bb348787c3d493c8cd
Parents: 183e816
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Wed Dec 16 16:41:05 2015 -0800
Committer: Todd Lipcon <to...@cloudera.com>
Committed: Wed Jan 6 20:25:23 2016 +0000

----------------------------------------------------------------------
 docs/kudu_impala_integration.adoc | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/4b6aec3a/docs/kudu_impala_integration.adoc
----------------------------------------------------------------------
diff --git a/docs/kudu_impala_integration.adoc b/docs/kudu_impala_integration.adoc
index 05b6599..cd7f971 100755
--- a/docs/kudu_impala_integration.adoc
+++ b/docs/kudu_impala_integration.adoc
@@ -428,7 +428,7 @@ has no mechanism for automatically (or manually) splitting a pre-existing tablet
 Until this feature has been implemented, you must pre-split your table when you create
 it, When designing your table schema, consider primary keys that will allow you to
 pre-split your table into tablets which grow at similar rates. You can provide split
-points using the `kudu.split_keys` table property when creating a table using Impala:
+points using a `DISTRIBUTE BY` clause when creating a table using Impala:
 
 NOTE: Impala keywords, such as `group`, are enclosed by back-tick characters when
 they are not used in their keyword sense.
@@ -450,15 +450,14 @@ CREATE TABLE cust_behavior (
   rating INT,
   fulfilled_date BIGINT
 )
+DISTRIBUTE BY RANGE(_id)
+SPLIT ROWS((1439560049342), (1439566253755), (1439572458168), (1439578662581), (1439584866994), (1439591071407))
 TBLPROPERTIES(
 'storage_handler' = 'com.cloudera.kudu.hive.KuduStorageHandler',
 'kudu.table_name' = 'cust_behavior',
 'kudu.master_addresses' = 'a1216.halxg.cloudera.com:7051',
 'kudu.key_columns' = '_id',
 'kudu.num_tablet_replicas' = '3',
-'kudu.split_keys' =
-'[[1439560049342], [1439566253755], [1439572458168], [1439578662581], [1439584866994], [1439591071407]],'
-
 );
 ----
 


[2/8] incubator-kudu git commit: cmake: disable new gcc 5 ABI

Posted by to...@apache.org.
cmake: disable new gcc 5 ABI

Code generation is broken when Kudu is built with gcc 5 on a distro where the
new ABI (new std::list and std::string) is on by default, such as my
machine's Ubuntu 15.10. That's because the precompiled code is built with
clang which doesn't (yet) support abi tags, and due to its dependency on glog,
it winds up with an undefined symbol that cannot be resolved at runtime.

Until clang supports abi tags [1], we must force the use of the old ABI in
Kudu and in any library code that interacts with Kudu.

Besides codegen, the new ABI also raised issues with slice comparison in
diskrowset-test and mt-diskrowset-test. These will also need to be addressed
if the new ABI is to be used in the future.

1. https://llvm.org/bugs/show_bug.cgi?id=23529

Change-Id: If5f4e0d3cd3285a54b185f13daed7463cb85505b
Reviewed-on: http://gerrit.cloudera.org:8080/1683
Tested-by: Dan Burkert <da...@danburkert.com>
Reviewed-by: Dan Burkert <da...@danburkert.com>


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

Branch: refs/heads/master
Commit: 8b04bf9753e400575e8497457feade91d2920103
Parents: 8ff679f
Author: Adar Dembo <ad...@cloudera.com>
Authored: Tue Dec 22 13:15:27 2015 -0800
Committer: Dan Burkert <da...@danburkert.com>
Committed: Tue Jan 5 22:46:48 2016 +0000

----------------------------------------------------------------------
 CMakeLists.txt                         | 15 +++++++++++++--
 src/kudu/client/samples/CMakeLists.txt |  3 +++
 thirdparty/build-thirdparty.sh         |  8 ++++++++
 3 files changed, 24 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/8b04bf97/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 58a4b2b..701b66b 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -56,13 +56,24 @@ endif()
 #    integers
 #  -Wno-deprecated: some of the gutil code includes old things like ext/hash_set, ignore that
 #  - pthread: enable multithreaded malloc
-#  - -D__STDC_FORMAT_MACROS: for PRI* print format macros
 #  -fno-strict-aliasing
 #     Assume programs do not follow strict aliasing rules.
 #     GCC cannot always verify whether strict aliasing rules are indeed followed due to
 #     fundamental limitations in escape analysis, which can result in subtle bad code generation.
 #     This has a small perf hit but worth it to avoid hard to debug crashes.
-set(CXX_COMMON_FLAGS "-fno-strict-aliasing -msse4.2 -Wall -Wno-sign-compare -Wno-deprecated -pthread -D__STDC_FORMAT_MACROS")
+set(CXX_COMMON_FLAGS "-fno-strict-aliasing -msse4.2 -Wall -Wno-sign-compare -Wno-deprecated -pthread")
+
+# We want access to the PRI* print format macros.
+add_definitions(-D__STDC_FORMAT_MACROS)
+
+# Explicitly disable the new gcc5 ABI. Until clang supports abi tags [1], Kudu's
+# generated code (which always uses clang) must be built against the old ABI.
+# There's no recourse for using both ABIs in the same process; gcc's advice [2]
+# is to build everything against the old ABI.
+#
+# 1. https://llvm.org/bugs/show_bug.cgi?id=23529
+# 2. https://gcc.gnu.org/onlinedocs/libstdc++/manual/using_dual_abi.html
+add_definitions(-D_GLIBCXX_USE_CXX11_ABI=0)
 
 # We want short macros from util/status.h.
 add_definitions(-DKUDU_HEADERS_USE_SHORT_STATUS_MACROS=1)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/8b04bf97/src/kudu/client/samples/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/client/samples/CMakeLists.txt b/src/kudu/client/samples/CMakeLists.txt
index cb549f6..d6f9c54 100644
--- a/src/kudu/client/samples/CMakeLists.txt
+++ b/src/kudu/client/samples/CMakeLists.txt
@@ -25,5 +25,8 @@ if ("${CMAKE_CXX_COMPILER_ID}" STREQUAL "Clang")
   set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libstdc++")
 endif()
 
+# The Kudu client library always uses the old gcc ABI.
+add_definitions(-D_GLIBCXX_USE_CXX11_ABI=0)
+
 add_executable(sample sample.cc)
 target_link_libraries(sample kudu_client)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/8b04bf97/thirdparty/build-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index 25debd4..622b55b 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -27,6 +27,14 @@ DEBUG_CFLAGS="-g -fno-omit-frame-pointer"
 EXTRA_CXXFLAGS="-O2 $DEBUG_CFLAGS $CXXFLAGS "
 if [[ "$OSTYPE" =~ ^linux ]]; then
   OS_LINUX=1
+  # Explicitly disable the new gcc5 ABI. Until clang supports abi tags [1],
+  # Kudu's generated code (which always uses clang) must be built against the
+  # old ABI. There's no recourse for using both ABIs in the same process; gcc's
+  # advice [2] is to build everything against the old ABI.
+  #
+  # 1. https://llvm.org/bugs/show_bug.cgi?id=23529
+  # 2. https://gcc.gnu.org/onlinedocs/libstdc++/manual/using_dual_abi.html
+  EXTRA_CXXFLAGS="$EXTRA_CXXFLAGS -D_GLIBCXX_USE_CXX11_ABI=0"
   DYLIB_SUFFIX="so"
 elif [[ "$OSTYPE" == "darwin"* ]]; then
   OS_OSX=1


[3/8] incubator-kudu git commit: DeleteTableTest should use ExternalMiniClusterITestBase

Posted by to...@apache.org.
DeleteTableTest should use ExternalMiniClusterITestBase

Removing duplicate code from DeleteTableTest. This is just test cleanup.

Change-Id: Iad374c10c2ee8ce46095b827ee4d6d11eb44605b
Reviewed-on: http://gerrit.cloudera.org:8080/1713
Tested-by: Internal Jenkins
Reviewed-by: Todd Lipcon <to...@cloudera.com>


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

Branch: refs/heads/master
Commit: dca76cadeb7a0c7ba6d08608ae42ec388584c0d0
Parents: 8b04bf9
Author: Mike Percy <mp...@cloudera.com>
Authored: Tue Jan 5 14:37:17 2016 -0800
Committer: Todd Lipcon <to...@cloudera.com>
Committed: Wed Jan 6 20:17:17 2016 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/delete_table-test.cc | 67 +-------------------
 1 file changed, 2 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/dca76cad/src/kudu/integration-tests/delete_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-test.cc b/src/kudu/integration-tests/delete_table-test.cc
index ce46432..249ea71 100644
--- a/src/kudu/integration-tests/delete_table-test.cc
+++ b/src/kudu/integration-tests/delete_table-test.cc
@@ -12,34 +12,24 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include <boost/assign/list_of.hpp>
 #include <boost/foreach.hpp>
 #include <boost/optional.hpp>
 #include <glog/stl_logging.h>
 #include <gtest/gtest.h>
 #include <tr1/memory>
-#include <tr1/unordered_map>
 #include <string>
 
-#include "kudu/client/client.h"
 #include "kudu/client/client-test-util.h"
 #include "kudu/common/wire_protocol-test-util.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/integration-tests/cluster_itest_util.h"
 #include "kudu/integration-tests/cluster_verifier.h"
-#include "kudu/integration-tests/external_mini_cluster.h"
-#include "kudu/integration-tests/external_mini_cluster_fs_inspector.h"
+#include "kudu/integration-tests/external_mini_cluster-itest-base.h"
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/util/curl_util.h"
-#include "kudu/util/pstack_watcher.h"
-#include "kudu/util/test_util.h"
 
-using boost::assign::list_of;
-using kudu::client::KuduClient;
-using kudu::client::KuduClientBuilder;
 using kudu::client::KuduSchema;
 using kudu::client::KuduSchemaFromSchema;
 using kudu::client::KuduTableCreator;
@@ -58,34 +48,12 @@ using kudu::tserver::ListTabletsResponsePB;
 using kudu::tserver::TabletServerErrorPB;
 using std::numeric_limits;
 using std::string;
-using std::tr1::shared_ptr;
-using std::tr1::unordered_map;
 using std::vector;
 using strings::Substitute;
 
 namespace kudu {
 
-class DeleteTableTest : public KuduTest {
- public:
-  virtual void TearDown() OVERRIDE {
-    if (HasFatalFailure()) {
-      for (int i = 0; i < 3; i++) {
-        if (!cluster_->tablet_server(i)->IsProcessAlive()) {
-          LOG(INFO) << "Tablet server " << i << " is not running. Cannot dump its stacks.";
-          continue;
-        }
-        LOG(INFO) << "Attempting to dump stacks of TS " << i
-                  << " with UUID " << cluster_->tablet_server(i)->uuid()
-                  << " and pid " << cluster_->tablet_server(i)->pid();
-        WARN_NOT_OK(PstackWatcher::DumpPidStacks(cluster_->tablet_server(i)->pid()),
-                    "Couldn't dump stacks");
-      }
-    }
-    if (cluster_) cluster_->Shutdown();
-    KuduTest::TearDown();
-    STLDeleteValues(&ts_map_);
-  }
-
+class DeleteTableTest : public ExternalMiniClusterITestBase {
  protected:
   enum IsCMetaExpected {
     CMETA_NOT_EXPECTED = 0,
@@ -97,10 +65,6 @@ class DeleteTableTest : public KuduTest {
     SUPERBLOCK_EXPECTED = 1
   };
 
-  void StartCluster(const vector<string>& extra_tserver_flags = vector<string>(),
-                    const vector<string>& extra_master_flags = vector<string>(),
-                    int num_tablet_servers = 3);
-
   // Get the UUID of the leader of the specified tablet, as seen by the TS with
   // the given 'ts_uuid'.
   string GetLeaderUUID(const string& ts_uuid, const string& tablet_id);
@@ -141,35 +105,8 @@ class DeleteTableTest : public KuduTest {
   // bootstrap, are running.
   void DeleteTabletWithRetries(const TServerDetails* ts, const string& tablet_id,
                                TabletDataState delete_type, const MonoDelta& timeout);
-
-  gscoped_ptr<ExternalMiniCluster> cluster_;
-  gscoped_ptr<itest::ExternalMiniClusterFsInspector> inspect_;
-  shared_ptr<KuduClient> client_;
-  unordered_map<string, TServerDetails*> ts_map_;
 };
 
-void DeleteTableTest::StartCluster(const vector<string>& extra_tserver_flags,
-                                   const vector<string>& extra_master_flags,
-                                   int num_tablet_servers) {
-  ExternalMiniClusterOptions opts;
-  opts.num_tablet_servers = num_tablet_servers;
-  opts.extra_tserver_flags = extra_tserver_flags;
-  opts.extra_master_flags = extra_master_flags;
-
-  // Disable fsyncing, since these tests do a lot of data writing and transfer,
-  // and the fsyncs (a) make them slower, and (b) make them flaky since the syncs
-  // can take a long time on loaded build slaves.
-  opts.extra_tserver_flags.push_back("--never_fsync");
-  cluster_.reset(new ExternalMiniCluster(opts));
-  ASSERT_OK(cluster_->Start());
-  inspect_.reset(new itest::ExternalMiniClusterFsInspector(cluster_.get()));
-  ASSERT_OK(itest::CreateTabletServerMap(cluster_->master_proxy().get(),
-                                          cluster_->messenger(),
-                                          &ts_map_));
-  KuduClientBuilder builder;
-  ASSERT_OK(cluster_->CreateClient(builder, &client_));
-}
-
 string DeleteTableTest::GetLeaderUUID(const string& ts_uuid, const string& tablet_id) {
   ConsensusStatePB cstate;
   CHECK_OK(itest::GetConsensusState(ts_map_[ts_uuid], tablet_id, CONSENSUS_CONFIG_COMMITTED,


[4/8] incubator-kudu git commit: Factor out function to get output from forked subprocess

Posted by to...@apache.org.
Factor out function to get output from forked subprocess

Similar to backticks in Perl or the shell.

Change-Id: Idba22393d86fc93e340a459ffe2806f8caf74596
Reviewed-on: http://gerrit.cloudera.org:8080/1714
Tested-by: Internal Jenkins
Reviewed-by: Todd Lipcon <to...@cloudera.com>


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

Branch: refs/heads/master
Commit: 183e81612317c521ab4e7a51080e5c15c192b036
Parents: dca76ca
Author: Mike Percy <mp...@cloudera.com>
Authored: Tue Jan 5 17:15:59 2016 -0800
Committer: Todd Lipcon <to...@cloudera.com>
Committed: Wed Jan 6 20:19:10 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/net/net_util.cc | 46 +++++---------------------------------
 src/kudu/util/subprocess.cc   | 37 ++++++++++++++++++++++++++++++
 src/kudu/util/subprocess.h    |  5 +++++
 3 files changed, 48 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/183e8161/src/kudu/util/net/net_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/net_util.cc b/src/kudu/util/net/net_util.cc
index 5f7b75c..56074d2 100644
--- a/src/kudu/util/net/net_util.cc
+++ b/src/kudu/util/net/net_util.cc
@@ -265,47 +265,13 @@ void TryRunLsof(const Sockaddr& addr, vector<string>* log) {
                            << "Trying to use lsof to find any processes listening "
                            << "on the same port:";
   LOG_STRING(INFO, log) << "$ " << cmd;
-  Subprocess p("/bin/bash",
-               boost::assign::list_of<string>("bash")("-c")(cmd));
-  p.ShareParentStdout(false);
-  Status s = p.Start();
-  if (!s.ok()) {
-    LOG_STRING(WARNING, log) << "Unable to fork bash: " << s.ToString();
-    return;
+  vector<string> argv = boost::assign::list_of<string>("/bin/bash")("-c")(cmd);
+  string results;
+  Status s = Subprocess::Call(argv, &results);
+  if (PREDICT_FALSE(!s.ok())) {
+    LOG_STRING(WARNING, log) << s.ToString();
   }
-
-  close(p.ReleaseChildStdinFd());
-
-  faststring results;
-  char buf[1024];
-  while (true) {
-    ssize_t n = read(p.from_child_stdout_fd(), buf, arraysize(buf));
-    if (n == 0) {
-      // EOF
-      break;
-    }
-    if (n < 0) {
-      if (errno == EINTR) continue;
-      LOG_STRING(WARNING, log) << "IO error reading from bash: " <<
-        ErrnoToString(errno);
-      close(p.ReleaseChildStdoutFd());
-      break;
-    }
-
-    results.append(buf, n);
-  }
-
-  int rc;
-  s = p.Wait(&rc);
-  if (!s.ok()) {
-    LOG_STRING(WARNING, log) << "Unable to wait for lsof: " << s.ToString();
-    return;
-  }
-  if (rc != 0) {
-    LOG_STRING(WARNING, log) << "lsof failed";
-  }
-
-  LOG_STRING(WARNING, log) << results.ToString();
+  LOG_STRING(WARNING, log) << results;
 }
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/183e8161/src/kudu/util/subprocess.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess.cc b/src/kudu/util/subprocess.cc
index a9e3d8e..0e62dfc 100644
--- a/src/kudu/util/subprocess.cc
+++ b/src/kudu/util/subprocess.cc
@@ -387,6 +387,43 @@ Status Subprocess::Call(const vector<string>& argv) {
   }
 }
 
+Status Subprocess::Call(const vector<string>& argv, string* stdout_out) {
+  Subprocess p(argv[0], argv);
+  p.ShareParentStdout(false);
+  RETURN_NOT_OK_PREPEND(p.Start(), "Unable to fork " + argv[0]);
+  int err = close(p.ReleaseChildStdinFd());
+  if (PREDICT_FALSE(err != 0)) {
+    return Status::IOError("Unable to close child process stdin", ErrnoToString(errno), errno);
+  }
+
+  stdout_out->clear();
+  char buf[1024];
+  while (true) {
+    ssize_t n = read(p.from_child_stdout_fd(), buf, arraysize(buf));
+    if (n == 0) {
+      // EOF
+      break;
+    }
+    if (n < 0) {
+      if (errno == EINTR) continue;
+      return Status::IOError("IO error reading from " + argv[0], ErrnoToString(errno), errno);
+    }
+
+    stdout_out->append(buf, n);
+  }
+
+  int retcode;
+  RETURN_NOT_OK_PREPEND(p.Wait(&retcode), "Unable to wait() for " + argv[0]);
+
+  if (PREDICT_FALSE(retcode != 0)) {
+    return Status::RuntimeError(Substitute(
+        "Subprocess '$0' terminated with non-zero exit status $1",
+        argv[0],
+        retcode));
+  }
+  return Status::OK();
+}
+
 int Subprocess::CheckAndOffer(int stdfd) const {
   CHECK_EQ(state_, kRunning);
   CHECK_EQ(fd_state_[stdfd], PIPED);

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/183e8161/src/kudu/util/subprocess.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/subprocess.h b/src/kudu/util/subprocess.h
index cd614fd..7f259ec 100644
--- a/src/kudu/util/subprocess.h
+++ b/src/kudu/util/subprocess.h
@@ -97,6 +97,11 @@ class Subprocess {
   // executable as argv[0] and the arguments to the program in argv[1..n].
   static Status Call(const std::vector<std::string>& argv);
 
+  // Same as above, but collects the output from the child process stdout into
+  // 'stdout_out'.
+  static Status Call(const std::vector<std::string>& argv,
+                     std::string* stdout_out);
+
   // Return the pipe fd to the child's standard stream.
   // Stream should not be disabled or shared.
   int to_child_stdin_fd()    const { return CheckAndOffer(STDIN_FILENO); }


[7/8] incubator-kudu git commit: Add a LOG_FATAL to the function so that new enum conditions are not missed.

Posted by to...@apache.org.
Add a LOG_FATAL to the function so that new enum conditions are not missed.

Change-Id: Ib376f21343a43536db0ee7a33e3105db7eca221b
Reviewed-on: http://gerrit.cloudera.org:8080/1702
Reviewed-by: Todd Lipcon <to...@cloudera.com>
Tested-by: Todd Lipcon <to...@cloudera.com>


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

Branch: refs/heads/master
Commit: abe94483483713465c3928259d4f70348125234d
Parents: 9a1d0a3
Author: Anup Gangwar <an...@gmail.com>
Authored: Tue Jan 5 13:25:22 2016 +0530
Committer: Todd Lipcon <to...@cloudera.com>
Committed: Wed Jan 6 21:31:38 2016 +0000

----------------------------------------------------------------------
 src/kudu/tablet/mvcc.cc | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/abe94483/src/kudu/tablet/mvcc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/mvcc.cc b/src/kudu/tablet/mvcc.cc
index 47a51c6..ea3ea19 100644
--- a/src/kudu/tablet/mvcc.cc
+++ b/src/kudu/tablet/mvcc.cc
@@ -330,6 +330,7 @@ bool MvccManager::IsDoneWaitingUnlocked(const WaitingState& waiter) const {
     case NONE_APPLYING:
       return !AnyApplyingAtOrBeforeUnlocked(waiter.timestamp);
   }
+  LOG(FATAL); // unreachable
 }
 
 bool MvccManager::AreAllTransactionsCommittedUnlocked(Timestamp ts) const {


[6/8] incubator-kudu git commit: Do not add thirdparty/bin to PATH in during builds

Posted by to...@apache.org.
Do not add thirdparty/bin to PATH in during builds

We sometimes build a custom Python version in thirdparty, which can make
scripts fail when it accidently ends up on the path.

This also adds the proper flags to TSAN to use the external llvm
symbolizer in thirdparty. As a result, we no longer need
asan_symbolizer.py, and it has been removed.

Change-Id: I0fabe02686eca5e1ec3dbf901d95f36d0443dc41
Reviewed-on: http://gerrit.cloudera.org:8080/1678
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 9a1d0a3736186ee151b8a4a5355a8e57011baeda
Parents: 4b6aec3
Author: Dan Burkert <da...@cloudera.com>
Authored: Mon Dec 21 11:37:44 2015 -0800
Committer: Dan Burkert <da...@cloudera.com>
Committed: Wed Jan 6 20:48:06 2016 +0000

----------------------------------------------------------------------
 LICENSE.txt                             |   2 +-
 build-support/jenkins/build-and-test.sh |  15 +-
 build-support/run-test.sh               |  15 +-
 src/kudu/scripts/benchmarks.sh          |   6 +-
 src/kudu/scripts/tpch.sh                |   6 +-
 thirdparty/asan_symbolize.py            | 360 ---------------------------
 6 files changed, 21 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index d864f10..347de4f 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -1091,7 +1091,7 @@ thirdparty/zlib-*/
   jloup@gzip.org          madler@alumni.caltech.edu
 
 --------------------------------------------------------------------------------
-thirdparty/asan_symbolize.py
+thirdparty/llvm-*
 
   ==============================================================================
   LLVM Release License

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/build-support/jenkins/build-and-test.sh
----------------------------------------------------------------------
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 5b1e50d..a64b0dc 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -134,8 +134,7 @@ fi
 thirdparty/build-if-necessary.sh
 
 THIRDPARTY_BIN=$(pwd)/thirdparty/installed/bin
-export PATH=$THIRDPARTY_BIN:$PATH
-export PPROF_PATH=$(pwd)/thirdparty/installed/bin/pprof
+export PPROF_PATH=$THIRDPARTY_BIN/pprof
 
 CLANG=$(pwd)/thirdparty/clang-toolchain/bin/clang
 
@@ -145,12 +144,12 @@ CLANG=$(pwd)/thirdparty/clang-toolchain/bin/clang
 # library (which the bindings depend on) is missing ASAN/TSAN symbols.
 if [ "$BUILD_TYPE" = "ASAN" ]; then
   CC=$CLANG CXX=$CLANG++ \
-   cmake -DKUDU_USE_ASAN=1 -DKUDU_USE_UBSAN=1 .
+   $THIRDPARTY_BIN/cmake -DKUDU_USE_ASAN=1 -DKUDU_USE_UBSAN=1 .
   BUILD_TYPE=fastdebug
   BUILD_PYTHON=0
 elif [ "$BUILD_TYPE" = "TSAN" ]; then
   CC=$CLANG CXX=$CLANG++ \
-   cmake -DKUDU_USE_TSAN=1 .
+   $THIRDPARTY_BIN/cmake -DKUDU_USE_TSAN=1 .
   BUILD_TYPE=fastdebug
   EXTRA_TEST_FLAGS="$EXTRA_TEST_FLAGS -LE no_tsan"
   BUILD_PYTHON=0
@@ -162,7 +161,7 @@ elif [ "$BUILD_TYPE" = "LEAKCHECK" ]; then
 elif [ "$BUILD_TYPE" = "COVERAGE" ]; then
   DO_COVERAGE=1
   BUILD_TYPE=debug
-  cmake -DKUDU_GENERATE_COVERAGE=1 .
+  $THIRDPARTY_BIN/cmake -DKUDU_GENERATE_COVERAGE=1 .
   # Reset coverage info from previous runs
   find src -name \*.gcda -o -name \*.gcno -exec rm {} \;
 elif [ "$BUILD_TYPE" = "LINT" ]; then
@@ -171,7 +170,7 @@ elif [ "$BUILD_TYPE" = "LINT" ]; then
   mkdir -p Testing/Temporary
   mkdir -p $TEST_LOGDIR
 
-  cmake .
+  $THIRDPARTY_BIN/cmake .
   make lint | tee $TEST_LOGDIR/lint.log
   exit $?
 fi
@@ -199,7 +198,7 @@ if [ "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
   fi
 fi
 
-cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
+$THIRDPARTY_BIN/cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
 
 # our tests leave lots of data lying around, clean up before we run
 make clean
@@ -237,7 +236,7 @@ fi
 EXIT_STATUS=0
 
 # Run the C++ unit tests.
-ctest -j$NUM_PROCS $EXTRA_TEST_FLAGS || EXIT_STATUS=$?
+$THIRDPARTY_BIN/ctest -j$NUM_PROCS $EXTRA_TEST_FLAGS || EXIT_STATUS=$?
 
 if [ $EXIT_STATUS != 0 ]; then
   # Tests that crash do not generate JUnit report XML files.

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/build-support/run-test.sh
----------------------------------------------------------------------
diff --git a/build-support/run-test.sh b/build-support/run-test.sh
index 805becf..62ffd1c 100755
--- a/build-support/run-test.sh
+++ b/build-support/run-test.sh
@@ -83,6 +83,12 @@ else
   pipe_cmd=cat
 fi
 
+# Suppressions require symbolization. We'll default to using the symbolizer in
+# thirdparty.
+if [ -z "$ASAN_SYMBOLIZER_PATH" ]; then
+  export ASAN_SYMBOLIZER_PATH=$ROOT/thirdparty/clang-toolchain/bin/llvm-symbolizer
+fi
+
 # Configure TSAN (ignored if this isn't a TSAN build).
 #
 # Deadlock detection (new in clang 3.5) is disabled because:
@@ -93,6 +99,7 @@ fi
 TSAN_OPTIONS="$TSAN_OPTIONS detect_deadlocks=0"
 TSAN_OPTIONS="$TSAN_OPTIONS suppressions=$ROOT/build-support/tsan-suppressions.txt"
 TSAN_OPTIONS="$TSAN_OPTIONS history_size=7"
+TSAN_OPTIONS="$TSAN_OPTIONS external_symbolizer_path=$ASAN_SYMBOLIZER_PATH"
 export TSAN_OPTIONS
 
 # Enable leak detection even under LLVM 3.4, where it was disabled by default.
@@ -104,12 +111,6 @@ export ASAN_OPTIONS
 LSAN_OPTIONS="$LSAN_OPTIONS suppressions=$ROOT/build-support/lsan-suppressions.txt"
 export LSAN_OPTIONS
 
-# Suppressions require symbolization. We'll default to using the symbolizer in
-# thirdparty.
-if [ -z "$ASAN_SYMBOLIZER_PATH" ]; then
-  export ASAN_SYMBOLIZER_PATH=$ROOT/thirdparty/installed/bin/llvm-symbolizer
-fi
-
 # Set a 15-minute timeout for tests run via 'make test'.
 # This keeps our jenkins builds from hanging in the case that there's
 # a deadlock or anything.
@@ -138,8 +139,6 @@ for ATTEMPT_NUMBER in $(seq 1 $TEST_EXECUTION_ATTEMPTS) ; do
   echo "Running $TEST_NAME, redirecting output into $LOGFILE" \
     "(attempt ${ATTEMPT_NUMBER}/$TEST_EXECUTION_ATTEMPTS)"
   $TEST_EXECUTABLE "$@" --test_timeout_after $KUDU_TEST_TIMEOUT 2>&1 \
-    | $ROOT/thirdparty/asan_symbolize.py \
-    | c++filt \
     | $ROOT/build-support/stacktrace_addr2line.pl $TEST_EXECUTABLE \
     | $pipe_cmd > $LOGFILE
   STATUS=$?

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/src/kudu/scripts/benchmarks.sh
----------------------------------------------------------------------
diff --git a/src/kudu/scripts/benchmarks.sh b/src/kudu/scripts/benchmarks.sh
index 6a2b28f..cbd9232 100755
--- a/src/kudu/scripts/benchmarks.sh
+++ b/src/kudu/scripts/benchmarks.sh
@@ -166,8 +166,8 @@ build_kudu() {
   thirdparty/build-if-necessary.sh
 
   # PATH=<thirdparty_stuff>:<toolchain_stuff>:$PATH
-  export PATH=$BASE_DIR/thirdparty/installed/bin:$PATH
-  export PPROF_PATH=$BASE_DIR/thirdparty/installed/bin/pprof
+  THIRDPARTY_BIN=$BASE_DIR/thirdparty/installed/bin
+  export PPROF_PATH=$THIRDPARTY_BIN/pprof
 
   # Build Kudu
   rm -rf CMakeCache.txt CMakeFiles
@@ -176,7 +176,7 @@ build_kudu() {
   # Workaround for gperftools issue #497
   export LD_BIND_NOW=1
 
-  cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
+  $THIRDPARTY_BIN/cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
   make clean
   # clean up before we run
   rm -Rf /tmp/kudutpch1-$UID

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/src/kudu/scripts/tpch.sh
----------------------------------------------------------------------
diff --git a/src/kudu/scripts/tpch.sh b/src/kudu/scripts/tpch.sh
index be8613c..0dd3550 100755
--- a/src/kudu/scripts/tpch.sh
+++ b/src/kudu/scripts/tpch.sh
@@ -88,8 +88,8 @@ fi
 $ROOT/thirdparty/build-if-necessary.sh
 
 # PATH=<thirdparty_stuff>:<toolchain_stuff>:$PATH
-export PATH=$(pwd)/thirdparty/installed/bin:$PATH
-export PPROF_PATH=$(pwd)/thirdparty/installed/bin/pprof
+THIRDPARTY_BIN=$(pwd)/thirdparty/installed/bin
+export PPROF_PATH=$THIRDPARTY_BIN/pprof
 
 # Build Kudu
 rm -rf CMakeCache.txt CMakeFiles
@@ -98,7 +98,7 @@ BUILD_TYPE=release
 # Workaround for gperftools issue #497
 export LD_BIND_NOW=1
 
-cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
+$THIRDPARTY_BIN/cmake . -DCMAKE_BUILD_TYPE=${BUILD_TYPE}
 make clean
 
 NUM_PROCS=$(cat /proc/cpuinfo | grep processor | wc -l)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/9a1d0a37/thirdparty/asan_symbolize.py
----------------------------------------------------------------------
diff --git a/thirdparty/asan_symbolize.py b/thirdparty/asan_symbolize.py
deleted file mode 100755
index 839a198..0000000
--- a/thirdparty/asan_symbolize.py
+++ /dev/null
@@ -1,360 +0,0 @@
-#!/usr/bin/env python
-#===- lib/asan/scripts/asan_symbolize.py -----------------------------------===#
-#
-#                     The LLVM Compiler Infrastructure
-#
-# This file is distributed under the University of Illinois Open Source
-# License. See LICENSE.TXT for details.
-#
-#===------------------------------------------------------------------------===#
-import bisect
-import os
-import re
-import subprocess
-import sys
-
-llvm_symbolizer = None
-symbolizers = {}
-filetypes = {}
-vmaddrs = {}
-DEBUG = False
-
-
-# FIXME: merge the code that calls fix_filename().
-def fix_filename(file_name):
-  for path_to_cut in sys.argv[1:]:
-    file_name = re.sub('.*' + path_to_cut, '', file_name)
-  file_name = re.sub('.*asan_[a-z_]*.cc:[0-9]*', '_asan_rtl_', file_name)
-  file_name = re.sub('.*crtstuff.c:0', '???:0', file_name)
-  return file_name
-
-
-class Symbolizer(object):
-  def __init__(self):
-    pass
-
-  def symbolize(self, addr, binary, offset):
-    """Symbolize the given address (pair of binary and offset).
-
-    Overriden in subclasses.
-    Args:
-        addr: virtual address of an instruction.
-        binary: path to executable/shared object containing this instruction.
-        offset: instruction offset in the @binary.
-    Returns:
-        list of strings (one string for each inlined frame) describing
-        the code locations for this instruction (that is, function name, file
-        name, line and column numbers).
-    """
-    return None
-
-
-class LLVMSymbolizer(Symbolizer):
-  def __init__(self, symbolizer_path):
-    super(LLVMSymbolizer, self).__init__()
-    self.symbolizer_path = symbolizer_path
-    self.pipe = self.open_llvm_symbolizer()
-
-  def open_llvm_symbolizer(self):
-    if not os.path.exists(self.symbolizer_path):
-      return None
-    cmd = [self.symbolizer_path,
-           '--use-symbol-table=true',
-           '--demangle=false',
-           '--functions=true',
-           '--inlining=true']
-    if DEBUG:
-      print ' '.join(cmd)
-    return subprocess.Popen(cmd, stdin=subprocess.PIPE,
-                            stdout=subprocess.PIPE)
-
-  def symbolize(self, addr, binary, offset):
-    """Overrides Symbolizer.symbolize."""
-    if not self.pipe:
-      return None
-    result = []
-    try:
-      symbolizer_input = '%s %s' % (binary, offset)
-      if DEBUG:
-        print symbolizer_input
-      print >> self.pipe.stdin, symbolizer_input
-      while True:
-        function_name = self.pipe.stdout.readline().rstrip()
-        if not function_name:
-          break
-        file_name = self.pipe.stdout.readline().rstrip()
-        file_name = fix_filename(file_name)
-        if (not function_name.startswith('??') and
-            not file_name.startswith('??')):
-          # Append only valid frames.
-          result.append('%s in %s %s' % (addr, function_name,
-                                         file_name))
-    except Exception:
-      result = []
-    if not result:
-      result = None
-    return result
-
-
-def LLVMSymbolizerFactory(system):
-  symbolizer_path = os.getenv('LLVM_SYMBOLIZER_PATH')
-  if not symbolizer_path:
-    # Assume llvm-symbolizer is in PATH.
-    symbolizer_path = 'llvm-symbolizer'
-  return LLVMSymbolizer(symbolizer_path)
-
-
-class Addr2LineSymbolizer(Symbolizer):
-  def __init__(self, binary):
-    super(Addr2LineSymbolizer, self).__init__()
-    self.binary = binary
-    self.pipe = self.open_addr2line()
-
-  def open_addr2line(self):
-    cmd = ['addr2line', '-f', '-e', self.binary]
-    if DEBUG:
-      print ' '.join(cmd)
-    return subprocess.Popen(cmd,
-                            stdin=subprocess.PIPE, stdout=subprocess.PIPE)
-
-  def symbolize(self, addr, binary, offset):
-    """Overrides Symbolizer.symbolize."""
-    if self.binary != binary:
-      return None
-    try:
-      print >> self.pipe.stdin, offset
-      function_name = self.pipe.stdout.readline().rstrip()
-      file_name = self.pipe.stdout.readline().rstrip()
-    except Exception:
-      function_name = ''
-      file_name = ''
-    file_name = fix_filename(file_name)
-    return ['%s in %s %s' % (addr, function_name, file_name)]
-
-
-class DarwinSymbolizer(Symbolizer):
-  def __init__(self, addr, binary):
-    super(DarwinSymbolizer, self).__init__()
-    self.binary = binary
-    # Guess which arch we're running. 10 = len('0x') + 8 hex digits.
-    if len(addr) > 10:
-      self.arch = 'x86_64'
-    else:
-      self.arch = 'i386'
-    self.vmaddr = None
-    self.pipe = None
-
-  def write_addr_to_pipe(self, offset):
-    print >> self.pipe.stdin, '0x%x' % int(offset, 16)
-
-  def open_atos(self):
-    if DEBUG:
-      print 'atos -o %s -arch %s' % (self.binary, self.arch)
-    cmdline = ['atos', '-o', self.binary, '-arch', self.arch]
-    self.pipe = subprocess.Popen(cmdline,
-                                 stdin=subprocess.PIPE,
-                                 stdout=subprocess.PIPE,
-                                 stderr=subprocess.PIPE)
-
-  def symbolize(self, addr, binary, offset):
-    """Overrides Symbolizer.symbolize."""
-    if self.binary != binary:
-      return None
-    self.open_atos()
-    self.write_addr_to_pipe(offset)
-    self.pipe.stdin.close()
-    atos_line = self.pipe.stdout.readline().rstrip()
-    # A well-formed atos response looks like this:
-    #   foo(type1, type2) (in object.name) (filename.cc:80)
-    match = re.match('^(.*) \(in (.*)\) \((.*:\d*)\)$', atos_line)
-    if DEBUG:
-      print 'atos_line: ', atos_line
-    if match:
-      function_name = match.group(1)
-      function_name = re.sub('\(.*?\)', '', function_name)
-      file_name = fix_filename(match.group(3))
-      return ['%s in %s %s' % (addr, function_name, file_name)]
-    else:
-      return ['%s in %s' % (addr, atos_line)]
-
-
-# Chain several symbolizers so that if one symbolizer fails, we fall back
-# to the next symbolizer in chain.
-class ChainSymbolizer(Symbolizer):
-  def __init__(self, symbolizer_list):
-    super(ChainSymbolizer, self).__init__()
-    self.symbolizer_list = symbolizer_list
-
-  def symbolize(self, addr, binary, offset):
-    """Overrides Symbolizer.symbolize."""
-    for symbolizer in self.symbolizer_list:
-      if symbolizer:
-        result = symbolizer.symbolize(addr, binary, offset)
-        if result:
-          return result
-    return None
-
-  def append_symbolizer(self, symbolizer):
-    self.symbolizer_list.append(symbolizer)
-
-
-def BreakpadSymbolizerFactory(binary):
-  suffix = os.getenv('BREAKPAD_SUFFIX')
-  if suffix:
-    filename = binary + suffix
-    if os.access(filename, os.F_OK):
-      return BreakpadSymbolizer(filename)
-  return None
-
-
-def SystemSymbolizerFactory(system, addr, binary):
-  if system == 'Darwin':
-    return DarwinSymbolizer(addr, binary)
-  elif system == 'Linux':
-    return Addr2LineSymbolizer(binary)
-
-
-class BreakpadSymbolizer(Symbolizer):
-  def __init__(self, filename):
-    super(BreakpadSymbolizer, self).__init__()
-    self.filename = filename
-    lines = file(filename).readlines()
-    self.files = []
-    self.symbols = {}
-    self.address_list = []
-    self.addresses = {}
-    # MODULE mac x86_64 A7001116478B33F18FF9BEDE9F615F190 t
-    fragments = lines[0].rstrip().split()
-    self.arch = fragments[2]
-    self.debug_id = fragments[3]
-    self.binary = ' '.join(fragments[4:])
-    self.parse_lines(lines[1:])
-
-  def parse_lines(self, lines):
-    cur_function_addr = ''
-    for line in lines:
-      fragments = line.split()
-      if fragments[0] == 'FILE':
-        assert int(fragments[1]) == len(self.files)
-        self.files.append(' '.join(fragments[2:]))
-      elif fragments[0] == 'PUBLIC':
-        self.symbols[int(fragments[1], 16)] = ' '.join(fragments[3:])
-      elif fragments[0] in ['CFI', 'STACK']:
-        pass
-      elif fragments[0] == 'FUNC':
-        cur_function_addr = int(fragments[1], 16)
-        if not cur_function_addr in self.symbols.keys():
-          self.symbols[cur_function_addr] = ' '.join(fragments[4:])
-      else:
-        # Line starting with an address.
-        addr = int(fragments[0], 16)
-        self.address_list.append(addr)
-        # Tuple of symbol address, size, line, file number.
-        self.addresses[addr] = (cur_function_addr,
-                                int(fragments[1], 16),
-                                int(fragments[2]),
-                                int(fragments[3]))
-    self.address_list.sort()
-
-  def get_sym_file_line(self, addr):
-    key = None
-    if addr in self.addresses.keys():
-      key = addr
-    else:
-      index = bisect.bisect_left(self.address_list, addr)
-      if index == 0:
-        return None
-      else:
-        key = self.address_list[index - 1]
-    sym_id, size, line_no, file_no = self.addresses[key]
-    symbol = self.symbols[sym_id]
-    filename = self.files[file_no]
-    if addr < key + size:
-      return symbol, filename, line_no
-    else:
-      return None
-
-  def symbolize(self, addr, binary, offset):
-    if self.binary != binary:
-      return None
-    res = self.get_sym_file_line(int(offset, 16))
-    if res:
-      function_name, file_name, line_no = res
-      result = ['%s in %s %s:%d' % (
-          addr, function_name, file_name, line_no)]
-      print result
-      return result
-    else:
-      return None
-
-
-class SymbolizationLoop(object):
-  def __init__(self, binary_name_filter=None):
-    # Used by clients who may want to supply a different binary name.
-    # E.g. in Chrome several binaries may share a single .dSYM.
-    self.binary_name_filter = binary_name_filter
-    self.system = os.uname()[0]
-    if self.system in ['Linux', 'Darwin']:
-      self.llvm_symbolizer = LLVMSymbolizerFactory(self.system)
-    else:
-      raise Exception('Unknown system')
-
-  def symbolize_address(self, addr, binary, offset):
-    # Use the chain of symbolizers:
-    # Breakpad symbolizer -> LLVM symbolizer -> addr2line/atos
-    # (fall back to next symbolizer if the previous one fails).
-    if not binary in symbolizers:
-      symbolizers[binary] = ChainSymbolizer(
-          [BreakpadSymbolizerFactory(binary), self.llvm_symbolizer])
-    result = symbolizers[binary].symbolize(addr, binary, offset)
-    if result is None:
-      # Initialize system symbolizer only if other symbolizers failed.
-      symbolizers[binary].append_symbolizer(
-          SystemSymbolizerFactory(self.system, addr, binary))
-      result = symbolizers[binary].symbolize(addr, binary, offset)
-    # The system symbolizer must produce some result.
-    assert result
-    return result
-
-  def print_symbolized_lines(self, symbolized_lines):
-    if not symbolized_lines:
-      print self.current_line
-    else:
-      for symbolized_frame in symbolized_lines:
-        print '    #' + str(self.frame_no) + ' ' + symbolized_frame.rstrip()
-        self.frame_no += 1
-
-  def process_stdin(self):
-    self.frame_no = 0
-    sys.stdout = os.fdopen(sys.stdout.fileno(), 'w', 0)
-
-    while True:
-      line = sys.stdin.readline()
-      if not line: break
-      self.current_line = line.rstrip()
-      #0 0x7f6e35cf2e45  (/blah/foo.so+0x11fe45)
-      stack_trace_line_format = (
-          '^( *#([0-9]+) *)(0x[0-9a-f]+) *\((.*)\+(0x[0-9a-f]+)\)')
-      match = re.match(stack_trace_line_format, line)
-      if not match:
-        print self.current_line
-        continue
-      if DEBUG:
-        print line
-      _, frameno_str, addr, binary, offset = match.groups()
-      if frameno_str == '0':
-        # Assume that frame #0 is the first frame of new stack trace.
-        self.frame_no = 0
-      original_binary = binary
-      if self.binary_name_filter:
-        binary = self.binary_name_filter(binary)
-      symbolized_line = self.symbolize_address(addr, binary, offset)
-      if not symbolized_line:
-        if original_binary != binary:
-          symbolized_line = self.symbolize_address(addr, binary, offset)
-      self.print_symbolized_lines(symbolized_line)
-
-
-if __name__ == '__main__':
-  loop = SymbolizationLoop()
-  loop.process_stdin()