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/10/17 20:43:08 UTC

[4/5] kudu git commit: KUDU-2411: (Part 1) Break out existing test utilities into a seperate module

KUDU-2411: (Part 1) Break out existing test utilities into a seperate module

This patch breaks out the test harness and test
util classes into a seperate module.

This cleans up the dependencies on test jars and
is a preliminary step to a public test utility. Though
all of the classes still remain marked as private
until the interfaces are fleshed out further.

Change-Id: Ifa935d6038b6d8756b332178347cec5cb70660a9
Reviewed-on: http://gerrit.cloudera.org:8080/11686
Reviewed-by: Dan Burkert <da...@apache.org>
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Grant Henke <gr...@apache.org>


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

Branch: refs/heads/master
Commit: 15f1416f67dcb714842d02647a1f2e06e675660d
Parents: 7072a85
Author: Grant Henke <gr...@apache.org>
Authored: Mon Oct 15 13:34:12 2018 -0500
Committer: Grant Henke <gr...@apache.org>
Committed: Wed Oct 17 20:40:42 2018 +0000

----------------------------------------------------------------------
 java/kudu-backup/build.gradle                   |   2 +-
 .../org/apache/kudu/backup/TestKuduBackup.scala |   2 +-
 java/kudu-client-tools/build.gradle             |   2 +-
 .../kudu/mapreduce/tools/ITExportCsv.java       |   2 +-
 .../kudu/mapreduce/tools/ITImportCsv.java       |   4 +-
 .../kudu/mapreduce/tools/ITImportParquet.java   |   4 +-
 .../tools/ITImportParquetPreCheck.java          |   2 +-
 .../kudu/mapreduce/tools/ITRowCounter.java      |   2 +-
 java/kudu-client/build.gradle                   |  16 +-
 .../java/org/apache/kudu/client/FakeDNS.java    | 192 ------
 .../java/org/apache/kudu/client/ITClient.java   |   6 +-
 .../org/apache/kudu/client/ITClientStress.java  |   8 +-
 .../kudu/client/ITScannerMultiTablet.java       |   2 +-
 .../org/apache/kudu/client/MiniKuduCluster.java | 642 ------------------
 .../org/apache/kudu/client/TestAlterTable.java  |   4 +-
 .../apache/kudu/client/TestAsyncKuduClient.java |  10 +-
 .../kudu/client/TestAsyncKuduSession.java       |  10 +-
 .../kudu/client/TestAuthnTokenReacquire.java    |  10 +-
 .../client/TestAuthnTokenReacquireOpen.java     |   8 +-
 .../kudu/client/TestClientFailoverSupport.java  |  14 +-
 .../kudu/client/TestConnectToCluster.java       |   1 +
 .../apache/kudu/client/TestConnectionCache.java |   3 +-
 .../apache/kudu/client/TestHandleTooBusy.java   |   6 +-
 .../org/apache/kudu/client/TestHybridTime.java  |   4 +-
 .../org/apache/kudu/client/TestKuduClient.java  |  22 +-
 .../org/apache/kudu/client/TestKuduSession.java |  12 +-
 .../org/apache/kudu/client/TestKuduTable.java   |  12 +-
 .../apache/kudu/client/TestLeaderFailover.java  |   8 +-
 .../apache/kudu/client/TestMasterFailover.java  |   6 +-
 .../apache/kudu/client/TestMiniKuduCluster.java | 130 ----
 .../kudu/client/TestMultipleLeaderFailover.java |  12 +-
 .../org/apache/kudu/client/TestNegotiation.java |   6 +-
 .../org/apache/kudu/client/TestPartialRow.java  |   2 +-
 .../apache/kudu/client/TestRemoteTablet.java    |   2 +-
 .../org/apache/kudu/client/TestRowErrors.java   |   8 +-
 .../org/apache/kudu/client/TestRowResult.java   |   4 +-
 .../org/apache/kudu/client/TestScanToken.java   |   8 +-
 .../kudu/client/TestScannerMultiTablet.java     |   2 +-
 .../org/apache/kudu/client/TestSecurity.java    |  20 +-
 .../client/TestSecurityContextRealUser.java     |   6 +-
 .../org/apache/kudu/client/TestServerInfo.java  |   1 +
 .../org/apache/kudu/client/TestStatistics.java  |   6 +-
 .../org/apache/kudu/client/TestTimeouts.java    |   6 +-
 .../java/org/apache/kudu/junit/RetryRule.java   |  83 ---
 .../org/apache/kudu/test/KuduTestHarness.java   | 445 -------------
 .../org/apache/kudu/util/AssertHelpers.java     |  46 --
 .../apache/kudu/util/CapturingLogAppender.java  |  82 ---
 .../org/apache/kudu/util/ClientTestUtil.java    | 375 -----------
 .../org/apache/kudu/util/KuduBinaryLocator.java |  94 ---
 .../org/apache/kudu/util/ProtobufUtils.java     |  64 --
 .../java/org/apache/kudu/util/RandomUtils.java  |  49 --
 java/kudu-flume-sink/build.gradle               |   2 +-
 .../sink/AvroKuduOperationsProducerTest.java    |   2 +-
 .../sink/KeyedKuduOperationsProducerTest.java   |   2 +-
 .../apache/kudu/flume/sink/KuduSinkTest.java    |   2 +-
 ...expKuduOperationsProducerParseErrorTest.java |   3 +-
 .../sink/RegexpKuduOperationsProducerTest.java  |   2 +-
 .../kudu/flume/sink/SecureKuduSinkTest.java     |   4 +-
 java/kudu-hive/build.gradle                     |   2 +-
 .../hive/metastore/TestKuduMetastorePlugin.java |   2 +-
 java/kudu-jepsen/build.gradle                   |   2 +-
 java/kudu-mapreduce/build.gradle                |   2 +-
 .../apache/kudu/mapreduce/ITInputFormatJob.java |   4 +-
 .../kudu/mapreduce/ITKuduTableInputFormat.java  |   4 +-
 .../kudu/mapreduce/ITKuduTableOutputFormat.java |   6 +-
 .../kudu/mapreduce/ITOutputFormatJob.java       |   6 +-
 java/kudu-spark-tools/build.gradle              |   2 +-
 java/kudu-spark/build.gradle                    |   6 +-
 java/kudu-test-utils/build.gradle               |  38 ++
 .../apache/kudu/test/CapturingLogAppender.java  |  82 +++
 .../org/apache/kudu/test/ClientTestUtil.java    | 376 +++++++++++
 .../org/apache/kudu/test/KuduTestHarness.java   | 444 +++++++++++++
 .../org/apache/kudu/test/ProtobufUtils.java     |  64 ++
 .../java/org/apache/kudu/test/RandomUtils.java  |  49 ++
 .../org/apache/kudu/test/cluster/FakeDNS.java   | 192 ++++++
 .../kudu/test/cluster/KuduBinaryLocator.java    |  94 +++
 .../kudu/test/cluster/MiniKuduCluster.java      | 643 +++++++++++++++++++
 .../apache/kudu/test/junit/AssertHelpers.java   |  46 ++
 .../org/apache/kudu/test/junit/RetryRule.java   |  83 +++
 .../apache/kudu/test/TestMiniKuduCluster.java   | 136 ++++
 java/settings.gradle                            |   1 +
 81 files changed, 2402 insertions(+), 2366 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-backup/build.gradle
