You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2018/07/09 19:22:07 UTC

[1/2] kudu git commit: KUDU-2420: Support parallel java tests

Repository: kudu
Updated Branches:
  refs/heads/master 32fabd3a4 -> c72711832


KUDU-2420: Support parallel java tests

Makes the changes neccessary to allow for parallel java tests.
- Ensures each MiniKuduCluster uses its own cluster root.
- Replaces usage of @BeforeClass and @AfterClass with
  @Before and @After ensuring tests are isolated.
- Fixes test usage of BaseKuduTest and its configurations.
- Replaces usage of BeforeAndAfterAll with BeforeAndAfterEach.

I tested this with `gradle test -PmaxParallelForks=8` and all tests passed locally.

Change-Id: I97c6718b022936a8839f957da0928f54ff6e7371
Reviewed-on: http://gerrit.cloudera.org:8080/10838
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: eb9c6442778d22eb2e6137fe0723d832b5cbedcb
Parents: 32fabd3
Author: Grant Henke <gr...@apache.org>
Authored: Wed Jun 27 13:06:16 2018 -0500
Committer: Grant Henke <gr...@apache.org>
Committed: Mon Jul 9 19:20:06 2018 +0000

----------------------------------------------------------------------
 java/gradle/tests.gradle                        |   1 -
 .../kudu/mapreduce/tools/ITExportCsv.java       |  21 +---
 .../kudu/mapreduce/tools/ITImportCsv.java       |  22 ++--
 .../kudu/mapreduce/tools/ITImportParquet.java   |  36 +++---
 .../tools/ITImportParquetPreCheck.java          |  42 +++----
 .../tools/ITIntegrationTestBigLinkedList.java   |  12 +-
 .../kudu/mapreduce/tools/ITRowCounter.java      |  18 +--
 .../org/apache/kudu/client/BaseKuduTest.java    | 115 +++++++------------
 .../java/org/apache/kudu/client/ITClient.java   |  12 +-
 .../kudu/client/ITScannerMultiTablet.java       |  14 +--
 .../org/apache/kudu/client/MiniKuduCluster.java |  33 +++++-
 .../org/apache/kudu/client/TestAlterTable.java  |   4 +-
 .../apache/kudu/client/TestAsyncKuduClient.java |   5 -
 .../kudu/client/TestAsyncKuduSession.java       |  18 ++-
 .../kudu/client/TestAuthnTokenReacquire.java    |  13 +--
 .../client/TestAuthnTokenReacquireOpen.java     |  24 ++--
 .../kudu/client/TestClientFailoverSupport.java  |  13 ---
 .../apache/kudu/client/TestHandleTooBusy.java   |  21 ++--
 .../org/apache/kudu/client/TestHybridTime.java  |  22 ++--
 .../org/apache/kudu/client/TestKuduClient.java  |  14 +--
 .../org/apache/kudu/client/TestKuduSession.java |   7 +-
 .../org/apache/kudu/client/TestKuduTable.java   |  14 +--
 .../apache/kudu/client/TestLeaderFailover.java  |   8 +-
 .../apache/kudu/client/TestMasterFailover.java  |   5 -
 .../kudu/client/TestMultipleLeaderFailover.java |   6 -
 .../org/apache/kudu/client/TestRowErrors.java   |   7 --
 .../org/apache/kudu/client/TestRowResult.java   |   7 +-
 .../org/apache/kudu/client/TestScanToken.java   |   7 +-
 .../kudu/client/TestScannerMultiTablet.java     |  24 ++--
 .../org/apache/kudu/client/TestSecurity.java    |   7 +-
 .../client/TestSecurityContextRealUser.java     |  44 +++----
 .../org/apache/kudu/client/TestStatistics.java  |   7 +-
 .../sink/AvroKuduOperationsProducerTest.java    |  14 +--
 .../apache/kudu/mapreduce/ITInputFormatJob.java |  20 +---
 .../kudu/mapreduce/ITKuduTableOutputFormat.java |   6 -
 .../kudu/mapreduce/ITOutputFormatJob.java       |  19 ++-
 .../apache/kudu/mapreduce/TestJarFinder.java    |  12 +-
 .../kudu/spark/kudu/DefaultSourceTest.scala     |  10 +-
 .../apache/kudu/spark/kudu/TestContext.scala    |  10 +-
 39 files changed, 270 insertions(+), 424 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/gradle/tests.gradle
----------------------------------------------------------------------
diff --git a/java/gradle/tests.gradle b/java/gradle/tests.gradle
index 901ea6e..f826c2b 100644
--- a/java/gradle/tests.gradle
+++ b/java/gradle/tests.gradle
@@ -109,7 +109,6 @@ task integrationTest(type: Test, group: "Verification") {
   description = "Runs the integration tests."
   testClassesDirs = sourceSets.integrationTest.output.classesDirs
   classpath = sourceSets.integrationTest.runtimeClasspath
-  maxParallelForks = 1
   mustRunAfter test
 }
 check.dependsOn(integrationTest)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITExportCsv.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITExportCsv.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITExportCsv.java
index f284205..3fa4b83 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITExportCsv.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITExportCsv.java
@@ -29,11 +29,9 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.GenericOptionsParser;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
 import org.junit.Test;
 
-import org.apache.kudu.Schema;
 import org.apache.kudu.client.BaseKuduTest;
 import org.apache.kudu.mapreduce.CommandLineParser;
 import org.apache.kudu.mapreduce.HadoopTestingUtility;
@@ -45,20 +43,9 @@ public class ITExportCsv extends BaseKuduTest {
 
   private static final HadoopTestingUtility HADOOP_UTIL = new HadoopTestingUtility();
 
-  private static Schema schema;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportCsv.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportCsv.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportCsv.java
index 503485b..0cc0740 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportCsv.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportCsv.java
@@ -32,8 +32,8 @@ import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.GenericOptionsParser;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.ColumnSchema;
@@ -54,10 +54,7 @@ public class ITImportCsv extends BaseKuduTest {
 
   private static Schema schema;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
+  static {
     ArrayList<ColumnSchema> columns = new ArrayList<ColumnSchema>(4);
     columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32)
         .key(true)
@@ -72,18 +69,17 @@ public class ITImportCsv extends BaseKuduTest {
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column4_b", Type.BOOL)
         .build());
     schema = new Schema(columns);
+  }
 
+  @Before
+  public void setUp() throws Exception {
     createTable(TABLE_NAME, schema,
                 new CreateTableOptions().setRangePartitionColumns(ImmutableList.of("key")));
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquet.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquet.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquet.java
index 632cee2..bcf7ddf 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquet.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquet.java
@@ -39,8 +39,8 @@ import org.apache.parquet.example.data.simple.SimpleGroupFactory;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.example.GroupWriteSupport;
 import org.apache.parquet.schema.MessageType;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.ColumnSchema;
@@ -60,36 +60,32 @@ public class ITImportParquet extends BaseKuduTest {
 
   private static Schema schema;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
+  static {
     ArrayList<ColumnSchema> columns = new ArrayList<ColumnSchema>(4);
     columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32)
-      .key(true)
-      .build());
+        .key(true)
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column1_i", Type.INT32)
-      .build());
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column2_d", Type.DOUBLE)
-      .build());
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column3_s", Type.STRING)
-      .nullable(true)
-      .build());
+        .nullable(true)
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column4_b", Type.BOOL)
-      .build());
+        .build());
     schema = new Schema(columns);
+  }
 
