You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/04/03 20:50:59 UTC

[kudu] branch master updated (984a3e1 -> c550fe8)

This is an automated email from the ASF dual-hosted git repository.

adar pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 984a3e1  KUDU-2757: retry OpenSSL downloads
     new d929aac  java: add support for flaky test reporting
     new 8532311  build: enable Java flaky test reporting
     new 16cfb7a  java: ensure KuduTestHarness or RetryRule in every test
     new c550fe8  build: adapt new Java flaky test infrastructure to existing controls

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 build-support/dist_test.py                         |  40 +++-
 build-support/jenkins/build-and-test.sh            |  65 +++++-
 build-support/report-test.sh                       |  51 ++---
 java/gradle/dependencies.gradle                    |   6 +
 java/gradle/tests.gradle                           |   3 -
 .../java/org/apache/kudu/TestColumnSchema.java     |  14 +-
 .../java/org/apache/kudu/client/TestBitSet.java    |   6 +
 .../org/apache/kudu/client/TestBloomFilter.java    |   8 +-
 .../java/org/apache/kudu/client/TestBytes.java     |   9 +-
 .../kudu/client/TestColumnRangePredicate.java      |   5 +
 .../apache/kudu/client/TestConnectToCluster.java   |  11 +-
 .../org/apache/kudu/client/TestErrorCollector.java |   6 +
 .../org/apache/kudu/client/TestKuduPredicate.java  |   5 +
 .../org/apache/kudu/client/TestNegotiator.java     |   5 +
 .../java/org/apache/kudu/client/TestOperation.java |   5 +
 .../org/apache/kudu/client/TestPartialRow.java     |   5 +
 .../org/apache/kudu/client/TestRemoteTablet.java   |   7 +-
 .../org/apache/kudu/client/TestRequestTracker.java |   7 +-
 .../java/org/apache/kudu/client/TestRpcTraces.java |   6 +
 .../org/apache/kudu/client/TestServerInfo.java     |  10 +-
 .../java/org/apache/kudu/client/TestStatus.java    |   6 +
 .../kudu/client/TestTableLocationsCache.java       |   9 +-
 .../org/apache/kudu/client/TestTimeoutTracker.java |   6 +
 .../java/org/apache/kudu/util/TestAsyncUtil.java   |   5 +
 .../java/org/apache/kudu/util/TestByteVec.java     |   6 +
 .../java/org/apache/kudu/util/TestMurmurHash.java  |  26 ++-
 .../java/org/apache/kudu/util/TestNetUtil.java     |   8 +-
 .../java/org/apache/kudu/util/TestStringUtil.java  |   6 +
 .../org/apache/kudu/util/TestTimestampUtil.java    |  10 +-
 .../org/apache/kudu/mapreduce/TestJarFinder.java   |   8 +-
 java/kudu-test-utils/build.gradle                  |   5 +
 .../org/apache/kudu/test/CapturingLogAppender.java |   8 +-
 .../kudu/test/CapturingToFileLogAppender.java      | 177 +++++++++++++++
 .../org/apache/kudu/test/junit/ResultReporter.java | 249 +++++++++++++++++++++
 .../java/org/apache/kudu/test/junit/RetryRule.java | 172 ++++++++++++--
 .../test/cluster/TestKuduBinaryJarExtractor.java   |  18 +-
 .../apache/kudu/test/junit/TestResultReporter.java | 205 +++++++++++++++++
 .../org/apache/kudu/test/junit/TestRetryRule.java  |   3 +-
 38 files changed, 1093 insertions(+), 108 deletions(-)
 create mode 100644 java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingToFileLogAppender.java
 create mode 100644 java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/ResultReporter.java
 create mode 100644 java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java


[kudu] 02/04: build: enable Java flaky test reporting

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 8532311f2739de0ebcfa68666f6a2d7ece3ef226
Author: Mike Percy <mp...@apache.org>
AuthorDate: Wed Dec 5 21:39:50 2018 -0800

    build: enable Java flaky test reporting
    
    This patch moves flaky test environment setup out of report-test.sh and
    into build-and-test.sh so that those environment variables can be
    inherited by the Java build environment. That change enables flaky test
    reporting for Java tests.
    
    This was tested in a RHEL6 DEBUG build environment. Example:
    
    http://dist-test.cloudera.org:8080/test_drilldown?test_name=testHiveMetastoreIntegration%28org.apache.kudu.test.TestMiniKuduCluster%29
    
    Change-Id: Ifef74fc9bf5453105c267418fa24daf4c33f73f3
    Reviewed-on: http://gerrit.cloudera.org:8080/12043
    Tested-by: Kudu Jenkins
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 build-support/jenkins/build-and-test.sh | 34 ++++++++++++++++++++++
 build-support/report-test.sh            | 51 ++++++++++++---------------------
 2 files changed, 53 insertions(+), 32 deletions(-)

diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 9c0c956..23b20e4 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -74,6 +74,9 @@
 #
 #   ERROR_ON_TEST_FAILURE    Default: 1
 #     Whether test failures will cause this script to return an error.
+#
+#   KUDU_REPORT_TEST_RESULTS Default: 0
+#     If non-zero, tests are reported to the central test server.
 
 # If a commit messages contains a line that says 'DONT_BUILD', exit
 # immediately.
@@ -292,6 +295,37 @@ $CMAKE
 mkdir -p Testing/Temporary
 mkdir -p $TEST_LOGDIR
 
+if [ -n "$KUDU_REPORT_TEST_RESULTS" ] && [ "$KUDU_REPORT_TEST_RESULTS" -ne 0 ]; then
+  # Export environment variables needed for flaky test reporting.
+  #
+  # The actual reporting happens in the test runners themselves.
+
+  # On Jenkins, we'll have this variable set. Otherwise,
+  # report the build tag as non-jenkins.
+  export BUILD_TAG=${BUILD_TAG:-non-jenkins}
+
+  # Figure out the current git revision, and append a "-dirty" tag if it's
+  # not a pristine checkout.
+  GIT_REVISION=$(cd $SOURCE_ROOT && git rev-parse HEAD)
+  if ! ( cd $SOURCE_ROOT && git diff --quiet .  && git diff --cached --quiet . ) ; then
+    GIT_REVISION="${GIT_REVISION}-dirty"
+  fi
+  export GIT_REVISION
+
+  # Parse out our "build config" - a space-separated list of tags
+  # which include the cmake build type as well as the list of configured
+  # sanitizers.
+
+  # Define BUILD_CONFIG for flaky test reporting.
+  BUILD_CONFIG="$CMAKE_BUILD"
+  if [ "$BUILD_TYPE" = "ASAN" ]; then
+    BUILD_CONFIG="$BUILD_CONFIG asan ubsan"
+  elif [ "$BUILD_TYPE" = "TSAN" ]; then
+    BUILD_CONFIG="$BUILD_CONFIG tsan"
+  fi
+  export BUILD_CONFIG
+fi
+
 # Short circuit for LINT builds.
 if [ "$BUILD_TYPE" = "LINT" ]; then
   make lint | tee $TEST_LOGDIR/lint.log
diff --git a/build-support/report-test.sh b/build-support/report-test.sh
index 41351d2..e283c0b 100755
--- a/build-support/report-test.sh
+++ b/build-support/report-test.sh
@@ -24,10 +24,24 @@
 #
 # Note that this may exit with a non-zero code if the network is flaky or the
 # test result server is down.
+#
+# Expects BUILD_TAG, GIT_REVISION, and BUILD_CONFIG environment variables to be set.
 
 set -e
 
