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/22 05:25:54 UTC

[1/4] incubator-kudu git commit: Send back an error when UpdateConsensus cannot prepare a single transaction

Repository: incubator-kudu
Updated Branches:
  refs/heads/master 8260093b9 -> 3d4e0ce8a


Send back an error when UpdateConsensus cannot prepare a single transaction

If for any reason an UpdateConsensus() call fails to prepare a single op,
it will still return OK. The leader doesn't detect that no progress
was made, so it sends a new batch right away... and it's the same one.

This patch makes it so that we detect this situation and so that we
go into the error-handling path on response. It also adds a test
where we manufacture the same conditions.

Change-Id: I546fd3069af974383c23acb7406ea621e6962bb3
Reviewed-on: http://gerrit.cloudera.org:8080/1785
Reviewed-by: Mike Percy <mp...@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/ae865a53
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/ae865a53
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/ae865a53

Branch: refs/heads/master
Commit: ae865a536a50e4c12daec39c24e053fe768e9c01
Parents: 8260093
Author: Jean-Daniel Cryans <jd...@cloudera.com>
Authored: Thu Jan 14 10:44:56 2016 -0800
Committer: Jean-Daniel Cryans <jd...@gerrit.cloudera.org>
Committed: Fri Jan 22 02:33:10 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus.proto              |  3 ++
 src/kudu/consensus/consensus_peers.cc           |  6 ++-
 src/kudu/consensus/raft_consensus.cc            | 25 +++++++++++
 .../integration-tests/raft_consensus-itest.cc   | 44 ++++++++++++++++++++
 4 files changed, 76 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/ae865a53/src/kudu/consensus/consensus.proto
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus.proto b/src/kudu/consensus/consensus.proto
index 8fe879d..cb10bde 100644
--- a/src/kudu/consensus/consensus.proto
+++ b/src/kudu/consensus/consensus.proto
@@ -65,6 +65,9 @@ message ConsensusErrorPB {
     // The local replica is in the middle of servicing either another vote
     // or an update from a valid leader.
     CONSENSUS_BUSY = 8;
+
+    // The local replica was unable to prepare a single transaction.
+    CANNOT_PREPARE = 9;
   }
 
   // The error code.

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/ae865a53/src/kudu/consensus/consensus_peers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_peers.cc b/src/kudu/consensus/consensus_peers.cc
index 6879451..c27ae83 100644
--- a/src/kudu/consensus/consensus_peers.cc
+++ b/src/kudu/consensus/consensus_peers.cc
@@ -244,8 +244,10 @@ void Peer::ProcessResponse() {
 
   // Pass through errors we can respond to, like not found, since in that case
   // we will need to remotely bootstrap. TODO: Handle DELETED response once implemented.
-  if (response_.has_error() &&
-      response_.error().code() != tserver::TabletServerErrorPB::TABLET_NOT_FOUND) {
+  if ((response_.has_error() &&
+      response_.error().code() != tserver::TabletServerErrorPB::TABLET_NOT_FOUND) ||
+      (response_.status().has_error() &&
+          response_.status().error().code() == consensus::ConsensusErrorPB::CANNOT_PREPARE)) {
     // Again, let the queue know that the remote is still responsive, since we
     // will not be sending this error response through to the queue.
     queue_->NotifyPeerIsResponsiveDespiteError(peer_pb_.permanent_uuid());

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/ae865a53/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 45567d8..58909f8 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -95,6 +95,11 @@ DEFINE_bool(follower_reject_update_consensus_requests, false,
             "Warning! This is only intended for testing.");
 TAG_FLAG(follower_reject_update_consensus_requests, unsafe);
 
+DEFINE_bool(follower_fail_all_prepare, false,
+            "Whether a follower will fail preparing all transactions. "
+            "Warning! This is only intended for testing.");
+TAG_FLAG(follower_fail_all_prepare, unsafe);
+
 DECLARE_int32(memory_limit_warn_threshold_percentage);
 
 METRIC_DEFINE_counter(tablet, follower_memory_pressure_rejections,
@@ -690,6 +695,11 @@ Status RaftConsensus::StartReplicaTransactionUnlocked(const ReplicateRefPtr& msg
     return StartConsensusOnlyRoundUnlocked(msg);
   }
 
+  if (PREDICT_FALSE(FLAGS_follower_fail_all_prepare)) {
+    return Status::IllegalState("Rejected: --follower_fail_all_prepare "
+                                "is set to true.");
+  }
+
   VLOG_WITH_PREFIX_UNLOCKED(1) << "Starting transaction: " << msg->get()->id().ShortDebugString();
   scoped_refptr<ConsensusRound> round(new ConsensusRound(this, msg));
   ConsensusRound* round_ptr = round.get();
@@ -1114,6 +1124,21 @@ Status RaftConsensus::UpdateReplica(const ConsensusRequestPB* request,
               " other warnings. Status for this op: " << prepare_status.ToString();
         }
       }
+
+      // If this is empty, it means we couldn't prepare a single de-duped message. There is nothing
+      // else we can do. The leader will detect this and retry later.
+      if (deduped_req.messages.empty()) {
+        string msg = Substitute("Rejecting Update request from peer $0 for term $1. "
+                                "Could not prepare a single transaction due to: $2",
+                                request->caller_uuid(),
+                                request->caller_term(),
+                                prepare_status.ToString());
+        LOG_WITH_PREFIX_UNLOCKED(INFO) << msg;
+        FillConsensusResponseError(response, ConsensusErrorPB::CANNOT_PREPARE,
+                                   Status::IllegalState(msg));
+        FillConsensusResponseOKUnlocked(response);
+        return Status::OK();
+      }
     }
 
     OpId last_from_leader;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/ae865a53/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 7a0ccd4..b49dc24 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -2417,6 +2417,50 @@ TEST_F(RaftConsensusITest, TestChangeConfigRejectedUnlessNoopReplicated) {
   ASSERT_STR_CONTAINS(s.ToString(), "Latest committed op is not from this term");
 }
 
+// Test that if for some reason none of the transactions can be prepared, that it will come
+// back as an error in UpdateConsensus().
+TEST_F(RaftConsensusITest, TestUpdateConsensusErrorNonePrepared) {
+  const int kNumOps = 10;
+
+  vector<string> ts_flags, master_flags;
+  ts_flags.push_back("--enable_leader_failure_detection=false");
+  master_flags.push_back("--catalog_manager_wait_for_new_tablets_to_elect_leader=false");
+  BuildAndStart(ts_flags, master_flags);
+
+  vector<TServerDetails*> tservers;
+  AppendValuesFromMap(tablet_servers_, &tservers);
+  ASSERT_EQ(3, tservers.size());
+
+  // Shutdown the other servers so they don't get chatty.
+  cluster_->tablet_server_by_uuid(tservers[1]->uuid())->Shutdown();
+  cluster_->tablet_server_by_uuid(tservers[2]->uuid())->Shutdown();
+
+  // Configure the first server to fail all on prepare.
+  TServerDetails *replica_ts = tservers[0];
+  ASSERT_OK(cluster_->SetFlag(cluster_->tablet_server(0),
+                "follower_fail_all_prepare", "true"));
+
+  // Pretend to be the leader and send a request that should return an error.
+  ConsensusRequestPB req;
+  ConsensusResponsePB resp;
+  RpcController rpc;
+  req.set_dest_uuid(replica_ts->uuid());
+  req.set_tablet_id(tablet_id_);
+  req.set_caller_uuid(tservers[2]->instance_id.permanent_uuid());
+  req.set_caller_term(0);
+  req.mutable_committed_index()->CopyFrom(MakeOpId(0, 0));
+  req.mutable_preceding_id()->CopyFrom(MakeOpId(0, 0));
+  for (int i = 0; i < kNumOps; i++) {
+    AddOp(MakeOpId(0, 1 + i), &req);
+  }
+
+  ASSERT_OK(replica_ts->consensus_proxy->UpdateConsensus(req, &resp, &rpc));
+  LOG(INFO) << resp.ShortDebugString();
+  ASSERT_TRUE(resp.status().has_error());
+  ASSERT_EQ(consensus::ConsensusErrorPB::CANNOT_PREPARE, resp.status().error().code());
+  ASSERT_STR_CONTAINS(resp.ShortDebugString(), "Could not prepare a single transaction");
+}
+
 }  // namespace tserver
 }  // namespace kudu
 


