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 02:21:51 UTC

incubator-kudu git commit: Fix up dist_test for C++11, add dist_test support for Jenkins builds

Repository: incubator-kudu
Updated Branches:
  refs/heads/master 173e1fde4 -> 113d5b092


Fix up dist_test for C++11, add dist_test support for Jenkins builds

This commit does two separate things, but wasn't worth the effort of teasing
the changes out -- I was working on Jenkins-ifying dist_test when the C++11
changes came in and broke everything, so the two projects got done simultaneously
along with a few other fixes.

Jenkins-related changes:

- Adds flaky-test retry support to dist_test. For flaky tests, we submit the
  task to the dist test server with a 'max_retries' field so that it will
  automatically resubmit after a failure.
- Adds kudu-ts-cli to the list of dependencies (fixes some tests which now
  shell out to it)
- Adds a 'no_dist_test' ctest label to the tests which currently won't run
  correctly in the distributed test sandbox. These are basically client tests
  which require various dev tools to be on the box at runtime, which aren't
  present in the slave environment.
- Adds an ENABLE_DIST_TEST flag to the Jenkins script. When this is passed,
  it runs the dist-test-capable tests on the cluster, collects the results,
  and moves them back into the correct directory layout such that Jenkins
  can parse the results.
- Passes through KUDU_COMPRESS_TEST_OUTPUT to the slave environment.
- For RELEASE builds, if dist test is enabled, dynamic link for better performance.

C++11-related changes:
- Fixes the path to llvm-symbolizer now that we build llvm 3.7 as part of our
  thirdparty build.
- Switches from passing the thirdparty directory as an LD_LIBRARY_PATH
  variable to instead tweak the RPATH of binaries on the slave. This was
  necessary so that llvm-symbolizer would run correctly in the TSAN environment.
- Fixed the way that we pass environment variables inside enable_devtoolset.sh
  to use the 'env' utility. Without this, on Ubuntu boxes, I was getting an error
  that 'CC=/path/to/clang' was not an executable.

Sort-of-unrelated:
- Adds a --no-wait flag to the dist test script. I'm planning on using this
  soon to have the Jenkins slaves submit the dist test job, then run python
  and Java tests locally while the other tests are running remotely, then
  later collect the results. I ended up ripping that out for now, but left
  the support in the dist-test code.

Change-Id: I816f87ca5d1c6147746cfa3230bfe449c849e561
Reviewed-on: http://gerrit.cloudera.org:8080/1791
Reviewed-by: Adar Dembo <ad...@cloudera.com>
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/113d5b09
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kudu/tree/113d5b09
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kudu/diff/113d5b09

Branch: refs/heads/master
Commit: 113d5b092097b6f9df20fdfc2cba3a551f906c07
Parents: 173e1fd
Author: Todd Lipcon <to...@cloudera.com>
Authored: Fri Jan 8 18:28:33 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Jan 22 01:20:54 2016 +0000

----------------------------------------------------------------------
 build-support/dist_test.py              | 137 ++++++++++++++++-----------
 build-support/jenkins/build-and-test.sh |  47 ++++++++-
 build-support/run_dist_test.py          |  74 +++++++++++++--
 src/kudu/client/CMakeLists.txt          |   4 +-
 src/kudu/client/client_samples-test.sh  |   3 +-
 src/kudu/twitter-demo/CMakeLists.txt    |   4 +-
 6 files changed, 201 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/build-support/dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index 2b74740..aa312ca 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -22,13 +22,13 @@
 #
 # See dist_test.py --help for usage information.
 
+import argparse
 import glob
 try:
   import simplejson as json
 except:
   import json
 import logging
-import optparse
 import os
 import pprint
 import re
@@ -38,12 +38,18 @@ import shutil
 import subprocess
 import time
 
-TEST_TIMEOUT_SECS = int(os.environ.get('TEST_TIMEOUT_SECS', '400'))
+TEST_TIMEOUT_SECS = int(os.environ.get('TEST_TIMEOUT_SECS', '900'))
+ARTIFACT_ARCHIVE_GLOBS = ["build/test-logs/*"]
 ISOLATE_SERVER = os.environ.get('ISOLATE_SERVER',
                                 "http://isolate.cloudera.org:4242/")
 DIST_TEST_HOME = os.environ.get('DIST_TEST_HOME',
                                 os.path.expanduser("~/dist_test"))
 