+  @Before
+  public void setUp() throws Exception {
     createTable(TABLE_NAME, schema,
       new CreateTableOptions().setRangePartitionColumns(ImmutableList.of("key")));
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquetPreCheck.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquetPreCheck.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquetPreCheck.java
index 784aeb8..bf52972 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquetPreCheck.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITImportParquetPreCheck.java
@@ -37,8 +37,8 @@ import org.apache.parquet.example.data.simple.SimpleGroupFactory;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.example.GroupWriteSupport;
 import org.apache.parquet.schema.MessageType;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -60,39 +60,35 @@ public class ITImportParquetPreCheck extends BaseKuduTest {
 
   private static Schema schema;
 
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
+  static {
     ArrayList<ColumnSchema> columns = new ArrayList<ColumnSchema>(4);
     columns.add(new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32)
-      .key(true)
-      .build());
+        .key(true)
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column1_i", Type.INT32)
-      .build());
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column2_d", Type.DOUBLE)
-      .build());
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column3_s", Type.STRING)
-      .nullable(true)
-      .build());
+        .nullable(true)
+        .build());
     columns.add(new ColumnSchema.ColumnSchemaBuilder("column4_b", Type.BOOL)
-      .build());
+        .build());
     schema = new Schema(columns);
+  }
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
     createTable(TABLE_NAME, schema,
       new CreateTableOptions().setRangePartitionColumns(ImmutableList.of("key")));
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITIntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITIntegrationTestBigLinkedList.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITIntegrationTestBigLinkedList.java
index 56fb813..1cfb8c4 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITIntegrationTestBigLinkedList.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITIntegrationTestBigLinkedList.java
@@ -19,7 +19,7 @@ package org.apache.kudu.mapreduce.tools;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -31,13 +31,9 @@ public class ITIntegrationTestBigLinkedList extends BaseKuduTest {
 
   private static final HadoopTestingUtility HADOOP_UTIL = new HadoopTestingUtility();
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITRowCounter.java
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITRowCounter.java b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITRowCounter.java
index b94d62f..7f62753 100644
--- a/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITRowCounter.java
+++ b/java/kudu-client-tools/src/test/java/org/apache/kudu/mapreduce/tools/ITRowCounter.java
@@ -24,8 +24,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.GenericOptionsParser;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
 import org.junit.Test;
 
 import org.apache.kudu.client.BaseKuduTest;
@@ -39,18 +38,9 @@ public class ITRowCounter extends BaseKuduTest {
 
   private static final HadoopTestingUtility HADOOP_UTIL = new HadoopTestingUtility();
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java b/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
index cd28736..9078085 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
@@ -27,10 +27,8 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.base.Stopwatch;
 import com.google.common.net.HostAndPort;
 import com.stumbleupon.async.Deferred;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.rules.TestName;
+import org.junit.After;
+import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,35 +45,42 @@ public class BaseKuduTest {
   // to complete, etc.
   protected static final int DEFAULT_SLEEP = 50000;
 
-  private static final Random randomForTSRestart = TestUtils.getRandom();
+  private final Random randomForTSRestart = TestUtils.getRandom();
 
-  protected static MiniKuduCluster miniCluster;
+  private static final int NUM_MASTERS = 3;
+  private static final int NUM_TABLET_SERVERS = 3;
 
-  // Expose the MiniKuduCluster builder so that subclasses can alter the builder.
-  protected static final MiniKuduCluster.MiniKuduClusterBuilder miniClusterBuilder =
-      new MiniKuduCluster.MiniKuduClusterBuilder();
+  protected MiniKuduCluster miniCluster;
 
   // Comma separate describing the master addresses and ports.
-  protected static String masterAddresses;
-  protected static List<HostAndPort> masterHostPorts;
+  protected String masterAddresses;
+  protected List<HostAndPort> masterHostPorts;
 
   // We create both versions of the client for ease of use.
-  protected static AsyncKuduClient client;
-  protected static KuduClient syncClient;
+  protected AsyncKuduClient client;
+  protected KuduClient syncClient;
   protected static final Schema basicSchema = getBasicSchema();
   protected static final Schema allTypesSchema = getSchemaWithAllTypes();
 
-  @Rule
-  public TestName testName = new TestName();
+  @Before
+  public void setUpBase() throws Exception {
+    FakeDNS.getInstance().install();
+
+    LOG.info("Creating a new MiniKuduCluster...");
+
+    miniCluster = getMiniClusterBuilder().build();
+    masterAddresses = miniCluster.getMasterAddresses();
+    masterHostPorts = miniCluster.getMasterHostPorts();
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    LOG.info("Setting up before class...");
-    doSetup(Integer.getInteger("NUM_MASTERS", 3), 3);
+    LOG.info("Creating a new Kudu client...");
+    client = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
+        .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
+        .build();
+    syncClient = client.syncClient();
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
+  @After
+  public void tearDownBase() throws Exception {
     try {
       if (client != null) {
         syncClient.shutdown();
@@ -90,50 +95,16 @@ public class BaseKuduTest {
   }
 
   /**
-   * This method is intended to be called from custom @BeforeClass method to setup Kudu mini cluster
-   * with the specified parameters. The #BaseKuduTest class calls it in its @BeforeClass method
-   * with the default parameters.
-   *
-   * @param numMasters number of masters in the cluster to start
-   * @param numTabletServers number of tablet servers in the cluster to start
-   * @throws Exception if something goes wrong
-   */
-  protected static void doSetup(int numMasters, int numTabletServers)
-      throws Exception {
-    FakeDNS.getInstance().install();
-
-    miniCluster = miniClusterBuilder
-        .numMasters(numMasters)
-        .numTservers(numTabletServers)
-        .build();
-    masterAddresses = miniCluster.getMasterAddresses();
-    masterHostPorts = miniCluster.getMasterHostPorts();
-
-    LOG.info("Creating new Kudu client...");
-    client = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
-        .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
-        .build();
-    syncClient = client.syncClient();
-  }
-
-  /**
-   * Returns the method name of the currently-running JUnit test.
-   * @return a test method name
-   */
-  protected String getTestMethodName() {
-    return testName.getMethodName();
-  }
-
-  /**
-   * Returns the method name of the currently-running JUnit test with a concatenated millisecond
-   * timestamp. Useful for table names in tests that are automatically retried.
-   * @return a test method name with a millisecond timestamp appended
+   * Returns a MiniKuduClusterBuilder to use when starting the MiniKuduCluster.
+   * Override this method to adjust to the MiniKuduClusterBuilder settings.
    */
-  protected String getTestMethodNameWithTimestamp() {
-    return getTestMethodName() + "-" + System.currentTimeMillis();
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
+    return new MiniKuduCluster.MiniKuduClusterBuilder()
+        .numMasters(NUM_MASTERS)
+        .numTservers(NUM_TABLET_SERVERS);
   }
 
-  protected static KuduTable createTable(String tableName, Schema schema,
+  protected KuduTable createTable(String tableName, Schema schema,
                                          CreateTableOptions builder) throws KuduException {
     LOG.info("Creating table: {}", tableName);
     return client.syncClient().createTable(tableName, schema, builder);
@@ -145,7 +116,7 @@ public class BaseKuduTest {
    * @return A KuduTable
    * @throws Exception MasterErrorException if the table doesn't exist
    */
-  protected static KuduTable openTable(String name) throws Exception {
+  protected KuduTable openTable(String name) throws Exception {
     Deferred<KuduTable> d = client.openTable(name);
     return d.join(DEFAULT_SLEEP);
   }
@@ -159,7 +130,7 @@ public class BaseKuduTest {
    * @param table a KuduTable which will get its single tablet's leader killed.
    * @throws Exception
    */
-  protected static void killTabletLeader(KuduTable table) throws Exception {
+  protected void killTabletLeader(KuduTable table) throws Exception {
     List<LocatedTablet> tablets = table.getTabletsLocations(DEFAULT_SLEEP);
     if (tablets.isEmpty() || tablets.size() > 1) {
       fail("Currently only support killing leaders for tables containing 1 tablet, table " +
@@ -183,7 +154,7 @@ public class BaseKuduTest {
    * @param tablet a RemoteTablet which will get its leader killed
    * @throws Exception
    */
-  protected static void killTabletLeader(RemoteTablet tablet) throws Exception {
+  protected void killTabletLeader(RemoteTablet tablet) throws Exception {
     HostAndPort hp = findLeaderTabletServerHostPort(new LocatedTablet(tablet));
     miniCluster.killTabletServerOnHostPort(hp);
   }
@@ -194,7 +165,7 @@ public class BaseKuduTest {
    * @return the host and port of the given tablet's leader tserver
    * @throws Exception if we are unable to find the leader tserver
    */
-  protected static HostAndPort findLeaderTabletServerHostPort(LocatedTablet tablet)
+  protected HostAndPort findLeaderTabletServerHostPort(LocatedTablet tablet)
       throws Exception {
     LocatedTablet.Replica leader = null;
     DeadlineTracker deadlineTracker = new DeadlineTracker();
@@ -220,7 +191,7 @@ public class BaseKuduTest {
    * This method is thread-safe.
    * @throws Exception if there is an error finding or killing the leader master.
    */
-  protected static void killMasterLeader() throws Exception {
+  protected void killMasterLeader() throws Exception {
     HostAndPort hp = findLeaderMasterHostPort();
     miniCluster.killMasterOnHostPort(hp);
   }
@@ -230,7 +201,7 @@ public class BaseKuduTest {
    * @return the host and port of the leader master
    * @throws Exception if we are unable to find the leader master
    */
-  protected static HostAndPort findLeaderMasterHostPort() throws Exception {
+  protected HostAndPort findLeaderMasterHostPort() throws Exception {
     Stopwatch sw = Stopwatch.createStarted();
     while (sw.elapsed(TimeUnit.MILLISECONDS) < DEFAULT_SLEEP) {
       Deferred<Master.GetTableLocationsResponsePB> masterLocD =
@@ -252,7 +223,7 @@ public class BaseKuduTest {
    * @param table table to query for a TS to restart
    * @throws Exception
    */
-  protected static void restartTabletServer(KuduTable table) throws Exception {
+  protected void restartTabletServer(KuduTable table) throws Exception {
     List<LocatedTablet> tablets = table.getTabletsLocations(DEFAULT_SLEEP);
     if (tablets.isEmpty()) {
       fail("Table " + table.getName() + " doesn't have any tablets");
@@ -270,7 +241,7 @@ public class BaseKuduTest {
    * @param tablet a RemoteTablet which will get its leader killed and restarted
    * @throws Exception
    */
-  protected static void restartTabletServer(RemoteTablet tablet) throws Exception {
+  protected void restartTabletServer(RemoteTablet tablet) throws Exception {
     HostAndPort hp = findLeaderTabletServerHostPort(new LocatedTablet(tablet));
     miniCluster.killTabletServerOnHostPort(hp);
     miniCluster.restartDeadTabletServerOnHostPort(hp);
@@ -280,7 +251,7 @@ public class BaseKuduTest {
    * Kills and restarts the leader master.
    * @throws Exception
    */
-  protected static void restartLeaderMaster() throws Exception {
+  protected void restartLeaderMaster() throws Exception {
     HostAndPort hp = findLeaderMasterHostPort();
     miniCluster.killMasterOnHostPort(hp);
     miniCluster.restartDeadMasterOnHostPort(hp);
@@ -291,7 +262,7 @@ public class BaseKuduTest {
    * config for this cluster.
    * @return The master config string.
    */
-  protected static String getMasterAddresses() {
+  protected String getMasterAddresses() {
     return masterAddresses;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
index e0e645b..b6d559c 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableList;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -65,8 +65,8 @@ public class ITClient extends BaseKuduTest {
 
   private volatile long sharedWriteTimestamp;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  @Before
+  public void setUp() throws Exception {
 
     String runtimeProp = System.getProperty(RUNTIME_PROPERTY_NAME);
     runtimeInSeconds = runtimeProp == null ? DEFAULT_RUNTIME_SECONDS : Long.parseLong(runtimeProp);
@@ -78,8 +78,6 @@ public class ITClient extends BaseKuduTest {
 
     LOG.info ("Test running for {} seconds", runtimeInSeconds);
 
-    BaseKuduTest.setUpBeforeClass();
-
     // Client we're using has low tolerance for read timeouts but a
     // higher overall operation timeout.
     localAsyncClient = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
@@ -206,7 +204,7 @@ public class ITClient extends BaseKuduTest {
      */
     private boolean restartTS() {
       try {
-        BaseKuduTest.restartTabletServer(table);
+        restartTabletServer(table);
       } catch (Exception e) {
         reportError("Couldn't restart a TS", e);
         return false;
@@ -220,7 +218,7 @@ public class ITClient extends BaseKuduTest {
      */
     private boolean restartMaster() {
       try {
-        BaseKuduTest.restartLeaderMaster();
+        restartLeaderMaster();
       } catch (Exception e) {
         reportError("Couldn't restart a master", e);
         return false;

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/ITScannerMultiTablet.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/ITScannerMultiTablet.java b/java/kudu-client/src/test/java/org/apache/kudu/client/ITScannerMultiTablet.java
index 1fee25d..dbaaefd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/ITScannerMultiTablet.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/ITScannerMultiTablet.java
@@ -23,8 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Random;
 
 import com.google.common.collect.Lists;
-import org.junit.After;
-import org.junit.BeforeClass;
+import org.junit.Before;
 
 import org.apache.kudu.Schema;
 
@@ -44,10 +43,8 @@ public class ITScannerMultiTablet extends BaseKuduTest {
 
   private static Random random = new Random(1234);
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
+  @Before
+  public void setUp() throws Exception {
     CreateTableOptions builder = new CreateTableOptions();
 
     builder.addHashPartitions(
@@ -75,11 +72,6 @@ public class ITScannerMultiTablet extends BaseKuduTest {
     assertEquals(0, session.countPendingErrors());
   }
 
-  @After
-  public void tearDown() throws Exception {
-    restartTabletServers();
-  }
-
   /**
    * Injecting failures (kill or restart TabletServer) while scanning, to verify:
    * fault tolerant scanner will continue scan and non-fault tolerant scanner will throw

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
index 2791b25..f922d25 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
@@ -19,9 +19,12 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -90,6 +93,7 @@ public class MiniKuduCluster implements AutoCloseable {
   private final int numTservers;
   private final ImmutableList<String> extraTserverFlags;
   private final ImmutableList<String> extraMasterFlags;
+  private final String clusterRoot;
 
   private MiniKdcOptionsPB kdcOptionsPb;
 
@@ -98,13 +102,27 @@ public class MiniKuduCluster implements AutoCloseable {
       int numTservers,
       List<String> extraTserverFlags,
       List<String> extraMasterFlags,
-      MiniKdcOptionsPB kdcOptionsPb) {
+      MiniKdcOptionsPB kdcOptionsPb,
+      String clusterRoot) {
     this.enableKerberos = enableKerberos;
     this.numMasters = numMasters;
     this.numTservers = numTservers;
     this.extraTserverFlags = ImmutableList.copyOf(extraTserverFlags);
     this.extraMasterFlags = ImmutableList.copyOf(extraMasterFlags);
     this.kdcOptionsPb = kdcOptionsPb;
+
+    if (clusterRoot == null) {
+      // If a cluster root was not set, create a  unique temp directory to use.
+      // The mini cluster will clean this directory up on exit.
+      try {
+        File tempRoot = Files.createTempDirectory("mini-kudu-cluster").toFile();
+        this.clusterRoot = tempRoot.toString();
+      } catch (IOException ex) {
+        throw new RuntimeException("Could not create cluster root directory", ex);
+      }
+    } else {
+      this.clusterRoot = clusterRoot;
+    }
   }
 
   /**
@@ -176,6 +194,7 @@ public class MiniKuduCluster implements AutoCloseable {
             .addAllExtraMasterFlags(extraMasterFlags)
             .addAllExtraTserverFlags(extraTserverFlags)
             .setMiniKdcOptions(kdcOptionsPb)
+            .setClusterRoot(clusterRoot)
             .build())
         .build());
     sendRequestToCluster(
@@ -496,6 +515,7 @@ public class MiniKuduCluster implements AutoCloseable {
     private boolean enableKerberos = false;
     private final List<String> extraTserverFlags = new ArrayList<>();
     private final List<String> extraMasterFlags = new ArrayList<>();
+    private String clusterRoot = null;
 
     private MiniKdcOptionsPB.Builder kdcOptionsPb =
         MiniKdcOptionsPB.newBuilder();
@@ -548,6 +568,15 @@ public class MiniKuduCluster implements AutoCloseable {
     }
 
     /**
+     * Sets the directory where the cluster's data and logs should be placed.
+     * @return this instance
+     */
+    public MiniKuduClusterBuilder clusterRoot(String clusterRoot) {
+      this.clusterRoot = clusterRoot;
+      return this;
+    }
+
+    /**
      * Builds and starts a new {@link MiniKuduCluster} using builder state.
      * @return the newly started {@link MiniKuduCluster}
      * @throws IOException if something went wrong starting the cluster
@@ -557,7 +586,7 @@ public class MiniKuduCluster implements AutoCloseable {
           new MiniKuduCluster(enableKerberos,
               numMasters, numTservers,
               extraTserverFlags, extraMasterFlags,
-              kdcOptionsPb.build());
+              kdcOptionsPb.build(), clusterRoot);
       try {
         cluster.start();
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
index 1680c2e..78fb9c3 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAlterTable.java
@@ -80,7 +80,7 @@ public class TestAlterTable extends BaseKuduTest {
       createOptions.addRangePartition(lower, upper);
     }
 
-    return BaseKuduTest.createTable(tableName, schema, createOptions);
+    return createTable(tableName, schema, createOptions);
   }
 
   /**
@@ -314,7 +314,7 @@ public class TestAlterTable extends BaseKuduTest {
                                     RangePartitionBound.EXCLUSIVE_BOUND,
                                     RangePartitionBound.INCLUSIVE_BOUND);
 
-    KuduTable table = BaseKuduTest.createTable(tableName, schema, createOptions);
+    KuduTable table = createTable(tableName, schema, createOptions);
 
     lower.addInt("c0", 199);
     upper.addInt("c0", 299);

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
index 53ba2fc..4fb7bab 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduClient.java
@@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.base.Stopwatch;
 import com.google.protobuf.ByteString;
 import com.stumbleupon.async.Deferred;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.kudu.ColumnSchema;
@@ -43,10 +42,6 @@ import org.apache.kudu.consensus.Metadata;
 import org.apache.kudu.master.Master;
 
 public class TestAsyncKuduClient extends BaseKuduTest {
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
 
   @Test(timeout = 100000)
   public void testDisconnect() throws Exception {

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
index cf7ee86..81e8946 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAsyncKuduSession.java
@@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.stumbleupon.async.Callback;
 import com.stumbleupon.async.Deferred;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.Schema;
@@ -59,13 +59,11 @@ public class TestAsyncKuduSession extends BaseKuduTest {
   private static Schema schema = getBasicSchema();
   private static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
+  @Before
+  public void setUp() throws Exception {
     table = createTable(TABLE_NAME, schema, getBasicCreateTableOptions());
   }
 
-
   @Test(timeout = 100000)
   public void testBackgroundErrors() throws Exception {
     try {
@@ -483,7 +481,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
     return delete;
   }
 
-  public static boolean exists(final int key) throws Exception {
+  public boolean exists(final int key) throws Exception {
 
     AsyncKuduScanner scanner = getScanner(key, key + 1);
     final AtomicBoolean exists = new AtomicBoolean(false);
@@ -517,7 +515,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
     return exists.get();
   }
 
-  public static int countNullColumns(final int startKey, final int endKey) throws Exception {
+  public int countNullColumns(final int startKey, final int endKey) throws Exception {
 
     AsyncKuduScanner scanner = getScanner(startKey, endKey);
     final AtomicInteger ai = new AtomicInteger();
@@ -546,17 +544,17 @@ public class TestAsyncKuduSession extends BaseKuduTest {
     return ai.get();
   }
 
-  public static int countInRange(final int start, final int exclusiveEnd) throws Exception {
+  public int countInRange(final int start, final int exclusiveEnd) throws Exception {
 
     AsyncKuduScanner scanner = getScanner(start, exclusiveEnd);
     return countRowsInScan(scanner);
   }
 
-  private static AsyncKuduScanner getScanner(int start, int exclusiveEnd) {
+  private AsyncKuduScanner getScanner(int start, int exclusiveEnd) {
     return getScanner(start, exclusiveEnd, null);
   }
 
-  private static AsyncKuduScanner getScanner(int start, int exclusiveEnd,
+  private AsyncKuduScanner getScanner(int start, int exclusiveEnd,
                                              List<String> columnNames) {
 
     PartialRow lowerBound = schema.newPartialRow();

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquire.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquire.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquire.java
index 3a85406..2bbe695 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquire.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquire.java
@@ -32,7 +32,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -45,26 +44,24 @@ public class TestAuthnTokenReacquire extends BaseKuduTest {
   private static final int TOKEN_TTL_SEC = 1;
   private static final int OP_TIMEOUT_MS = 60 * TOKEN_TTL_SEC * 1000;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  @Override
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
     // Inject additional INVALID_AUTHENTICATION_TOKEN responses from both the master and tablet
     // servers, even for not-yet-expired tokens.
-    miniClusterBuilder
+    return super.getMiniClusterBuilder()
         .enableKerberos()
         .addMasterFlag(String.format("--authn_token_validity_seconds=%d", TOKEN_TTL_SEC))
         .addMasterFlag("--rpc_inject_invalid_authn_token_ratio=0.5")
         .addTserverFlag("--rpc_inject_invalid_authn_token_ratio=0.5");
-
-    BaseKuduTest.setUpBeforeClass();
   }
 
-  private static void dropConnections() {
+  private void dropConnections() {
     for (Connection c : client.getConnectionListCopy()) {
       c.disconnect();
     }
   }
 
-  private static void dropConnectionsAndExpireToken() throws InterruptedException {
+  private void dropConnectionsAndExpireToken() throws InterruptedException {
     // Drop all connections from the client to Kudu servers.
     dropConnections();
     // Wait for authn token expiration.

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquireOpen.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquireOpen.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquireOpen.java
index 8e94b76..16dc1aa 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquireOpen.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestAuthnTokenReacquireOpen.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -41,26 +40,23 @@ public class TestAuthnTokenReacquireOpen extends BaseKuduTest {
   private static final int OP_TIMEOUT_MS = 60 * TOKEN_TTL_SEC * 1000;
   private static final int KEEPALIVE_TIME_MS = 2 * OP_TIMEOUT_MS;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Set appropriate TTL for authn token and connection keep-alive property, so the client could
-    // keep an open connection to the master when its authn token is already expired. Inject
-    // additional INVALID_AUTHENTICATION_TOKEN responses from the tablet server even for
-    // not-yet-expired tokens for an extra stress on the client.
-    miniClusterBuilder
+  @Override
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
+    return super.getMiniClusterBuilder()
+        // We want to have a cluster with a single master.
+        .numMasters(1)
+        // Set appropriate TTL for authn token and connection keep-alive property, so the client could
+        // keep an open connection to the master when its authn token is already expired. Inject
+        // additional INVALID_AUTHENTICATION_TOKEN responses from the tablet server even for
+        // not-yet-expired tokens for an extra stress on the client.
         .enableKerberos()
         .addMasterFlag(String.format("--authn_token_validity_seconds=%d", TOKEN_TTL_SEC))
         .addMasterFlag(String.format("--rpc_default_keepalive_time_ms=%d", KEEPALIVE_TIME_MS))
         .addTserverFlag(String.format("--rpc_default_keepalive_time_ms=%d", KEEPALIVE_TIME_MS))
         .addTserverFlag("--rpc_inject_invalid_authn_token_ratio=0.5");
-
-    // We want to have a cluster with a single master.
-    final int NUM_MASTERS = 1;
-    final int NUM_TABLET_SERVERS = 3;
-    doSetup(NUM_MASTERS, NUM_TABLET_SERVERS);
   }
 
-  private static void dropConnections() {
+  private void dropConnections() {
     for (Connection c : client.getConnectionListCopy()) {
       c.disconnect();
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestClientFailoverSupport.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestClientFailoverSupport.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestClientFailoverSupport.java
index 1dd8cf1..315a010 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestClientFailoverSupport.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestClientFailoverSupport.java
@@ -30,7 +30,6 @@ import org.apache.kudu.util.AssertHelpers.BooleanExpression;
 import org.apache.kudu.util.CapturingLogAppender;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.net.HostAndPort;
@@ -44,24 +43,12 @@ public class TestClientFailoverSupport extends BaseKuduTest {
     KILL
   }
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    final int NUM_TABLET_SERVERS = 3;
-    BaseKuduTest.doSetup(3, NUM_TABLET_SERVERS);
-  }
-
   @Before
   public void attachToLog() {
     claAttach = cla.attach();
   }
 
   @After
-  public void restartKilledMaster() throws IOException {
-    miniCluster.restartDeadMasters();
-    miniCluster.restartDeadTservers();
-  }
-
-  @After
   public void checkLogs() throws IOException {
     claAttach.close();
     String log = cla.getAppendedText();

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestHandleTooBusy.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestHandleTooBusy.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestHandleTooBusy.java
index 41e595b..1738585 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestHandleTooBusy.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestHandleTooBusy.java
@@ -23,7 +23,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import com.google.common.collect.Lists;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
@@ -35,17 +34,15 @@ import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
 public class TestHandleTooBusy extends BaseKuduTest {
   private static final String TABLE_NAME = "TestHandleTooBusy";
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    miniClusterBuilder
-      // Short queue to provoke overflow.
-      .addMasterFlag("--rpc_service_queue_length=1")
-      // Low number of service threads, so things stay in the queue.
-      .addMasterFlag("--rpc_num_service_threads=3")
-      // inject latency so lookups process slowly.
-      .addMasterFlag("--master_inject_latency_on_tablet_lookups_ms=100");
-
-    BaseKuduTest.setUpBeforeClass();
+  @Override
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
+    return super.getMiniClusterBuilder()
+        // Short queue to provoke overflow.
+        .addMasterFlag("--rpc_service_queue_length=1")
+        // Low number of service threads, so things stay in the queue.
+        .addMasterFlag("--rpc_num_service_threads=3")
+        // inject latency so lookups process slowly.
+        .addMasterFlag("--master_inject_latency_on_tablet_lookups_ms=100");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestHybridTime.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestHybridTime.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestHybridTime.java
index 303f7a5..8662c55 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestHybridTime.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestHybridTime.java
@@ -31,7 +31,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableList;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,16 +53,18 @@ public class TestHybridTime extends BaseKuduTest {
   protected static final Schema schema = getSchema();
   protected static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Before starting the cluster, disable automatic safe time advancement in the
-    // absence of writes. This test does snapshot reads in the present and expects
-    // certain timestamps to be assigned to the scans. If safe time was allowed
-    // to move automatically the scans might not be assigned the expected timestamps.
-    miniClusterBuilder.addTserverFlag("--safe_time_advancement_without_writes=false");
-
-    BaseKuduTest.setUpBeforeClass();
+  @Override
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
+    return super.getMiniClusterBuilder()
+        // Before starting the cluster, disable automatic safe time advancement in the
+        // absence of writes. This test does snapshot reads in the present and expects
+        // certain timestamps to be assigned to the scans. If safe time was allowed
+        // to move automatically the scans might not be assigned the expected timestamps.
+        .addTserverFlag("--safe_time_advancement_without_writes=false");
+  }
 
+  @Before
+  public void setUp() throws Exception {
     // Using multiple tablets doesn't work with the current way this test works since we could
     // jump from one TS to another which changes the logical clock.
     CreateTableOptions builder =

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
index ce88fd7..282ec03 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduClient.java
@@ -66,19 +66,19 @@ import org.apache.kudu.util.CapturingLogAppender;
 import org.apache.kudu.util.DecimalUtil;
 
 public class TestKuduClient extends BaseKuduTest {
-  private static final Logger LOG = LoggerFactory.getLogger(TestKuduClient.class);
-  private String tableName;
-
-  @Before
-  public void setUp() {
-    tableName = getTestMethodNameWithTimestamp();
-  }
+  private static final String tableName = "TestKuduClient";
 
   /**
    * Test setting and reading the most recent propagated timestamp.
    */
   @Test(timeout = 100000)
   public void testLastPropagatedTimestamps() throws Exception {
+    // Scan a table to ensure a timestamp is propagated.
+    KuduTable table = syncClient.createTable(tableName, basicSchema, getBasicCreateTableOptions());
+    syncClient.newScannerBuilder(table).build().nextRows().getNumRows();
+    assertTrue(syncClient.hasLastPropagatedTimestamp());
+    assertTrue(client.hasLastPropagatedTimestamp());
+
     long initial_ts = syncClient.getLastPropagatedTimestamp();
 
     // Check that the initial timestamp is consistent with the asynchronous client.

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduSession.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduSession.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduSession.java
index d1afea7..5f9d942 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduSession.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduSession.java
@@ -35,12 +35,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TestKuduSession extends BaseKuduTest {
-  private String tableName;
-
-  @Before
-  public void setUp() {
-    tableName = getTestMethodNameWithTimestamp();
-  }
+  private static final String tableName = "TestKuduSession";
 
   @Test(timeout = 100000)
   public void testBasicOps() throws Exception {

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
index 7b67b39..aa7adf7 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
@@ -43,17 +43,7 @@ import org.apache.kudu.Type;
 
 public class TestKuduTable extends BaseKuduTest {
   private static final Schema BASIC_SCHEMA = getBasicSchema();
-  private String tableName;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
-  @Before
-  public void setUp() {
-    tableName = getTestMethodNameWithTimestamp();
-  }
+  private static final String tableName = "TestKuduTable";
 
   @Test(timeout = 100000)
   public void testAlterColumn() throws Exception {
@@ -680,7 +670,7 @@ public class TestKuduTable extends BaseKuduTest {
     for (int i = 1; i <= 3; i++) {
       // Ignore even numbers.
       if (i % 2 != 0) {
-        String tableName = getTestMethodNameWithTimestamp() + "-" + i;
+        String tableName = "testNumReplicas" + "-" + i;
         CreateTableOptions options = getBasicCreateTableOptions();
         options.setNumReplicas(i);
         createTable(tableName, basicSchema, options);

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestLeaderFailover.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestLeaderFailover.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestLeaderFailover.java
index 94305c0..bb2b954 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestLeaderFailover.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestLeaderFailover.java
@@ -22,7 +22,7 @@ import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestLeaderFailover extends BaseKuduTest {
@@ -31,10 +31,8 @@ public class TestLeaderFailover extends BaseKuduTest {
       TestLeaderFailover.class.getName() + "-" + System.currentTimeMillis();
   private static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
+  @Before
+  public void setUp() throws Exception {
     CreateTableOptions builder = getBasicCreateTableOptions();
     createTable(TABLE_NAME, basicSchema, builder);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestMasterFailover.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMasterFailover.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMasterFailover.java
index 1731bc3..81b2383 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMasterFailover.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMasterFailover.java
@@ -34,11 +34,6 @@ public class TestMasterFailover extends BaseKuduTest {
     SCAN_TABLE
   }
 
-  @After
-  public void restartDeadServers() throws Exception {
-    miniCluster.restartDeadMasters();
-  }
-
   @Test(timeout = 30000)
   public void testKillLeaderBeforeCreateClient() throws Exception {
     doTestKillLeader(KillBefore.CREATE_CLIENT);

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestMultipleLeaderFailover.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMultipleLeaderFailover.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMultipleLeaderFailover.java
index a2c07d3..85526f9 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMultipleLeaderFailover.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMultipleLeaderFailover.java
@@ -25,18 +25,12 @@ import static org.junit.Assert.fail;
 
 import java.util.List;
 import org.apache.kudu.util.AssertHelpers.BooleanExpression;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.net.HostAndPort;
 
 public class TestMultipleLeaderFailover extends BaseKuduTest {
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
   private void waitUntilRowCount(final KuduTable table, final int rowCount, long timeoutMs)
       throws Exception {
     assertEventuallyTrue(String.format("Read count should be %s", rowCount),

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowErrors.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowErrors.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowErrors.java
index 78636a4..4d9c41f 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowErrors.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowErrors.java
@@ -24,19 +24,12 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.List;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestRowErrors extends BaseKuduTest {
 
   private static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-
-  }
-
   @Test(timeout = 100000)
   public void singleTabletTest() throws Exception {
     String tableName = TestRowErrors.class.getName() + "-" + System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowResult.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowResult.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowResult.java
index a9cb588..47fa15e 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowResult.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestRowResult.java
@@ -26,7 +26,7 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.sql.Timestamp;
 
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.Type;
@@ -39,9 +39,8 @@ public class TestRowResult extends BaseKuduTest {
 
   private static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
+  @Before
+  public void setUp() throws Exception {
     createTable(TABLE_NAME, allTypesSchema, getAllTypesCreateTableOptions());
     table = openTable(TABLE_NAME);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
index cf8ba44..0e3794f 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScanToken.java
@@ -37,12 +37,7 @@ import static org.junit.Assert.fail;
 
 public class TestScanToken extends BaseKuduTest {
 
-  private String testTableName;
-
-  @Before
-  public void setup() {
-    testTableName = getTestMethodNameWithTimestamp();
-  }
+  private static final String testTableName = "TestScanToken";
 
   /**
    * Tests scan tokens by creating a set of scan tokens, serializing them, and

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestScannerMultiTablet.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScannerMultiTablet.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScannerMultiTablet.java
index 92ea72b..54a1cf3 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestScannerMultiTablet.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestScannerMultiTablet.java
@@ -30,7 +30,6 @@ import com.google.common.collect.Lists;
 import com.stumbleupon.async.Deferred;
 
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -47,20 +46,18 @@ public class TestScannerMultiTablet extends BaseKuduTest {
   private static Schema schema = getSchema();
 
   /**
-   * The timestamp after inserting the rows into the test table during
-   * setupBeforeClass().
+   * The timestamp after inserting the rows into the test table during setUp().
    */
-  private static long beforeClassWriteTimestamp;
+  private static long beforeWriteTimestamp;
   private KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
+  @Before
+  public void setUp() throws Exception {
     // create a 4-tablets table for scanning
     CreateTableOptions builder =
         new CreateTableOptions().setRangePartitionColumns(ImmutableList.of("key1", "key2"));
 
-    for (int i = 1; i < 4; i++){
+    for (int i = 1; i < 4; i++) {
       PartialRow splitRow = schema.newPartialRow();
       splitRow.addString("key1", "" + i);
       splitRow.addString("key2", "");
@@ -69,7 +66,7 @@ public class TestScannerMultiTablet extends BaseKuduTest {
 
     createTable(TABLE_NAME, schema, builder);
 
-    KuduTable table = openTable(TABLE_NAME);
+    KuduTable insertTable = openTable(TABLE_NAME);
     AsyncKuduSession session = client.newSession();
     session.setFlushMode(AsyncKuduSession.FlushMode.AUTO_FLUSH_SYNC);
 
@@ -81,7 +78,7 @@ public class TestScannerMultiTablet extends BaseKuduTest {
     String[] keys = new String[] {"1", "2", "3"};
     for (String key1 : keys) {
       for (String key2 : keys) {
-        Insert insert = table.newInsert();
+        Insert insert = insertTable.newInsert();
         PartialRow row = insert.getRow();
         row.addString(0, key1);
         row.addString(1, key2);
@@ -91,11 +88,8 @@ public class TestScannerMultiTablet extends BaseKuduTest {
       }
     }
 
-    beforeClassWriteTimestamp = client.getLastPropagatedTimestamp();
-  }
+    beforeWriteTimestamp = client.getLastPropagatedTimestamp();
 
-  @Before
-  public void setup() throws Exception {
     // Reset the clients in order to clear the propagated timestamp, which may
     // have been set if other test cases ran before this one. This ensures
     // that all tests set their own state.
@@ -293,7 +287,7 @@ public class TestScannerMultiTablet extends BaseKuduTest {
   // client-local read-your-writes.
   @Test(timeout = 100000)
   public void testReadYourWrites() throws Exception {
-    long preTs = beforeClassWriteTimestamp;
+    long preTs = beforeWriteTimestamp;
 
     // Update the propagated timestamp to ensure we see the rows written
     // in the constructor.

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
index 81251ca..f6345c1 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
@@ -38,6 +38,7 @@ import org.apache.kudu.util.SecurityUtil;
 import org.hamcrest.CoreMatchers;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -86,13 +87,13 @@ public class TestSecurity {
     client.listTabletServers();
   }
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  @Before
+  public void setUp() {
     FakeDNS.getInstance().install();
   }
 
   @After
-  public void reset() throws IOException, InterruptedException {
+  public void tearDown() throws IOException {
     if (client != null) {
       client.close();
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurityContextRealUser.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurityContextRealUser.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurityContextRealUser.java
index 3b9f507..c48a1f3 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurityContextRealUser.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurityContextRealUser.java
@@ -18,10 +18,7 @@
 package org.apache.kudu.client;
 
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 
@@ -36,35 +33,32 @@ import static org.junit.Assert.fail;
  * SASL PLAIN negotiations, and is imported from the SecurityCredentialsPB.
  */
 public class TestSecurityContextRealUser extends BaseKuduTest {
-  private static final Logger LOG = LoggerFactory.getLogger(TestSecurityContextRealUser.class);
-
   private String tableName;
 
+  @Override
+  protected MiniKuduCluster.MiniKuduClusterBuilder getMiniClusterBuilder() {
+    return super.getMiniClusterBuilder()
+        // This test requires a delicate setup. We enable Kerberos, make
+        // authentication optional, and set the superuser ACL to test-admin so that
+        // the external mini-cluster is able to connect to the master while creating
+        // the cluster. The user ACL is scoped to a different user so that we can
+        // test real user name propagation.
+        .enableKerberos()
+        .addMasterFlag("--user-acl=token-user")
+        .addMasterFlag("--superuser-acl=test-admin")
+        .addMasterFlag("--rpc-authentication=optional")
+        .addMasterFlag("--rpc-trace-negotiation")
+        .addTserverFlag("--user-acl=token-user")
+        .addTserverFlag("--superuser-acl=test-admin")
+        .addTserverFlag("--rpc-authentication=optional")
+        .addTserverFlag("--rpc-trace-negotiation");
+  }
+
   @Before
   public void setTableName() {
     tableName = TestSecurityContextRealUser.class.getName() + "-" + System.currentTimeMillis();
   }
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // This test requires a delicate setup. We enable Kerberos, make
-    // authentication optional, and set the superuser ACL to test-admin so that
-    // the external mini-cluster is able to connect to the master while creating
-    // the cluster. The user ACL is scoped to a different user so that we can
-    // test real user name propagation.
-    miniClusterBuilder.enableKerberos()
-                      .addMasterFlag("--user-acl=token-user")
-                      .addMasterFlag("--superuser-acl=test-admin")
-                      .addMasterFlag("--rpc-authentication=optional")
-                      .addMasterFlag("--rpc-trace-negotiation")
-                      .addTserverFlag("--user-acl=token-user")
-                      .addTserverFlag("--superuser-acl=test-admin")
-                      .addTserverFlag("--rpc-authentication=optional")
-                      .addTserverFlag("--rpc-trace-negotiation");
-
-    BaseKuduTest.setUpBeforeClass();
-  }
-
   @Test
   public void test() throws Exception {
     // Clear out the Kerberos credentials in the environment.

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
index 7758b6c..4c7c641 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestStatistics.java
@@ -20,7 +20,7 @@ import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
 import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.client.Statistics.Statistic;
@@ -34,9 +34,8 @@ public class TestStatistics extends BaseKuduTest {
       + System.currentTimeMillis();
   private static KuduTable table;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
+  @Before
+  public void setUp() throws Exception {
     CreateTableOptions options = getBasicCreateTableOptions().setNumReplicas(1);
     table = createTable(TABLE_NAME, basicSchema, options);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/AvroKuduOperationsProducerTest.java
----------------------------------------------------------------------
diff --git a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/AvroKuduOperationsProducerTest.java b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/AvroKuduOperationsProducerTest.java
index 0070003..8e3c8dc 100644
--- a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/AvroKuduOperationsProducerTest.java
+++ b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/AvroKuduOperationsProducerTest.java
@@ -58,6 +58,7 @@ import org.apache.flume.channel.MemoryChannel;
 import org.apache.flume.conf.Configurables;
 import org.apache.flume.event.EventBuilder;
 import org.apache.kudu.util.DecimalUtil;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -72,19 +73,18 @@ public class AvroKuduOperationsProducerTest extends BaseKuduTest {
   private static final String schemaPath = "src/test/avro/testAvroKuduOperationsProducer.avsc";
   private static String schemaLiteral;
 
-  enum SchemaLocation {
-    GLOBAL, URL, LITERAL
-  }
-
-  @BeforeClass
-  public static void setupAvroSchemaBeforeClass() {
+  static {
     try {
       schemaLiteral = Files.toString(new File(schemaPath), UTF_8);
     } catch (IOException e) {
-      throw new FlumeException("Unable to read schema file!", e);
+      throw new RuntimeException(e);
     }
   }
 
+  enum SchemaLocation {
+    GLOBAL, URL, LITERAL
+  }
+
   @Test
   public void testEmptyChannel() throws Exception {
     testEvents(0, SchemaLocation.GLOBAL);

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITInputFormatJob.java
----------------------------------------------------------------------
diff --git a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITInputFormatJob.java b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITInputFormatJob.java
index 64eefa4..69696e4 100644
--- a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITInputFormatJob.java
+++ b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITInputFormatJob.java
@@ -30,8 +30,7 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,20 +48,11 @@ public class ITInputFormatJob extends BaseKuduTest {
   private static final HadoopTestingUtility HADOOP_UTIL = new HadoopTestingUtility();
 
   /** Counter enumeration to count the actual rows. */
-  private static enum Counters { ROWS }
+  private enum Counters { ROWS }
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITKuduTableOutputFormat.java
----------------------------------------------------------------------
diff --git a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITKuduTableOutputFormat.java b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITKuduTableOutputFormat.java
index 4038d16..f97ee0a 100644
--- a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITKuduTableOutputFormat.java
+++ b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITKuduTableOutputFormat.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertNotNull;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.RecordWriter;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.kudu.client.AsyncKuduScanner;
@@ -40,11 +39,6 @@ public class ITKuduTableOutputFormat extends BaseKuduTest {
   private static final String TABLE_NAME =
       ITKuduTableOutputFormat.class.getName() + "-" + System.currentTimeMillis();
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
-  }
-
   @Test
   public void test() throws Exception {
     createTable(TABLE_NAME, getBasicSchema(), getBasicCreateTableOptions());

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITOutputFormatJob.java
----------------------------------------------------------------------
diff --git a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITOutputFormatJob.java b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITOutputFormatJob.java
index 8e16eb9..7b2745b 100644
--- a/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITOutputFormatJob.java
+++ b/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/ITOutputFormatJob.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.kudu.client.AsyncKuduScanner;
@@ -54,19 +54,14 @@ public class ITOutputFormatJob extends BaseKuduTest {
 
   private static final HadoopTestingUtility HADOOP_UTIL = new HadoopTestingUtility();
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    BaseKuduTest.setUpBeforeClass();
+  @Before
+  public void setUp() throws Exception {
     createTable(TABLE_NAME, getBasicSchema(), getBasicCreateTableOptions());
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    try {
-      BaseKuduTest.tearDownAfterClass();
-    } finally {
-      HADOOP_UTIL.cleanup();
-    }
+  @After
+  public void tearDown() throws Exception {
+    HADOOP_UTIL.cleanup();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-mapreduce/src/test/java/org/apache/kudu/mapreduce/TestJarFinder.java
----------------------------------------------------------------------
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 73401c1..35bd0c0 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
@@ -32,9 +32,9 @@ import java.util.jar.Manifest;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.LogFactory;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -44,14 +44,14 @@ public class TestJarFinder {
 
   private static File testDir;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     testDir = Files.createTempDirectory("test-dir").toFile();
     System.setProperty(JarFinder.FILE_DIR_PROPERTY, testDir.getAbsolutePath());
   }
 
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     FileUtils.deleteDirectory(testDir);
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/DefaultSourceTest.scala
----------------------------------------------------------------------
diff --git a/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/DefaultSourceTest.scala b/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/DefaultSourceTest.scala
index 19a1948..4e7d5a4 100644
--- a/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/DefaultSourceTest.scala
+++ b/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/DefaultSourceTest.scala
@@ -19,28 +19,28 @@ package org.apache.kudu.spark.kudu
 import scala.collection.JavaConverters._
 import scala.collection.immutable.IndexedSeq
 import scala.util.control.NonFatal
-
 import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.types.{DataTypes, StructField, StructType}
 import org.junit.Assert._
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
-import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
-
+import org.scalatest.{BeforeAndAfterEach, FunSuite, Matchers}
 import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder
 import org.apache.kudu.client.CreateTableOptions
 import org.apache.kudu.{Schema, Type}
 
 @RunWith(classOf[JUnitRunner])
-class DefaultSourceTest extends FunSuite with TestContext with BeforeAndAfter with Matchers {
+class DefaultSourceTest extends FunSuite with TestContext with BeforeAndAfterEach with Matchers {
 
   val rowCount = 10
   var sqlContext : SQLContext = _
   var rows : IndexedSeq[(Int, Int, String, Long)] = _
   var kuduOptions : Map[String, String] = _
 
-  before {
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+
     rows = insertRows(rowCount)
 
     sqlContext = ss.sqlContext

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb9c6442/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/TestContext.scala
----------------------------------------------------------------------
diff --git a/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/TestContext.scala b/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/TestContext.scala
index 62b2c72..3ef0e45 100644
--- a/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/TestContext.scala
+++ b/java/kudu-spark/src/test/scala/org/apache/kudu/spark/kudu/TestContext.scala
@@ -21,10 +21,8 @@ import java.util.Date
 
 import scala.collection.JavaConverters._
 import scala.collection.immutable.IndexedSeq
-
 import org.apache.spark.SparkConf
-import org.scalatest.{BeforeAndAfterAll, Suite}
-
+import org.scalatest.{BeforeAndAfterEach, Suite}
 import org.apache.kudu.ColumnSchema.ColumnSchemaBuilder
 import org.apache.kudu.ColumnTypeAttributes.ColumnTypeAttributesBuilder
 import org.apache.kudu.client.KuduClient.KuduClientBuilder
@@ -34,7 +32,7 @@ import org.apache.kudu.{Schema, Type}
 import org.apache.kudu.util.DecimalUtil
 import org.apache.spark.sql.SparkSession
 
-trait TestContext extends BeforeAndAfterAll { self: Suite =>
+trait TestContext extends BeforeAndAfterEach { self: Suite =>
 
   var ss: SparkSession = _
   var miniCluster: MiniKuduCluster = _
@@ -88,7 +86,7 @@ trait TestContext extends BeforeAndAfterAll { self: Suite =>
     set("spark.ui.enabled", "false").
     set("spark.app.id", appID)
 
-  override def beforeAll() {
+  override def beforeEach() {
     miniCluster = new MiniKuduClusterBuilder()
       .numMasters(1)
       .numTservers(1)
@@ -119,7 +117,7 @@ trait TestContext extends BeforeAndAfterAll { self: Suite =>
     kuduClient.createTable(simpleTableName, simpleSchema, simpleTableOptions)
   }
 
-  override def afterAll() {
+  override def afterEach() {
     if (kuduClient != null) kuduClient.shutdown()
     if (miniCluster != null) miniCluster.shutdown()
     if (ss != null) ss.stop()


[2/2] kudu git commit: [Java] Support re-running tests in the Gradle build

Posted by gr...@apache.org.
[Java] Support re-running tests in the Gradle build

Gradle does not have built in support for re-running tests like
Maven Surefire/Failsafe does. This patch adds a Junit Rule to
BaseKuduTest to support retries and configures the Gradle
build and Jenkins script to set the retry property to match
the Maven build.

Change-Id: Ie11e61c40ee991b00616012f0decaaf2c5fe5ef0
Reviewed-on: http://gerrit.cloudera.org:8080/10824
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: c727118320b03fc87befd9941841cdddf8b30762
Parents: eb9c644
Author: Grant Henke <gr...@apache.org>
Authored: Tue Jun 26 11:08:00 2018 -0500
Committer: Grant Henke <gr...@apache.org>
Committed: Mon Jul 9 19:20:11 2018 +0000

----------------------------------------------------------------------
 build-support/jenkins/build-and-test.sh         |  2 +-
 java/gradle/tests.gradle                        |  3 +
 .../org/apache/kudu/client/BaseKuduTest.java    | 16 ++++-
 .../java/org/apache/kudu/junit/RetryRule.java   | 74 ++++++++++++++++++++
 4 files changed, 93 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c7271183/build-support/jenkins/build-and-test.sh
----------------------------------------------------------------------
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index cf60d40..e545394 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -389,7 +389,7 @@ if [ "$BUILD_JAVA" == "1" ]; then
 
   # Run the full Gradle build.
   if [ "$BUILD_GRADLE" == "1" ]; then
-     GRADLE_FLAGS="$GRADLE_FLAGS --console=plain --no-daemon"
+     GRADLE_FLAGS="$GRADLE_FLAGS --console=plain --no-daemon -DrerunFailingTestsCount=3"
      # TODO: Run `gradle check` in BUILD_TYPE DEBUG when static code analysis is fixed
      if ! ./gradlew $GRADLE_FLAGS clean test integrationTest ; then
        EXIT_STATUS=1

http://git-wip-us.apache.org/repos/asf/kudu/blob/c7271183/java/gradle/tests.gradle
----------------------------------------------------------------------
diff --git a/java/gradle/tests.gradle b/java/gradle/tests.gradle
index f826c2b..c2df581 100644
--- a/java/gradle/tests.gradle
+++ b/java/gradle/tests.gradle
@@ -57,6 +57,9 @@ 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)
+
   // Don't fail the build if a --tests filter doesn't match any tests.
   // This is useful for filtering tests from the top of the project where some
   // subprojects won't match the filter. Without this, those subprojects would fail.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c7271183/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java b/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
index 9078085..8d91e01 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/BaseKuduTest.java
@@ -6,7 +6,8 @@
 // "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 //
+//   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
@@ -27,8 +28,10 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.base.Stopwatch;
 import com.google.common.net.HostAndPort;
 import com.stumbleupon.async.Deferred;
+import org.apache.kudu.junit.RetryRule;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,6 +65,17 @@ public class BaseKuduTest {
   protected static final Schema basicSchema = getBasicSchema();
   protected static final Schema allTypesSchema = getSchemaWithAllTypes();
 
+  // Add a rule to rerun 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.
+  private static final String RETRY_PROPERTY_NAME = "rerunFailingTestsCount";
+  private static final int DEFAULT_RETRY_COUNT = 0;
+  private static final String retryPropVal = System.getProperty(RETRY_PROPERTY_NAME);
+  private static final int retryCount =
+      retryPropVal != null ? Integer.parseInt(retryPropVal) : DEFAULT_RETRY_COUNT;
+  @Rule
+  public RetryRule retryRule = new RetryRule(retryCount);
+
   @Before
   public void setUpBase() throws Exception {
     FakeDNS.getInstance().install();

http://git-wip-us.apache.org/repos/asf/kudu/blob/c7271183/java/kudu-client/src/test/java/org/apache/kudu/junit/RetryRule.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/junit/RetryRule.java b/java/kudu-client/src/test/java/org/apache/kudu/junit/RetryRule.java
new file mode 100644
index 0000000..24defed
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/junit/RetryRule.java
@@ -0,0 +1,74 @@
+// 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.junit;
+
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+/**
+ * A JUnit rule to retry failed tests.
+ */
+public class RetryRule implements TestRule {
+  private int retryCount;
+
+  public RetryRule (int retryCount) {
+    this.retryCount = retryCount;
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    return new RetryStatement(base, description, retryCount);
+  }
+
+  private static class RetryStatement extends Statement {
+
+    private final Statement base;
+    private final Description description;
+    private final int retryCount;
+
+    RetryStatement(final Statement base, final Description description, final int retryCount) {
+      this.base = base;
+      this.description = description;
+      this.retryCount = retryCount;
+    }
+
+    @Override
+    public void evaluate() throws Throwable {
+      // If there are no retries, just pass through to evaluate as usual.
+      if (retryCount == 0) {
+        base.evaluate();
+        return;
+      }
+
+      // To retry we catch the exception for the evaluate, log a message, and retry.
+      // We track and throw the last failure if all tries fail.
+      Throwable lastException = null;
+      for (int i = 0; i < retryCount; i++) {
+        try {
+          base.evaluate();
+          return;
+        } catch (Throwable t) {
+          lastException = t;
+          System.err.println(description.getDisplayName() + ": run " + (i + 1) + " failed.");
+        }
+      }
+      System.err.println(description.getDisplayName() + ": giving up after " + retryCount + " failures.");
+      throw lastException;
+    }
+  }
+}