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

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

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();