+# The number of times that flaky tests will be retried.
+# Our non-distributed implementation sets a number of _attempts_, not a number
+# of retries, so we have to subtract 1.
+FLAKY_TEST_RETRIES = int(os.environ.get('KUDU_FLAKY_TEST_ATTEMPTS', 1)) - 1
+
 PATH_TO_REPO = "../"
 
 TEST_COMMAND_RE = re.compile('Test command: (.+)$')
@@ -56,15 +62,20 @@ DEPS_FOR_ALL = \
      "build-support/tsan-suppressions.txt",
      "build-support/lsan-suppressions.txt",
 
+     # The LLVM symbolizer is necessary for suppressions to work
+     "thirdparty/installed/bin/llvm-symbolizer",
+
      # Tests that use the external minicluster require these.
      # TODO: declare these dependencies per-test.
      "build/latest/kudu-tserver",
      "build/latest/kudu-master",
+     "build/latest/kudu-ts-cli",
 
      # parser-test requires these data files.
      # TODO: again, we should do this with some per-test metadata file.
-     "src/kudu/twitter-demo/example-deletes.txt",
-     "src/kudu/twitter-demo/example-tweets.txt",
+     # TODO: these are broken now that we separate source and build trees.
+     #".../example-deletes.txt",
+     #".../example-tweets.txt",
 
      # Tests that require tooling require these.
      "build/latest/kudu-admin",
@@ -110,7 +121,7 @@ def abs_to_rel(abs_path, staging):
 
 def get_test_commandlines():
   ctest_bin = os.path.join(rel_to_abs("thirdparty/installed/bin/ctest"))
-  p = subprocess.Popen([ctest_bin, "-V", "-N"], stdout=subprocess.PIPE)
+  p = subprocess.Popen([ctest_bin, "-V", "-N", "-LE", "no_dist_test"], stdout=subprocess.PIPE)
   out, err = p.communicate()
   if p.returncode != 0:
     print >>sys.stderr, "Unable to list tests with ctest"
@@ -197,6 +208,10 @@ def ldd_deps(exe):
 def num_shards_for_test(test_name):
   if 'raft_consensus-itest' in test_name:
     return 8
+  if 'cfile-test' in test_name:
+    return 4
+  if 'mt-tablet-test' in test_name:
+    return 4
   return 1
 
 
