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/09/21 13:45:36 UTC

[4/4] kudu git commit: [test] Clean up MiniKuduCluster and BaseKuduTest

[test] Clean up MiniKuduCluster and BaseKuduTest

This patch cleans up the MiniKuduCluster and
BaseKuduTest API in preparation for creating a
public test API and improving per-test
MiniKuduCluster parameters.

This patch also removes the use of Guava’s
HostAndPort as part of the clean up. It replaces
it with our own HostAndPort which wraps an
unresolved InetSocketAddress. We can’t use
Guava’s HostAndPort because it is shaded and
relocated in all of our libraries. This means it shouldn’t
be a part of the “public” API. In this case that means
the test API too. Additionally HostAndPort is marked
as beta.

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


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

Branch: refs/heads/master
Commit: fd1ffd0fb65e138f1f015a55aa96ae870c1d51cd
Parents: e48157a
Author: Grant Henke <gr...@apache.org>
Authored: Wed Sep 19 12:47:38 2018 -0500
Committer: Grant Henke <gr...@apache.org>
Committed: Fri Sep 21 13:42:35 2018 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/backup/TestKuduBackup.scala |   4 +-
 .../kudu/mapreduce/tools/ITExportCsv.java       |   2 +-
 .../kudu/mapreduce/tools/ITImportCsv.java       |   2 +-
 .../kudu/mapreduce/tools/ITImportParquet.java   |   2 +-
 .../tools/ITImportParquetPreCheck.java          |   2 +-
 .../tools/ITIntegrationTestBigLinkedList.java   |   2 +-
 .../kudu/mapreduce/tools/ITRowCounter.java      |   2 +-
 .../org/apache/kudu/client/AsyncKuduClient.java |   5 +-
 .../apache/kudu/client/ConnectToCluster.java    |   9 +-
 .../kudu/client/ConnectToClusterResponse.java   |   3 +-
 .../org/apache/kudu/client/HostAndPort.java     |  74 +++++++
 .../org/apache/kudu/client/ProtobufHelper.java  |  11 +-
 .../java/org/apache/kudu/client/ServerInfo.java |   1 -
 .../main/java/org/apache/kudu/util/NetUtil.java |  13 +-
 .../org/apache/kudu/client/BaseKuduTest.java    | 140 ++++++++----
 .../java/org/apache/kudu/client/ITClient.java   |   2 +-
 .../org/apache/kudu/client/ITClientStress.java  |   2 +-
 .../org/apache/kudu/client/MiniKuduCluster.java | 213 ++++++++++---------
 .../apache/kudu/client/TestAsyncKuduClient.java |   2 +-
 .../kudu/client/TestAsyncKuduSession.java       |  10 +-
 .../kudu/client/TestClientFailoverSupport.java  |   7 +-
 .../kudu/client/TestConnectToCluster.java       |  13 +-
 .../apache/kudu/client/TestConnectionCache.java |   5 +-
 .../apache/kudu/client/TestHandleTooBusy.java   |   2 +-
 .../org/apache/kudu/client/TestKuduClient.java  |  14 +-
 .../apache/kudu/client/TestMasterFailover.java  |  14 +-
 .../apache/kudu/client/TestMiniKuduCluster.java |  24 +--
 .../kudu/client/TestMultipleLeaderFailover.java |   7 +-
 .../org/apache/kudu/client/TestNegotiation.java |   2 +-
 .../apache/kudu/client/TestRemoteTablet.java    |   3 +-
 .../org/apache/kudu/client/TestSecurity.java    |  22 +-
 .../client/TestSecurityContextRealUser.java     |   6 +-
 .../org/apache/kudu/client/TestServerInfo.java  |   5 +-
 .../org/apache/kudu/client/TestTimeouts.java    |   2 +-
 .../java/org/apache/kudu/util/TestNetUtil.java  |  13 +-
 .../sink/AvroKuduOperationsProducerTest.java    |   2 +-
 .../sink/KeyedKuduOperationsProducerTest.java   |   2 +-
 .../apache/kudu/flume/sink/KuduSinkTest.java    |   2 +-
 .../sink/RegexpKuduOperationsProducerTest.java  |   2 +-
 .../apache/kudu/mapreduce/ITInputFormatJob.java |   2 +-
 .../kudu/mapreduce/ITKuduTableInputFormat.java  |   2 +-
 .../kudu/mapreduce/ITKuduTableOutputFormat.java |   2 +-
 .../kudu/mapreduce/ITOutputFormatJob.java       |   2 +-
 .../kudu/spark/tools/ITBigLinkedListTest.scala  |   4 +-
 .../spark/tools/TestImportExportFiles.scala     |   2 +-
 .../kudu/spark/kudu/DefaultSourceTest.scala     |  39 ++--
 .../kudu/spark/kudu/KuduContextTest.scala       |   2 +-
 .../apache/kudu/spark/kudu/KuduTestSuite.scala  |   4 +-
 48 files changed, 415 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 4c4e5f0..aa4d603 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