-ROOT=$(dirname $BASH_SOURCE)/..
+# Verify required environment variables.
+if [ -z "$BUILD_TAG" ]; then
+  echo "BUILD_TAG environment variable must be set"
+  exit 1
+fi
+if [ -z "$GIT_REVISION" ]; then
+  echo "GIT_REVISION environment variable must be set"
+  exit 1
+fi
+if [ -z "$BUILD_CONFIG" ]; then
+  echo "BUILD_CONFIG environment variable must be set"
+  exit 1
+fi
 
 # Verify and parse command line and options
 if [ $# -ne 3 ]; then
@@ -43,35 +57,6 @@ STATUS=$3
 TEST_RESULT_SERVER=${TEST_RESULT_SERVER:-localhost:8080}
 REPORT_TIMEOUT=${REPORT_TIMEOUT:-10}
 
-# On Jenkins, we'll have this variable set. Otherwise,
-# report the build ID as non-jenkins.
-BUILD_ID=${BUILD_TAG:-non-jenkins}
-
-# Figure out the current git revision, and append a "-dirty" tag if it's
-# not a pristine checkout
-REVISION=$(cd $ROOT && git rev-parse HEAD)
-if ! ( cd $ROOT && git diff --quiet .  && git diff --cached --quiet . ) ; then
-  REVISION="${REVISION}-dirty"
-fi
-
-BUILD_ROOT=$(dirname $TEST_EXECUTABLE)/..
-
-# Parse out our "build config" - a space-separated list of tags
-# which include the cmake build type as well as the list of configured
-# sanitizers
-
-CMAKECACHE=$BUILD_ROOT/CMakeCache.txt
-BUILD_CONFIG=$(grep '^CMAKE_BUILD_TYPE:' $CMAKECACHE | cut -f 2 -d=)
-if grep -q "KUDU_USE_ASAN:UNINITIALIZED=1" $CMAKECACHE ; then
-  BUILD_CONFIG="$BUILD_CONFIG asan"
-fi
-if grep -q "KUDU_USE_TSAN:UNINITIALIZED=1" $CMAKECACHE ; then
-  BUILD_CONFIG="$BUILD_CONFIG tsan"
-fi
-if grep -q "KUDU_USE_UBSAN:UNINITIALIZED=1" $CMAKECACHE ; then
-  BUILD_CONFIG="$BUILD_CONFIG ubsan"
-fi
-
 # We sometimes have flaky infrastructure where NTP is broken. In that case
 # do not report it as a failed test.
 if zgrep -q 'Clock considered unsynchronized' $LOGFILE ; then
@@ -88,13 +73,15 @@ else
   LOG_PARAM=""
 fi
 
+# In the backend, the BUILD_TAG field is called 'build_id', but we can't use
+# that as an env variable because it'd collide with Jenkins' BUILD_ID.
 curl -s \
     --max-time $REPORT_TIMEOUT \
     $LOG_PARAM \
-    -F "build_id=$BUILD_ID" \
+    -F "build_id=$BUILD_TAG" \
     -F "hostname=$(hostname)" \
     -F "test_name=$(basename $TEST_EXECUTABLE)" \
     -F "status=$STATUS" \
-    -F "revision=$REVISION" \
+    -F "revision=$GIT_REVISION" \
     -F "build_config=$BUILD_CONFIG" \
     http://$TEST_RESULT_SERVER/add_result


[kudu] 04/04: build: adapt new Java flaky test infrastructure to existing controls

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit c550fe8f55ee0a4c6838d246e00e541ef4cff460
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Apr 1 21:59:21 2019 -0700

    build: adapt new Java flaky test infrastructure to existing controls
    
    Now that Java tests are reporting success/failure, we can use the existing
    flaky test controls to drive it. As a refresher, the C++ tests rely on these
    environment variables:
    - RUN_FLAKY_ONLY: whether to run just flaky tests or all tests
    - KUDU_FLAKY_TEST_ATTEMPTS: number of attempts for flaky tests
    - KUDU_FLAKY_TEST_LIST: path to list of flaky tests, one on each line
    - KUDU_RETRY_ALL_FAILED_TESTS: whether to retry all tests or just the ones
                                   in the flaky test list
    
    The algorithm is roughly:
      if RUN_FLAKY_ONLY or KUDU_FLAKY_TEST_ATTEMPTS > 1:
        populate KUDU_FLAKY_TEST_LIST from test result server
    
      if RUN_FLAKY_ONLY:
        testset = tests listed in KUDU_FLAKY_TEST_LIST
      else:
        testset = all tests
    
      for t in testset:
        if KUDU_RETRY_ALL_FAILED_TESTS or (KUDU_FLAKY_TEST_LIST and
                                           t in KUDU_FLAKY_TEST_LIST):
          num_attempts = KUDU_FLAKY_TEST_ATTEMPTS (or 1 if unset)
        else:
          num_attempts = 1
    
        run t up to num_attempts times
    
    You can see it at work in build-and-test.sh/run-test.sh. You can also see it
    in dist-test.py though notably, it doesn't care about RUN_FLAKY_ONLY because
    we never used that particular combination (presumably the list of flaky
    tests is short enough that it wouldn't benefit from distributed testing).
    
    This patch attempts to mirror these exact semantics for Java tests. Here are
    the interesting changes:
    - In RetryRule, rerunFailingTestsCount is gone. The behavior is informed via
      the aforementioned environment variables instead.
    - In build-and-test.sh, if RUN_FLAKY_ONLY is set, parse the flaky test list
      into a series of --tests gradle command line arguments.
    - In dist-test.py, opt into the C++ flaky test handling (which reflects the
      above algorithm). There are also some small changes to flaky handling to
      accommodate Java's per-method flaky test tracking.
    
    Note: all of this assumes that there's no overlap between the names of any
    C++ or Java tests, which is currently true as all C++ tests have names like
    "tablet-test" or "master_cert_authority-itest" while all Java tests are
    prefixed with "org.apache.kudu...". If this were to change, we'd need to
    properly "namespace" the test results in the reporting infrastructure and
    fetch the flaky test lists separately for C++ and Java tests. For now
    there's just one flaky test list, and both ctest and gradle are OK with
    being asked to run irrelevant tests (they'll just be ignored).
    
    Change-Id: Ia89598d7eeb5ab642ab4ebb7aa583adcce770eae
    Reviewed-on: http://gerrit.cloudera.org:8080/12917
    Reviewed-by: Grant Henke <gr...@apache.org>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 build-support/dist_test.py                         |  40 ++++++--
 build-support/jenkins/build-and-test.sh            |  31 +++++--
 java/gradle/tests.gradle                           |   3 -
 .../java/org/apache/kudu/test/junit/RetryRule.java | 102 ++++++++++++++++++---
 4 files changed, 145 insertions(+), 31 deletions(-)

diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index 992da9c..2e207f5 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -70,13 +70,17 @@ GRADLE_FLAGS = os.environ.get('EXTRA_GRADLE_FLAGS', "")
 PATH_TO_REPO = "../"
 
 # Matches the command line listings in 'ctest -V -N'. For example:
-#   262: Test command: /src/kudu/build-support/run-test.sh "/src/kudu/build/debug/bin/jsonwriter-test"
+#  262: Test command: /src/kudu/build-support/run-test.sh "/src/kudu/build/debug/bin/jsonwriter-test"
 TEST_COMMAND_RE = re.compile('Test command: (.+)$')
 
 # Matches the environment variable listings in 'ctest -V -N'. For example:
 #  262:  GTEST_TOTAL_SHARDS=1
 TEST_ENV_RE = re.compile('^\d+:  (\S+)=(.+)')
 
+# Matches test names that have a shard suffix. For example:
+#  master-stress-test.8
+TEST_SHARD_RE = re.compile("\.\d+$")
+
 DEPS_FOR_ALL = \
     ["build-support/stacktrace_addr2line.pl",
      "build-support/run-test.sh",
@@ -347,6 +351,10 @@ def create_task_json(staging,
 
   If 'replicate_tasks' is higher than one, each .isolate file will be
   submitted multiple times. This can be useful for looping tests.
+
+  The test name is compared with the contents of 'flaky_test_set' to decide
+  how many times the execution service should retry the test on failure.
+  Alternatively, if 'retry_all_tests' is True, all tests will be retried.
   """
   tasks = []
   with file(staging.archive_dump_path(), "r") as isolate_dump:
@@ -356,9 +364,9 @@ 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])
+    # The key may be 'foo-test.<shard>'. So, chop off the last component
+    # to get the test name.
+    test_name = ".".join(k.split(".")[:-1]) if TEST_SHARD_RE.search(k) else k
     max_retries = 0
     if test_name in flaky_test_set or retry_all_tests:
       max_retries = FLAKY_TEST_RETRIES
@@ -426,7 +434,20 @@ def get_flakies():
   path = os.getenv('KUDU_FLAKY_TEST_LIST')
   if not path:
     return set()
-  return set(l.strip() for l in file(path))
+
+  # dist-test can only retry tests on a per-class basis, but
+  # KUDU_FLAKY_TEST_LIST lists Java flakes on a per-method basis.
+  flaky_classes = []
+  with open(path) as f:
+    for l in f:
+      l = l.strip()
+      if '.' in l:
+        # "o.a.k.client.TestKuduClient.testFoo" -> "o.a.k.client.TestKuduClient"
+        flaky_classes.append('.'.join(l.split('.')[:-1]))
+      else:
+        flaky_classes.append(l)
+
+  return set(flaky_classes)
 
 def run_tests(parser, options):
   """
@@ -545,10 +566,11 @@ def run_java_tests(parser, options):
                         cwd=rel_to_abs("java"))
   staging = StagingDir(rel_to_abs("java/build/dist-test"))
   run_isolate(staging)
-  # TODO(ghenke): Add Java tests to the flaky dashboard
-  # KUDU_FLAKY_TEST_LIST doesn't included Java tests.
-  # Instead we will retry all Java tests in case they are flaky.
-  create_task_json(staging, 1, retry_all_tests=True)
+  retry_all = RETRY_ALL_TESTS > 0
+  create_task_json(staging,
+                   flaky_test_set=get_flakies(),
+                   replicate_tasks=1,
+                   retry_all_tests=retry_all)
   submit_tasks(staging, options)
 
 def loop_java_test(parser, options):
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 23b20e4..4eaf556 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -223,6 +223,13 @@ if [ "$RUN_FLAKY_ONLY" == "1" -o "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
   fi
 
   if [ "$RUN_FLAKY_ONLY" == "1" ]; then
+    # TODO(adar): we can't yet pass the flaky test list into the dist-test
+    # machinery (in order to control which tests are executed).
+    if [ "$ENABLE_DIST_TEST" == "1" ]; then
+      echo "Distributed testing is incompatible with RUN_FLAKY_ONLY=1"
+      exit 1
+    fi
+
     test_regex=$(perl -e '
       chomp(my @lines = <>);
       print join("|", map { "^" . quotemeta($_) } @lines);
@@ -232,14 +239,25 @@ if [ "$RUN_FLAKY_ONLY" == "1" -o "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
       exit 0
     fi
 
+    # Set up ctest/gradle to run only those tests found in the flaky test list.
+    #
+    # Note: the flaky test list contains both C++ and Java tests and we pass it
+    # in its entirety to both ctest and gradle. This is safe because:
+    # 1. There are no test name collisions between C++ and Java tests.
+    # 2. Both ctest and gradle will happily ignore tests they can't find.
+    #
+    # If either of these assumptions changes, we'll need to explicitly split the
+    # test list into two lists, either here or in the test result server.
     EXTRA_TEST_FLAGS="$EXTRA_TEST_FLAGS -R $test_regex"
+    while IFS="" read t || [ -n "$t" ]
+    do
+      EXTRA_GRADLE_TEST_FLAGS="--tests $t $EXTRA_GRADLE_TEST_FLAGS"
+    done < $KUDU_FLAKY_TEST_LIST
 
-    # We don't support detecting java and python flaky tests at the moment.
-    echo "RUN_FLAKY_ONLY=1: running flaky tests only,"\
-         "disabling Java and python builds."
+    # We don't support detecting python flaky tests at the moment.
+    echo "RUN_FLAKY_ONLY=1: running flaky tests only, disabling python build."
     BUILD_PYTHON=0
     BUILD_PYTHON3=0
-    BUILD_JAVA=0
   elif [ "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
     echo Will retry the flaky tests up to $KUDU_FLAKY_TEST_ATTEMPTS times.
   fi
@@ -373,7 +391,7 @@ EXIT_STATUS=0
 FAILURES=""
 
 # If we're running distributed C++ tests, submit them asynchronously while
-# we run the Java and Python tests.
+# we run any local tests.
 if [ "$ENABLE_DIST_TEST" == "1" ]; then
   echo
   echo Submitting C++ distributed-test job.
@@ -424,7 +442,6 @@ if [ "$BUILD_JAVA" == "1" ]; then
   export EXTRA_GRADLE_FLAGS="--console=plain"
   EXTRA_GRADLE_FLAGS="$EXTRA_GRADLE_FLAGS --no-daemon"
   EXTRA_GRADLE_FLAGS="$EXTRA_GRADLE_FLAGS --continue"
-  EXTRA_GRADLE_FLAGS="$EXTRA_GRADLE_FLAGS -DrerunFailingTestsCount=3"
   # KUDU-2524: temporarily disable scalafmt until we can work out its JDK
   # incompatibility issue.
   EXTRA_GRADLE_FLAGS="$EXTRA_GRADLE_FLAGS -DskipFormat"
@@ -443,7 +460,7 @@ if [ "$BUILD_JAVA" == "1" ]; then
     fi
   else
     # TODO: Run `gradle check` in BUILD_TYPE DEBUG when static code analysis is fixed
-    if ! ./gradlew $EXTRA_GRADLE_FLAGS clean test ; then
+    if ! ./gradlew $EXTRA_GRADLE_FLAGS clean test $EXTRA_GRADLE_TEST_FLAGS; then
       TESTS_FAILED=1
       FAILURES="$FAILURES"$'Java Gradle build/test failed\n'
     fi
diff --git a/java/gradle/tests.gradle b/java/gradle/tests.gradle
index 2f43d96..2b541c4 100644
--- a/java/gradle/tests.gradle
+++ b/java/gradle/tests.gradle
@@ -62,9 +62,6 @@ tasks.withType(Test) {
   systemProperty "java.net.preferIPv4Stack", true
   systemProperty "java.security.egd", "file:/dev/urandom" // Improve RNG generation speed.
 
-  // Set rerunFailingTestsCount for use in BaseKuduTest.java to rerun failing tests.
-  systemProperty "rerunFailingTestsCount", propertyWithDefault("rerunFailingTestsCount", 0)
-
   // Set kuduBinDir to the binaries to use with the MiniKuduCluster.
   systemProperty "kuduBinDir", propertyWithDefault("kuduBinDir", "$project.rootDir/../build/latest/bin")
 
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
index 31d8a95..a4611db 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
@@ -16,6 +16,8 @@
 // under the License.
 package org.apache.kudu.test.junit;
 
+import com.google.common.base.Preconditions;
+
 import org.apache.kudu.test.CapturingToFileLogAppender;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -24,15 +26,23 @@ import org.junit.runner.Description;
 import org.junit.runners.model.Statement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import java.io.BufferedReader;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
 
 /**
  * JUnit rule to retry failed tests.
  *
- * The number of retries is controlled by the "rerunFailingTestsCount" system
- * property, mimicking Surefire in that regard.
+ * Uses the KUDU_FLAKY_TEST_LIST and KUDU_RETRY_ALL_FAILED_TESTS environment
+ * variables to determine whether a test should be retried, and the
+ * KUDU_FLAKY_TEST_ATTEMPTS environment variable to determine how many times.
  *
  * By default will use ResultReporter to report success/failure of each test
  * attempt to an external server; this may be skipped if desired.
@@ -40,44 +50,112 @@ import java.io.IOException;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class RetryRule implements TestRule {
-
-  private static final String RETRY_PROP = "rerunFailingTestsCount";
-
   private static final Logger LOG = LoggerFactory.getLogger(RetryRule.class);
+  private static final int DEFAULT_RETRY_COUNT = 0;
+  private static final Set<String> FLAKY_TESTS = new HashSet<>();
 
   private final int retryCount;
   private final ResultReporter reporter;
 
+  static {
+    // Initialize the flaky test set if it exists. The file wil have one test
+    // name per line.
+    String value = System.getenv("KUDU_FLAKY_TEST_LIST");
+    if (value != null) {
+      try (BufferedReader br = Files.newBufferedReader(Paths.get(value), UTF_8)) {
+        for (String l = br.readLine(); l != null; l = br.readLine()) {
+          FLAKY_TESTS.add(l);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
   public RetryRule() {
-    this(Integer.getInteger(RETRY_PROP, 0), /*skipReporting=*/ false);
+    this(DEFAULT_RETRY_COUNT, /*skipReporting=*/ false);
   }
 
   @InterfaceAudience.LimitedPrivate("Test")
   RetryRule(int retryCount, boolean skipReporting) {
+    Preconditions.checkArgument(retryCount >= 0);
     this.retryCount = retryCount;
     this.reporter = skipReporting ? null : new ResultReporter();
   }
 
+  private static boolean retryAllTests() {
+    String value = System.getenv("KUDU_RETRY_ALL_FAILED_TESTS");
+    return value != null && !value.isEmpty();
+  }
+
+  private static boolean retryThisTest(String humanReadableTestName) {
+    return FLAKY_TESTS.contains(humanReadableTestName);
+  }
+
+  private static int getActualRetryCount() {
+    String value = System.getenv("KUDU_FLAKY_TEST_ATTEMPTS");
+    if (value == null) {
+      return DEFAULT_RETRY_COUNT;
+    }
+    try {
+      int val = Integer.parseInt(value);
+      if (val < 1) {
+        throw new NumberFormatException(
+            String.format("expected non-zero positive value, got %d", val));
+      }
+
+      // Convert from number of "attempts" to number of "retries".
+      return Integer.parseInt(value) - 1;
+    } catch (NumberFormatException e) {
+      LOG.warn("Could not parse KUDU_FLAKY_TEST_ATTEMPTS, using default value ({})",
+               DEFAULT_RETRY_COUNT, e);
+      return DEFAULT_RETRY_COUNT;
+    }
+  }
+
   @Override
   public Statement apply(Statement base, Description description) {
-    return new RetryStatement(base, description, retryCount, reporter);
+    String humanReadableTestName =
+        description.getClassName() + "." + description.getMethodName();
+
+    // Retrying and reporting are independent; the RetryStatement is used if
+    // either is enabled. We'll retry the test under one of the following
+    // circumstances:
+    //
+    // 1. The RetryRule was constructed with an explicit retry count.
+    // 2. We've been asked to retry all tests via KUDU_RETRY_ALL_FAILED_TESTS.
+    // 3. We've been asked to retry this test via KUDU_FLAKY_TEST_LIST.
+    //
+    // In the latter two cases, we consult KUDU_FLAKY_TEST_ATTEMPTS for the retry count.
+    boolean retryExplicit = retryCount != DEFAULT_RETRY_COUNT;
+    boolean retryAll = retryAllTests();
+    boolean retryThis = retryThisTest(humanReadableTestName);
+    if (retryExplicit || retryAll || retryThis || reporter != null) {
+      int actualRetryCount = (retryAll || retryThis) ? getActualRetryCount() : retryCount;
+      LOG.info("Creating RetryStatement {} result reporter and retry count of {} ({})",
+               reporter != null ? "with" : "without",
+               actualRetryCount,
+               retryExplicit ? "explicit" :
+                 retryAll ? "all tests" :
+                   retryThis ? "this test" : "no retries");
+      return new RetryStatement(base, actualRetryCount, reporter, humanReadableTestName);
+    }
+    return base;
   }
 
   private static class RetryStatement extends Statement {
 
     private final Statement base;
-    private final Description description;
     private final int retryCount;
     private final ResultReporter reporter;
     private final String humanReadableTestName;
 
-    RetryStatement(Statement base, Description description,
-                   int retryCount, ResultReporter reporter) {
+    RetryStatement(Statement base, int retryCount, ResultReporter reporter,
+                   String humanReadableTestName) {
       this.base = base;
-      this.description = description;
       this.retryCount = retryCount;
       this.reporter = reporter;
-      this.humanReadableTestName = description.getClassName() + "." + description.getMethodName();
+      this.humanReadableTestName = humanReadableTestName;
     }
 
     private void report(ResultReporter.Result result, File logFile) {


[kudu] 03/04: java: ensure KuduTestHarness or RetryRule in every test

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 16cfb7a19aa0e817dceeb44d70062c880f701375
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Wed Mar 27 13:21:44 2019 -0700

    java: ensure KuduTestHarness or RetryRule in every test
    
    Now that test reporting is built into the RetryRule, we should ensure that
    every test uses either RetryRule or KuduTestHarness (which wraps RetryRule).
    This patch adds RetryRule to all tests that were missing one of the two.
    
    Change-Id: I951f9fbb516abdb24a74d5a2acd7e1f1cd8a6fa5
    Reviewed-on: http://gerrit.cloudera.org:8080/12872
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 .../java/org/apache/kudu/TestColumnSchema.java     | 14 ++++++++----
 .../java/org/apache/kudu/client/TestBitSet.java    |  6 +++++
 .../org/apache/kudu/client/TestBloomFilter.java    |  8 ++++++-
 .../java/org/apache/kudu/client/TestBytes.java     |  9 ++++++--
 .../kudu/client/TestColumnRangePredicate.java      |  5 +++++
 .../apache/kudu/client/TestConnectToCluster.java   | 11 ++++++---
 .../org/apache/kudu/client/TestErrorCollector.java |  6 +++++
 .../org/apache/kudu/client/TestKuduPredicate.java  |  5 +++++
 .../org/apache/kudu/client/TestNegotiator.java     |  5 +++++
 .../java/org/apache/kudu/client/TestOperation.java |  5 +++++
 .../org/apache/kudu/client/TestPartialRow.java     |  5 +++++
 .../org/apache/kudu/client/TestRemoteTablet.java   |  7 +++++-
 .../org/apache/kudu/client/TestRequestTracker.java |  7 +++++-
 .../java/org/apache/kudu/client/TestRpcTraces.java |  6 +++++
 .../org/apache/kudu/client/TestServerInfo.java     | 10 +++++++--
 .../java/org/apache/kudu/client/TestStatus.java    |  6 +++++
 .../kudu/client/TestTableLocationsCache.java       |  9 ++++++--
 .../org/apache/kudu/client/TestTimeoutTracker.java |  6 +++++
 .../java/org/apache/kudu/util/TestAsyncUtil.java   |  5 +++++
 .../java/org/apache/kudu/util/TestByteVec.java     |  6 +++++
 .../java/org/apache/kudu/util/TestMurmurHash.java  | 26 +++++++++++++---------
 .../java/org/apache/kudu/util/TestNetUtil.java     |  8 ++++++-
 .../java/org/apache/kudu/util/TestStringUtil.java  |  6 +++++
 .../org/apache/kudu/util/TestTimestampUtil.java    | 10 +++++++--
 .../org/apache/kudu/mapreduce/TestJarFinder.java   |  8 ++++++-
 .../test/cluster/TestKuduBinaryJarExtractor.java   | 18 ++++++++++-----
 .../apache/kudu/test/junit/TestResultReporter.java | 16 +++++++------
 27 files changed, 190 insertions(+), 43 deletions(-)

diff --git a/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java b/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
index 51d5cb8..d227f45 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
@@ -16,15 +16,21 @@
 // under the License.
 package org.apache.kudu;
 
-import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder;
-import org.apache.kudu.util.DecimalUtil;
-import org.junit.Test;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder;
+import org.apache.kudu.util.DecimalUtil;
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestColumnSchema {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testToString() {
     ColumnSchema col1 = new ColumnSchemaBuilder("col1", Type.STRING).build();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBitSet.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBitSet.java
index bfc4131..80002dd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBitSet.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBitSet.java
@@ -22,10 +22,16 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.BitSet;
 
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestBitSet {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   /**
    * Test out BitSet-related operations
    */
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBloomFilter.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBloomFilter.java
index 4458c57..d2e2a85 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBloomFilter.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBloomFilter.java
@@ -21,15 +21,21 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Random;
 
-import org.apache.kudu.util.BloomFilter;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.util.BloomFilter;
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestBloomFilter {
 
   private int nBytes = 32 * 1024;
   private long kRandomSeed = System.currentTimeMillis();
   private int nKeys = 2000;
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testNumberOfHashes() {
     assertEquals(BloomFilter.byCountAndFPRate(10, 0.1).getNHashes(), 3);
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBytes.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBytes.java
index e6461c7..a27c1da 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestBytes.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestBytes.java
@@ -24,13 +24,18 @@ import java.math.BigInteger;
 import java.math.MathContext;
 import java.math.RoundingMode;
 
-import org.apache.kudu.util.DecimalUtil;
-
+import org.junit.Rule;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.kudu.util.DecimalUtil;
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestBytes {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void test() {
     byte[] bytes = new byte[16];
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestColumnRangePredicate.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestColumnRangePredicate.java
index 979ce5d..e1f004d 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestColumnRangePredicate.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestColumnRangePredicate.java
@@ -25,14 +25,19 @@ import java.util.List;
 
 import com.google.common.collect.Lists;
 import org.junit.Test;
+import org.junit.Rule;
 
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.ColumnTypeAttributes;
 import org.apache.kudu.Type;
 import org.apache.kudu.tserver.Tserver;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestColumnRangePredicate {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testRawLists() {
     ColumnSchema col1 = new ColumnSchema.ColumnSchemaBuilder("col1", Type.INT32).build();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectToCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectToCluster.java
index c755f63..63430bc 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectToCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectToCluster.java
@@ -25,13 +25,15 @@ import static org.junit.Assert.fail;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.stumbleupon.async.Callback;
-
-import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+
 import org.apache.kudu.consensus.Metadata;
 import org.apache.kudu.master.Master.ConnectToMasterResponsePB;
-import org.hamcrest.CoreMatchers;
+import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestConnectToCluster {
 
@@ -40,6 +42,9 @@ public class TestConnectToCluster {
       new HostAndPort("1", 9000),
       new HostAndPort("2", 9000));
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   /**
    * Test that the client properly falls back to the old GetMasterRegistration
    * RPC when connecting to a master which does not support the new
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestErrorCollector.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestErrorCollector.java
index 09e8ef7..c742e9e 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestErrorCollector.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestErrorCollector.java
@@ -17,10 +17,16 @@
 package org.apache.kudu.client;
 
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestErrorCollector {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testErrorCollector() {
     int maxErrors = 10;
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
index 0cb2333..9dcb7bc 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
@@ -31,11 +31,13 @@ import java.util.Arrays;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Type;
 import org.apache.kudu.util.DecimalUtil;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestKuduPredicate {
 
@@ -81,6 +83,9 @@ public class TestKuduPredicate {
           .typeAttributes(DecimalUtil.typeAttributes(DecimalUtil.MAX_DECIMAL128_PRECISION, 2))
           .build();
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   private static KuduPredicate intRange(int lower, int upper) {
     Preconditions.checkArgument(lower < upper);
     return new KuduPredicate(RANGE, intCol, Bytes.fromInt(lower), Bytes.fromInt(upper));
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
index e021712..976512e 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
@@ -42,6 +42,7 @@ import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.handler.codec.embedder.DecoderEmbedder;
 import org.jboss.netty.handler.ssl.SslHandler;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,6 +56,7 @@ import org.apache.kudu.rpc.RpcHeader.NegotiatePB.SaslMechanism;
 import org.apache.kudu.rpc.RpcHeader.ResponseHeader;
 import org.apache.kudu.rpc.RpcHeader.RpcFeatureFlag;
 import org.apache.kudu.security.Token.SignedTokenPB;
+import org.apache.kudu.test.junit.RetryRule;
 import org.apache.kudu.util.SecurityUtil;
 
 public class TestNegotiator {
@@ -91,6 +93,9 @@ public class TestNegotiator {
       "nrjox4GmC3JJaA==\n" +
       "-----END CERTIFICATE-----";
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Before
   public void setup() {
     serverEngine = createServerEngine();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestOperation.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestOperation.java
index 1ac314c..ec8599e 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestOperation.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestOperation.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.primitives.Longs;
+import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -31,6 +32,7 @@ import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
 import org.apache.kudu.WireProtocol.RowOperationsPB;
 import org.apache.kudu.client.Operation.ChangeType;
+import org.apache.kudu.test.junit.RetryRule;
 import org.apache.kudu.tserver.Tserver.WriteRequestPBOrBuilder;
 
 /**
@@ -38,6 +40,9 @@ import org.apache.kudu.tserver.Tserver.WriteRequestPBOrBuilder;
  */
 public class TestOperation {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   private Schema createManyStringsSchema() {
     ArrayList<ColumnSchema> columns = new ArrayList<ColumnSchema>(4);
     columns.add(new ColumnSchema.ColumnSchemaBuilder("c0", Type.STRING).key(true).build());
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
index d696e1f..3c93c03 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
@@ -29,14 +29,19 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.sql.Timestamp;
 
+import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestPartialRow {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testGetters() {
     PartialRow partialRow = getPartialRowWithAllTypes();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRemoteTablet.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRemoteTablet.java
index 368ecbc..43ca37a 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRemoteTablet.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRemoteTablet.java
@@ -28,11 +28,13 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.google.protobuf.ByteString;
-import org.apache.kudu.test.ProtobufUtils;
+import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.kudu.consensus.Metadata;
 import org.apache.kudu.master.Master;
+import org.apache.kudu.test.ProtobufUtils;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestRemoteTablet {
   private static final String kClientLocation = "/fake-client";
@@ -40,6 +42,9 @@ public class TestRemoteTablet {
   private static final String kNoLocation = "";
   private static final String[] kUuids = { "uuid-0", "uuid-1", "uuid-2" };
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testLeaderLastRemovedLast() {
     RemoteTablet tablet = getTablet(2);
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRequestTracker.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRequestTracker.java
index 980f447..1989b24 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRequestTracker.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRequestTracker.java
@@ -26,13 +26,18 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.collect.Lists;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestRequestTracker {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test(timeout = 10000)
   public void test() {
     RequestTracker tracker = new RequestTracker("test");
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRpcTraces.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRpcTraces.java
index faba44d..5a25951 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRpcTraces.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRpcTraces.java
@@ -23,10 +23,16 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.List;
 
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestRpcTraces {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testLimit() {
     PingRequest ping = PingRequest.makeMasterPingRequest();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
index 8a67cec..e3e0644 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
@@ -17,14 +17,20 @@
 package org.apache.kudu.client;
 
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 
-import org.apache.kudu.test.cluster.FakeDNS;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
 
-import java.net.UnknownHostException;
+import org.apache.kudu.test.cluster.FakeDNS;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestServerInfo {
+
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   /**
    * Test for KUDU-2103. Checks if the original hostnames is returned if unknown.
    */
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatus.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatus.java
index d4b0703..0b51b68 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatus.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatus.java
@@ -22,10 +22,16 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Arrays;
 
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestStatus {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testOKStatus() {
     Status s = Status.OK();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
index c9de4d3..c231bbd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTableLocationsCache.java
@@ -20,17 +20,22 @@ import static org.junit.Assert.*;
 
 import java.util.List;
 
+import com.google.common.base.Ticker;
+import com.google.common.collect.ImmutableList;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import com.google.common.base.Ticker;
-import com.google.common.collect.ImmutableList;
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestTableLocationsCache {
   private TableLocationsCache cache = new TableLocationsCache();
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   /**
    * Prevent time from advancing during the test by mocking the time.
    */
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeoutTracker.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeoutTracker.java
index b43c8d7..38618b1 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeoutTracker.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeoutTracker.java
@@ -24,10 +24,16 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Ticker;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestTimeoutTracker {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testTimeout() {
     final AtomicLong timeToReturn = new AtomicLong();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestAsyncUtil.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestAsyncUtil.java
index 794a985..7c859e1 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestAsyncUtil.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestAsyncUtil.java
@@ -23,11 +23,16 @@ import com.stumbleupon.async.Deferred;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 /**
  * Test for {@link AsyncUtil}.
  */
 public class TestAsyncUtil {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test(expected = IllegalStateException.class)
   public void testAddCallbacksDeferring() throws Exception {
     Deferred<String> d = new Deferred<String>();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestByteVec.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestByteVec.java
index f77258a..e99df2d 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestByteVec.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestByteVec.java
@@ -25,11 +25,17 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Random;
 
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestByteVec {
   private static final Random RAND = new Random();
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   private void assertBytesEqual(byte a, byte b) {
     if (a != b) throw new AssertionError(String.format("%s != %s", a, b));
   }
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestMurmurHash.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestMurmurHash.java
index cf3ac22..19116f9 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestMurmurHash.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestMurmurHash.java
@@ -20,8 +20,11 @@ import static org.junit.Assert.assertEquals;
 
 import com.google.common.primitives.UnsignedLongs;
 import com.sangupta.murmur.Murmur2;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 /**
  * Test Murmur2 Hash64 returns the expected values for inputs.
  *
@@ -30,17 +33,20 @@ import org.junit.Test;
  */
 public class TestMurmurHash {
 
-    @Test
-    public void testMurmur2Hash64() throws Exception {
-      long hash;
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
+  @Test
+  public void testMurmur2Hash64() throws Exception {
+    long hash;
 
-      hash = Murmur2.hash64("ab".getBytes("UTF-8"), 2, 0);
-      assertEquals(UnsignedLongs.parseUnsignedLong("7115271465109541368"), hash);
+    hash = Murmur2.hash64("ab".getBytes("UTF-8"), 2, 0);
+    assertEquals(UnsignedLongs.parseUnsignedLong("7115271465109541368"), hash);
 
-      hash = Murmur2.hash64("abcdefg".getBytes("UTF-8"), 7, 0);
-      assertEquals(UnsignedLongs.parseUnsignedLong("2601573339036254301"), hash);
+    hash = Murmur2.hash64("abcdefg".getBytes("UTF-8"), 7, 0);
+    assertEquals(UnsignedLongs.parseUnsignedLong("2601573339036254301"), hash);
 
-      hash = Murmur2.hash64("quick brown fox".getBytes("UTF-8"), 15, 42);
-      assertEquals(UnsignedLongs.parseUnsignedLong("3575930248840144026"), hash);
-    }
+    hash = Murmur2.hash64("quick brown fox".getBytes("UTF-8"), 15, 42);
+    assertEquals(UnsignedLongs.parseUnsignedLong("3575930248840144026"), hash);
+  }
 }
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
index 4220bfe..8af7d3c 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
@@ -25,14 +25,20 @@ import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.kudu.client.HostAndPort;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.client.HostAndPort;
+import org.apache.kudu.test.junit.RetryRule;
+
 /**
  * Test for {@link NetUtil}.
  */
 public class TestNetUtil {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   /**
    * Tests parsing strings into {@link HostAndPort} objects with and without specifying
    * the port in the string.
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestStringUtil.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestStringUtil.java
index 0e1a1a7..1f6a8df 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestStringUtil.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestStringUtil.java
@@ -18,10 +18,16 @@ package org.apache.kudu.util;
 
 import static org.junit.Assert.assertEquals;
 
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 public class TestStringUtil {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   private String escapeSQLString(String s) {
     StringBuilder sb = new StringBuilder();
     StringUtil.appendEscapedSQLString(s, sb);
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestTimestampUtil.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestTimestampUtil.java
index 46d6e69..3731393 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestTimestampUtil.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestTimestampUtil.java
@@ -16,16 +16,22 @@
 // under the License.
 package org.apache.kudu.util;
 
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
 import java.util.TimeZone;
 
-import static org.junit.Assert.assertEquals;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestTimestampUtil {
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Test
   public void testTimestampConversion() throws Exception {
     Timestamp epoch = new Timestamp(0);
diff --git a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/TestJarFinder.java b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/TestJarFinder.java
index 35bd0c0..f6e5d7b 100644
--- a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/TestJarFinder.java
+++ b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/TestJarFinder.java
@@ -35,8 +35,11 @@ import org.apache.commons.logging.LogFactory;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.test.junit.RetryRule;
+
 /**
  * This file was forked from hbase/branches/master@4ce6f48.
  */
@@ -44,6 +47,9 @@ public class TestJarFinder {
 
   private static File testDir;
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   @Before
   public void setUp() throws Exception {
     testDir = Files.createTempDirectory("test-dir").toFile();
@@ -112,4 +118,4 @@ public class TestJarFinder {
     Assert.assertNotNull(jis.getManifest());
     jis.close();
   }
-}
\ No newline at end of file
+}
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/cluster/TestKuduBinaryJarExtractor.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/cluster/TestKuduBinaryJarExtractor.java
index 0b6bdda..598c3fd 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/cluster/TestKuduBinaryJarExtractor.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/cluster/TestKuduBinaryJarExtractor.java
@@ -17,9 +17,9 @@
 
 package org.apache.kudu.test.cluster;
 
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -40,14 +40,20 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kudu.test.junit.RetryRule;
 
 public class TestKuduBinaryJarExtractor {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestKuduBinaryJarExtractor.class);
 
+  @Rule
+  public RetryRule retryRule = new RetryRule();
+
   private Path createKuduBinaryJar(final String os) throws IOException, URISyntaxException {
     String baseName = "fake-" + os + "-kudu-binary";
     Path tempDir = Files.createTempDirectory(baseName);
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java
index bad2aad..1adf77d 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java
@@ -16,6 +16,7 @@
 // under the License.
 package org.apache.kudu.test.junit;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -26,6 +27,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,10 +51,15 @@ import java.util.Map;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 /** Unit test for ResultReporter. */
 public class TestResultReporter {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TestResultReporter.class);
+  private static final String BIND_ADDR = "127.0.0.1";
+  private Server server;
+  private MockFlakyTestServlet flakyTestServlet;
+
+  @Rule
+  public RetryRule retryRule = new RetryRule();
 
   /** Record of a specific test run. */
   private static class TestRecord {
@@ -117,11 +124,6 @@ public class TestResultReporter {
     }
   }
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(TestResultReporter.class);
-  private static final String BIND_ADDR = "127.0.0.1";
-  private Server server;
-  private MockFlakyTestServlet flakyTestServlet;
-
   @Before
   public void setup() throws Exception {
     flakyTestServlet = new MockFlakyTestServlet();


[kudu] 01/04: java: add support for flaky test reporting

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit d929aac91831c12e7675faf2dbfd6c578996f84f
Author: Mike Percy <mp...@apache.org>
AuthorDate: Tue Dec 4 18:57:47 2018 -0800

    java: add support for flaky test reporting
    
    This patch hooks into the existing RetryRule to report test results to
    the flaky test server inline as the tests are executed. All of the
    actual reporting logic is factored out into a separate ResultReporter class.
    
    The interface for the test reporter to pass relevant information about
    the build environment to the flaky test server is based on environment
    variables. This includes configuration and build metadata such as flaky
    test server address, git revision, build id, build host, and build type.
    
    This patch also includes a simple integration test for the reporter
    using a mocked-up flaky test server HTTP endpoint.
    
    This patch does not integrate the above functionality into the build.
    That will happen in a follow-up patch.
    
    Change-Id: I34f88363dbf52c2f7bba50fbfb5de059f88e7f74
    Reviewed-on: http://gerrit.cloudera.org:8080/12042
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 java/gradle/dependencies.gradle                    |   6 +
 java/kudu-test-utils/build.gradle                  |   5 +
 .../org/apache/kudu/test/CapturingLogAppender.java |   8 +-
 .../kudu/test/CapturingToFileLogAppender.java      | 177 +++++++++++++++
 .../org/apache/kudu/test/junit/ResultReporter.java | 249 +++++++++++++++++++++
 .../java/org/apache/kudu/test/junit/RetryRule.java |  88 ++++++--
 .../apache/kudu/test/junit/TestResultReporter.java | 203 +++++++++++++++++
 .../org/apache/kudu/test/junit/TestRetryRule.java  |   3 +-
 8 files changed, 721 insertions(+), 18 deletions(-)

diff --git a/java/gradle/dependencies.gradle b/java/gradle/dependencies.gradle
index 181a146..5b945cc 100755
--- a/java/gradle/dependencies.gradle
+++ b/java/gradle/dependencies.gradle
@@ -39,7 +39,9 @@ versions += [
     hamcrest       : "2.1",
     hdrhistogram   : "2.1.11",
     hive           : "2.3.4",
+    httpClient     : "4.5.7",
     jepsen         : "0.1.5",
+    jetty          : "9.4.15.v20190215",
     jsr305         : "3.0.2",
     junit          : "4.12",
     log4j          : "1.2.17",
@@ -94,7 +96,11 @@ libs += [
     hdrhistogram         : "org.hdrhistogram:HdrHistogram:$versions.hdrhistogram",
     hiveMetastore        : "org.apache.hive:hive-metastore:$versions.hive",
     hiveMetastoreTest    : "org.apache.hive:hive-metastore:$versions.hive:tests",
+    httpClient           : "org.apache.httpcomponents:httpclient:$versions.httpClient",
+    httpMime             : "org.apache.httpcomponents:httpmime:$versions.httpClient",
     jepsen               : "jepsen:jepsen:$versions.jepsen",
+    jetty                : "org.eclipse.jetty:jetty-server:$versions.jetty",
+    jettyServlet         : "org.eclipse.jetty:jetty-servlet:$versions.jetty",
     jsr305               : "com.google.code.findbugs:jsr305:$versions.jsr305",
     junit                : "junit:junit:$versions.junit",
     log4j                : "log4j:log4j:$versions.log4j",
diff --git a/java/kudu-test-utils/build.gradle b/java/kudu-test-utils/build.gradle
index e172c2a..9db3651 100644
--- a/java/kudu-test-utils/build.gradle
+++ b/java/kudu-test-utils/build.gradle
@@ -21,6 +21,8 @@ dependencies {
   compile project(path: ":kudu-client")
   compile libs.commonsIo
   compile libs.guava
+  compile libs.httpClient
+  compile libs.httpMime
   compile libs.osdetector
 
   compileUnshaded libs.junit
@@ -32,6 +34,9 @@ dependencies {
 
   optional libs.jsr305
   optional libs.yetusAnnotations
+
+  testCompile libs.jetty
+  testCompile libs.jettyServlet
 }
 
 // kudu-test-utils has no public Javadoc.
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingLogAppender.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingLogAppender.java
index 81c9bc9..056ef80 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingLogAppender.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingLogAppender.java
@@ -36,8 +36,9 @@ import org.apache.yetus.audience.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class CapturingLogAppender extends AppenderSkeleton {
+  private static final Layout LAYOUT = new SimpleLayout();
+
   private StringBuilder appended = new StringBuilder();
-  private static final Layout layout = new SimpleLayout();
 
   @Override
   public void close() {
@@ -50,13 +51,16 @@ public class CapturingLogAppender extends AppenderSkeleton {
 
   @Override
   protected void append(LoggingEvent event) {
-    appended.append(layout.format(event));
+    appended.append(LAYOUT.format(event));
     if (event.getThrowableInformation() != null) {
       appended.append(Throwables.getStackTraceAsString(
           event.getThrowableInformation().getThrowable())).append("\n");
     }
   }
 
+  /**
+   * @return all of the appended messages captured thus far, joined together.
+   */
   public String getAppendedText() {
     return appended.toString();
   }
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingToFileLogAppender.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingToFileLogAppender.java
new file mode 100644
index 0000000..6eea54a
--- /dev/null
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/CapturingToFileLogAppender.java
@@ -0,0 +1,177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.kudu.test;
+
+import java.io.BufferedWriter;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.zip.GZIPOutputStream;
+
+import com.google.common.base.Throwables;
+import org.apache.commons.io.IOUtils;
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.Layout;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.apache.log4j.spi.LoggingEvent;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * Test utility which wraps Log4j and captures all messages logged while
+ * attached, storing them in an (optionally gzipped) temporary file.
+ *
+ * The typical lifecycle is as follows:
+ *
+ * constructor: temporary file is created and opened.
+ * append():    a new log event is captured. It may or may not be flushed to disk.
+ * finish():    all events previously captured in append() are now guaranteed to
+ *              be on disk and visible to readers. No more events may be appended.
+ * close():     the temporary file is deleted.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CapturingToFileLogAppender extends AppenderSkeleton implements AutoCloseable {
+  // This is the standard layout used in Kudu tests.
+  private static final Layout LAYOUT = new PatternLayout(
+      "%d{HH:mm:ss.SSS} [%p - %t] (%F:%L) %m%n");
+
+  private File outputFile;
+  private Writer outputFileWriter;
+
+  /**
+   * Creates a new appender. The temporary file is created immediately; it may
+   * be obtained via getOutputFile().
+   *
+   * Appended messages are buffered; they must be flushed to disk via finish().
+   *
+   * @param useGzip whether to gzip-compress messages when appended
+   */
+  public CapturingToFileLogAppender(boolean useGzip) throws IOException {
+    outputFile = File.createTempFile("captured_output", ".txt.gz");
+    try {
+      OutputStream os = new FileOutputStream(outputFile.getPath());
+      try {
+        if (useGzip) {
+          os = new GZIPOutputStream(os);
+        }
+
+        // As per the recommendation in OutputStreamWriter's Javadoc, we wrap in a
+        // BufferedWriter to buffer up character conversions.
+        outputFileWriter = new BufferedWriter(new OutputStreamWriter(os, UTF_8));
+      } catch (Throwable t) {
+        IOUtils.closeQuietly(os);
+        throw t;
+      }
+    } catch (Throwable t) {
+      outputFile.delete();
+      throw t;
+    }
+  }
+
+  @Override
+  public void close() {
+    // Just do the cleanup; we don't care about exceptions/logging.
+
+    if (outputFileWriter != null) {
+      IOUtils.closeQuietly(outputFileWriter);
+      outputFileWriter = null;
+    }
+    if (outputFile != null) {
+      outputFile.delete();
+      outputFile = null;
+    }
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+
+  @Override
+  protected void append(LoggingEvent event) {
+    assert outputFileWriter != null;
+    try {
+      outputFileWriter.write(LAYOUT.format(event));
+      if (event.getThrowableInformation() != null) {
+        outputFileWriter.write(Throwables.getStackTraceAsString(
+            event.getThrowableInformation().getThrowable()));
+        outputFileWriter.write("\n");
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Flushes any buffered appended events to the on-disk temporary file and
+   * closes it.
+   *
+   * After calling this function, all appended events will be visible to new
+   * readers.
+   *
+   * @throws IOException if close failed
+   */
+  public void finish() throws IOException {
+    // As per the Writer contract, this will also flush the output stream as
+    // well as the compressor (if gzip-compression is used).
+    //
+    // Why close() and not flush()? It turns out to be remarkably hard to
+    // flush a GZIPOutputStream [1]. At the very least it also requires calling
+    // finish(), which is not a generic OutputStream method. But for our use
+    // case (multiple append() calls followed by a single file access) it's
+    // easier to just close() when we're done appending.
+    //
+    // 1. https://stackoverflow.com/questions/3640080/force-flush-on-a-gzipoutputstream-in-java
+    //
+    outputFileWriter.close();
+    outputFileWriter = null;
+  }
+
+  /**
+   * @return the temporary file opened in the appender's constructor
+   */
+  public File getOutputFile() {
+    return outputFile;
+  }
+
+  /**
+   * Temporarily attach the capturing appender to the Log4j root logger.
+   * This can be used in a 'try-with-resources' block:
+   * <code>
+   *   try (Closeable c = capturer.attach()) {
+   *     ...
+   *   }
+   * </code>
+   */
+  public Closeable attach() {
+    Logger.getRootLogger().addAppender(this);
+    return new Closeable() {
+      @Override
+      public void close() throws IOException {
+        Logger.getRootLogger().removeAppender(CapturingToFileLogAppender.this);
+      }
+    };
+  }
+}
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/ResultReporter.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/ResultReporter.java
new file mode 100644
index 0000000..483b6ca
--- /dev/null
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/ResultReporter.java
@@ -0,0 +1,249 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.kudu.test.junit;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import org.apache.http.StatusLine;
+import org.apache.http.util.EntityUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.mime.MultipartEntityBuilder;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/** Class to report test results to the flaky test server. */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ResultReporter {
+  public enum Result {
+    SUCCESS,
+    FAILURE
+  }
+
+  public static class Options {
+    private boolean reportResults = true;
+    private String httpEndpoint;
+    private String buildTag;
+    private String revision;
+    private String hostname;
+    private String buildConfig;
+
+    public Options reportResults(boolean reportResults) {
+      this.reportResults = reportResults;
+      return this;
+    }
+    public Options httpEndpoint(String httpEndpoint) {
+      this.httpEndpoint = httpEndpoint;
+      return this;
+    }
+    public Options buildTag(String buildTag) {
+      this.buildTag = buildTag;
+      return this;
+    }
+    public Options revision(String revision) {
+      this.revision = revision;
+      return this;
+    }
+    public Options hostname(String hostname) {
+      this.hostname = hostname;
+      return this;
+    }
+    public Options buildConfig(String buildConfig) {
+      this.buildConfig = buildConfig;
+      return this;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ResultReporter.class);
+  private static final String KUDU_REPORT_TEST_RESULTS_VAR = "KUDU_REPORT_TEST_RESULTS";
+  private static final String TEST_RESULT_SERVER_VAR = "TEST_RESULT_SERVER";
+  private static final String BUILD_TAG_VAR = "BUILD_TAG";
+  private static final String GIT_REVISION_VAR = "GIT_REVISION";
+  private static final String BUILD_CONFIG_VAR = "BUILD_CONFIG";
+
+  private final Options options;
+
+  public ResultReporter() {
+    this(new Options()
+        .reportResults(isReportingConfigured())
+        .httpEndpoint(getEnvStringWithDefault(TEST_RESULT_SERVER_VAR,
+                                              "localhost:8080"))
+        .buildTag(System.getenv(BUILD_TAG_VAR))
+        .revision(System.getenv(GIT_REVISION_VAR))
+        .buildConfig(System.getenv(BUILD_CONFIG_VAR))
+        .hostname(getLocalHostname()));
+  }
+
+  @InterfaceAudience.LimitedPrivate("Test")
+  ResultReporter(Options options) {
+    this.options = options;
+  }
+
+  private static boolean isVarSetAndNonEmpty(String name) {
+    String var = System.getenv(name);
+    return var != null && !var.equals("");
+  }
+
+  private static boolean areRequiredReportingVarsSetAndNonEmpty() {
+    return isVarSetAndNonEmpty(BUILD_TAG_VAR) &&
+           isVarSetAndNonEmpty(GIT_REVISION_VAR) &&
+           isVarSetAndNonEmpty(BUILD_CONFIG_VAR);
+  }
+
+  private static String reportingVarDump() {
+    List<String> vars = new ArrayList<>();
+    for (String var : ImmutableList.of(TEST_RESULT_SERVER_VAR,
+                                       BUILD_TAG_VAR,
+                                       GIT_REVISION_VAR,
+                                       BUILD_CONFIG_VAR)) {
+      vars.add(var + ": \"" + System.getenv(var) + "\"");
+    }
+    return Joiner.on(", ").join(vars);
+  }
+
+  private static boolean isReportingConfigured() {
+    if (getEnvIntegerWithDefault(KUDU_REPORT_TEST_RESULTS_VAR, 0) == 0) {
+      return false;
+    }
+    if (!areRequiredReportingVarsSetAndNonEmpty()) {
+      throw new IllegalStateException("Not all required variables are set: " +
+                                      reportingVarDump());
+    }
+    return true;
+  }
+
+  private static String getEnvStringWithDefault(String name,
+                                                String defaultValue) {
+    String value = System.getenv(name);
+    if (value == null || value.isEmpty()) {
+      return defaultValue;
+    }
+    return value;
+  }
+
+  private static int getEnvIntegerWithDefault(String name, int defaultValue) {
+    return Integer.parseInt(getEnvStringWithDefault(
+        name, String.valueOf(defaultValue)));
+  }
+
+  /**
+   * Invokes the `hostname` UNIX utility to retrieve the machine's hostname.
+   *
+   * Note: this is not the same as InetAddress.getLocalHost().getHostName(),
+   * which performs a reverse DNS lookup and may return a different result,
+   * depending on the machine's networking configuration. The equivalent C++
+   * code uses `hostname`, so it's important we do the same here for parity.
+   *
+   * @returns the local hostname
+   */
+  @InterfaceAudience.LimitedPrivate("Test")
+  static String getLocalHostname() {
+    ProcessBuilder pb = new ProcessBuilder("hostname");
+    try {
+      Process p = pb.start();
+      try (InputStreamReader isr = new InputStreamReader(p.getInputStream(), UTF_8);
+           BufferedReader br = new BufferedReader(isr)) {
+        int rv = p.waitFor();
+        if (rv != 0) {
+          throw new IllegalStateException(String.format(
+              "Process 'hostname' exited with exit status %d", rv));
+        }
+        return br.readLine();
+      }
+    } catch (InterruptedException | IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Reports a test result to the flaky test server.
+   *
+   * @param testName the display name of the JUnit test
+   * @param result success or failure
+   * @param logFile optionally, file containing log messages generated by the test
+   * @throws IOException if test reporting failed
+   */
+  public void reportResult(String testName, Result result, File logFile)
+      throws IOException {
+    if (!options.reportResults) return;
+
+    try (CloseableHttpClient client = HttpClients.createDefault()) {
+      HttpPost post = new HttpPost("http://" + options.httpEndpoint + "/add_result");
+
+      // Set up the request with all form parts.
+      MultipartEntityBuilder meb = MultipartEntityBuilder.create();
+      // In the backend, the BUILD_TAG field is called 'build_id', but we can't use
+      // that as an env variable because it'd collide with Jenkins' BUILD_ID.
+      meb.addTextBody("build_id", options.buildTag);
+      meb.addTextBody("hostname", options.hostname);
+      meb.addTextBody("revision", options.revision);
+      meb.addTextBody("build_config", options.buildConfig);
+      meb.addTextBody("test_name", testName);
+      // status=0 indicates success, status=1 indicates failure.
+      meb.addTextBody("status", Integer.toString(result == Result.SUCCESS ? 0 : 1));
+      if (logFile != null) {
+        meb.addBinaryBody("log", logFile, ContentType.APPLICATION_OCTET_STREAM,
+                          testName + ".txt.gz");
+      }
+      post.setEntity(meb.build());
+
+      // Send the request and process the response.
+      try (CloseableHttpResponse resp = client.execute(post)) {
+        StatusLine sl = resp.getStatusLine();
+        if (sl.getStatusCode() != 200) {
+          throw new IOException("Bad response from server: " + sl.getStatusCode() + ": " +
+                                EntityUtils.toString(resp.getEntity(), UTF_8));
+        }
+      }
+    }
+  }
+
+  /**
+   * Same as {@link #reportResult(String, Result)} but never throws an exception.
+   * Logs a warning message on failure.
+   */
+  public void tryReportResult(String testName, Result result, File logFile) {
+    try {
+      reportResult(testName, result, logFile);
+    } catch (IOException ex) {
+      LOG.warn("Failed to record test result for {} as {}", testName, result, ex);
+    }
+  }
+
+  /**
+   * @return whether result reporting is enabled for this reporter
+   */
+  public boolean isReportingEnabled() {
+    return options.reportResults;
+  }
+}
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
index cf1bab7..31d8a95 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/junit/RetryRule.java
@@ -16,6 +16,7 @@
 // under the License.
 package org.apache.kudu.test.junit;
 
+import org.apache.kudu.test.CapturingToFileLogAppender;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.junit.rules.TestRule;
@@ -23,32 +24,43 @@ import org.junit.runner.Description;
 import org.junit.runners.model.Statement;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
 
 /**
- * A JUnit rule to retry failed tests.
- * We use this with Gradle because it doesn't support
- * Surefire/Failsafe rerunFailingTestsCount like Maven does. We use the system
- * property rerunFailingTestsCount to mimic the maven arguments closely.
+ * JUnit rule to retry failed tests.
+ *
+ * The number of retries is controlled by the "rerunFailingTestsCount" system
+ * property, mimicking Surefire in that regard.
+ *
+ * By default will use ResultReporter to report success/failure of each test
+ * attempt to an external server; this may be skipped if desired.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class RetryRule implements TestRule {
 
+  private static final String RETRY_PROP = "rerunFailingTestsCount";
+
   private static final Logger LOG = LoggerFactory.getLogger(RetryRule.class);
+
   private final int retryCount;
+  private final ResultReporter reporter;
 
   public RetryRule() {
-    this(Integer.getInteger("rerunFailingTestsCount", 0));
+    this(Integer.getInteger(RETRY_PROP, 0), /*skipReporting=*/ false);
   }
 
-  // Visible for testing.
-  RetryRule(int retryCount) {
+  @InterfaceAudience.LimitedPrivate("Test")
+  RetryRule(int retryCount, boolean skipReporting) {
     this.retryCount = retryCount;
+    this.reporter = skipReporting ? null : new ResultReporter();
   }
 
   @Override
   public Statement apply(Statement base, Description description) {
-    return new RetryStatement(base, description, retryCount);
+    return new RetryStatement(base, description, retryCount, reporter);
   }
 
   private static class RetryStatement extends Statement {
@@ -56,11 +68,57 @@ public class RetryRule implements TestRule {
     private final Statement base;
     private final Description description;
     private final int retryCount;
+    private final ResultReporter reporter;
+    private final String humanReadableTestName;
 
-    RetryStatement(Statement base, Description description, int retryCount) {
+    RetryStatement(Statement base, Description description,
+                   int retryCount, ResultReporter reporter) {
       this.base = base;
       this.description = description;
       this.retryCount = retryCount;
+      this.reporter = reporter;
+      this.humanReadableTestName = description.getClassName() + "." + description.getMethodName();
+    }
+
+    private void report(ResultReporter.Result result, File logFile) {
+      reporter.tryReportResult(humanReadableTestName, result, logFile);
+    }
+
+    private void doOneAttemptAndReport(int attempt) throws Throwable {
+      try (CapturingToFileLogAppender capturer =
+           new CapturingToFileLogAppender(/*useGzip=*/ true)) {
+        try {
+          try (Closeable c = capturer.attach()) {
+            base.evaluate();
+          }
+
+          // The test succeeded.
+          //
+          // We skip the file upload; this saves space and network bandwidth,
+          // and we don't need the logs of successful tests.
+          report(ResultReporter.Result.SUCCESS, /*logFile=*/ null);
+          return;
+        } catch (Throwable t) {
+          // The test failed.
+          //
+          // Before reporting, capture the failing exception too.
+          try (Closeable c = capturer.attach()) {
+            LOG.error("{}: failed attempt {}", humanReadableTestName, attempt, t);
+          }
+          capturer.finish();
+          report(ResultReporter.Result.FAILURE, capturer.getOutputFile());
+          throw t;
+        }
+      }
+    }
+
+    private void doOneAttempt(int attempt) throws Throwable {
+      try {
+        base.evaluate();
+      } catch (Throwable t) {
+        LOG.error("{}: failed attempt {}", humanReadableTestName, attempt, t);
+        throw t;
+      }
     }
 
     @Override
@@ -70,17 +128,17 @@ public class RetryRule implements TestRule {
       do {
         attempt++;
         try {
-          base.evaluate();
+          if (reporter != null && reporter.isReportingEnabled()) {
+            doOneAttemptAndReport(attempt);
+          } else {
+            doOneAttempt(attempt);
+          }
           return;
-
         } catch (Throwable t) {
-          // To retry, we catch the exception from evaluate(), log an error, and loop.
-          // We retain and rethrow the last failure if all attempts fail.
           lastException = t;
-          LOG.error("{}: failed attempt {}", description.getDisplayName(), attempt, t);
         }
       } while (attempt <= retryCount);
-      LOG.error("{}: giving up after {} attempts", description.getDisplayName(), attempt);
+      LOG.error("{}: giving up after {} attempts", humanReadableTestName, attempt);
       throw lastException;
     }
   }
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java
new file mode 100644
index 0000000..bad2aad
--- /dev/null
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestResultReporter.java
@@ -0,0 +1,203 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.kudu.test.junit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.io.IOUtils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.MultipartConfigElement;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.Part;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/** Unit test for ResultReporter. */
+public class TestResultReporter {
+
+  /** Record of a specific test run. */
+  private static class TestRecord {
+    public String testName;
+    public String buildTag;
+    public String revision;
+    public String hostname;
+    public String buildConfig;
+    public int status;
+    public String log;
+
+    public TestRecord(Map<String, String> params) {
+      testName = params.get("test_name");
+      buildTag = params.get("build_id");
+      revision = params.get("revision");
+      hostname = params.get("hostname");
+      buildConfig = params.get("build_config");
+      status = Integer.parseInt(params.get("status"));
+      log = params.get("log");
+    }
+
+    @Override
+    public String toString() {
+      List<String> required = ImmutableList.of(
+          testName, buildTag, revision, hostname, buildConfig, Integer.toString(status));
+      List<String> all = new ArrayList<>(required);
+      if (log != null) {
+        all.add(log);
+      }
+      return Joiner.on(" ").join(all);
+    }
+  }
+
+  /**
+   * Mock implementation of the flaky test server.
+   *
+   * Must be a servlet (not just a Jetty handler) to support multipart forms.
+   */
+  private static class MockFlakyTestServlet extends HttpServlet {
+    private static final Logger LOG = LoggerFactory.getLogger(MockFlakyTestServlet.class);
+    private final List<TestRecord> records = new ArrayList<>();
+
+    public List<TestRecord> getRecords() {
+      return records;
+    }
+
+    @Override
+    protected void doPost(HttpServletRequest request,
+                          HttpServletResponse response) throws IOException, ServletException {
+      LOG.debug("Handling request {}: ", request);
+
+      // Process the form parts into key/value pairs.
+      Map<String, String> params = new HashMap<>();
+      for (Part p : request.getParts()) {
+        params.put(p.getName(), IOUtils.toString(p.getInputStream(), UTF_8));
+      }
+
+      // We're done processing the request.
+      records.add(new TestRecord(params));
+      response.setContentType("text/html; charset=utf-8");
+      response.setStatus(HttpServletResponse.SC_OK);
+    }
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(TestResultReporter.class);
+  private static final String BIND_ADDR = "127.0.0.1";
+  private Server server;
+  private MockFlakyTestServlet flakyTestServlet;
+
+  @Before
+  public void setup() throws Exception {
+    flakyTestServlet = new MockFlakyTestServlet();
+
+    // This Enterprise Java nonsense is to enable multipart form submission. The
+    // servlet is configured to only spill parts to disk if they exceed 1 MB in
+    // size, which isn't a concern for this test.
+    ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+    context.setContextPath("/");
+    ServletHolder holder = new ServletHolder(flakyTestServlet);
+    holder.getRegistration().setMultipartConfig(new MultipartConfigElement(
+        "",            // location
+        1024 * 1024,   // maxFileSize
+        1024 * 1024,   // maxRequestSize
+        1024 * 1024)); // fileSizeThreshold
+    context.addServlet(holder, "/*");
+
+    server = new Server(new InetSocketAddress(BIND_ADDR, 0));
+    server.setHandler(context);
+    server.start();
+  }
+
+  @After
+  public void teardown() throws Exception {
+    server.stop();
+    server.join();
+  }
+
+  @Test
+  public void testRoundTrip() throws IOException {
+    ResultReporter.Options options = new ResultReporter.Options();
+    assertNotNull(server);
+    assertTrue(server.isStarted());
+    assertNotNull(server.getURI());
+    options.httpEndpoint(BIND_ADDR + ":" + server.getURI().getPort())
+           .buildTag("shark")
+           .revision("do")
+           .hostname("do-do")
+           .buildConfig("do-do-do");
+    ResultReporter.Result[] expectedResults = {
+        ResultReporter.Result.SUCCESS, ResultReporter.Result.FAILURE };
+    String[] testNames = { "baby", "mommy", "daddy"};
+    String logFormat = "%s: a log message";
+    ResultReporter reporter = new ResultReporter(options);
+    int expectedRecords = 0;
+    for (ResultReporter.Result result : expectedResults) {
+      for (String testName : testNames) {
+        File tempLogFile = null;
+        if (result == ResultReporter.Result.FAILURE) {
+          tempLogFile = File.createTempFile("test_log", ".txt");
+          tempLogFile.deleteOnExit();
+          FileOutputStream fos = new FileOutputStream(tempLogFile);
+          IOUtils.write(String.format(logFormat, testName), fos, UTF_8);
+        }
+        reporter.reportResult(testName, result, tempLogFile);
+        expectedRecords++;
+      }
+    }
+    assertEquals(expectedRecords, flakyTestServlet.getRecords().size());
+    Iterator<TestRecord> iterator = flakyTestServlet.getRecords().iterator();
+    for (ResultReporter.Result result : expectedResults) {
+      for (String testName : testNames) {
+        assertTrue(iterator.hasNext());
+        TestRecord record = iterator.next();
+        LOGGER.info(record.toString());
+        assertEquals(testName, record.testName);
+        assertEquals(result == ResultReporter.Result.SUCCESS ? 0 : 1, record.status);
+        assertEquals(result == ResultReporter.Result.FAILURE ?
+                     String.format(logFormat, testName) : null, record.log);
+      }
+    }
+  }
+
+  @Test
+  public void testHostName() {
+    // Just tests that this doesn't throw an exception.
+    LOGGER.info(ResultReporter.getLocalHostname());
+  }
+}
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestRetryRule.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestRetryRule.java
index 92693db..b5a3d29 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestRetryRule.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/junit/TestRetryRule.java
@@ -29,8 +29,9 @@ public class TestRetryRule {
   // an assertion exception.
   private int failures = 0;
 
+  // We skip flaky test reporting for this test because it is designed to fail.
   @Rule
-  public RetryRule retryRule = new RetryRule(MAX_FAILURES);
+  public RetryRule retryRule = new RetryRule(MAX_FAILURES, /*skipReporting=*/ true);
 
   // Ensure that the RetryRule prevents test failures as long as we don't exceed MAX_FAILURES
   // failures.