@@ -246,7 +261,10 @@ def create_archive_input(staging, argv,
                '-e', 'GTEST_TOTAL_SHARDS=%d' % num_shards,
                '-e', 'KUDU_TEST_TIMEOUT=%d' % (TEST_TIMEOUT_SECS - 30),
                '-e', 'KUDU_ALLOW_SLOW_TESTS=%s' % os.environ.get('KUDU_ALLOW_SLOW_TESTS', 1),
-               "--"] + argv[1:]
+               '-e', 'KUDU_COMPRESS_TEST_OUTPUT=%s' % \
+                      os.environ.get('KUDU_COMPRESS_TEST_OUTPUT', 0)]
+    command.append('--')
+    command += argv[1:]
 
     archive_json = dict(args=["-i", out_isolate,
                               "-s", out_isolate + "d"],
@@ -262,7 +280,8 @@ def create_archive_input(staging, argv,
 
 
 def create_task_json(staging,
-                     replicate_tasks=1):
+                     replicate_tasks=1,
+                     flaky_test_set=set()):
   """
   Create a task JSON file suitable for submitting to the distributed
   test execution service.
@@ -278,9 +297,18 @@ def create_task_json(staging,
   # the dumped JSON. Others list it in an 'items' dictionary.
   items = inmap.get('items', inmap)
   for k, v in items.iteritems():
+    # The key is 'foo-test.<shard>'. So, chop off the last component
+    # to get the test name
+    test_name = ".".join(k.split(".")[:-1])
+    max_retries = 0
+    if test_name in flaky_test_set:
+      max_retries = FLAKY_TEST_RETRIES
+
     tasks += [{"isolate_hash": str(v),
                "description": str(k),
-               "timeout": TEST_TIMEOUT_SECS
+               "artifact_archive_globs": ARTIFACT_ARCHIVE_GLOBS,
+               "timeout": TEST_TIMEOUT_SECS + 30,
+               "max_retries": max_retries
                }] * replicate_tasks
 
   outmap = {"tasks": tasks}
@@ -307,7 +335,7 @@ def run_isolate(staging):
     print >>sys.stderr, "Failed to run", isolate_path
     raise
 
-def submit_tasks(staging):
+def submit_tasks(staging, options):
   """
   Runs the distributed testing tool to submit the tasks in the
   provided staging directory.
@@ -322,80 +350,79 @@ def submit_tasks(staging):
     raise OSError("Cannot find path to dist_test tools")
   client_py_path = os.path.join(DIST_TEST_HOME, "client.py")
   try:
-    subprocess.check_call([client_py_path,
-                           "submit",
-                           staging.tasks_json_path()])
+    cmd = [client_py_path, "submit"]
+    if options.no_wait:
+      cmd.append('--no-wait')
+    cmd.append(staging.tasks_json_path())
+    subprocess.check_call(cmd)
   except:
     print >>sys.stderr, "Failed to run", client_py_path
     raise
 
-def run_all_tests(argv):
+def get_flakies():
+  path = os.getenv('KUDU_FLAKY_TEST_LIST')
+  if not path:
+    return set()
+  return set(l.strip() for l in file(path))
+
+def run_all_tests(parser, options):
   """
   Gets all of the test command lines from 'ctest', isolates them,
   creates a task list, and submits the tasks to the testing service.
   """
-  if len(argv) != 1:
-    print >>sys.stderr, "run-all-tests takes no arguments"
-    sys.exit(1)
-
   commands = get_test_commandlines()
   staging = StagingDir.new()
   for command in commands:
-    create_archive_input(staging, command)
+    create_archive_input(staging, command,
+        disable_sharding=options.disable_sharding)
 
   run_isolate(staging)
-  create_task_json(staging)
-  submit_tasks(staging)
+  create_task_json(staging, flaky_test_set=get_flakies())
+  submit_tasks(staging, options)
 
+def add_run_all_subparser(subparsers):
+  p = subparsers.add_parser('run-all', help='Run all of the dist-test-enabled tests')
+  p.set_defaults(func=run_all_tests)
 
-def loop_test(argv):
+def loop_test(parser, options):
   """
   Runs many instances of a user-provided test case on the testing service.
   """
-  p = optparse.OptionParser(
-      usage="usage: %prog loop [--] <test-path> [<args>]",
-      epilog="if passing arguments to the test, you may want to use a '--' " +
-             "argument before <test-path>. e.g: loop -- foo-test --gtest_opt=123")
-  p.add_option("-n", "--num-instances", dest="num_instances", type="int",
-               help="number of test instances to start", metavar="NUM",
-               default=100)
-  p.add_option("--disable-sharding", dest="disable_sharding", action="store_true",
-               help="Disable automatic sharding of tests", default=False)
-  options, args = p.parse_args()
   if options.num_instances < 1:
-    p.error("--num-instances must be >= 1")
-  if len(args) < 1:
-    p.error("no test command specified")
-    sys.exit(1)
-
-  command = ["run-test.sh"] + args
+    parser.error("--num-instances must be >= 1")
+  command = ["run-test.sh", options.cmd] + options.args
   staging = StagingDir.new()
   create_archive_input(staging, command,
                        disable_sharding=options.disable_sharding)
   run_isolate(staging)
   create_task_json(staging, options.num_instances)
-  submit_tasks(staging)
-
+  submit_tasks(staging, options)
 
-def usage(argv):
-  print >>sys.stderr, "usage: %s <command> [<args>]" % os.path.basename(argv[0])
-  print >>sys.stderr, """Commands:
-    run-all Run all unit tests defined by ctest
-    loop    Run a single test many times"""
-  print >>sys.stderr, "%s <command> --help may provide further info" % argv[0]
+def add_loop_test_subparser(subparsers):
+  p = subparsers.add_parser('loop', help='Run many instances of the same test',
+      epilog="if passing arguments to the test, you may want to use a '--' " +
+             "argument before <test-path>. e.g: loop -- foo-test --gtest_opt=123")
+  p.add_argument("--num-instances", "-n", dest="num_instances", type=int,
+                 help="number of test instances to start", metavar="NUM",
+                 default=100)
+  p.add_argument("cmd", help="test binary")
+  p.add_argument("args", nargs=argparse.REMAINDER, help="test arguments")
+  p.set_defaults(func=loop_test)
 
 
 def main(argv):
-  if len(argv) < 2:
-    usage(argv)
-    sys.exit(1)
-  command = argv[1]
-  del argv[1]
-  if command == "run-all":
-    run_all_tests(argv)
-  elif command == "loop":
-    loop_test(argv)
+  logging.basicConfig(level=logging.INFO)
+  p = argparse.ArgumentParser()
+  p.add_argument("--disable-sharding", dest="disable_sharding", action="store_true",
+                 help="Disable automatic sharding of tests", default=False)
+  p.add_argument("--no-wait", dest="no_wait", action="store_true",
+                 help="Return without waiting for the job to complete", default=False)
+  sp = p.add_subparsers()
+  add_loop_test_subparser(sp)
+  add_run_all_subparser(sp)
+  args = p.parse_args(argv)
+  args.func(p, args)
 
 
 if __name__ == "__main__":
-  main(sys.argv)
+  main(sys.argv[1:])

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/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 2844763..a3f7480 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -47,6 +47,12 @@
 #     This must be configured for flaky test resistance or test result
 #     reporting to work.
 #
+#   ENABLE_DIST_TEST  Default: 0
+#     If set to 1, will submit C++ tests to be run by the distributed
+#     test runner instead of running them locally. This requires that
+#     $DIST_TEST_HOME be set to a working dist_test checkout (and that
+#     dist_test itself be appropriately configured to point to a cluster)
+#
 #   BUILD_JAVA        Default: 1
 #     Build and test java code if this is set to 1.
 #
@@ -156,12 +162,12 @@ rm -rf $SOURCE_ROOT/CMakeCache.txt $SOURCE_ROOT/CMakeFiles
 cd $BUILD_ROOT
 if [ "$BUILD_TYPE" = "ASAN" ]; then
   $SOURCE_ROOT/build-support/enable_devtoolset.sh \
-    "CC=$CLANG CXX=$CLANG++ $THIRDPARTY_BIN/cmake -DKUDU_USE_ASAN=1 -DKUDU_USE_UBSAN=1 $SOURCE_ROOT"
+    "env CC=$CLANG CXX=$CLANG++ $THIRDPARTY_BIN/cmake -DKUDU_USE_ASAN=1 -DKUDU_USE_UBSAN=1 $SOURCE_ROOT"
   BUILD_TYPE=fastdebug
   BUILD_PYTHON=0
 elif [ "$BUILD_TYPE" = "TSAN" ]; then
   $SOURCE_ROOT/build-support/enable_devtoolset.sh \
-    "CC=$CLANG CXX=$CLANG++ $THIRDPARTY_BIN/cmake -DKUDU_USE_TSAN=1 $SOURCE_ROOT"
+    "env CC=$CLANG CXX=$CLANG++ $THIRDPARTY_BIN/cmake -DKUDU_USE_TSAN=1 $SOURCE_ROOT"
   BUILD_TYPE=fastdebug
   EXTRA_TEST_FLAGS="$EXTRA_TEST_FLAGS -LE no_tsan"
   BUILD_PYTHON=0
@@ -210,7 +216,15 @@ if [ "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
   fi
 fi
 
-$SOURCE_ROOT/build-support/enable_devtoolset.sh "$THIRDPARTY_BIN/cmake -DCMAKE_BUILD_TYPE=${BUILD_TYPE} $SOURCE_ROOT"
+# On distributed tests, force dynamic linking even for release builds. Otherwise,
+# the test binaries are too large and we spend way too much time uploading them
+# to the test slaves.
+LINK_FLAGS=
+if [ "$ENABLE_DIST_TEST" == "1" ]; then
+  LINK_FLAGS="-DKUDU_LINK=dynamic"
+fi
+
+$SOURCE_ROOT/build-support/enable_devtoolset.sh "$THIRDPARTY_BIN/cmake -DCMAKE_BUILD_TYPE=${BUILD_TYPE} $LINK_FLAGS $SOURCE_ROOT"
 
 # our tests leave lots of data lying around, clean up before we run
 if [ -d "$TEST_TMPDIR" ]; then
@@ -247,6 +261,33 @@ fi
 EXIT_STATUS=0
 
 # Run the C++ unit 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
+  # 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

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/build-support/run_dist_test.py
----------------------------------------------------------------------
diff --git a/build-support/run_dist_test.py b/build-support/run_dist_test.py
index db369ee..7e25072 100755
--- a/build-support/run_dist_test.py
+++ b/build-support/run_dist_test.py
@@ -29,6 +29,7 @@
 
 import optparse
 import os
+import re
 import shutil
 import subprocess
 import sys
@@ -36,6 +37,57 @@ import sys
 ME = os.path.abspath(__file__)
 ROOT = os.path.abspath(os.path.join(os.path.dirname(ME), ".."))
 
+def is_elf_binary(path):
+  """ Determine if the given path is an ELF binary (executable or shared library) """
+  if not os.path.isfile(path) or os.path.islink(path):
+    return False
+  try:
+    with file(path, "rb") as f:
+      magic = f.read(4)
+      return magic == "\x7fELF"
+  except:
+    # Ignore unreadable files
+    return False
+
+def fix_rpath_component(bin_path, path):
+  """
+  Given an RPATH component 'path' of the binary located at 'bin_path',
+  fix the thirdparty dir to be relative to the binary rather than absolute.
+  """
+  rel_tp = os.path.relpath(os.path.join(ROOT, "thirdparty/"),
+                           os.path.dirname(bin_path))
+  path = re.sub(r".*thirdparty/", "$ORIGIN/"+rel_tp + "/", path)
+  return path
+
+def fix_rpath(path):
+  """
+  Fix the RPATH/RUNPATH of the binary located at 'path' so that
+  the thirdparty/ directory is properly found, even though we will
+  run the binary at a different path than it was originally built.
+  """
+  # Fetch the original rpath.
+  p = subprocess.Popen(["chrpath", path],
+                       stdout=subprocess.PIPE,
+                       stderr=subprocess.PIPE)
+  stdout, stderr = p.communicate()
+  if p.returncode != 0:
+    return
+  rpath = re.search("R(?:UN)?PATH=(.+)", stdout.strip()).group(1)
+  # Fix it to be relative.
+  new_path = ":".join(fix_rpath_component(path, c) for c in rpath.split(":"))
+  # Write the new rpath back into the binary.
+  subprocess.check_call(["chrpath", "-r", new_path, path])
+
+def fixup_rpaths(root):
+  """
+  Recursively walk the directory tree 'root' and fix the RPATH for any
+  ELF files (binaries/libraries) that are found.
+  """
+  for dirpath, dirnames, filenames in os.walk(root):
+    for f in filenames:
+      p = os.path.join(dirpath, f)
+      if is_elf_binary(p):
+        fix_rpath(p)
 
 def main():
   p = optparse.OptionParser(usage="usage: %prog [options] <test-name>")
@@ -54,15 +106,25 @@ def main():
     (k, v) = env_pair.split("=", 1)
     env[k] = v
 
+  # Fix the RPATHs of any binaries. During the build, we end up with
+  # absolute paths from the build machine. This fixes the paths to be
+  # binary-relative so that we can run it on the new location.
+  #
+  # It's important to do this rather than just putting all of the thirdparty
+  # lib directories into $LD_LIBRARY_PATH below because we need to make sure
+  # that non-TSAN-instrumented runtime tools (like 'llvm-symbolizer') do _NOT_
+  # pick up the TSAN-instrumented libraries, whereas TSAN-instrumented test
+  # binaries (like 'foo_test' or 'kudu-tserver') _DO_ pick them up.
+  fixup_rpaths(os.path.join(ROOT, "build"))
+  fixup_rpaths(os.path.join(ROOT, "thirdparty"))
+
   env['LD_LIBRARY_PATH'] = ":".join(
-      [os.path.join(ROOT, "thirdparty/installed/lib"),
-       os.path.join(ROOT, "build/dist-test-system-libs/"),
-       os.path.abspath(os.path.dirname(test_exe))])
+    [os.path.join(ROOT, "build/dist-test-system-libs/"),
+     os.path.abspath(os.path.dirname(test_exe))])
+  env['GTEST_OUTPUT'] = 'xml:' + os.path.join(ROOT, 'build/test-logs/')
+  env['ASAN_SYMBOLIZER_PATH'] = os.path.join(ROOT, "thirdparty/installed/bin/llvm-symbolizer")
   rc = subprocess.call([os.path.join(ROOT, "build-support/run-test.sh")] + args,
                        env=env)
-  # 'cat' the test logs to stdout so that the user can grab them.
-  with file(os.path.join(ROOT, "build/test-logs/%s.txt" % test_name), "r") as f:
-    shutil.copyfileobj(f, sys.stdout)
   sys.exit(rc)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/src/kudu/client/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/client/CMakeLists.txt b/src/kudu/client/CMakeLists.txt
index 3299e22..84cf429 100644
--- a/src/kudu/client/CMakeLists.txt
+++ b/src/kudu/client/CMakeLists.txt
@@ -222,12 +222,12 @@ target_link_libraries(kudu_client_test_util
 if (NOT APPLE)
   # The OS X system compiler does not support source symbol maps, so the client
   # leaks internal symbols globally.
-  ADD_KUDU_TEST(client_symbol-test.sh)
+  ADD_KUDU_TEST(client_symbol-test.sh LABELS no_dist_test)
 endif()
 
 # The samples are never built with ASAN/TSAN.
 if(NOT "${KUDU_USE_ASAN}" AND NOT "${KUDU_USE_TSAN}")
-  ADD_KUDU_TEST(client_samples-test.sh RUN_SERIAL true)
+  ADD_KUDU_TEST(client_samples-test.sh RUN_SERIAL true LABELS no_dist_test)
 endif()
 set(KUDU_TEST_LINK_LIBS kudu_client integration-tests ${KUDU_MIN_TEST_LIBS})
 ADD_KUDU_TEST(client-test)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/src/kudu/client/client_samples-test.sh
----------------------------------------------------------------------
diff --git a/src/kudu/client/client_samples-test.sh b/src/kudu/client/client_samples-test.sh
index f1dafef..7925261 100755
--- a/src/kudu/client/client_samples-test.sh
+++ b/src/kudu/client/client_samples-test.sh
@@ -62,7 +62,8 @@ popd
 # use the one from thirdparty.
 CMAKE=$(which cmake || :)
 if [ -z "$CMAKE" ]; then
-  CMAKE=$ROOT/thirdparty/installed/bin/cmake
+  # TODO: temporary hack which assumes this script is in src/build/latest
+  CMAKE=$OUTPUT_DIR/../../thirdparty/installed/bin/cmake
 fi
 
 # Build the client samples using the client library.

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/113d5b09/src/kudu/twitter-demo/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/twitter-demo/CMakeLists.txt b/src/kudu/twitter-demo/CMakeLists.txt
index e6b4471..5c261e2 100644
--- a/src/kudu/twitter-demo/CMakeLists.txt
+++ b/src/kudu/twitter-demo/CMakeLists.txt
@@ -47,7 +47,9 @@ else()
 
   # Tests
   ADD_KUDU_TEST(oauth-test)
-  ADD_KUDU_TEST(parser-test)
+  # parser-test relies on symlinked data files which we can't currently copy correctly
+  # to the cluster.
+  ADD_KUDU_TEST(parser-test LABELS no_dist_test)
   if(NOT "${NO_TESTS}")
     target_link_libraries(oauth-test
       twitter_demo)