[3/4] incubator-kudu git commit: build: disable check for leaked tmp dirs

Posted by to...@apache.org.
build: disable check for leaked tmp dirs

The Python tests appear to be leaking tmp directories occasionally due to some
race in cleanup (KUDU-1301). In order to fix this source of precommit
flakiness, we'll just disable this for now.

Change-Id: I9ea325da5866adb494e5419ce54a50b964454f7f
Reviewed-on: http://gerrit.cloudera.org:8080/1817
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 89380b53d48b085dfe394b9966360d1ef1a4b038
Parents: 23c6c66
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Jan 19 15:53:33 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jan 22 04:21:52 2016 +0000

----------------------------------------------------------------------
 build-support/jenkins/build-and-test.sh | 24 ++++++++++++++----------
 1 file changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/89380b53/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 a3f7480..6401a33 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -305,16 +305,20 @@ if [ $EXIT_STATUS != 0 ]; then
 fi
 
 # If all tests passed, ensure that they cleaned up their test output.
-if [ $EXIT_STATUS == 0 ]; then
-  TEST_TMPDIR_CONTENTS=$(ls $TEST_TMPDIR)
-  if [ -n "$TEST_TMPDIR_CONTENTS" ]; then
-    echo "All tests passed, yet some left behind their test output:"
-    for SUBDIR in $TEST_TMPDIR_CONTENTS; do
-      echo $SUBDIR
-    done
-    EXIT_STATUS=1
-  fi
-fi
+#
+# TODO: Python is currently leaking a tmp directory sometimes (KUDU-1301).
+# Temporarily disabled until that's fixed.
+#
+# if [ $EXIT_STATUS == 0 ]; then
+#   TEST_TMPDIR_CONTENTS=$(ls $TEST_TMPDIR)
+#   if [ -n "$TEST_TMPDIR_CONTENTS" ]; then
+#     echo "All tests passed, yet some left behind their test output:"
+#     for SUBDIR in $TEST_TMPDIR_CONTENTS; do
+#       echo $SUBDIR
+#     done
+#     EXIT_STATUS=1
+#   fi
+# fi
 
 if [ "$DO_COVERAGE" == "1" ]; then
   echo Generating coverage report...