@@ -351,11 +351,11 @@ class TestKuduBackup extends KuduTestSuite {
     val path = dir.toUri.toString
 
     val backupOptions =
-      new KuduBackupOptions(tableNames, path, miniCluster.getMasterAddresses)
+      new KuduBackupOptions(tableNames, path, miniCluster.getMasterAddressesAsString)
     KuduBackup.run(backupOptions, ss)
 
     val restoreOptions =
-      new KuduRestoreOptions(tableNames, path, miniCluster.getMasterAddresses)
+      new KuduRestoreOptions(tableNames, path, miniCluster.getMasterAddressesAsString)
     KuduRestore.run(restoreOptions, ss)
 
     FileUtils.deleteDirectory(dir.toFile)

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 3fa4b83..16e548e 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
@@ -57,7 +57,7 @@ public class ITExportCsv extends BaseKuduTest {
     // create a table with on empty tablet and 3 tablets of 3 rows each.
     createFourTabletsTableWithNineRows(client, TABLE_NAME, DEFAULT_SLEEP);
     String[] args = new String[] {
-      "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddresses(),
+      "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddressesAsString(),
       "*", TABLE_NAME, testHome + "/exportdata"};
 
     GenericOptionsParser parser = new GenericOptionsParser(conf, args);

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 0cc0740..670880e 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
@@ -99,7 +99,7 @@ public class ITImportCsv extends BaseKuduTest {
     }
     sb.deleteCharAt(sb.length() - 1);
     String[] args = new String[] {
-        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddresses(),
+        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddressesAsString(),
         sb.toString(), TABLE_NAME, data.toString()};
 
     GenericOptionsParser parser = new GenericOptionsParser(conf, args);

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 bcf7ddf..7156d3b 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
@@ -104,7 +104,7 @@ public class ITImportParquet extends BaseKuduTest {
       sb.append(",");
     }
     sb.deleteCharAt(sb.length() - 1);
-    String[] args = new String[] { "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddresses(),
+    String[] args = new String[] { "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddressesAsString(),
       TABLE_NAME, data.toString()};
 
     GenericOptionsParser parser = new GenericOptionsParser(conf, args);

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 bf52972..c286edb 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
@@ -108,7 +108,7 @@ public class ITImportParquetPreCheck extends BaseKuduTest {
     }
     sb.deleteCharAt(sb.length() - 1);
     String[] args = new String[] { "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" +
-      getMasterAddresses(), TABLE_NAME, data.toString()};
+      getMasterAddressesAsString(), TABLE_NAME, data.toString()};
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("The column column1_i does not exist in Parquet schema");

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 1cfb8c4..a7d1973 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
@@ -43,7 +43,7 @@ public class ITIntegrationTestBigLinkedList extends BaseKuduTest {
         ITIntegrationTestBigLinkedList.class.getName(),conf).getAbsolutePath();
 
     String[] args = new String[] {
-        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddresses(),
+        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddressesAsString(),
         "Loop",
         "2", // Two iterations
         "1", // 1 mapper

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 7f62753..011921b 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
@@ -51,7 +51,7 @@ public class ITRowCounter extends BaseKuduTest {
     createFourTabletsTableWithNineRows(client, TABLE_NAME, DEFAULT_SLEEP);
 
     String[] args = new String[] {
-        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddresses(), TABLE_NAME};
+        "-D" + CommandLineParser.MASTER_ADDRESSES_KEY + "=" + getMasterAddressesAsString(), TABLE_NAME};
     GenericOptionsParser parser = new GenericOptionsParser(conf, args);
     Job job = RowCounter.createSubmittableJob(parser.getConfiguration(), parser.getRemainingArgs());
     assertTrue("Job did not end properly", job.waitForCompletion(true));

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
index 62425a4..3407ad9 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
@@ -51,7 +51,6 @@ import javax.annotation.concurrent.GuardedBy;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.net.HostAndPort;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Message;
@@ -705,8 +704,8 @@ public class AsyncKuduClient implements AutoCloseable {
   /**
    * Gets a table's schema either by ID or by name. Note: the name must be
    * provided, even if the RPC should be sent by ID.
-   * @param name name of table
-   * @param id immutable ID of table
+   * @param tableName name of table
+   * @param tableId immutable ID of table
    * @param parent parent RPC (for tracing), if any
    * @return a deferred object that yields the schema
    */

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToCluster.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToCluster.java
index eac5d37..aecb51f 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToCluster.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToCluster.java
@@ -28,7 +28,6 @@ import com.google.common.base.Functions;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.net.HostAndPort;
 import com.stumbleupon.async.Callback;
 import com.stumbleupon.async.Deferred;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -188,8 +187,8 @@ final class ConnectToCluster {
   /**
    * Creates a callback for a ConnectToMaster RPC that was sent to 'hostAndPort'.
    * @see ConnectToMasterCB
-   * @param hostAndPort Host and part for the RPC we're attaching this to. Host and port must
-   *                    be valid.
+   * @param hostAndPort Host and port for the RPC we're attaching this to.
+   *                    Host and port must be valid.
    * @return The callback object that can be added to the RPC request.
    */
   @InterfaceAudience.LimitedPrivate("Test")
@@ -200,8 +199,8 @@ final class ConnectToCluster {
   /**
    * Creates an errback for a ConnectToMaster that was sent to 'hostAndPort'.
    * @see ConnectToMasterErrCB
-   * @param hostAndPort Host and port for the RPC we're attaching this to. Used for debugging
-   *                    purposes.
+   * @param hostAndPort Host and port for the RPC we're attaching this to.
+   *                Used for debugging purposes.
    * @return The errback object that can be added to the RPC request.
    */
   @InterfaceAudience.LimitedPrivate("Test")

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
index bfa1f2b..97dd14c 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ConnectToClusterResponse.java
@@ -17,7 +17,6 @@
 
 package org.apache.kudu.client;
 
-import com.google.common.net.HostAndPort;
 import com.google.protobuf.ByteString;
 
 import org.apache.kudu.Common.PartitionPB;
@@ -41,7 +40,7 @@ class ConnectToClusterResponse {
   private final ConnectToMasterResponsePB connectResponse;
 
   public ConnectToClusterResponse(HostAndPort hostAndPort,
-      ConnectToMasterResponsePB connectResponse) {
+                                  ConnectToMasterResponsePB connectResponse) {
     super();
     this.leaderHostAndPort = hostAndPort;
     this.connectResponse = connectResponse;

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/HostAndPort.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/HostAndPort.java b/java/kudu-client/src/main/java/org/apache/kudu/client/HostAndPort.java
new file mode 100644
index 0000000..b6960a6
--- /dev/null
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/HostAndPort.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.client;
+
+import com.google.common.base.Objects;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.net.InetSocketAddress;
+
+/**
+ * A simple wrapper around InetSocketAddress to prevent
+ * accidentally introducing DNS lookups.
+ *
+ * The HostAndPort implementation in Guava is not used
+ * because Guava is shaded and relocated in Kudu preventing
+ * it from being used as a parameter or return value on
+ * public methods. Additionally Guava's HostAndPort
+ * implementation is marked as beta.
+ */
+@InterfaceAudience.Private
+public class HostAndPort {
+
+  private final InetSocketAddress address;
+
+  public HostAndPort(String host, int port) {
+    // Using createUnresolved ensures no lookups will occur.
+    this.address = InetSocketAddress.createUnresolved(host, port);
+  }
+
+  public String getHost() {
+    // Use getHostString to ensure no reverse lookup is done.
+    return address.getHostString();
+  }
+
+  public int getPort() {
+    return address.getPort();
+  }
+
+  public InetSocketAddress getAddress() {
+    return address;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    HostAndPort that = (HostAndPort) o;
+    return Objects.equal(address, that.address);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(address);
+  }
+
+  @Override
+  public String toString() {
+    return address.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
index d49fa8f..432ad28 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
@@ -27,7 +27,6 @@ import java.util.List;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
-import com.google.common.net.HostAndPort;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.UnsafeByteOperations;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -322,7 +321,7 @@ public class ProtobufHelper {
   }
 
   /**
-   * Convert a {@link com.google.common.net.HostAndPort} to
+   * Convert a {@link HostAndPort} to
    *     {@link org.apache.kudu.Common.HostPortPB}
    * protobuf message for serialization.
    * @param hostAndPort The host and port object. Both host and port must be specified.
@@ -337,13 +336,13 @@ public class ProtobufHelper {
 
   /**
    * Convert a {@link org.apache.kudu.Common.HostPortPB} to
-   *     {@link com.google.common.net.HostAndPort}.
-   * @param hostPortPB The fully initialized HostPortPB object. Must have both host and port
-   *                   specified.
+   *     {@link HostAndPort}.
+   * @param hostPortPB The fully initialized HostPortPB object.
+   *                   Must have both host and port specified.
    * @return An initialized initialized HostAndPort object.
    */
   public static HostAndPort hostAndPortFromPB(Common.HostPortPB hostPortPB) {
-    return HostAndPort.fromParts(hostPortPB.getHost(), hostPortPB.getPort());
+    return new HostAndPort(hostPortPB.getHost(), hostPortPB.getPort());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/client/ServerInfo.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ServerInfo.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ServerInfo.java
index bed3e21..e2e4162 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ServerInfo.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ServerInfo.java
@@ -23,7 +23,6 @@ import java.net.UnknownHostException;
 import java.util.concurrent.ConcurrentHashMap;
 
 import com.google.common.base.Preconditions;
-import com.google.common.net.HostAndPort;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.kudu.util.NetUtil;

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/main/java/org/apache/kudu/util/NetUtil.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/util/NetUtil.java b/java/kudu-client/src/main/java/org/apache/kudu/util/NetUtil.java
index 2a0101c..ced1b70 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/util/NetUtil.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/util/NetUtil.java
@@ -28,7 +28,7 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.net.HostAndPort;
+import org.apache.kudu.client.HostAndPort;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,16 +53,19 @@ public class NetUtil {
   }
 
   /**
-   * Parse a "host:port" pair into a {@link HostAndPort} object. If there is no
-   * port specified in the string, then 'defaultPort' is used.
+   * Parse a "host:port" pair into a {@link HostAndPort} object.
+   * If there is no port specified in the string, then 'defaultPort' is used.
    *
    * @param addrString  A host or a "host:port" pair.
    * @param defaultPort Default port to use if no port is specified in addrString.
    * @return The HostAndPort object constructed from addrString.
    */
   public static HostAndPort parseString(String addrString, int defaultPort) {
-    return addrString.indexOf(':') == -1 ? HostAndPort.fromParts(addrString, defaultPort) :
-               HostAndPort.fromString(addrString);
+    // Use Guava's HostAndPort so we don't need to handle parsing ourselves.
+    com.google.common.net.HostAndPort hostAndPort = addrString.indexOf(':') == -1 ?
+        com.google.common.net.HostAndPort.fromParts(addrString, defaultPort) :
+        com.google.common.net.HostAndPort.fromString(addrString);
+    return new HostAndPort(hostAndPort.getHost(), hostAndPort.getPort());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 17d6c3c..d66a65e 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
@@ -26,7 +26,6 @@ import java.util.Random;
 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.apache.yetus.audience.InterfaceAudience;
@@ -57,11 +56,7 @@ public class BaseKuduTest {
   private static final int NUM_MASTERS = 3;
   private static final int NUM_TABLET_SERVERS = 3;
 
-  protected MiniKuduCluster miniCluster;
-
-  // Comma separate describing the master addresses and ports.
-  protected String masterAddresses;
-  protected List<HostAndPort> masterHostPorts;
+  private MiniKuduCluster miniCluster;
 
   // We create both versions of the client for ease of use.
   protected AsyncKuduClient client;
@@ -81,11 +76,9 @@ public class BaseKuduTest {
     LOG.info("Creating a new MiniKuduCluster...");
 
     miniCluster = getMiniClusterBuilder().build();
-    masterAddresses = miniCluster.getMasterAddresses();
-    masterHostPorts = miniCluster.getMasterHostPorts();
 
     LOG.info("Creating a new Kudu client...");
-    client = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
+    client = new AsyncKuduClient.AsyncKuduClientBuilder(miniCluster.getMasterAddressesAsString())
         .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
         .build();
     syncClient = client.syncClient();
@@ -153,8 +146,8 @@ public class BaseKuduTest {
       fail("Table " + table.getName() + " only has 1 tablet, please enable replication");
     }
 
-    HostAndPort hp = findLeaderTabletServerHostPort(tablet);
-    miniCluster.killTabletServerOnHostPort(hp);
+    HostAndPort hp = findLeaderTabletServer(tablet);
+    miniCluster.killTabletServer(hp);
   }
 
   /**
@@ -167,8 +160,21 @@ public class BaseKuduTest {
    * @throws Exception
    */
   protected void killTabletLeader(RemoteTablet tablet) throws Exception {
-    HostAndPort hp = findLeaderTabletServerHostPort(new LocatedTablet(tablet));
-    miniCluster.killTabletServerOnHostPort(hp);
+    killTabletLeader(new LocatedTablet(tablet));
+  }
+
+  /**
+   * Helper method to kill a tablet server that serves the given tablet's
+   * leader. The currently running test case will be failed if the tablet has no
+   * leader after some retries, or if the tablet server was already killed.
+   *
+   * This method is thread-safe.
+   * @param tablet a LocatedTablet which will get its leader killed
+   * @throws Exception
+   */
+  protected void killTabletLeader(LocatedTablet tablet) throws Exception {
+    HostAndPort hp = findLeaderTabletServer(tablet);
+    miniCluster.killTabletServer(hp);
   }
 
   /**
@@ -177,7 +183,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 HostAndPort findLeaderTabletServerHostPort(LocatedTablet tablet)
+  protected HostAndPort findLeaderTabletServer(LocatedTablet tablet)
       throws Exception {
     LocatedTablet.Replica leader = null;
     DeadlineTracker deadlineTracker = new DeadlineTracker();
@@ -194,7 +200,7 @@ public class BaseKuduTest {
         Thread.sleep(50);
       }
     }
-    return HostAndPort.fromParts(leader.getRpcHost(), leader.getRpcPort());
+    return new HostAndPort(leader.getRpcHost(), leader.getRpcPort());
   }
 
   /**
@@ -203,9 +209,9 @@ public class BaseKuduTest {
    * This method is thread-safe.
    * @throws Exception if there is an error finding or killing the leader master.
    */
-  protected void killMasterLeader() throws Exception {
-    HostAndPort hp = findLeaderMasterHostPort();
-    miniCluster.killMasterOnHostPort(hp);
+  protected void killLeaderMasterServer() throws Exception {
+    HostAndPort hp = findLeaderMasterServer();
+    miniCluster.killMasterServer(hp);
   }
 
   /**
@@ -213,7 +219,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 HostAndPort findLeaderMasterHostPort() throws Exception {
+  protected HostAndPort findLeaderMasterServer() throws Exception {
     Stopwatch sw = Stopwatch.createStarted();
     while (sw.elapsed(TimeUnit.MILLISECONDS) < DEFAULT_SLEEP) {
       Deferred<Master.GetTableLocationsResponsePB> masterLocD =
@@ -224,7 +230,7 @@ public class BaseKuduTest {
           .getTsInfo()
           .getRpcAddresses(0);
       if (pb.getPort() != -1) {
-        return HostAndPort.fromParts(pb.getHost(), pb.getPort());
+        return new HostAndPort(pb.getHost(), pb.getPort());
       }
     }
     throw new IOException(String.format("No leader master found after %d ms", DEFAULT_SLEEP));
@@ -243,9 +249,9 @@ public class BaseKuduTest {
 
     LocatedTablet tablet = tablets.get(0);
     Replica replica = tablet.getReplicas().get(randomForTSRestart.nextInt(tablet.getReplicas().size()));
-    HostAndPort hp = HostAndPort.fromParts(replica.getRpcHost(), replica.getRpcPort());
-    miniCluster.killTabletServerOnHostPort(hp);
-    miniCluster.restartDeadTabletServerOnHostPort(hp);
+    HostAndPort hp = new HostAndPort(replica.getRpcHost(), replica.getRpcPort());
+    miniCluster.killTabletServer(hp);
+    miniCluster.startTabletServer(hp);
   }
 
   /**
@@ -254,9 +260,9 @@ public class BaseKuduTest {
    * @throws Exception
    */
   protected void restartTabletServer(RemoteTablet tablet) throws Exception {
-    HostAndPort hp = findLeaderTabletServerHostPort(new LocatedTablet(tablet));
-    miniCluster.killTabletServerOnHostPort(hp);
-    miniCluster.restartDeadTabletServerOnHostPort(hp);
+    HostAndPort hp = findLeaderTabletServer(new LocatedTablet(tablet));
+    miniCluster.killTabletServer(hp);
+    miniCluster.startTabletServer(hp);
   }
 
   /**
@@ -264,9 +270,9 @@ public class BaseKuduTest {
    * @throws Exception
    */
   protected void restartLeaderMaster() throws Exception {
-    HostAndPort hp = findLeaderMasterHostPort();
-    miniCluster.killMasterOnHostPort(hp);
-    miniCluster.restartDeadMasterOnHostPort(hp);
+    HostAndPort hp = findLeaderMasterServer();
+    miniCluster.killMasterServer(hp);
+    miniCluster.startMasterServer(hp);
   }
 
   /**
@@ -274,24 +280,78 @@ public class BaseKuduTest {
    * config for this cluster.
    * @return The master config string.
    */
-  protected String getMasterAddresses() {
-    return masterAddresses;
+  protected String getMasterAddressesAsString() {
+    return miniCluster.getMasterAddressesAsString();
   }
 
   /**
-   * Kills all tablet servers in the cluster.
-   * @throws InterruptedException
+   * @return the list of master servers
    */
-  protected void killTabletServers() throws IOException {
-    miniCluster.killTservers();
+  public List<HostAndPort> getMasterServers() {
+    return miniCluster.getMasterServers();
   }
 
   /**
-   * Restarts killed tablet servers in the cluster.
-   * @throws Exception
+   * @return the list of tablet servers
+   */
+  public List<HostAndPort> getTabletServers() {
+    return miniCluster.getMasterServers();
+  }
+
+  /**
+   * Kills all the master servers.
+   * Does nothing to the servers that are already dead.
+   *
+   * @throws IOException
+   */
+  protected void killAllMasterServers() throws IOException {
+    miniCluster.killAllMasterServers();
+  }
+
+  /**
+   * Starts all the master servers.
+   * Does nothing to the servers that are already running.
+   *
+   * @throws IOException
+   */
+  protected void startAllMasterServers() throws IOException {
+    miniCluster.startAllMasterServers();
+  }
+
+  /**
+   * Kills all the tablet servers.
+   * Does nothing to the servers that are already dead.
+   *
+   * @throws IOException
+   */
+  protected void killAllTabletServers() throws IOException {
+    miniCluster.killAllTabletServers();
+  }
+
+  /**
+   * Starts all the tablet servers.
+   * Does nothing to the servers that are already running.
+   *
+   * @throws IOException
+   */
+  protected void startAllTabletServers() throws IOException {
+    miniCluster.startAllTabletServers();
+  }
+
+  /**
+   * Removes all credentials for all principals from the Kerberos credential cache.
+   */
+  protected void kdestroy() throws IOException {
+    miniCluster.kdestroy();
+  }
+
+  /**
+   * Re-initialize Kerberos credentials for the given username, writing them
+   * into the Kerberos credential cache.
+   * @param username the username to kinit as
    */
-  protected void restartTabletServers() throws IOException {
-    miniCluster.restartDeadTservers();
+  protected void kinit(String username) throws IOException {
+    miniCluster.kinit(username);
   }
 
   /**
@@ -300,7 +360,7 @@ public class BaseKuduTest {
    */
   protected void resetClients() throws IOException {
     syncClient.shutdown();
-    client = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
+    client = new AsyncKuduClient.AsyncKuduClientBuilder(miniCluster.getMasterAddressesAsString())
                                 .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
                                 .build();
     syncClient = client.syncClient();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 b6d559c..42dae2a 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
@@ -80,7 +80,7 @@ public class ITClient extends BaseKuduTest {
 
     // Client we're using has low tolerance for read timeouts but a
     // higher overall operation timeout.
-    localAsyncClient = new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
+    localAsyncClient = new AsyncKuduClient.AsyncKuduClientBuilder(getMasterAddressesAsString())
         .defaultSocketReadTimeoutMs(500)
         .build();
     localClient = new KuduClient(localAsyncClient);

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 fb053cc..cd08a94 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
@@ -106,7 +106,7 @@ public class ITClientStress extends BaseKuduTest {
           @Override
           public Void call() throws Exception {
             try (AsyncKuduClient client =
-                  new AsyncKuduClient.AsyncKuduClientBuilder(masterAddresses)
+                  new AsyncKuduClient.AsyncKuduClientBuilder(getMasterAddressesAsString())
                   .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
                   .build()) {
               KuduTable t = client.openTable(TABLE_NAME).join();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 ee664fb..5bc960a 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
@@ -36,10 +36,8 @@ 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 com.google.common.net.HostAndPort;
 
 import org.apache.kudu.Common;
-import org.apache.kudu.Common.HostPortPB;
 import org.apache.kudu.tools.Tool.ControlShellRequestPB;
 import org.apache.kudu.tools.Tool.ControlShellResponsePB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniKdcOptionsPB;
@@ -89,10 +87,10 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   // Map of master addresses to daemon information.
-  private final Map<HostAndPort, DaemonInfo> masters = Maps.newHashMap();
+  private final Map<HostAndPort, DaemonInfo> masterServers = Maps.newHashMap();
 
   // Map of tserver addresses to daemon information.
-  private final Map<HostAndPort, DaemonInfo> tservers = Maps.newHashMap();
+  private final Map<HostAndPort, DaemonInfo> tabletServers = Maps.newHashMap();
 
   // Builder-provided cluster configuration state.
   private final boolean enableKerberos;
@@ -243,7 +241,7 @@ public class MiniKuduCluster implements AutoCloseable {
       }
     }
 
-    // Initialize the maps of masters and tservers.
+    // Initialize the maps of master and tablet servers.
     ControlShellResponsePB resp = sendRequestToCluster(
         ControlShellRequestPB.newBuilder()
         .setGetMasters(GetMastersRequestPB.newBuilder().build())
@@ -252,7 +250,7 @@ public class MiniKuduCluster implements AutoCloseable {
       DaemonInfo d = new DaemonInfo();
       d.id = info.getId();
       d.isRunning = true;
-      masters.put(hostAndPortFromPB(info.getBoundRpcAddress()), d);
+      masterServers.put(ProtobufHelper.hostAndPortFromPB(info.getBoundRpcAddress()), d);
     }
     resp = sendRequestToCluster(
         ControlShellRequestPB.newBuilder()
@@ -262,23 +260,44 @@ public class MiniKuduCluster implements AutoCloseable {
       DaemonInfo d = new DaemonInfo();
       d.id = info.getId();
       d.isRunning = true;
-      tservers.put(hostAndPortFromPB(info.getBoundRpcAddress()), d);
+      tabletServers.put(ProtobufHelper.hostAndPortFromPB(info.getBoundRpcAddress()), d);
     }
   }
 
   /**
-   * Restarts a master identified by hostname and port. The master must already be dead.
-   * @param hp unique hostname and port identifying the master
-   * @throws IOException if the master is believed to be alive
+   * @return comma-separated list of master server addresses
    */
-  public void restartDeadMasterOnHostPort(HostAndPort hp) throws IOException {
-    DaemonInfo d = masters.get(hp);
-    if (d == null) {
-      throw new IOException(String.format("Master %s not found", hp));
-    }
+  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) {
-      throw new IOException(String.format("Master %s is already running", hp));
+      return;
     }
+    LOG.info("Starting master server {}", hp);
     sendRequestToCluster(ControlShellRequestPB.newBuilder()
         .setStartDaemon(StartDaemonRequestPB.newBuilder().setId(d.id).build())
         .build());
@@ -286,19 +305,18 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * Kills a master identified by hostname and port. Does nothing if the master
-   * was already dead.
-   * @param hp unique hostname and port identifying the master
+   * 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 killMasterOnHostPort(HostAndPort hp) throws IOException {
-    DaemonInfo d = masters.get(hp);
-    if (d == null) {
-      throw new IOException(String.format("Master %s not found", hp));
-    }
+  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());
@@ -306,18 +324,18 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * Restarts a tserver identified by hostname and port. The tserver must already be dead.
-   * @param hp unique hostname and port identifying the tserver
-   * @throws IOException if the tserver is believed to be alive
+   * 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 restartDeadTabletServerOnHostPort(HostAndPort hp) throws IOException {
-    DaemonInfo d = tservers.get(hp);
-    if (d == null) {
-      throw new IOException(String.format("Tserver %s not found", hp));
-    }
+  public void startTabletServer(HostAndPort hp) throws IOException {
+    DaemonInfo d = getTabletServer(hp);
     if (d.isRunning) {
-      throw new IOException(String.format("Tserver %s is already running", hp));
+      return;
     }
+    LOG.info("Starting tablet server {}", hp);
     sendRequestToCluster(ControlShellRequestPB.newBuilder()
         .setStartDaemon(StartDaemonRequestPB.newBuilder().setId(d.id).build())
         .build());
@@ -325,20 +343,18 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * Kills a tserver identified by hostname and port. Does nothing if the tserver
-   * was already dead.
-   * @param hp unique hostname and port identifying the tserver
+   * 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 killTabletServerOnHostPort(HostAndPort hp) throws IOException {
-    DaemonInfo d = tservers.get(hp);
-    if (d == null) {
-      throw new IOException(String.format("Tserver %s not found", hp));
-    }
+  public void killTabletServer(HostAndPort hp) throws IOException {
+    DaemonInfo d = getTabletServer(hp);
     if (!d.isRunning) {
       return;
     }
-    LOG.info("Killing tserver {}", hp);
+    LOG.info("Killing tablet server {}", hp);
     sendRequestToCluster(ControlShellRequestPB.newBuilder()
         .setStopDaemon(StopDaemonRequestPB.newBuilder().setId(d.id).build())
         .build());
@@ -346,66 +362,50 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * Kills all masters not already stopped.
+   * Kills all the master servers.
+   * Does nothing to the servers that are already dead.
+   *
    * @throws IOException if something went wrong in transit
    */
-  public void killMasters() throws IOException {
-    List<HostAndPort> toKill = Lists.newArrayList();
-    for (Map.Entry<HostAndPort, DaemonInfo> e : masters.entrySet()) {
-      if (e.getValue().isRunning) {
-        toKill.add(e.getKey());
-      }
-    }
-    for (HostAndPort hp : toKill) {
-      killMasterOnHostPort(hp);
+  public void killAllMasterServers() throws IOException {
+    for (Map.Entry<HostAndPort, DaemonInfo> e : masterServers.entrySet()) {
+      killMasterServer(e.getKey());
     }
   }
 
   /**
-   * Starts all currently stopped masters.
+   * Starts all the master servers.
+   * Does nothing to the servers that are already running.
+   *
    * @throws IOException if something went wrong in transit
    */
-  public void restartDeadMasters() throws IOException {
-    List<HostAndPort> toRestart = Lists.newArrayList();
-    for (Map.Entry<HostAndPort, DaemonInfo> e : masters.entrySet()) {
-      if (!e.getValue().isRunning) {
-        toRestart.add(e.getKey());
-      }
-    }
-    for (HostAndPort hp : toRestart) {
-      restartDeadMasterOnHostPort(hp);
+  public void startAllMasterServers() throws IOException {
+    for (Map.Entry<HostAndPort, DaemonInfo> e : masterServers.entrySet()) {
+      startMasterServer(e.getKey());
     }
   }
 
   /**
-   * Kills all tservers not already stopped.
+   * Kills all tablet servers.
+   * Does nothing to the servers that are already dead.
+   *
    * @throws IOException if something went wrong in transit
    */
-  public void killTservers() throws IOException {
-    List<HostAndPort> toKill = Lists.newArrayList();
-    for (Map.Entry<HostAndPort, DaemonInfo> e : tservers.entrySet()) {
-      if (e.getValue().isRunning) {
-        toKill.add(e.getKey());
-      }
-    }
-    for (HostAndPort hp : toKill) {
-      killTabletServerOnHostPort(hp);
+  public void killAllTabletServers() throws IOException {
+    for (Map.Entry<HostAndPort, DaemonInfo> e : tabletServers.entrySet()) {
+      killTabletServer(e.getKey());
     }
   }
 
   /**
-   * Starts all currently stopped tservers.
+   * Starts all the tablet servers.
+   * Does nothing to the servers that are already running.
+   *
    * @throws IOException if something went wrong in transit
    */
-  public void restartDeadTservers() throws IOException {
-    List<HostAndPort> toRestart = Lists.newArrayList();
-    for (Map.Entry<HostAndPort, DaemonInfo> e : tservers.entrySet()) {
-      if (!e.getValue().isRunning) {
-        toRestart.add(e.getKey());
-      }
-    }
-    for (HostAndPort hp : toRestart) {
-      restartDeadTabletServerOnHostPort(hp);
+  public void startAllTabletServers() throws IOException {
+    for (Map.Entry<HostAndPort, DaemonInfo> e : tabletServers.entrySet()) {
+      startTabletServer(e.getKey());
     }
   }
 
@@ -413,9 +413,10 @@ public class MiniKuduCluster implements AutoCloseable {
    * 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());
+        .setKdestroy(KdestroyRequestPB.getDefaultInstance())
+        .build());
   }
 
   /**
@@ -424,6 +425,7 @@ public class MiniKuduCluster implements AutoCloseable {
    * @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());
@@ -473,24 +475,33 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * @return comma-separated list of master addresses
-   */
-  public String getMasterAddresses() {
-    return Joiner.on(',').join(masters.keySet());
-  }
-
-  /**
-   * @return list of all masters, uniquely identified by hostname and port
+   * 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
    */
-  public List<HostAndPort> getMasterHostPorts() {
-    return new ArrayList<>(masters.keySet());
+  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;
   }
 
   /**
-   * @return list of all tservers, uniquely identified by hostname and port
+   * 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
    */
-  public List<HostAndPort> getTserverHostPorts() {
-    return new ArrayList<>(tservers.keySet());
+  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;
   }
 
   /**
@@ -523,14 +534,6 @@ public class MiniKuduCluster implements AutoCloseable {
   }
 
   /**
-   * TODO(KUDU-2186): If used directly from {@link ProtobufHelper}, tests from
-   * other modules break when run by Gradle.
-   */
-  private static HostAndPort hostAndPortFromPB(HostPortPB hostPortPB) {
-    return HostAndPort.fromParts(hostPortPB.getHost(), hostPortPB.getPort());
-  }
-
-  /**
    * Builder for {@link MiniKuduCluster}
    */
   public static class MiniKuduClusterBuilder {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 4fb7bab..56d5d6e 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
@@ -202,7 +202,7 @@ public class TestAsyncKuduClient extends BaseKuduTest {
     assertEquals(0, countRowsInScan(syncClient.newScannerBuilder(table).build()));
 
     // Make it impossible to use Kudu.
-    killTabletServers();
+    killAllTabletServers();
 
     // Create a scan with a short timeout.
     KuduScanner scanner = syncClient.newScannerBuilder(table).scanRequestTimeout(1000).build();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 81e8946..0e99254 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
@@ -165,7 +165,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
   /** Regression test for a failure to correctly handle a timeout when flushing a batch. */
   @Test
   public void testInsertIntoUnavailableTablet() throws Exception {
-    killTabletServers();
+    killAllTabletServers();
     try {
       AsyncKuduSession session = client.newSession();
       session.setTimeoutMillis(1);
@@ -180,7 +180,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
       assertEquals(1, responses.size());
       assertTrue(responses.get(0).getRowError().getErrorStatus().isTimedOut());
     } finally {
-      restartTabletServers();
+      startAllTabletServers();
     }
   }
 
@@ -210,8 +210,8 @@ public class TestAsyncKuduSession extends BaseKuduTest {
       int numClientsBefore = client.getConnectionListCopy().size();
 
       // Restart all the tablet servers.
-      killTabletServers();
-      restartTabletServers();
+      killAllTabletServers();
+      startAllTabletServers();
 
       // Perform another write, which will require reconnecting to the same
       // tablet server that we wrote to above.
@@ -221,7 +221,7 @@ public class TestAsyncKuduSession extends BaseKuduTest {
       int numClientsAfter = client.getConnectionListCopy().size();
       assertEquals(numClientsBefore, numClientsAfter);
     } finally {
-      restartTabletServers();
+      startAllTabletServers();
 
       client.deleteTable("non-replicated").join();
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 315a010..1da9fc3 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
@@ -32,8 +32,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.net.HostAndPort;
-
 public class TestClientFailoverSupport extends BaseKuduTest {
   private CapturingLogAppender cla = new CapturingLogAppender();
   private Closeable claAttach;
@@ -108,7 +106,7 @@ public class TestClientFailoverSupport extends BaseKuduTest {
     // Kill or restart the leader master.
     switch (failureType) {
     case KILL:
-      killMasterLeader();
+      killLeaderMasterServer();
       break;
     case RESTART:
       restartLeaderMaster();
@@ -121,8 +119,7 @@ public class TestClientFailoverSupport extends BaseKuduTest {
     // to the new one.
     List<LocatedTablet> tablets = table.getTabletsLocations(DEFAULT_SLEEP);
     assertEquals(1, tablets.size());
-    HostAndPort hp = findLeaderTabletServerHostPort(tablets.get(0));
-    miniCluster.killTabletServerOnHostPort(hp);
+    killTabletLeader(tablets.get(0));
 
     // Insert some more rows.
     for (int i = TOTAL_ROWS_TO_INSERT; i < 2*TOTAL_ROWS_TO_INSERT; i++) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 6bda971..a3379e5 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
@@ -22,11 +22,8 @@ import static org.apache.kudu.consensus.Metadata.RaftPeerPB.Role.LEADER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.util.List;
-
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
-import com.google.common.net.HostAndPort;
 import com.stumbleupon.async.Callback;
 
 import org.junit.Assert;
@@ -38,9 +35,9 @@ import org.hamcrest.CoreMatchers;
 public class TestConnectToCluster {
 
   private static final ImmutableList<HostAndPort> MASTERS = ImmutableList.of(
-      HostAndPort.fromParts("0", 9000),
-      HostAndPort.fromParts("1", 9000),
-      HostAndPort.fromParts("2", 9000));
+      new HostAndPort("0", 9000),
+      new HostAndPort("1", 9000),
+      new HostAndPort("2", 9000));
 
   /**
    * Test that the client properly falls back to the old GetMasterRegistration
@@ -56,7 +53,7 @@ public class TestConnectToCluster {
         .build();
     KuduClient c = null;
     try {
-      c = new KuduClient.KuduClientBuilder(cluster.getMasterAddresses())
+      c = new KuduClient.KuduClientBuilder(cluster.getMasterAddressesAsString())
           .build();
       // Call some method which uses the master. This forces us to connect
       // and verifies that the fallback works.
@@ -83,7 +80,7 @@ public class TestConnectToCluster {
         .build();
     int successes = 0;
     try {
-      String[] masterAddrs = cluster.getMasterAddresses().split(",", -1);
+      String[] masterAddrs = cluster.getMasterAddressesAsString().split(",", -1);
       assertEquals(3, masterAddrs.length);
       for (String masterAddr : masterAddrs) {
         KuduClient c = null;

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 84dacbf..11eb4bc 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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 
-import com.google.common.net.HostAndPort;
 import com.stumbleupon.async.Deferred;
 import org.junit.Test;
 
@@ -37,13 +36,13 @@ public class TestConnectionCache {
       cluster = new MiniKuduCluster.MiniKuduClusterBuilder().numMasters(3).build();
 
       final AsyncKuduClient client =
-          new AsyncKuduClient.AsyncKuduClientBuilder(cluster.getMasterAddresses()).build();
+          new AsyncKuduClient.AsyncKuduClientBuilder(cluster.getMasterAddressesAsString()).build();
       // Below we ping the masters directly using RpcProxy, so if they aren't ready to process
       // RPCs we'll get an error. Here by listing the tables we make sure this won't happen since
       // it won't return until a master leader is found.
       client.getTablesList().join();
 
-      HostAndPort masterHostPort = cluster.getMasterHostPorts().get(0);
+      HostAndPort masterHostPort = cluster.getMasterServers().get(0);
       ServerInfo firstMaster = new ServerInfo("fake-uuid", masterHostPort,
           NetUtil.getInetAddress(masterHostPort.getHost()));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 1738585..7b1b3a9 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
@@ -59,7 +59,7 @@ public class TestHandleTooBusy extends BaseKuduTest {
         @Override
         public Void call() throws Exception {
           for (int i = 0; i < 5; i++) {
-            try (KuduClient c = new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses())
+            try (KuduClient c = new KuduClient.KuduClientBuilder(getMasterAddressesAsString())
                 .build()) {
               KuduTable table = c.openTable(TABLE_NAME);
               for (int j = 0; j < 5; j++) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 80c0843..738cdcc 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
@@ -795,7 +795,7 @@ public class TestKuduClient extends BaseKuduTest {
    */
   @Test(timeout = 100000)
   public void testAutoClose() throws Exception {
-    try (KuduClient localClient = new KuduClient.KuduClientBuilder(masterAddresses).build()) {
+    try (KuduClient localClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
       localClient.createTable(TABLE_NAME, basicSchema, getBasicCreateTableOptions());
       KuduTable table = localClient.openTable(TABLE_NAME);
       KuduSession session = localClient.newSession();
@@ -818,7 +818,7 @@ public class TestKuduClient extends BaseKuduTest {
   public void testCloseShortlyAfterOpen() throws Exception {
     CapturingLogAppender cla = new CapturingLogAppender();
     try (Closeable c = cla.attach()) {
-      try (KuduClient localClient = new KuduClient.KuduClientBuilder(masterAddresses).build()) {
+      try (KuduClient localClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
         // Force the client to connect to the masters.
         localClient.exportAuthenticationCredentials();
       }
@@ -837,8 +837,8 @@ public class TestKuduClient extends BaseKuduTest {
   public void testNoLogSpewOnConnectionRefused() throws Exception {
     CapturingLogAppender cla = new CapturingLogAppender();
     try (Closeable c = cla.attach()) {
-      miniCluster.killMasters();
-      try (KuduClient localClient = new KuduClient.KuduClientBuilder(masterAddresses).build()) {
+      killAllMasterServers();
+      try (KuduClient localClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
         // Force the client to connect to the masters.
         localClient.exportAuthenticationCredentials();
         fail("Should have failed to connect.");
@@ -849,7 +849,7 @@ public class TestKuduClient extends BaseKuduTest {
                 ".*Connection refused.*"));
       }
     } finally {
-      miniCluster.restartDeadMasters();
+      startAllMasterServers();
     }
     // Ensure there is no log spew due to an unexpected lost connection.
     String logText = cla.getAppendedText();
@@ -862,7 +862,7 @@ public class TestKuduClient extends BaseKuduTest {
   @Test(timeout = 100000)
   public void testCustomNioExecutor() throws Exception {
     long startTime = System.nanoTime();
-    final KuduClient localClient = new KuduClient.KuduClientBuilder(masterAddresses)
+    final KuduClient localClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString())
         .nioExecutors(Executors.newFixedThreadPool(1), Executors.newFixedThreadPool(2))
         .bossCount(1)
         .workerCount(2)
@@ -922,7 +922,7 @@ public class TestKuduClient extends BaseKuduTest {
 
     // Add a range partition with a separate client. The new client is necessary
     // in order to avoid clearing the meta cache as part of the alter operation.
-    try (KuduClient alterClient = new KuduClient.KuduClientBuilder(masterAddresses)
+    try (KuduClient alterClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString())
                                                 .defaultAdminOperationTimeoutMs(DEFAULT_SLEEP)
                                                 .build()) {
       AlterTableOptions alter = new AlterTableOptions();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 81b2383..7b9a8ec 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
@@ -20,7 +20,6 @@ import static org.apache.kudu.util.ClientTestUtil.countRowsInScan;
 import static org.apache.kudu.util.ClientTestUtil.getBasicCreateTableOptions;
 import static org.junit.Assert.assertEquals;
 
-import org.junit.After;
 import org.junit.Test;
 
 /**
@@ -53,32 +52,31 @@ public class TestMasterFailover extends BaseKuduTest {
 
   private void doTestKillLeader(KillBefore killBefore) throws Exception {
     String tableName = "TestMasterFailover-killBefore=" + killBefore;
-    int countMasters = masterHostPorts.size();
+    int countMasters = getMasterServers().size();
     if (countMasters < 3) {
       throw new Exception("This test requires at least 3 master servers, but only "
         + countMasters + " are specified.");
     }
 
     if (killBefore == KillBefore.CREATE_CLIENT) {
-      killMasterLeader();
+      killLeaderMasterServer();
     }
-    try (KuduClient c = new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses())
-          .build()) {
+    try (KuduClient c = new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
       if (killBefore == KillBefore.CREATE_TABLE) {
-        killMasterLeader();
+        killLeaderMasterServer();
       }
 
       createTable(tableName, basicSchema, getBasicCreateTableOptions());
 
       if (killBefore == KillBefore.OPEN_TABLE) {
-        killMasterLeader();
+        killLeaderMasterServer();
       }
 
       // Test that we can open a previously created table after killing the leader master.
       KuduTable table = openTable(tableName);
 
       if (killBefore == KillBefore.SCAN_TABLE) {
-        killMasterLeader();
+        killLeaderMasterServer();
       }
       assertEquals(0, countRowsInScan(client.newScannerBuilder(table).build()));
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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
index 9579c0e..c445379 100644
--- 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
@@ -25,8 +25,6 @@ import java.net.Socket;
 import org.apache.kudu.client.KuduClient.KuduClientBuilder;
 import org.junit.Test;
 
-import com.google.common.net.HostAndPort;
-
 public class TestMiniKuduCluster {
 
   private static final int NUM_TABLET_SERVERS = 3;
@@ -39,19 +37,19 @@ public class TestMiniKuduCluster {
                                                       .numMasters(NUM_MASTERS)
                                                       .numTservers(NUM_TABLET_SERVERS)
                                                       .build()) {
-      assertEquals(NUM_MASTERS, cluster.getMasterHostPorts().size());
-      assertEquals(NUM_TABLET_SERVERS, cluster.getTserverHostPorts().size());
+      assertEquals(NUM_MASTERS, cluster.getMasterServers().size());
+      assertEquals(NUM_TABLET_SERVERS, cluster.getTabletServers().size());
 
       {
         // Kill the master.
-        HostAndPort masterHostPort = cluster.getMasterHostPorts().get(0);
+        HostAndPort masterHostPort = cluster.getMasterServers().get(0);
         testHostPort(masterHostPort, true);
-        cluster.killMasterOnHostPort(masterHostPort);
+        cluster.killMasterServer(masterHostPort);
 
         testHostPort(masterHostPort, false);
 
         // Restart the master.
-        cluster.restartDeadMasterOnHostPort(masterHostPort);
+        cluster.startMasterServer(masterHostPort);
 
         // Test we can reach it.
         testHostPort(masterHostPort, true);
@@ -59,14 +57,14 @@ public class TestMiniKuduCluster {
 
       {
         // Kill the first TS.
-        HostAndPort tsHostPort = cluster.getTserverHostPorts().get(0);
+        HostAndPort tsHostPort = cluster.getTabletServers().get(0);
         testHostPort(tsHostPort, true);
-        cluster.killTabletServerOnHostPort(tsHostPort);
+        cluster.killTabletServer(tsHostPort);
 
         testHostPort(tsHostPort, false);
 
         // Restart it.
-        cluster.restartDeadTabletServerOnHostPort(tsHostPort);
+        cluster.startTabletServer(tsHostPort);
 
         testHostPort(tsHostPort, true);
       }
@@ -81,7 +79,7 @@ public class TestMiniKuduCluster {
                                                       .numTservers(NUM_TABLET_SERVERS)
                                                       .enableKerberos()
                                                       .build()) {
-      KuduClient client = new KuduClientBuilder(cluster.getMasterAddresses()).build();
+      KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build();
       ListTablesResponse resp = client.getTablesList();
       assertTrue(resp.getTablesList().isEmpty());
       assertNull(client.getHiveMetastoreConfig());
@@ -95,7 +93,7 @@ public class TestMiniKuduCluster {
                                                       .numTservers(NUM_TABLET_SERVERS)
                                                       .enableHiveMetastoreIntegration()
                                                       .build()) {
-      KuduClient client = new KuduClientBuilder(cluster.getMasterAddresses()).build();
+      KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build();
       assertNotNull(client.getHiveMetastoreConfig());
     }
   }
@@ -106,7 +104,7 @@ public class TestMiniKuduCluster {
    * @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 {
+                                   boolean testIsOpen) throws InterruptedException {
     DeadlineTracker tracker = new DeadlineTracker();
     while (tracker.getElapsedMillis() < SLEEP_TIME_MS) {
       try {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 85526f9..a9f5cd8 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
@@ -27,8 +27,6 @@ import java.util.List;
 import org.apache.kudu.util.AssertHelpers.BooleanExpression;
 import org.junit.Test;
 
-import com.google.common.net.HostAndPort;
-
 public class TestMultipleLeaderFailover extends BaseKuduTest {
 
   private void waitUntilRowCount(final KuduTable table, final int rowCount, long timeoutMs)
@@ -73,8 +71,7 @@ public class TestMultipleLeaderFailover extends BaseKuduTest {
     for (int i = 0; i < NUM_ITERATIONS; i++) {
       List<LocatedTablet> tablets = table.getTabletsLocations(DEFAULT_SLEEP);
       assertEquals(1, tablets.size());
-      HostAndPort hp = findLeaderTabletServerHostPort(tablets.get(0));
-      miniCluster.killTabletServerOnHostPort(hp);
+      killTabletLeader(tablets.get(0));
 
       for (int j = 0; j < ROWS_PER_ITERATION; j++) {
         OperationResponse resp = session.apply(createBasicSchemaInsert(table, currentRows));
@@ -84,7 +81,7 @@ public class TestMultipleLeaderFailover extends BaseKuduTest {
         currentRows++;
       }
 
-      miniCluster.restartDeadTabletServerOnHostPort(hp);
+      startAllTabletServers();
       // Read your writes hasn't been enabled, so we need to use a helper function to poll.
       waitUntilRowCount(table, currentRows, DEFAULT_SLEEP);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 633c4d9..9d01e37 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
@@ -48,7 +48,7 @@ public class TestNegotiation {
     try (MiniKuduCluster cluster = clusterBuilder.build()) {
       cluster.kdestroy();
       try (Closeable c = cla.attach();
-           KuduClient client = new KuduClient.KuduClientBuilder(cluster.getMasterAddresses())
+           KuduClient client = new KuduClient.KuduClientBuilder(cluster.getMasterAddressesAsString())
                                              .build()
       ) {
         assertTrue(client.getTablesList().getTablesList().isEmpty());

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 c835f27..b9cbd46 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
@@ -27,7 +27,6 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.google.common.net.HostAndPort;
 import com.google.protobuf.ByteString;
 import org.junit.Test;
 
@@ -172,7 +171,7 @@ public class TestRemoteTablet {
 
       String uuid = kUuids[i];
       servers.add(new ServerInfo(uuid,
-                                 HostAndPort.fromParts("host", 1000 + i),
+                                 new HostAndPort("host", 1000 + i),
                                  addr));
       tabletPb.addReplicas(TestUtils.getFakeTabletReplicaPB(
           uuid, "host", i,

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 a6d0f95..5594177 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
@@ -40,11 +40,9 @@ import org.hamcrest.CoreMatchers;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableSet;
 import com.stumbleupon.async.Deferred;
@@ -79,7 +77,7 @@ public class TestSecurity {
         .numTservers(opts.contains(Option.START_TSERVERS) ? 3 : 0)
         .build();
     miniCluster.kinit("test-admin");
-    client = new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses()).build();
+    client = new KuduClient.KuduClientBuilder(miniCluster.getMasterAddressesAsString()).build();
 
     // TODO(todd): it seems that exportAuthenticationCredentials() doesn't properly retry
     // in the case that there is no leader, even though NoLeaderFoundException is a RecoverableException.
@@ -122,7 +120,7 @@ public class TestSecurity {
   }
 
   private KuduClient createClient() {
-    return new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses()).build();
+    return new KuduClient.KuduClientBuilder(miniCluster.getMasterAddressesAsString()).build();
   }
 
   private void checkClientCanReconnect(KuduClient client) throws IOException {
@@ -132,8 +130,8 @@ public class TestSecurity {
     // would continue to work even though our credentials might have
     // expired (we only authenticate when a connection is negotiated, not
     // for each call).
-    miniCluster.killMasters();
-    miniCluster.restartDeadMasters();
+    miniCluster.killAllMasterServers();
+    miniCluster.startAllMasterServers();
     client.listTabletServers();
   }
 
@@ -152,7 +150,7 @@ public class TestSecurity {
     System.clearProperty(SecurityUtil.KUDU_TICKETCACHE_PROPERTY);
     try {
       KuduClient newClient = new KuduClient.KuduClientBuilder(
-          miniCluster.getMasterAddresses()).build();
+          miniCluster.getMasterAddressesAsString()).build();
 
       // Test that a client with no credentials cannot list servers.
       try {
@@ -255,7 +253,7 @@ public class TestSecurity {
           new BooleanExpression() {
             @Override
             public boolean get() throws Exception {
-              ConnectToCluster connector = new ConnectToCluster(miniCluster.getMasterHostPorts());
+              ConnectToCluster connector = new ConnectToCluster(miniCluster.getMasterServers());
               List<Deferred<ConnectToMasterResponsePB>> deferreds =
                       connector.connectToMasters(newClient.asyncClient.getMasterTable(), null,
                       /* timeout = */50000,
@@ -293,8 +291,8 @@ public class TestSecurity {
       newClient.importAuthenticationCredentials(authnData);
       System.err.println("=> imported auth");
 
-      miniCluster.killMasters();
-      miniCluster.restartDeadMasters();
+      miniCluster.killAllMasterServers();
+      miniCluster.startAllMasterServers();
       newClient.listTabletServers();
       System.err.println("=> listTabletServers");
     } finally {
@@ -364,8 +362,8 @@ public class TestSecurity {
           @Override
           public boolean get() throws Exception {
             Thread.sleep(3000);
-            miniCluster.killMasters();
-            miniCluster.restartDeadMasters();
+            miniCluster.killAllMasterServers();
+            miniCluster.startAllMasterServers();
             try {
               client.listTabletServers();
             } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 c48a1f3..5957bcb 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
@@ -62,12 +62,12 @@ public class TestSecurityContextRealUser extends BaseKuduTest {
   @Test
   public void test() throws Exception {
     // Clear out the Kerberos credentials in the environment.
-    miniCluster.kdestroy();
+    kdestroy();
 
     // Create a new client instance with the logged in user, and ensure that it
     // fails to connect (the logged in user is not in the user-acl).
     try (KuduClient client =
-             new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses()).build()) {
+             new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
       client.listTabletServers();
       fail();
     } catch (KuduException e) {
@@ -77,7 +77,7 @@ public class TestSecurityContextRealUser extends BaseKuduTest {
 
     // Try again with a correct real user.
     try (KuduClient client =
-             new KuduClient.KuduClientBuilder(miniCluster.getMasterAddresses()).build()) {
+             new KuduClient.KuduClientBuilder(getMasterAddressesAsString()).build()) {
       Client.AuthenticationCredentialsPB credentials =
           Client.AuthenticationCredentialsPB.newBuilder().setRealUser("token-user").build();
       client.importAuthenticationCredentials(credentials.toByteArray());

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
index 9aea64f..2d82272 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestServerInfo.java
@@ -15,7 +15,6 @@ package org.apache.kudu.client;
 
 import java.net.InetAddress;
 
-import com.google.common.net.HostAndPort;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -31,7 +30,7 @@ public class TestServerInfo {
 
     ServerInfo serverInfo = new ServerInfo(
         "nevermind",
-        HostAndPort.fromParts("master2.example.com", 12345),
+        new HostAndPort("master2.example.com", 12345),
         InetAddress.getByName("10.1.2.3"));
 
     Assert.assertEquals("master2.example.com", serverInfo.getAndCanonicalizeHostname());
@@ -47,7 +46,7 @@ public class TestServerInfo {
 
     ServerInfo serverInfo = new ServerInfo(
         "abcdef", // uuid
-        HostAndPort.fromParts("master1.example.com", 12345),
+        new HostAndPort("master1.example.com", 12345),
         InetAddress.getByName("10.1.2.3"));
 
     Assert.assertEquals("server123.example.com", serverInfo.getAndCanonicalizeHostname());

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeouts.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeouts.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeouts.java
index 9b4bdba..86c03dd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeouts.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestTimeouts.java
@@ -35,7 +35,7 @@ public class TestTimeouts extends BaseKuduTest {
    */
   @Test(timeout = 100000)
   public void testLowTimeouts() throws Exception {
-    KuduClient lowTimeoutsClient = new KuduClient.KuduClientBuilder(masterAddresses)
+    KuduClient lowTimeoutsClient = new KuduClient.KuduClientBuilder(getMasterAddressesAsString())
         .defaultAdminOperationTimeoutMs(1)
         .defaultOperationTimeoutMs(1)
         .build();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java b/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
index 8c91245..4220bfe 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/util/TestNetUtil.java
@@ -25,7 +25,7 @@ import java.net.InetAddress;
 import java.util.Arrays;
 import java.util.List;
 
-import com.google.common.net.HostAndPort;
+import org.apache.kudu.client.HostAndPort;
 import org.junit.Test;
 
 /**
@@ -59,17 +59,18 @@ public class TestNetUtil {
     String testAddrs = "1.2.3.4.5,10.0.0.1:5555,127.0.0.1:7777";
     List<HostAndPort> hostsAndPorts = NetUtil.parseStrings(testAddrs, 3333);
     assertArrayEquals(hostsAndPorts.toArray(),
-                         new HostAndPort[] { HostAndPort.fromParts("1.2.3.4.5", 3333),
-                           HostAndPort.fromParts("10.0.0.1", 5555),
-                           HostAndPort.fromParts("127.0.0.1", 7777) }
+        new HostAndPort[] {
+            new HostAndPort("1.2.3.4.5", 3333),
+            new HostAndPort("10.0.0.1", 5555),
+            new HostAndPort("127.0.0.1", 7777)}
     );
   }
 
   @Test
   public void testHostsAndPortsToString() {
     List<HostAndPort> hostsAndPorts = Arrays.asList(
-        HostAndPort.fromParts("127.0.0.1", 1111),
-        HostAndPort.fromParts("1.2.3.4.5", 0)
+        new HostAndPort("127.0.0.1", 1111),
+        new HostAndPort("1.2.3.4.5", 0)
     );
     assertEquals(NetUtil.hostsAndPortsToString(hostsAndPorts), "127.0.0.1:1111,1.2.3.4.5:0");
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/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 42c94cd..7b20a2c 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
@@ -164,7 +164,7 @@ public class AvroKuduOperationsProducerTest extends BaseKuduTest {
     KuduSink sink = new KuduSink(syncClient);
     HashMap<String, String> parameters = new HashMap<>();
     parameters.put(TABLE_NAME, tableName);
-    parameters.put(MASTER_ADDRESSES, getMasterAddresses());
+    parameters.put(MASTER_ADDRESSES, getMasterAddressesAsString());
     parameters.put(PRODUCER, AvroKuduOperationsProducer.class.getName());
     Context context = new Context(parameters);
     context.putAll(ctx.getParameters());

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KeyedKuduOperationsProducerTest.java
----------------------------------------------------------------------
diff --git a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KeyedKuduOperationsProducerTest.java b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KeyedKuduOperationsProducerTest.java
index ea7ca73..b16a209 100644
--- a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KeyedKuduOperationsProducerTest.java
+++ b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KeyedKuduOperationsProducerTest.java
@@ -218,7 +218,7 @@ public class KeyedKuduOperationsProducerTest extends BaseKuduTest {
     KuduSink sink = new KuduSink(syncClient);
     HashMap<String, String> parameters = new HashMap<>();
     parameters.put(TABLE_NAME, tableName);
-    parameters.put(MASTER_ADDRESSES, getMasterAddresses());
+    parameters.put(MASTER_ADDRESSES, getMasterAddressesAsString());
     parameters.put(PRODUCER, SimpleKeyedKuduOperationsProducer.class.getName());
     Context context = new Context(parameters);
     context.putAll(ctx.getParameters());

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd1ffd0f/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KuduSinkTest.java
----------------------------------------------------------------------
diff --git a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KuduSinkTest.java b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KuduSinkTest.java
index 6551010..d4dce94 100644
--- a/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KuduSinkTest.java
+++ b/java/kudu-flume-sink/src/test/java/org/apache/kudu/flume/sink/KuduSinkTest.java
@@ -235,7 +235,7 @@ public class KuduSinkTest extends BaseKuduTest {
     KuduSink sink = new KuduSink(syncClient);
     HashMap<String, String> parameters = new HashMap<>();
     parameters.put(KuduSinkConfigurationConstants.TABLE_NAME, tableName);
-    parameters.put(KuduSinkConfigurationConstants.MASTER_ADDRESSES, getMasterAddresses());
+    parameters.put(KuduSinkConfigurationConstants.MASTER_ADDRESSES, getMasterAddressesAsString());
     Context context = new Context(parameters);
     context.putAll(ctx.getParameters());
     Configurables.configure(sink, context);