----------------------------------------------------------------------
diff --git a/java/kudu-backup/build.gradle b/java/kudu-backup/build.gradle
index 6a63f75..53b59af 100644
--- a/java/kudu-backup/build.gradle
+++ b/java/kudu-backup/build.gradle
@@ -40,7 +40,7 @@ dependencies {
   provided libs.sparkSql
   provided libs.slf4jApi
 
-  testCompile project(path: ":kudu-client", configuration: "shadowTest")
+  testCompile project(path: ":kudu-test-utils", configuration: "shadow")
   testCompile project(path: ":kudu-spark", configuration: "test")
   testCompile libs.junit
   testCompile libs.log4j

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
----------------------------------------------------------------------
diff --git a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
index 0650ad9..e74fdcd 100644
--- a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
+++ b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
@@ -34,8 +34,8 @@ import org.apache.kudu.ColumnSchema
 import org.apache.kudu.Schema
 import org.apache.kudu.Type
 import org.apache.kudu.spark.kudu._
+import org.apache.kudu.test.RandomUtils
 import org.apache.kudu.util.DecimalUtil
-import org.apache.kudu.util.RandomUtils
 import org.junit.Assert._
 import org.junit.Test
 import org.slf4j.Logger

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client-tools/build.gradle
----------------------------------------------------------------------
diff --git a/java/kudu-client-tools/build.gradle b/java/kudu-client-tools/build.gradle
index 99b9fcf..01ed399 100644
--- a/java/kudu-client-tools/build.gradle
+++ b/java/kudu-client-tools/build.gradle
@@ -28,7 +28,7 @@ dependencies {
 
   optional libs.yetusAnnotations
 
-  testCompile project(path: ":kudu-client", configuration: "shadowTest")
+  testCompile project(path: ":kudu-test-utils", configuration: "shadow")
   testCompile project(path: ":kudu-mapreduce", configuration: "test")
   testCompile libs.commonsIo
   testCompile libs.junit

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 e44876d..3bdda0d 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
@@ -17,8 +17,8 @@
 
 package org.apache.kudu.mapreduce.tools;
 
+import static org.apache.kudu.test.ClientTestUtil.createFourTabletsTableWithNineRows;
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.createFourTabletsTableWithNineRows;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 969f521..fc544c2 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
@@ -17,9 +17,9 @@
 
 package org.apache.kudu.mapreduce.tools;
 
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 2599426..23fdf50 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
@@ -17,8 +17,8 @@
 
 package org.apache.kudu.mapreduce.tools;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
 import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 34625e0..6bfde78 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
@@ -17,7 +17,7 @@
 
 package org.apache.kudu.mapreduce.tools;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
 import static org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
 import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 3b5f01b..789c47d 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
@@ -17,8 +17,8 @@
 
 package org.apache.kudu.mapreduce.tools;
 
+import static org.apache.kudu.test.ClientTestUtil.createFourTabletsTableWithNineRows;
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.createFourTabletsTableWithNineRows;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/build.gradle
----------------------------------------------------------------------
diff --git a/java/kudu-client/build.gradle b/java/kudu-client/build.gradle
index 3d4c25c..077e050 100644
--- a/java/kudu-client/build.gradle
+++ b/java/kudu-client/build.gradle
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-import com.github.jengelman.gradle.plugins.shadow.tasks.ShadowJar
-
 apply from: "$rootDir/gradle/protobuf.gradle"
 apply from: "$rootDir/gradle/shadow.gradle"
 
@@ -37,6 +35,7 @@ dependencies {
   optional libs.jsr305
   optional libs.yetusAnnotations
 
+  testCompile project(":kudu-test-utils")
   testCompile libs.hamcrestCore
   testCompile libs.junit
   testCompile libs.log4j
@@ -54,16 +53,3 @@ sourceSets {
     }
   }
 }
-
-// Configure a shaded test jar for use in the other modules.
-// We only do this for kudu-client because it has common test utilities.
-task shadowTestJar(type: ShadowJar) {
-  classifier = "tests-shaded"
-  from sourceSets.test.output
-  configurations = [project.configurations.testRuntime]
-}
-// Create a configuration so that the shaded test jar can be referenced in other modules.
-configurations.create("shadowTest")
-artifacts {
-  shadowTest shadowTestJar
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/FakeDNS.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/FakeDNS.java b/java/kudu-client/src/test/java/org/apache/kudu/client/FakeDNS.java
deleted file mode 100644
index 72444ae..0000000
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/FakeDNS.java
+++ /dev/null
@@ -1,192 +0,0 @@
-// 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.client;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import javax.annotation.concurrent.GuardedBy;
-
-import com.google.common.base.Throwables;
-import com.google.common.net.InetAddresses;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-
-/**
- * Fake DNS resolver which allows our tests to work well even though we use
- * strange loopback IP addresses (127.x.y.z) with no corresponding reverse
- * DNS.
- *
- * This overrides the reverse lookups for such IPs to return the same address
- * in String form.
- *
- * Without this class, reverse DNS lookups for such addresses often take
- * 5 seconds to return, causing timeouts and overall test slowness.
- *
- * In the future this class might also be extended to test more interesting
- * DNS-related scenarios.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class FakeDNS {
-  static FakeDNS instance = new FakeDNS();
-
-  @GuardedBy("this")
-  private Map<String, InetAddress> forwardResolutions = new HashMap<>();
-
-  @GuardedBy("this")
-  private Map<InetAddress, String> reverseResolutions = new HashMap<>();
-
-  /** whether the fake resolver has been installed */
-  @GuardedBy("this")
-  private boolean installed = false;
-
-  private FakeDNS() {}
-  public static FakeDNS getInstance() {
-    return instance;
-  }
-
-  public synchronized void addForwardResolution(String hostname, InetAddress ip) {
-    forwardResolutions.put(hostname, ip);
-  }
-
-  public synchronized void addReverseResolution(InetAddress ip, String hostname) {
-    reverseResolutions.put(ip, hostname);
-  }
-
-  /**
-   * Install the fake DNS resolver into the Java runtime.
-   */
-  public synchronized void install() {
-    if (installed) return;
-    try {
-      try {
-        // Override the NameService in Java 9 or later.
-        final Class<?> nameServiceInterface = Class.forName("java.net.InetAddress$NameService");
-        Field field = InetAddress.class.getDeclaredField("nameService");
-        // Get the default NameService to fallback to.
-        Method method = InetAddress.class.getDeclaredMethod("createNameService");
-        method.setAccessible(true);
-        Object fallbackNameService = method.invoke(null);
-        // Create a proxy instance to set on the InetAddress field which will handle
-        // all NameService calls.
-        Object proxy = Proxy.newProxyInstance(nameServiceInterface.getClassLoader(),
-            new Class<?>[]{nameServiceInterface}, new NameServiceListener(fallbackNameService));
-        field.setAccessible(true);
-        field.set(InetAddress.class, proxy);
-      } catch (final ClassNotFoundException | NoSuchFieldException e) {
-        // Override the NameService in Java 8 or earlier.
-        final Class<?> nameServiceInterface = Class.forName("sun.net.spi.nameservice.NameService");
-        Field field = InetAddress.class.getDeclaredField("nameServices");
-        // Get the default NameService to fallback to.
-        Method method = InetAddress.class.getDeclaredMethod("createNSProvider", String.class);
-        method.setAccessible(true);
-        Object fallbackNameService = method.invoke(null, "default");
-        // Create a proxy instance to set on the InetAddress field which will handle
-        // all NameService calls.
-        Object proxy = Proxy.newProxyInstance(nameServiceInterface.getClassLoader(),
-            new Class<?>[]{nameServiceInterface}, new NameServiceListener(fallbackNameService));
-        field.setAccessible(true);
-        // Java 8 or earlier takes a list of NameServices
-        field.set(InetAddress.class, Arrays.asList(proxy));
-      }
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    installed = true;
-  }
-
-  /**
-   * The NameService in all versions of Java has the same interface, so we
-   * can use the same InvocationHandler as our proxy instance for both
-   * java.net.InetAddress$NameService and sun.net.spi.nameservice.NameService.
-   */
-  private class NameServiceListener implements InvocationHandler {
-
-    private final Object fallbackNameService;
-
-    // Creates a NameServiceListener with a NameService implementation to
-    // fallback to. The parameter is untyped so we can handle the NameService
-    // type in all versions of Java with reflection.
-    NameServiceListener(Object fallbackNameService) {
-      this.fallbackNameService = fallbackNameService;
-    }
-
-    private InetAddress[] lookupAllHostAddr(String host) throws UnknownHostException {
-      InetAddress inetAddress;
-      synchronized(FakeDNS.this) {
-        inetAddress = forwardResolutions.get(host);
-      }
-      if (inetAddress != null) {
-        return new InetAddress[]{inetAddress};
-      }
-
-      try {
-        Method method = fallbackNameService.getClass()
-            .getDeclaredMethod("lookupAllHostAddr", String.class);
-        method.setAccessible(true);
-        return (InetAddress[]) method.invoke(fallbackNameService, host);
-      } catch (ReflectiveOperationException e) {
-        Throwables.propagateIfPossible(e.getCause(), UnknownHostException.class);
-        throw new AssertionError("unexpected reflection issue", e);
-      }
-    }
-
-    private String getHostByAddr(byte[] addr) throws UnknownHostException {
-      if (addr[0] == 127) {
-        return InetAddresses.toAddrString(InetAddress.getByAddress(addr));
-      }
-
-      String hostname;
-      synchronized (FakeDNS.this) {
-        hostname = reverseResolutions.get(InetAddress.getByAddress(addr));
-      }
-      if (hostname != null) {
-        return hostname;
-      }
-
-      try {
-        Method method = fallbackNameService.getClass()
-            .getDeclaredMethod("getHostByAddr", byte[].class);
-        method.setAccessible(true);
-        return (String) method.invoke(fallbackNameService, (Object) addr);
-      } catch (ReflectiveOperationException e) {
-        Throwables.propagateIfPossible(e.getCause(), UnknownHostException.class);
-        throw new AssertionError("unexpected reflection issue", e);
-      }
-    }
-
-    @Override
-    public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
-      switch (method.getName()) {
-        case "lookupAllHostAddr":
-          return lookupAllHostAddr((String) args[0]);
-        case "getHostByAddr":
-          return getHostByAddr((byte[]) args[0]);
-        default:
-          throw new UnsupportedOperationException();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 6e379f7..06f25f8 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
@@ -32,9 +32,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 
 /**
  * Integration test for the client. RPCs are sent to Kudu from multiple threads while processes

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/ITClientStress.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClientStress.java b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClientStress.java
index 9b637e6..f4e476b 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClientStress.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClientStress.java
@@ -17,8 +17,8 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.createFourTabletsTableWithNineRows;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.createFourTabletsTableWithNineRows;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -36,11 +36,11 @@ import com.google.common.base.Supplier;
 
 import org.apache.kudu.Schema;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.apache.kudu.client.SessionConfiguration.FlushMode;
-import org.apache.kudu.util.CapturingLogAppender;
+import org.apache.kudu.test.CapturingLogAppender;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 02d8dcc..2a5c0fc 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
@@ -16,7 +16,7 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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
deleted file mode 100644
index 95b310f..0000000
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ /dev/null
@@ -1,642 +0,0 @@
-// 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.client;
-
-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.Paths;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import org.apache.kudu.Common;
-import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.tools.Tool.ControlShellRequestPB;
-import org.apache.kudu.tools.Tool.ControlShellResponsePB;
-import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniKdcOptionsPB;
-import org.apache.kudu.tools.Tool.CreateClusterRequestPB;
-import org.apache.kudu.tools.Tool.DaemonIdentifierPB;
-import org.apache.kudu.tools.Tool.DaemonInfoPB;
-import org.apache.kudu.tools.Tool.GetKDCEnvVarsRequestPB;
-import org.apache.kudu.tools.Tool.GetMastersRequestPB;
-import org.apache.kudu.tools.Tool.GetTServersRequestPB;
-import org.apache.kudu.tools.Tool.KdestroyRequestPB;
-import org.apache.kudu.tools.Tool.KinitRequestPB;
-import org.apache.kudu.tools.Tool.StartClusterRequestPB;
-import org.apache.kudu.tools.Tool.StartDaemonRequestPB;
-import org.apache.kudu.tools.Tool.StopDaemonRequestPB;
-import org.apache.kudu.util.KuduBinaryLocator;
-import org.apache.kudu.util.SecurityUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility class to start and manipulate Kudu clusters. Depends on precompiled
- * kudu, kudu-master, and kudu-tserver binaries. {@link KuduTestHarness}
- * should be used instead of directly using this class in almost all cases.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class MiniKuduCluster implements AutoCloseable {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MiniKuduCluster.class);
-
-  // Control shell process.
-  private Process miniCluster;
-
-  // Request channel to the control shell.
-  private DataOutputStream miniClusterStdin;
-
-  // Response channel from the control shell.
-  private DataInputStream miniClusterStdout;
-
-  // Thread that reads and logs stderr from the control shell.
-  private Thread miniClusterErrorPrinter;
-
-  private static class DaemonInfo {
-    DaemonIdentifierPB id;
-    boolean isRunning;
-  }
-
-  // Map of master addresses to daemon information.
-  private final Map<HostAndPort, DaemonInfo> masterServers = Maps.newHashMap();
-
-  // Map of tserver addresses to daemon information.
-  private final Map<HostAndPort, DaemonInfo> tabletServers = Maps.newHashMap();
-
-  // Builder-provided cluster configuration state.
-  private final boolean enableKerberos;
-  private final int numMasters;
-  private final int numTservers;
-  private final ImmutableList<String> extraTserverFlags;
-  private final ImmutableList<String> extraMasterFlags;
-  private final String clusterRoot;
-
-  private MiniKdcOptionsPB kdcOptionsPb;
-  private final Common.HmsMode hmsMode;
-
-  private MiniKuduCluster(boolean enableKerberos,
-      int numMasters,
-      int numTservers,
-      List<String> extraTserverFlags,
-      List<String> extraMasterFlags,
-      MiniKdcOptionsPB kdcOptionsPb,
-      String clusterRoot,
-      Common.HmsMode hmsMode) {
-    this.enableKerberos = enableKerberos;
-    this.numMasters = numMasters;
-    this.numTservers = numTservers;
-    this.extraTserverFlags = ImmutableList.copyOf(extraTserverFlags);
-    this.extraMasterFlags = ImmutableList.copyOf(extraMasterFlags);
-    this.kdcOptionsPb = kdcOptionsPb;
-    this.hmsMode = hmsMode;
-
-    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 = getTempDirectory("mini-kudu-cluster");
-        this.clusterRoot = tempRoot.toString();
-      } catch (IOException ex) {
-        throw new RuntimeException("Could not create cluster root directory", ex);
-      }
-    } else {
-      this.clusterRoot = clusterRoot;
-    }
-  }
-
-  // Match the C++ MiniCluster test functionality for overriding the tmp directory used.
-  // See MakeClusterRoot in src/kudu/tools/tool_action_test.cc.
-  // If the TEST_TMPDIR environment variable is defined that directory will be used
-  // instead of the default temp directory.
-  private File getTempDirectory(String prefix) throws IOException  {
-    String testTmpdir = System.getenv("TEST_TMPDIR");
-    if (testTmpdir != null) {
-      LOG.info("Using the temp directory defined by TEST_TMPDIR: " + testTmpdir);
-      return Files.createTempDirectory(Paths.get(testTmpdir), prefix).toFile();
-    } else {
-      return Files.createTempDirectory(prefix).toFile();
-    }
-  }
-
-  /**
-   * Sends a command to the control shell and receives its response.
-   * <p>
-   * The method is synchronized to prevent interleaving of requests and responses.
-   * @param req control shell request
-   * @return control shell response
-   * @throws IOException if there was some kind of transport error, or if the
-   *                     response indicates an error
-   */
-  private synchronized ControlShellResponsePB sendRequestToCluster(ControlShellRequestPB req)
-      throws IOException {
-    // Send the request's size (4 bytes, big endian) followed by the request.
-    LOG.debug("Request: {}", req);
-    miniClusterStdin.writeInt(req.getSerializedSize());
-    miniClusterStdin.write(req.toByteArray());
-    miniClusterStdin.flush();
-
-    // Read the response's size (4 bytes, big endian) followed by the response.
-    int respLength = miniClusterStdout.readInt();
-    byte[] respBody = new byte[respLength];
-    miniClusterStdout.readFully(respBody);
-    ControlShellResponsePB resp = ControlShellResponsePB.parseFrom(respBody);
-    LOG.debug("Response: {}", resp);
-
-    // Convert any error into an exception.
-    if (resp.hasError()) {
-      throw new NonRecoverableException(Status.fromPB(resp.getError()));
-    }
-    return resp;
-  }
-
-  /**
-   * Starts this Kudu cluster.
-   * @throws IOException if something went wrong in transit
-   */
-  private void start() throws IOException {
-    Preconditions.checkArgument(numMasters > 0, "Need at least one master");
-
-    // Start the control shell and the communication channel to it.
-    List<String> commandLine = Lists.newArrayList(
-        KuduBinaryLocator.findBinary("kudu"),
-        "test",
-        "mini_cluster",
-        "--serialization=pb");
-    LOG.info("Starting process: {}", commandLine);
-    ProcessBuilder processBuilder = new ProcessBuilder(commandLine);
-    miniCluster = processBuilder.start();
-    miniClusterStdin = new DataOutputStream(miniCluster.getOutputStream());
-    miniClusterStdout = new DataInputStream(miniCluster.getInputStream());
-
-    // Set up a thread that logs stderr from the control shell; this will
-    // include all cluster logging.
-    ProcessInputStreamLogPrinterRunnable printer =
-        new ProcessInputStreamLogPrinterRunnable(miniCluster.getErrorStream());
-    miniClusterErrorPrinter = new Thread(printer);
-    miniClusterErrorPrinter.setDaemon(true);
-    miniClusterErrorPrinter.setName("cluster stderr printer");
-    miniClusterErrorPrinter.start();
-
-    // Create and start the cluster.
-    sendRequestToCluster(
-        ControlShellRequestPB.newBuilder()
-        .setCreateCluster(CreateClusterRequestPB.newBuilder()
-            .setNumMasters(numMasters)
-            .setNumTservers(numTservers)
-            .setEnableKerberos(enableKerberos)
-            .setHmsMode(hmsMode)
-            .addAllExtraMasterFlags(extraMasterFlags)
-            .addAllExtraTserverFlags(extraTserverFlags)
-            .setMiniKdcOptions(kdcOptionsPb)
-            .setClusterRoot(clusterRoot)
-            .build())
-        .build());
-    sendRequestToCluster(
-        ControlShellRequestPB.newBuilder()
-        .setStartCluster(StartClusterRequestPB.newBuilder().build())
-        .build());
-
-    // If the cluster is Kerberized, retrieve the KDC's environment variables
-    // and adapt them into certain security-related system properties.
-    if (enableKerberos) {
-      ControlShellResponsePB resp = sendRequestToCluster(
-          ControlShellRequestPB.newBuilder()
-          .setGetKdcEnvVars(GetKDCEnvVarsRequestPB.newBuilder().build())
-          .build());
-      for (Map.Entry<String, String> e : resp.getGetKdcEnvVars().getEnvVarsMap().entrySet()) {
-        if (e.getKey().equals("KRB5_CONFIG")) {
-          System.setProperty("java.security.krb5.conf", e.getValue());
-        } else if (e.getKey().equals("KRB5CCNAME")) {
-          System.setProperty(SecurityUtil.KUDU_TICKETCACHE_PROPERTY, e.getValue());
-        }
-      }
-    }
-
-    // Initialize the maps of master and tablet servers.
-    ControlShellResponsePB resp = sendRequestToCluster(
-        ControlShellRequestPB.newBuilder()
-        .setGetMasters(GetMastersRequestPB.newBuilder().build())
-        .build());
-    for (DaemonInfoPB info : resp.getGetMasters().getMastersList()) {
-      DaemonInfo d = new DaemonInfo();
-      d.id = info.getId();
-      d.isRunning = true;
-      masterServers.put(ProtobufHelper.hostAndPortFromPB(info.getBoundRpcAddress()), d);
-    }
-    resp = sendRequestToCluster(
-        ControlShellRequestPB.newBuilder()
-        .setGetTservers(GetTServersRequestPB.newBuilder().build())
-        .build());
-    for (DaemonInfoPB info : resp.getGetTservers().getTserversList()) {
-      DaemonInfo d = new DaemonInfo();
-      d.id = info.getId();
-      d.isRunning = true;
-      tabletServers.put(ProtobufHelper.hostAndPortFromPB(info.getBoundRpcAddress()), d);
-    }
-  }
-
-  /**
-   * @return comma-separated list of master server addresses
-   */
-  public String getMasterAddressesAsString() {
-    return Joiner.on(',').join(masterServers.keySet());
-  }
-
-  /**
-   * @return the list of master servers
-   */
-  public List<HostAndPort> getMasterServers() {
-    return new ArrayList(masterServers.keySet());
-  }
-
-  /**
-   * @return the list of tablet servers
-   */
-  public List<HostAndPort> getTabletServers() {
-    return new ArrayList(tabletServers.keySet());
-  }
-
-  /**
-   * Starts a master identified by a host and port.
-   * Does nothing if the server was already running.
-   *
-   * @param hp unique host and port identifying the server
-   * @throws IOException if something went wrong in transit
-   */
-  public void startMasterServer(HostAndPort hp) throws IOException {
-    DaemonInfo d = getMasterServer(hp);
-    if (d.isRunning) {
-      return;
-    }
-    LOG.info("Starting master server {}", hp);
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setStartDaemon(StartDaemonRequestPB.newBuilder().setId(d.id).build())
-        .build());
-    d.isRunning = true;
-  }
-
-  /**
-   * Kills a master identified identified by an host and port.
-   * Does nothing if the master was already dead.
-   *
-   * @param hp unique host and port identifying the server
-   * @throws IOException if something went wrong in transit
-   */
-  public void killMasterServer(HostAndPort hp) throws IOException {
-    DaemonInfo d = getMasterServer(hp);
-    if (!d.isRunning) {
-      return;
-    }
-    LOG.info("Killing master server {}", hp);
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setStopDaemon(StopDaemonRequestPB.newBuilder().setId(d.id).build())
-        .build());
-    d.isRunning = false;
-  }
-
-  /**
-   * Starts a tablet server identified by an host and port.
-   * Does nothing if the server was already running.
-   *
-   * @param hp unique host and port identifying the server
-   * @throws IOException if something went wrong in transit
-   */
-  public void startTabletServer(HostAndPort hp) throws IOException {
-    DaemonInfo d = getTabletServer(hp);
-    if (d.isRunning) {
-      return;
-    }
-    LOG.info("Starting tablet server {}", hp);
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setStartDaemon(StartDaemonRequestPB.newBuilder().setId(d.id).build())
-        .build());
-    d.isRunning = true;
-  }
-
-  /**
-   * Kills a tablet server identified by an host and port.
-   * Does nothing if the tablet server was already dead.
-   *
-   * @param hp unique host and port identifying the server
-   * @throws IOException if something went wrong in transit
-   */
-  public void killTabletServer(HostAndPort hp) throws IOException {
-    DaemonInfo d = getTabletServer(hp);
-    if (!d.isRunning) {
-      return;
-    }
-    LOG.info("Killing tablet server {}", hp);
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setStopDaemon(StopDaemonRequestPB.newBuilder().setId(d.id).build())
-        .build());
-    d.isRunning = false;
-  }
-
-  /**
-   * Kills all the master servers.
-   * Does nothing to the servers that are already dead.
-   *
-   * @throws IOException if something went wrong in transit
-   */
-  public void killAllMasterServers() throws IOException {
-    for (Map.Entry<HostAndPort, DaemonInfo> e : masterServers.entrySet()) {
-      killMasterServer(e.getKey());
-    }
-  }
-
-  /**
-   * Starts all the master servers.
-   * Does nothing to the servers that are already running.
-   *
-   * @throws IOException if something went wrong in transit
-   */
-  public void startAllMasterServers() throws IOException {
-    for (Map.Entry<HostAndPort, DaemonInfo> e : masterServers.entrySet()) {
-      startMasterServer(e.getKey());
-    }
-  }
-
-  /**
-   * Kills all tablet servers.
-   * Does nothing to the servers that are already dead.
-   *
-   * @throws IOException if something went wrong in transit
-   */
-  public void killAllTabletServers() throws IOException {
-    for (Map.Entry<HostAndPort, DaemonInfo> e : tabletServers.entrySet()) {
-      killTabletServer(e.getKey());
-    }
-  }
-
-  /**
-   * Starts all the tablet servers.
-   * Does nothing to the servers that are already running.
-   *
-   * @throws IOException if something went wrong in transit
-   */
-  public void startAllTabletServers() throws IOException {
-    for (Map.Entry<HostAndPort, DaemonInfo> e : tabletServers.entrySet()) {
-      startTabletServer(e.getKey());
-    }
-  }
-
-  /**
-   * Removes all credentials for all principals from the Kerberos credential cache.
-   */
-  public void kdestroy() throws IOException {
-    LOG.info("Destroying all Kerberos credentials");
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setKdestroy(KdestroyRequestPB.getDefaultInstance())
-        .build());
-  }
-
-  /**
-   * Re-initialize Kerberos credentials for the given username, writing them
-   * into the Kerberos credential cache.
-   * @param username the username to kinit as
-   */
-  public void kinit(String username) throws IOException {
-    LOG.info("Running kinit for user {}", username);
-    sendRequestToCluster(ControlShellRequestPB.newBuilder()
-        .setKinit(KinitRequestPB.newBuilder().setUsername(username).build())
-        .build());
-  }
-
-
-  /** {@override} */
-  @Override
-  public void close() {
-    shutdown();
-  }
-
-  /**
-   * Shuts down a Kudu cluster.
-   */
-  public void shutdown() {
-    // Closing stdin should cause the control shell process to terminate.
-    if (miniClusterStdin != null) {
-      try {
-        miniClusterStdin.close();
-      } catch (IOException e) {
-        LOG.info("Caught exception while closing minicluster stdin", e);
-      }
-    }
-    if (miniClusterStdout != null) {
-      try {
-        miniClusterStdout.close();
-      } catch (IOException e) {
-        LOG.info("Caught exception while closing minicluster stdout", e);
-      }
-    }
-    if (miniClusterErrorPrinter != null) {
-      try {
-        miniClusterErrorPrinter.join();
-      } catch (InterruptedException e) {
-        LOG.info("Caught exception while closing minicluster stderr", e);
-      }
-    }
-    if (miniCluster != null) {
-      try {
-        miniCluster.waitFor();
-      } catch (InterruptedException e) {
-        LOG.warn("Minicluster process did not exit, destroying");
-        miniCluster.destroy();
-      }
-    }
-  }
-
-  /**
-   * Returns a master server identified by an address.
-   *
-   * @param hp unique host and port identifying the server
-   * @return the DaemonInfo of the server
-   * @throws RuntimeException if the server is not found
-   */
-  private DaemonInfo getMasterServer(HostAndPort hp) throws RuntimeException {
-    DaemonInfo d = masterServers.get(hp);
-    if (d == null) {
-      throw new RuntimeException(String.format("Master server %s not found", hp));
-    }
-    return d;
-  }
-
-  /**
-   * Returns a tablet server identified by an address.
-   *
-   * @param hp unique host and port identifying the server
-   * @return the DaemonInfo of the server
-   * @throws RuntimeException if the server is not found
-   */
-  private DaemonInfo getTabletServer(HostAndPort hp) throws RuntimeException {
-    DaemonInfo d = tabletServers.get(hp);
-    if (d == null) {
-      throw new RuntimeException(String.format("Tablet server %s not found", hp));
-    }
-    return d;
-  }
-
-  /**
-   * @return path to the mini cluster root directory
-   */
-  public String getClusterRoot() {
-    return clusterRoot;
-  }
-
-  /**
-   * Helper runnable that receives stderr and logs it along with the process' identifier.
-   */
-  public static class ProcessInputStreamLogPrinterRunnable implements Runnable {
-
-    private final InputStream is;
-
-    public ProcessInputStreamLogPrinterRunnable(InputStream is) {
-      this.is = is;
-    }
-
-    @Override
-    public void run() {
-      try {
-        String line;
-        BufferedReader in = new BufferedReader(
-            new InputStreamReader(is, UTF_8));
-        while ((line = in.readLine()) != null) {
-          LOG.info(line);
-        }
-        in.close();
-      } catch (Exception e) {
-        if (!e.getMessage().contains("Stream closed")) {
-          LOG.error("Caught error while reading a process' output", e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Builder for {@link MiniKuduCluster}
-   */
-  public static class MiniKuduClusterBuilder {
-
-    private int numMasterServers = 1;
-    private int numTabletServers = 3;
-    private boolean enableKerberos = false;
-    private final List<String> extraTabletServerFlags = new ArrayList<>();
-    private final List<String> extraMasterServerFlags = new ArrayList<>();
-    private String clusterRoot = null;
-
-    private MiniKdcOptionsPB.Builder kdcOptionsPb = MiniKdcOptionsPB.newBuilder();
-    private Common.HmsMode hmsMode = Common.HmsMode.NONE;
-
-    public MiniKuduClusterBuilder numMasterServers(int numMasterServers) {
-      this.numMasterServers = numMasterServers;
-      return this;
-    }
-
-    public MiniKuduClusterBuilder numTabletServers(int numTabletServers) {
-      this.numTabletServers = numTabletServers;
-      return this;
-    }
-
-    /**
-     * Enables Kerberos on the mini cluster and acquire client credentials for this process.
-     * @return this instance
-     */
-    public MiniKuduClusterBuilder enableKerberos() {
-      enableKerberos = true;
-      return this;
-    }
-
-    public MiniKuduClusterBuilder enableHiveMetastoreIntegration() {
-      hmsMode = Common.HmsMode.ENABLE_METASTORE_INTEGRATION;
-      return this;
-    }
-
-    /**
-     * Adds a new flag to be passed to the Tablet Server daemons on start.
-     * @return this instance
-     */
-    public MiniKuduClusterBuilder addTabletServerFlag(String flag) {
-      this.extraTabletServerFlags.add(flag);
-      return this;
-    }
-
-    /**
-     * Adds a new flag to be passed to the Master daemons on start.
-     * @return this instance
-     */
-    public MiniKuduClusterBuilder addMasterServerFlag(String flag) {
-      this.extraMasterServerFlags.add(flag);
-      return this;
-    }
-
-    public MiniKuduClusterBuilder kdcTicketLifetime(String lifetime) {
-      this.kdcOptionsPb.setTicketLifetime(lifetime);
-      return this;
-    }
-
-    public MiniKuduClusterBuilder kdcRenewLifetime(String lifetime) {
-      this.kdcOptionsPb.setRenewLifetime(lifetime);
-      return this;
-    }
-
-    /**
-     * 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
-     */
-    public MiniKuduCluster build() throws IOException {
-      MiniKuduCluster cluster =
-          new MiniKuduCluster(enableKerberos,
-              numMasterServers, numTabletServers,
-              extraTabletServerFlags, extraMasterServerFlags,
-              kdcOptionsPb.build(), clusterRoot, hmsMode);
-      try {
-        cluster.start();
-      } catch (IOException e) {
-        // MiniKuduCluster.close should not throw, so no need for a nested try/catch.
-        cluster.close();
-        throw e;
-      }
-      return cluster;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 8227fce..ac0610f 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
@@ -16,8 +16,8 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInTable;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInTable;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 9a0feac..a622459 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
@@ -18,9 +18,9 @@ package org.apache.kudu.client;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
@@ -34,8 +34,8 @@ import com.google.common.base.Stopwatch;
 import com.google.protobuf.ByteString;
 import com.stumbleupon.async.Deferred;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
-import org.apache.kudu.util.ProtobufUtils;
+import org.apache.kudu.test.ClientTestUtil;
+import org.apache.kudu.test.ProtobufUtils;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 ae4f161..a7c9112 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
@@ -17,11 +17,11 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.defaultErrorCB;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.defaultErrorCB;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 5b765f8..7d332b0 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
@@ -23,9 +23,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.List;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -33,9 +33,9 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import org.apache.kudu.Schema;
-import org.apache.kudu.client.MiniKuduCluster.MiniKuduClusterBuilder;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 f019e43..32a499c 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
@@ -17,17 +17,17 @@
 
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.kudu.Schema;
-import org.apache.kudu.client.MiniKuduCluster.MiniKuduClusterBuilder;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 c30ef72..e24cb5e 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
@@ -17,10 +17,10 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.AssertHelpers.assertEventuallyTrue;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.junit.AssertHelpers.assertEventuallyTrue;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
@@ -30,9 +30,9 @@ import java.util.List;
 
 import org.apache.kudu.Schema;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.AssertHelpers.BooleanExpression;
-import org.apache.kudu.util.CapturingLogAppender;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.junit.AssertHelpers.BooleanExpression;
+import org.apache.kudu.test.CapturingLogAppender;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectToCluster.java
----------------------------------------------------------------------
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 57ac3cd..bb73585 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
@@ -26,6 +26,7 @@ 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.junit.Assert;
 import org.junit.Test;
 import org.apache.kudu.consensus.Metadata;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectionCache.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectionCache.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectionCache.java
index 72ff6a3..7cdc6f0 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectionCache.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestConnectionCache.java
@@ -23,7 +23,8 @@ import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 
 import com.stumbleupon.async.Deferred;
-import org.apache.kudu.junit.RetryRule;
+import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.junit.RetryRule;
 import org.junit.Rule;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 e506809..d4d8687 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
@@ -24,13 +24,13 @@ import java.util.concurrent.Future;
 
 import com.google.common.collect.Lists;
 import org.apache.kudu.Schema;
-import org.apache.kudu.client.MiniKuduCluster.MiniKuduClusterBuilder;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
 
 /**
  * Tests which provoke RPC queue overflow errors on the server side

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 14fb1c4..f73d780 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
@@ -18,7 +18,7 @@ package org.apache.kudu.client;
 
 import static org.apache.kudu.Type.STRING;
 import static org.apache.kudu.client.ExternalConsistencyMode.CLIENT_PROPAGATED;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
 import static org.apache.kudu.util.HybridTimeUtil.HTTimestampToPhysicalAndLogical;
 import static org.apache.kudu.util.HybridTimeUtil.clockTimestampToHTTimestamp;
 import static org.apache.kudu.util.HybridTimeUtil.physicalAndLogicalToHTTimestamp;
@@ -31,7 +31,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.kudu.client.MiniKuduCluster.MiniKuduClusterBuilder;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.KuduTestHarness;
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 49357f6..018d1e6 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
@@ -21,15 +21,15 @@ import static org.apache.kudu.client.KuduPredicate.ComparisonOp.GREATER;
 import static org.apache.kudu.client.KuduPredicate.ComparisonOp.GREATER_EQUAL;
 import static org.apache.kudu.client.KuduPredicate.ComparisonOp.LESS;
 import static org.apache.kudu.client.KuduPredicate.ComparisonOp.LESS_EQUAL;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.createManyStringsSchema;
-import static org.apache.kudu.util.ClientTestUtil.createSchemaWithBinaryColumns;
-import static org.apache.kudu.util.ClientTestUtil.createSchemaWithDecimalColumns;
-import static org.apache.kudu.util.ClientTestUtil.createSchemaWithTimestampColumns;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.createManyStringsSchema;
+import static org.apache.kudu.test.ClientTestUtil.createSchemaWithBinaryColumns;
+import static org.apache.kudu.test.ClientTestUtil.createSchemaWithDecimalColumns;
+import static org.apache.kudu.test.ClientTestUtil.createSchemaWithTimestampColumns;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -55,7 +55,7 @@ import com.stumbleupon.async.Deferred;
 
 import org.apache.kudu.test.KuduTestHarness;
 import org.apache.kudu.test.KuduTestHarness.TabletServerConfig;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.apache.kudu.util.TimestampUtil;
 import org.junit.Before;
 import org.junit.Rule;
@@ -64,7 +64,7 @@ import org.junit.Test;
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
-import org.apache.kudu.util.CapturingLogAppender;
+import org.apache.kudu.test.CapturingLogAppender;
 import org.apache.kudu.util.DecimalUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 46624c0..09bacb1 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
@@ -16,11 +16,11 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -33,7 +33,7 @@ import java.util.List;
 import com.google.common.collect.ImmutableList;
 import org.apache.kudu.Schema;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 4d95f9a..d49fdda 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
@@ -17,11 +17,11 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
-import static org.apache.kudu.util.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
-import static org.apache.kudu.util.ClientTestUtil.scanTableToStrings;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.getBasicTableOptionsWithNonCoveredRange;
+import static org.apache.kudu.test.ClientTestUtil.scanTableToStrings;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -35,7 +35,7 @@ import java.util.List;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.ClientTestUtil;
+import org.apache.kudu.test.ClientTestUtil;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 ba108c9..663d512 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
@@ -16,10 +16,10 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 7c3237f..74004b9 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
@@ -16,9 +16,9 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.kudu.test.KuduTestHarness;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
deleted file mode 100644
index bc03d3d..0000000
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed 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. See accompanying LICENSE file.
- */
-package org.apache.kudu.client;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.net.Socket;
-
-import org.apache.kudu.client.KuduClient.KuduClientBuilder;
-import org.apache.kudu.junit.RetryRule;
-import org.junit.Rule;
-import org.junit.Test;
-
-public class TestMiniKuduCluster {
-
-  private static final int NUM_TABLET_SERVERS = 3;
-  private static final int NUM_MASTERS = 1;
-  private static final long SLEEP_TIME_MS = 10000;
-
-  @Rule
-  public RetryRule retryRule = new RetryRule();
-
-  @Test(timeout = 50000)
-  public void test() throws Exception {
-    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
-                                                      .numMasterServers(NUM_MASTERS)
-                                                      .numTabletServers(NUM_TABLET_SERVERS)
-                                                      .build()) {
-      assertEquals(NUM_MASTERS, cluster.getMasterServers().size());
-      assertEquals(NUM_TABLET_SERVERS, cluster.getTabletServers().size());
-
-      {
-        // Kill the master.
-        HostAndPort masterHostPort = cluster.getMasterServers().get(0);
-        testHostPort(masterHostPort, true);
-        cluster.killMasterServer(masterHostPort);
-
-        testHostPort(masterHostPort, false);
-
-        // Restart the master.
-        cluster.startMasterServer(masterHostPort);
-
-        // Test we can reach it.
-        testHostPort(masterHostPort, true);
-      }
-
-      {
-        // Kill the first TS.
-        HostAndPort tsHostPort = cluster.getTabletServers().get(0);
-        testHostPort(tsHostPort, true);
-        cluster.killTabletServer(tsHostPort);
-
-        testHostPort(tsHostPort, false);
-
-        // Restart it.
-        cluster.startTabletServer(tsHostPort);
-
-        testHostPort(tsHostPort, true);
-      }
-    }
-  }
-
-  @Test(timeout = 50000)
-  public void testKerberos() throws Exception {
-    FakeDNS.getInstance().install();
-    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
-                                                      .numMasterServers(NUM_MASTERS)
-                                                      .numTabletServers(NUM_TABLET_SERVERS)
-                                                      .enableKerberos()
-                                                      .build()) {
-      KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build();
-      ListTablesResponse resp = client.getTablesList();
-      assertTrue(resp.getTablesList().isEmpty());
-      assertNull(client.getHiveMetastoreConfig());
-    }
-  }
-
-  @Test(timeout = 50000)
-  public void testHiveMetastoreIntegration() throws Exception {
-    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
-                                                      .numMasterServers(NUM_MASTERS)
-                                                      .numTabletServers(NUM_TABLET_SERVERS)
-                                                      .enableHiveMetastoreIntegration()
-                                                      .build()) {
-      KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build();
-      assertNotNull(client.getHiveMetastoreConfig());
-    }
-  }
-
-  /**
-   * Test whether the specified host and port is open or closed, waiting up to a certain time.
-   * @param hp the host and port to test
-   * @param testIsOpen true if we should want it to be open, false if we want it closed
-   */
-  private static void testHostPort(HostAndPort hp,
-                                   boolean testIsOpen) throws InterruptedException {
-    DeadlineTracker tracker = new DeadlineTracker();
-    while (tracker.getElapsedMillis() < SLEEP_TIME_MS) {
-      try {
-        Socket socket = new Socket(hp.getHost(), hp.getPort());
-        socket.close();
-        if (testIsOpen) {
-          return;
-        }
-      } catch (IOException e) {
-        if (!testIsOpen) {
-          return;
-        }
-      }
-      Thread.sleep(200);
-    }
-    fail("HostAndPort " + hp + " is still " + (testIsOpen ? "closed " : "open"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 3152c33..5d47fc7 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
@@ -17,18 +17,18 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.AssertHelpers.assertEventuallyTrue;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.junit.AssertHelpers.assertEventuallyTrue;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.util.List;
 
 import org.apache.kudu.test.KuduTestHarness;
-import org.apache.kudu.util.AssertHelpers.BooleanExpression;
+import org.apache.kudu.test.junit.AssertHelpers.BooleanExpression;
 import org.junit.Rule;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
index 91f8273..bfcf716 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
@@ -17,8 +17,10 @@
 
 package org.apache.kudu.client;
 
-import org.apache.kudu.junit.RetryRule;
-import org.apache.kudu.util.CapturingLogAppender;
+import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.junit.RetryRule;
+import org.apache.kudu.test.cluster.FakeDNS;
+import org.apache.kudu.test.CapturingLogAppender;
 import org.junit.Rule;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
----------------------------------------------------------------------
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 adb2efc..78f2768 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
@@ -17,7 +17,7 @@
 
 package org.apache.kudu.client;
 
-import static org.apache.kudu.util.ClientTestUtil.getSchemaWithAllTypes;
+import static org.apache.kudu.test.ClientTestUtil.getSchemaWithAllTypes;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/java/kudu-client/src/test/java/org/apache/kudu/client/TestRemoteTablet.java
----------------------------------------------------------------------
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 c5cc27f..a9b3747 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,7 +28,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.google.protobuf.ByteString;
-import org.apache.kudu.util.ProtobufUtils;
+import org.apache.kudu.test.ProtobufUtils;
 import org.junit.Test;
 
 import org.apache.kudu.consensus.Metadata;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 3290bea..5391e82 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
@@ -17,10 +17,10 @@
 package org.apache.kudu.client;
 
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.createBasicSchemaInsert;
-import static org.apache.kudu.util.ClientTestUtil.createFourTabletsTableWithNineRows;
-import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getBasicSchema;
+import static org.apache.kudu.test.ClientTestUtil.createBasicSchemaInsert;
+import static org.apache.kudu.test.ClientTestUtil.createFourTabletsTableWithNineRows;
+import static org.apache.kudu.test.ClientTestUtil.getBasicCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getBasicSchema;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 f64a1d4..82c5956 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
@@ -17,8 +17,8 @@
 package org.apache.kudu.client;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.kudu.util.ClientTestUtil.getAllTypesCreateTableOptions;
-import static org.apache.kudu.util.ClientTestUtil.getSchemaWithAllTypes;
+import static org.apache.kudu.test.ClientTestUtil.getAllTypesCreateTableOptions;
+import static org.apache.kudu.test.ClientTestUtil.getSchemaWithAllTypes;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 55f6f78..87a9086 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
@@ -31,10 +31,10 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.List;
 
-import static org.apache.kudu.util.ClientTestUtil.countScanTokenRows;
-import static org.apache.kudu.util.ClientTestUtil.createDefaultTable;
-import static org.apache.kudu.util.ClientTestUtil.createManyStringsSchema;
-import static org.apache.kudu.util.ClientTestUtil.loadDefaultTable;
+import static org.apache.kudu.test.ClientTestUtil.countScanTokenRows;
+import static org.apache.kudu.test.ClientTestUtil.createDefaultTable;
+import static org.apache.kudu.test.ClientTestUtil.createManyStringsSchema;
+import static org.apache.kudu.test.ClientTestUtil.loadDefaultTable;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;

http://git-wip-us.apache.org/repos/asf/kudu/blob/15f1416f/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 4856e25..dabe437 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
@@ -18,7 +18,7 @@ package org.apache.kudu.client;
 
 import static org.apache.kudu.Type.STRING;
 import static org.apache.kudu.test.KuduTestHarness.DEFAULT_SLEEP;
-import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
+import static org.apache.kudu.test.ClientTestUtil.countRowsInScan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;