[4/4] incubator-kudu git commit: build: run local tests parallel with distributed tests

Posted by to...@apache.org.
build: run local tests parallel with distributed tests

This changes the order of the build slightly so that, if distributed
testing is enabled, it asynchronously submits the job to the dist-test
cluster, and then runs the non-distributed tests locally. When the
local tests are finished, it fetches and processes the results of the
dist-test job.

This shaves several minutes off the precommit build.

Change-Id: Iaac391210589a68b43900deb28abe9aff2fe5c76
Reviewed-on: http://gerrit.cloudera.org:8080/1815
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 3d4e0ce8af061eb574ca0bbe735e99c59e444266
Parents: 89380b5
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Jan 15 18:06:18 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jan 22 04:23:03 2016 +0000

----------------------------------------------------------------------
 build-support/jenkins/build-and-test.sh | 141 +++++++++++++++------------
 1 file changed, 77 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/3d4e0ce8/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 6401a33..9e7ce15 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -260,66 +260,18 @@ fi
 
 EXIT_STATUS=0
 
-# Run the C++ unit tests.
+# If we're running distributed tests, submit them asynchronously while
+# we run the Java and Python tests.
 if [ "$ENABLE_DIST_TEST" == "1" ]; then
   export DIST_TEST_JOB_PATH=$BUILD_ROOT/dist-test-job-id
-  $SOURCE_ROOT/build-support/dist_test.py run-all || EXIT_STATUS=$?
-  $DIST_TEST_HOME/client.py fetch --artifacts -d $TEST_LOGDIR
-  # Fetching the artifacts expands each log into its own directory.
-  # Move them back into the main log directory
-  rm -f $TEST_LOGDIR/*zip
-  for arch_dir in $TEST_LOGDIR/* ; do
-    # In the case of sharded tests, we'll have multiple subdirs
-    # which contain files of the same name. We need to disambiguate
-    # when we move back. We can grab the shard index from the task name
-    # which is in the archive directory name.
-    shard_idx=$(echo $arch_dir | perl -ne '
-      if (/(\d+)$/) {
-        print $1;
-      } else {
-        print "unknown_shard";
-      }')
-    for log_file in $arch_dir/build/test-logs/* ; do
-      mv $log_file $TEST_LOGDIR/${shard_idx}_$(basename $log_file)
-    done
-    rm -Rf $arch_dir
-  done
+  rm -f $DIST_TEST_JOB_PATH
+  $SOURCE_ROOT/build-support/dist_test.py --no-wait run-all || EXIT_STATUS=$?
   # Still need to run a few non-dist-test-capable tests locally.
   EXTRA_TEST_FLAGS="$EXTRA_TEST_FLAGS -L no_dist_test"
 fi
 
 $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.
-  # We go through and generate a kind of poor-man's version of them in those cases.
-  for GTEST_OUTFILE in $TEST_LOGDIR/*.txt.gz; do
-    TEST_EXE=$(basename $GTEST_OUTFILE .txt.gz)
-    GTEST_XMLFILE="$TEST_LOGDIR/$TEST_EXE.xml"
-    if [ ! -f "$GTEST_XMLFILE" ]; then
-      echo "JUnit report missing:" \
-           "generating fake JUnit report file from $GTEST_OUTFILE and saving it to $GTEST_XMLFILE"
-      zcat $GTEST_OUTFILE | $SOURCE_ROOT/build-support/parse_test_failure.py -x > $GTEST_XMLFILE
-    fi
-  done
-fi
-
-# If all tests passed, ensure that they cleaned up their test output.
-#
-# TODO: Python is currently leaking a tmp directory sometimes (KUDU-1301).
-# Temporarily disabled until that's fixed.
-#
-# if [ $EXIT_STATUS == 0 ]; then
-#   TEST_TMPDIR_CONTENTS=$(ls $TEST_TMPDIR)
-#   if [ -n "$TEST_TMPDIR_CONTENTS" ]; then
-#     echo "All tests passed, yet some left behind their test output:"
-#     for SUBDIR in $TEST_TMPDIR_CONTENTS; do
-#       echo $SUBDIR
-#     done
-#     EXIT_STATUS=1
-#   fi
-# fi
-
 if [ "$DO_COVERAGE" == "1" ]; then
   echo Generating coverage report...
   ./thirdparty/gcovr-3.0/scripts/gcovr -r .  -e '.*\.pb\..*' --xml \
@@ -346,18 +298,6 @@ if [ "$BUILD_JAVA" == "1" ]; then
   popd
 fi
 
-if [ "$HEAPCHECK" = normal ]; then
-  FAILED_TESTS=$(zgrep -L -- "WARNING: Perftools heap leak checker is active -- Performance may suffer" build/test-logs/*-test.txt*)
-  if [ -n "$FAILED_TESTS" ]; then
-    echo "Some tests didn't heap check properly:"
-    for FTEST in $FAILED_TESTS; do
-      echo $FTEST
-    done
-    EXIT_STATUS=1
-  else
-    echo "All tests heap checked properly"
-  fi
-fi
 
 if [ "$BUILD_PYTHON" == "1" ]; then
   # Failing to compile the Python client should result in a build failure
@@ -379,8 +319,81 @@ if [ "$BUILD_PYTHON" == "1" ]; then
   python setup.py test \
     --addopts="kudu --junit-xml=$KUDU_BUILD/test-logs/python_client.xml" \
     2> $KUDU_BUILD/test-logs/python_client.log || EXIT_STATUS=$?
+
+  popd
+fi
+
+# If we submitted the tasks earlier, go fetch the results now
+if [ "$ENABLE_DIST_TEST" == "1" ]; then
+  echo Fetching previously submitted dist-test results...
+  $DIST_TEST_HOME/client.py watch || EXIT_STATUS=$?
+  DT_DIR=$TEST_LOGDIR/dist-test-out
+  rm -Rf $DT_DIR
+  $DIST_TEST_HOME/client.py fetch --artifacts -d $DT_DIR
+  # Fetching the artifacts expands each log into its own directory.
+  # Move them back into the main log directory
+  rm -f $DT_DIR/*zip
+  for arch_dir in $DT_DIR/* ; do
+    # In the case of sharded tests, we'll have multiple subdirs
+    # which contain files of the same name. We need to disambiguate
+    # when we move back. We can grab the shard index from the task name
+    # which is in the archive directory name.
+    shard_idx=$(echo $arch_dir | perl -ne '
+      if (/(\d+)$/) {
+        print $1;
+      } else {
+        print "unknown_shard";
+      }')
+    for log_file in $arch_dir/build/test-logs/* ; do
+      mv $log_file $TEST_LOGDIR/${shard_idx}_$(basename $log_file)
+    done
+    rm -Rf $arch_dir
+  done
+fi
+
+if [ "$HEAPCHECK" = normal ]; then
+  FAILED_TESTS=$(zgrep -L -- "WARNING: Perftools heap leak checker is active -- Performance may suffer" build/test-logs/*-test.txt*)
+  if [ -n "$FAILED_TESTS" ]; then
+    echo "Some tests didn't heap check properly:"
+    for FTEST in $FAILED_TESTS; do
+      echo $FTEST
+    done
+    EXIT_STATUS=1
+  else
+    echo "All tests heap checked properly"
+  fi
+fi
+
+if [ $EXIT_STATUS != 0 ]; then
+  # Tests that crash do not generate JUnit report XML files.
+  # We go through and generate a kind of poor-man's version of them in those cases.
+  for GTEST_OUTFILE in $TEST_LOGDIR/*.txt.gz; do
+    TEST_EXE=$(basename $GTEST_OUTFILE .txt.gz)
+    GTEST_XMLFILE="$TEST_LOGDIR/$TEST_EXE.xml"
+    if [ ! -f "$GTEST_XMLFILE" ]; then
+      echo "JUnit report missing:" \
+           "generating fake JUnit report file from $GTEST_OUTFILE and saving it to $GTEST_XMLFILE"
+      zcat $GTEST_OUTFILE | $SOURCE_ROOT/build-support/parse_test_failure.py -x > $GTEST_XMLFILE
+    fi
+  done
 fi
 
+# If all tests passed, ensure that they cleaned up their test output.
+#
+# TODO: Python is currently leaking a tmp directory sometimes (KUDU-1301).
+# Temporarily disabled until that's fixed.
+#
+# if [ $EXIT_STATUS == 0 ]; then
+#   TEST_TMPDIR_CONTENTS=$(ls $TEST_TMPDIR)
+#   if [ -n "$TEST_TMPDIR_CONTENTS" ]; then
+#     echo "All tests passed, yet some left behind their test output:"
+#     for SUBDIR in $TEST_TMPDIR_CONTENTS; do
+#       echo $SUBDIR
+#     done
+#     EXIT_STATUS=1
+#   fi
+# fi
+
 set -e
 
 exit $EXIT_STATUS


[2/4] incubator-kudu git commit: Fix run-test and dist-test to make paths absolute

Posted by to...@apache.org.
Fix run-test and dist-test to make paths absolute

This fixes issues with run-test.sh (and transitively with dist-test)
when trying to run them from within a working directory which isn't
the repository root. The recent out-of-tree build patch broke some
of this behavior.

Tested by verifying I can now run:
  ../build-support/run-test.sh ./latest/cbtree-test

from my build directory. I also successfully submitted a dist-test task.

Change-Id: Ief6768d05fc4ae5016813909f819fc9e99f9ebd6
Reviewed-on: http://gerrit.cloudera.org:8080/1857
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 23c6c66d60c9addbd6c58f2bc73c71d496fb80a9
Parents: ae865a5
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 20 23:13:18 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jan 22 04:21:26 2016 +0000

----------------------------------------------------------------------
 README.adoc                |  8 ++++++++
 build-support/dist_test.py |  2 +-
 build-support/run-test.sh  | 20 +++++++++++---------
 3 files changed, 20 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/23c6c66d/README.adoc
----------------------------------------------------------------------
diff --git a/README.adoc b/README.adoc
index ae80060..0478a26 100644
--- a/README.adoc
+++ b/README.adoc
@@ -212,6 +212,14 @@ Instead, use a command like:
 $ ctest -R foo-test
 ----
 
+or
+
+[source,bash]
+----
+$ build-support/run-test.sh build/tsan/latest/foo-test [--test-arguments-here]
+----
+
+
 ...and then view the logs in _build/tsan/test-logs/_
 
 In order for all of the suppressions to work, you need libraries with debug

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/23c6c66d/build-support/dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index aa312ca..ceb37b7 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -236,7 +236,7 @@ def create_archive_input(staging, argv,
   files.append(rel_test_exe)
   deps = ldd_deps(abs_test_exe)
   for d in DEPS_FOR_ALL:
-    d = os.path.realpath(d)
+    d = os.path.realpath(rel_to_abs(d))
     if os.path.isdir(d):
       d += "/"
     deps.append(d)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/23c6c66d/build-support/run-test.sh
----------------------------------------------------------------------
diff --git a/build-support/run-test.sh b/build-support/run-test.sh
index 605a325..f9f7f60 100755
--- a/build-support/run-test.sh
+++ b/build-support/run-test.sh
@@ -33,13 +33,14 @@
 # central test server.
 
 # Path to the test executable or script to be run.
+# May be relative or absolute.
 TEST_PATH=$1
 
-# Path to the root source directory. This script is expected to live within it.
-SOURCE_ROOT=$(dirname "$BASH_SOURCE")/..
+# Absolute path to the root source directory. This script is expected to live within it.
+SOURCE_ROOT=$(cd $(dirname "$BASH_SOURCE")/.. ; pwd)
 
-# Path to the root build directory. The test path is expected to be within it.
-BUILD_ROOT=$(dirname "$TEST_PATH")/..
+# Absolute path to the root build directory. The test path is expected to be within it.
+BUILD_ROOT=$(cd $(dirname "$TEST_PATH")/.. ; pwd)
 
 TEST_LOGDIR=$BUILD_ROOT/test-logs
 mkdir -p $TEST_LOGDIR
@@ -49,6 +50,7 @@ mkdir -p $TEST_DEBUGDIR
 
 TEST_DIRNAME=$(cd $(dirname $TEST_PATH); pwd)
 TEST_FILENAME=$(basename $TEST_PATH)
+ABS_TEST_PATH=$TEST_DIRNAME/$TEST_FILENAME
 shift
 TEST_NAME=$(echo $TEST_FILENAME | perl -pe 's/\..+?$//') # Remove path and extension (if any).
 
@@ -148,8 +150,8 @@ 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_PATH "$@" --test_timeout_after $KUDU_TEST_TIMEOUT 2>&1 \
-    | $SOURCE_ROOT/build-support/stacktrace_addr2line.pl $TEST_PATH \
+  $ABS_TEST_PATH "$@" --test_timeout_after $KUDU_TEST_TIMEOUT 2>&1 \
+    | $SOURCE_ROOT/build-support/stacktrace_addr2line.pl $ABS_TEST_PATH \
     | $pipe_cmd > $LOGFILE
   STATUS=$?
 
@@ -189,7 +191,7 @@ for ATTEMPT_NUMBER in $(seq 1 $TEST_EXECUTION_ATTEMPTS) ; do
 
   if [ -n "$KUDU_REPORT_TEST_RESULTS" ]; then
     echo Reporting results
-    $SOURCE_ROOT/build-support/report-test.sh "$TEST_PATH" "$LOGFILE" "$STATUS" &
+    $SOURCE_ROOT/build-support/report-test.sh "$ABS_TEST_PATH" "$LOGFILE" "$STATUS" &
 
     # On success, we'll do "best effort" reporting, and disown the subprocess.
     # On failure, we want to upload the failed test log. So, in that case,
@@ -229,12 +231,12 @@ fi
 COREFILES=$(ls | grep ^core)
 if [ -n "$COREFILES" ]; then
   echo Found core dump. Saving executable and core files.
-  gzip < $TEST_PATH > "$TEST_DEBUGDIR/$TEST_NAME.gz" || exit $?
+  gzip < $ABS_TEST_PATH > "$TEST_DEBUGDIR/$TEST_NAME.gz" || exit $?
   for COREFILE in $COREFILES; do
     gzip < $COREFILE > "$TEST_DEBUGDIR/$TEST_NAME.$COREFILE.gz" || exit $?
   done
   # Pull in any .so files as well.
-  for LIB in $(ldd $TEST_PATH | grep $BUILD_ROOT | awk '{print $3}'); do
+  for LIB in $(ldd $ABS_TEST_PATH | grep $BUILD_ROOT | awk '{print $3}'); do
     LIB_NAME=$(basename $LIB)
     gzip < $LIB > "$TEST_DEBUGDIR/$LIB_NAME.gz" || exit $?
   done