You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/12/12 22:36:29 UTC

[GitHub] mikewalch closed pull request #825: #820 - Make sure AccumuloClient is closed

mikewalch closed pull request #825:  #820 - Make sure AccumuloClient is closed
URL: https://github.com/apache/accumulo/pull/825
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 21e15612d4..dbc98bad61 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -153,7 +153,6 @@ public void parseArgs(String programName, String[] args, Object... others) {
     startTracing(programName);
   }
 
-  private AccumuloClient cachedAccumuloClient = null;
   private Properties cachedProps = null;
 
   public String getPrincipal() {
@@ -168,11 +167,11 @@ public void setClientProperties(Properties clientProps) {
     this.cachedProps = clientProps;
   }
 
-  public AccumuloClient getClient() {
-    if (cachedAccumuloClient == null) {
-      cachedAccumuloClient = Accumulo.newClient().from(getClientProperties()).build();
-    }
-    return cachedAccumuloClient;
+  /**
+   * @return {@link AccumuloClient} that must be closed by user
+   */
+  public AccumuloClient createClient() {
+    return Accumulo.newClient().from(getClientProperties()).build();
   }
 
   public String getClientConfigFile() {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index dbbc8c3bea..24360dc74d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -81,8 +81,7 @@ public void start(String[] args) throws MergeException {
     Opts opts = new Opts();
     opts.parseArgs(Merge.class.getName(), args);
 
-    try {
-      AccumuloClient client = opts.getClient();
+    try (AccumuloClient client = opts.createClient()) {
 
       if (!client.tableOperations().exists(opts.getTableName())) {
         System.err.println("table " + opts.getTableName() + " does not exist");
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
index 26af9d3baf..f380c05fd2 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
@@ -46,12 +46,10 @@
 
   @Override
   public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
-    try {
-      // if the instance isn't configured, it will complain here
-      ClientInfo clientInfo = getClientInfo(job);
-      String principal = clientInfo.getPrincipal();
-      AuthenticationToken token = clientInfo.getAuthenticationToken();
-      AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
+    ClientInfo clientInfo = getClientInfo(job);
+    String principal = clientInfo.getPrincipal();
+    AuthenticationToken token = clientInfo.getAuthenticationToken();
+    try (AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build()) {
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException | AccumuloSecurityException e) {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
index 0bd279ab18..601b671fff 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
@@ -57,13 +57,10 @@
 
   @Override
   public void checkOutputSpecs(JobContext job) throws IOException {
-    try {
-      // if the instance isn't configured, it will complain here
-      ClientInfo clientInfo = getClientInfo(job);
-      String principal = clientInfo.getPrincipal();
-      AuthenticationToken token = clientInfo.getAuthenticationToken();
-      AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
-
+    ClientInfo clientInfo = getClientInfo(job);
+    String principal = clientInfo.getPrincipal();
+    AuthenticationToken token = clientInfo.getAuthenticationToken();
+    try (AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build()) {
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException | AccumuloSecurityException e) {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
index 63ac313b7f..559df5ddf3 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
@@ -182,8 +182,9 @@ protected static Authorizations getScanAuthorizations(JobConf job) {
    * @since 1.5.0
    */
   public static void validateOptions(JobConf job) throws IOException {
-    AccumuloClient client = InputConfigurator.getClient(CLASS, job);
-    InputConfigurator.validatePermissions(CLASS, job, client);
+    try (AccumuloClient client = InputConfigurator.createClient(CLASS, job)) {
+      InputConfigurator.validatePermissions(CLASS, job, client);
+    }
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
index e1a26b500d..7b7cbed3a4 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
@@ -346,6 +346,8 @@ public void close(Reporter reporter) throws IOException {
           log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
         }
         throw new IOException(e);
+      } finally {
+        client.close();
       }
     }
   }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
index 2e91e5d285..56ed8b9f6d 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
@@ -210,8 +210,10 @@ protected static InputTableConfig getInputTableConfig(JobContext context, String
    * @since 1.5.0
    */
   public static void validateOptions(JobContext context) throws IOException {
-    AccumuloClient client = InputConfigurator.getClient(CLASS, context.getConfiguration());
-    InputConfigurator.validatePermissions(CLASS, context.getConfiguration(), client);
+    try (
+        AccumuloClient client = InputConfigurator.createClient(CLASS, context.getConfiguration())) {
+      InputConfigurator.validatePermissions(CLASS, context.getConfiguration(), client);
+    }
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
index dc956bb019..80a292b0e6 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
@@ -348,6 +348,8 @@ public void close(TaskAttemptContext attempt) throws IOException {
           log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
         }
         throw new IOException(e);
+      } finally {
+        client.close();
       }
     }
   }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
index dc4bf4ecc3..3d85205dfb 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
@@ -247,8 +247,9 @@ private Authorizations getUserAuths(Authorizations scanAuths, ClientInfo clientI
       throws AccumuloSecurityException, AccumuloException {
     if (scanAuths != null)
       return scanAuths;
-    AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
-    return c.securityOperations().getUserAuthorizations(clientInfo.getPrincipal());
+    try (AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build()) {
+      return c.securityOperations().getUserAuthorizations(clientInfo.getPrincipal());
+    }
   }
 
   private static class BuilderBooleans {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
index 8ac9457bb8..dd2c3dc438 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
@@ -107,8 +107,7 @@ public static ClientInfo updateToken(org.apache.hadoop.security.Credentials cred
     ClientInfo result = info;
     if (info.getAuthenticationToken() instanceof KerberosToken) {
       log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        AccumuloClient client = Accumulo.newClient().from(info.getProperties()).build();
+      try (AccumuloClient client = Accumulo.newClient().from(info.getProperties()).build()) {
         AuthenticationToken token = client.securityOperations()
             .getDelegationToken(new DelegationTokenConfig());
         result = ClientInfo.from(Accumulo.newClientProperties().from(info.getProperties())
@@ -287,16 +286,16 @@ public static AuthenticationToken getAuthenticationToken(Class<?> implementingCl
   }
 
   /**
-   * Creates an Accumulo {@link AccumuloClient} based on the configuration
+   * Creates an {@link AccumuloClient} based on the configuration that must be closed by user
    *
    * @param implementingClass
    *          class whose name will be used as a prefix for the property configuration
    * @param conf
    *          Hadoop configuration object
-   * @return Accumulo connector
+   * @return {@link AccumuloClient} that must be closed by user
    * @since 2.0.0
    */
-  public static AccumuloClient getClient(Class<?> implementingClass, Configuration conf) {
+  public static AccumuloClient createClient(Class<?> implementingClass, Configuration conf) {
     return Accumulo.newClient().from(getClientProperties(implementingClass, conf)).build();
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
index 49a7fa427c..e376697b3a 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
@@ -58,24 +58,25 @@ public AuthenticationToken getToken() {
         log.info("Obtaining delegation token for {}", newPrincipal);
 
         setPrincipal(newPrincipal);
-        AccumuloClient client = Accumulo.newClient().from(getClientProperties())
-            .as(newPrincipal, krbToken).build();
+        try (AccumuloClient client = Accumulo.newClient().from(getClientProperties())
+            .as(newPrincipal, krbToken).build()) {
 
-        // Do the explicit check to see if the user has the permission to get a delegation token
-        if (!client.securityOperations().hasSystemPermission(client.whoami(),
-            SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
-          log.error(
-              "{} doesn't have the {} SystemPermission neccesary to obtain a delegation"
-                  + " token. MapReduce tasks cannot automatically use the client's"
-                  + " credentials on remote servers. Delegation tokens provide a means to run"
-                  + " MapReduce without distributing the user's credentials.",
-              user.getUserName(), SystemPermission.OBTAIN_DELEGATION_TOKEN.name());
-          throw new IllegalStateException(
-              client.whoami() + " does not have permission to obtain a delegation token");
-        }
+          // Do the explicit check to see if the user has the permission to get a delegation token
+          if (!client.securityOperations().hasSystemPermission(client.whoami(),
+              SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
+            log.error(
+                "{} doesn't have the {} SystemPermission neccesary to obtain a delegation"
+                    + " token. MapReduce tasks cannot automatically use the client's"
+                    + " credentials on remote servers. Delegation tokens provide a means to run"
+                    + " MapReduce without distributing the user's credentials.",
+                user.getUserName(), SystemPermission.OBTAIN_DELEGATION_TOKEN.name());
+            throw new IllegalStateException(
+                client.whoami() + " does not have permission to obtain a delegation token");
+          }
 
-        // Get the delegation token from Accumulo
-        return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
+          // Get the delegation token from Accumulo
+          return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
+        }
       } catch (Exception e) {
         final String msg = "Failed to acquire DelegationToken for use with MapReduce";
         log.error(msg, e);
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
index 0be85ade24..cb32ca5960 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -20,8 +20,6 @@
 import java.util.Properties;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.server.ServerContext;
@@ -53,12 +51,10 @@
   ServerContext getServerContext();
 
   /**
-   * Utility method to get a client connection to the cluster.
-   *
+   * @return {@link AccumuloClient} that must be closed by user
    * @since 2.0
    */
-  AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException;
+  AccumuloClient createAccumuloClient(String user, AuthenticationToken token);
 
   /**
    * Get the client configuration for the cluster
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index 0dab985209..8160e2814f 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -139,7 +139,7 @@ public synchronized ServerContext getServerContext() {
   }
 
   @Override
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
+  public AccumuloClient createAccumuloClient(String user, AuthenticationToken token) {
     return Accumulo.newClient().to(getInstanceName(), getZooKeepers()).as(user, token).build();
   }
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index e830551ae7..8ba0cc5832 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -118,22 +118,23 @@ public MiniAccumuloConfig getConfig() {
    * Utility method to get a connector to the MAC.
    *
    * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #getAccumuloClient(String, AuthenticationToken)}
+   * @deprecated since 2.0.0, replaced by {@link #createAccumuloClient(String, AuthenticationToken)}
    */
   @Deprecated
   public org.apache.accumulo.core.client.Connector getConnector(String user, String passwd)
       throws AccumuloException, AccumuloSecurityException {
     return org.apache.accumulo.core.client.Connector
-        .from(impl.getAccumuloClient(user, new PasswordToken(passwd)));
+        .from(impl.createAccumuloClient(user, new PasswordToken(passwd)));
   }
 
   /**
-   * Utility method to get a client connection to the MAC.
+   * Utility method to create an {@link AccumuloClient} with connection to the MAC. The
+   * AccumuloClient object should be closed by user
    *
    * @since 2.0.0
    */
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
-    return impl.getAccumuloClient(user, token);
+  public AccumuloClient createAccumuloClient(String user, AuthenticationToken token) {
+    return impl.createAccumuloClient(user, token);
   }
 
   /**
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index fb4a3aaa2c..590d4f9de5 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -780,7 +780,7 @@ public MiniAccumuloConfigImpl getConfig() {
   }
 
   @Override
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
+  public AccumuloClient createAccumuloClient(String user, AuthenticationToken token) {
     return Accumulo.newClient().from(getClientProperties()).as(user, token).build();
   }
 
diff --git a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
index 25a5e66a30..3c5f443ed6 100644
--- a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
@@ -73,7 +73,7 @@ public static void setupMiniCluster() throws Exception {
     accumulo = new MiniAccumuloClusterImpl(config);
     accumulo.start();
     // create a table to ensure there are some entries in the !0 table
-    AccumuloClient client = accumulo.getAccumuloClient("root", new PasswordToken("superSecret"));
+    AccumuloClient client = accumulo.createAccumuloClient("root", new PasswordToken("superSecret"));
     TableOperations tableops = client.tableOperations();
     tableops.create(TEST_TABLE);
     testTableID = tableops.tableIdMap().get(TEST_TABLE);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index 2af41c0f39..a59a40edf8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -247,8 +247,7 @@ public void execute(final String[] args) {
       } else if (cl.getParsedCommand().equals("volumes")) {
         ListVolumesUsed.listVolumes(context);
       } else if (cl.getParsedCommand().equals("randomizeVolumes")) {
-        rc = RandomizeVolumes.randomize(context, context.getClient(),
-            randomizeVolumesOpts.tableName);
+        rc = RandomizeVolumes.randomize(context, randomizeVolumesOpts.tableName);
       } else {
         everything = cl.getParsedCommand().equals("stopAll");
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
index e61eae28bc..1bcfabdb5e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
@@ -23,6 +23,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
@@ -97,59 +98,60 @@ public static void checkMetadataAndRootTableEntries(String tableNameToCheck, Ser
     System.out.println("Checking table: " + tableNameToCheck);
     Map<String,TreeSet<KeyExtent>> tables = new HashMap<>();
 
-    Scanner scanner;
+    try (AccumuloClient client = opts.createClient()) {
 
-    scanner = opts.getClient().createScanner(tableNameToCheck, Authorizations.EMPTY);
+      Scanner scanner = client.createScanner(tableNameToCheck, Authorizations.EMPTY);
 
-    scanner.setRange(MetadataSchema.TabletsSection.getRange());
-    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-    scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.setRange(MetadataSchema.TabletsSection.getRange());
+      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 
-    Text colf = new Text();
-    Text colq = new Text();
-    boolean justLoc = false;
+      Text colf = new Text();
+      Text colq = new Text();
+      boolean justLoc = false;
 
-    int count = 0;
+      int count = 0;
 
-    for (Entry<Key,Value> entry : scanner) {
-      colf = entry.getKey().getColumnFamily(colf);
-      colq = entry.getKey().getColumnQualifier(colq);
+      for (Entry<Key,Value> entry : scanner) {
+        colf = entry.getKey().getColumnFamily(colf);
+        colq = entry.getKey().getColumnQualifier(colq);
 
-      count++;
+        count++;
 
-      String tableName = (new KeyExtent(entry.getKey().getRow(), (Text) null)).getTableId()
-          .canonicalID();
+        String tableName = (new KeyExtent(entry.getKey().getRow(), (Text) null)).getTableId()
+            .canonicalID();
 
-      TreeSet<KeyExtent> tablets = tables.get(tableName);
-      if (tablets == null) {
-        Set<Entry<String,TreeSet<KeyExtent>>> es = tables.entrySet();
+        TreeSet<KeyExtent> tablets = tables.get(tableName);
+        if (tablets == null) {
+          Set<Entry<String,TreeSet<KeyExtent>>> es = tables.entrySet();
 
-        for (Entry<String,TreeSet<KeyExtent>> entry2 : es) {
-          checkTable(entry2.getKey(), entry2.getValue(), opts);
-        }
+          for (Entry<String,TreeSet<KeyExtent>> entry2 : es) {
+            checkTable(entry2.getKey(), entry2.getValue(), opts);
+          }
 
-        tables.clear();
+          tables.clear();
 
-        tablets = new TreeSet<>();
-        tables.put(tableName, tablets);
-      }
+          tablets = new TreeSet<>();
+          tables.put(tableName, tablets);
+        }
 
-      if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.equals(colf, colq)) {
-        KeyExtent tabletKe = new KeyExtent(entry.getKey().getRow(), entry.getValue());
-        tablets.add(tabletKe);
-        justLoc = false;
-      } else if (colf.equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
-        if (justLoc) {
-          System.out.println("Problem at key " + entry.getKey());
-          sawProblems = true;
+        if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.equals(colf, colq)) {
+          KeyExtent tabletKe = new KeyExtent(entry.getKey().getRow(), entry.getValue());
+          tablets.add(tabletKe);
+          justLoc = false;
+        } else if (colf.equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
+          if (justLoc) {
+            System.out.println("Problem at key " + entry.getKey());
+            sawProblems = true;
+          }
+          justLoc = true;
         }
-        justLoc = true;
       }
-    }
 
-    if (count == 0) {
-      System.err.println("ERROR : " + tableNameToCheck + " table is empty");
-      sawProblems = true;
+      if (count == 0) {
+        System.err.println("ERROR : " + tableNameToCheck + " table is empty");
+        sawProblems = true;
+      }
     }
 
     Set<Entry<String,TreeSet<KeyExtent>>> es = tables.entrySet();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
index 261a0f53b3..00af1df77c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
@@ -45,35 +45,36 @@ public int run(String[] args) throws Exception {
     opts.parseArgs(LocalityCheck.class.getName(), args);
 
     VolumeManager fs = opts.getServerContext().getVolumeManager();
-    AccumuloClient accumuloClient = opts.getClient();
-    Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
-    scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-    scanner.setRange(MetadataSchema.TabletsSection.getRange());
+    try (AccumuloClient accumuloClient = opts.createClient()) {
+      Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+      scanner.setRange(MetadataSchema.TabletsSection.getRange());
 
-    Map<String,Long> totalBlocks = new HashMap<>();
-    Map<String,Long> localBlocks = new HashMap<>();
-    ArrayList<String> files = new ArrayList<>();
+      Map<String,Long> totalBlocks = new HashMap<>();
+      Map<String,Long> localBlocks = new HashMap<>();
+      ArrayList<String> files = new ArrayList<>();
 
-    for (Entry<Key,Value> entry : scanner) {
-      Key key = entry.getKey();
-      if (key.compareColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME) == 0) {
-        String location = entry.getValue().toString();
-        String[] parts = location.split(":");
-        String host = parts[0];
-        addBlocks(fs, host, files, totalBlocks, localBlocks);
-        files.clear();
-      } else if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
+      for (Entry<Key,Value> entry : scanner) {
+        Key key = entry.getKey();
+        if (key.compareColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME) == 0) {
+          String location = entry.getValue().toString();
+          String[] parts = location.split(":");
+          String host = parts[0];
+          addBlocks(fs, host, files, totalBlocks, localBlocks);
+          files.clear();
+        } else if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
 
-        files.add(fs.getFullPath(key).toString());
+          files.add(fs.getFullPath(key).toString());
+        }
+      }
+      System.out.println(" Server         %local  total blocks");
+      for (Entry<String,Long> entry : totalBlocks.entrySet()) {
+        final String host = entry.getKey();
+        final Long blocksForHost = entry.getValue();
+        System.out.println(String.format("%15s %5.1f %8d", host,
+            (localBlocks.get(host) * 100.) / blocksForHost, blocksForHost));
       }
-    }
-    System.out.println(" Server         %local  total blocks");
-    for (Entry<String,Long> entry : totalBlocks.entrySet()) {
-      final String host = entry.getKey();
-      final Long blocksForHost = entry.getValue();
-      System.out.println(String.format("%15s %5.1f %8d", host,
-          (localBlocks.get(host) * 100.) / blocksForHost, blocksForHost));
     }
     return 0;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
index 7fc35682c4..19f9d51dc3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
@@ -100,8 +100,7 @@ public static void main(String[] args) throws Exception {
 
     long start = System.currentTimeMillis();
     log.info("starting at {} for user {}", start, opts.getPrincipal());
-    try {
-      AccumuloClient accumuloClient = opts.getClient();
+    try (AccumuloClient accumuloClient = opts.createClient()) {
       BatchWriter bw = accumuloClient.createBatchWriter(opts.getTableName(),
           bwOpts.getBatchWriterConfig());
       log.info("Writing {} mutations...", opts.count);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
index 81c9b2aeea..86283860b1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
@@ -23,7 +23,6 @@
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -50,18 +49,12 @@
 public class RandomizeVolumes {
   private static final Logger log = LoggerFactory.getLogger(RandomizeVolumes.class);
 
-  public static void main(String[] args) throws AccumuloException, AccumuloSecurityException {
+  public static void main(String[] args) {
     ServerUtilOnRequiredTable opts = new ServerUtilOnRequiredTable();
     opts.parseArgs(RandomizeVolumes.class.getName(), args);
     ServerContext context = opts.getServerContext();
-    AccumuloClient c;
-    if (opts.getToken() == null) {
-      c = context.getClient();
-    } else {
-      c = opts.getClient();
-    }
     try {
-      int status = randomize(context, c, opts.getTableName());
+      int status = randomize(context, opts.getTableName());
       System.exit(status);
     } catch (Exception ex) {
       log.error("{}", ex.getMessage(), ex);
@@ -69,14 +62,14 @@ public static void main(String[] args) throws AccumuloException, AccumuloSecurit
     }
   }
 
-  public static int randomize(ServerContext context, AccumuloClient c, String tableName)
-      throws IOException, AccumuloSecurityException, AccumuloException, TableNotFoundException {
+  public static int randomize(ServerContext context, String tableName)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     final VolumeManager vm = context.getVolumeManager();
     if (vm.getVolumes().size() < 2) {
       log.error("There are not enough volumes configured");
       return 1;
     }
-    String tblStr = c.tableOperations().tableIdMap().get(tableName);
+    String tblStr = context.getClient().tableOperations().tableIdMap().get(tableName);
     if (null == tblStr) {
       log.error("Could not determine the table ID for table {}", tableName);
       return 2;
@@ -85,15 +78,15 @@ public static int randomize(ServerContext context, AccumuloClient c, String tabl
     TableState tableState = context.getTableManager().getTableState(tableId);
     if (TableState.OFFLINE != tableState) {
       log.info("Taking {} offline", tableName);
-      c.tableOperations().offline(tableName, true);
+      context.getClient().tableOperations().offline(tableName, true);
       log.info("{} offline", tableName);
     }
     SimpleThreadPool pool = new SimpleThreadPool(50, "directory maker");
     log.info("Rewriting entries for {}", tableName);
-    Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner scanner = context.getClient().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     DIRECTORY_COLUMN.fetch(scanner);
     scanner.setRange(TabletsSection.getRange(tableId));
-    BatchWriter writer = c.createBatchWriter(MetadataTable.NAME, null);
+    BatchWriter writer = context.getClient().createBatchWriter(MetadataTable.NAME, null);
     int count = 0;
     for (Entry<Key,Value> entry : scanner) {
       String oldLocation = entry.getValue().toString();
@@ -122,14 +115,11 @@ public static int randomize(ServerContext context, AccumuloClient c, String tabl
         log.trace("Replacing {} with {}", oldLocation, newLocation);
       }
       writer.addMutation(m);
-      pool.submit(new Runnable() {
-        @Override
-        public void run() {
-          try {
-            vm.mkdirs(new Path(newLocation));
-          } catch (IOException ex) {
-            // nevermind
-          }
+      pool.submit(() -> {
+        try {
+          vm.mkdirs(new Path(newLocation));
+        } catch (IOException ex) {
+          // nevermind
         }
       });
       count++;
@@ -147,7 +137,7 @@ public void run() {
     }
     log.info("Updated {} entries for table {}", count, tableName);
     if (TableState.OFFLINE != tableState) {
-      c.tableOperations().online(tableName, true);
+      context.getClient().tableOperations().online(tableName, true);
       log.info("table {} back online", tableName);
     }
     return 0;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
index b03765aed6..e7c310e475 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
@@ -299,9 +299,10 @@ public static void printDiskUsage(Collection<String> tableNames, VolumeManager f
   public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(TableDiskUsage.class.getName(), args);
-    AccumuloClient client = opts.getClient();
-    VolumeManager fs = opts.getServerContext().getVolumeManager();
-    org.apache.accumulo.server.util.TableDiskUsage.printDiskUsage(opts.tables, fs, client, false);
+    try (AccumuloClient client = opts.createClient()) {
+      VolumeManager fs = opts.getServerContext().getVolumeManager();
+      org.apache.accumulo.server.util.TableDiskUsage.printDiskUsage(opts.tables, fs, client, false);
+    }
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index 68941bc933..f956cd0275 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -73,11 +73,11 @@ public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(VerifyTabletAssignments.class.getName(), args);
 
-    AccumuloClient client = opts.getClient();
-    ClientContext context = new ClientContext(client);
-    for (String table : client.tableOperations().list())
-      checkTable(context, opts, table, null);
-
+    try (AccumuloClient client = opts.createClient()) {
+      ClientContext context = new ClientContext(client);
+      for (String table : client.tableOperations().list())
+        checkTable(context, opts, table, null);
+    }
   }
 
   private static void checkTable(final ClientContext context, final Opts opts, String tableName,
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index a4622268ba..b79f6c4d4c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@ -262,22 +262,23 @@ public static void main(String[] args) throws Exception {
     ServerUtilOpts opts = new ServerUtilOpts();
     opts.parseArgs(MergeStats.class.getName(), args);
 
-    AccumuloClient client = opts.getClient();
-    Map<String,String> tableIdMap = client.tableOperations().tableIdMap();
-    ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooReaderWriter();
-    for (Entry<String,String> entry : tableIdMap.entrySet()) {
-      final String table = entry.getKey(), tableId = entry.getValue();
-      String path = ZooUtil.getRoot(client.getInstanceID()) + Constants.ZTABLES + "/" + tableId
-          + "/merge";
-      MergeInfo info = new MergeInfo();
-      if (zooReaderWriter.exists(path)) {
-        byte[] data = zooReaderWriter.getData(path, new Stat());
-        DataInputBuffer in = new DataInputBuffer();
-        in.reset(data, data.length);
-        info.readFields(in);
+    try (AccumuloClient client = opts.createClient()) {
+      Map<String,String> tableIdMap = client.tableOperations().tableIdMap();
+      ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooReaderWriter();
+      for (Entry<String,String> entry : tableIdMap.entrySet()) {
+        final String table = entry.getKey(), tableId = entry.getValue();
+        String path = ZooUtil.getRoot(client.getInstanceID()) + Constants.ZTABLES + "/" + tableId
+            + "/merge";
+        MergeInfo info = new MergeInfo();
+        if (zooReaderWriter.exists(path)) {
+          byte[] data = zooReaderWriter.getData(path, new Stat());
+          DataInputBuffer in = new DataInputBuffer();
+          in.reset(data, data.length);
+          info.readFields(in);
+        }
+        System.out.println(String.format("%25s  %10s %10s %s", table, info.getState(),
+            info.getOperation(), info.getExtent()));
       }
-      System.out.println(String.format("%25s  %10s %10s %s", table, info.getState(),
-          info.getOperation(), info.getExtent()));
     }
   }
 }
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceDump.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceDump.java
index 33cb3d5004..c26d761c10 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceDump.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceDump.java
@@ -82,18 +82,19 @@ private static int listSpans(Opts opts, ScannerOpts scanOpts) throws Exception {
     PrintStream out = System.out;
     long endTime = System.currentTimeMillis();
     long startTime = endTime - opts.length;
-    AccumuloClient client = opts.getClient();
-    Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
-    scanner.setBatchSize(scanOpts.scanBatchSize);
-    Range range = new Range(new Text("start:" + Long.toHexString(startTime)),
-        new Text("start:" + Long.toHexString(endTime)));
-    scanner.setRange(range);
-    out.println("Trace            Day/Time                 (ms)  Start");
-    for (Entry<Key,Value> entry : scanner) {
-      RemoteSpan span = TraceFormatter.getRemoteSpan(entry);
-      out.println(String.format("%016x %s %5d %s", span.traceId,
-          TraceFormatter.formatDate(new Date(span.getStart())), span.stop - span.start,
-          span.description));
+    try (AccumuloClient client = opts.createClient()) {
+      Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
+      scanner.setBatchSize(scanOpts.scanBatchSize);
+      Range range = new Range(new Text("start:" + Long.toHexString(startTime)),
+          new Text("start:" + Long.toHexString(endTime)));
+      scanner.setRange(range);
+      out.println("Trace            Day/Time                 (ms)  Start");
+      for (Entry<Key,Value> entry : scanner) {
+        RemoteSpan span = TraceFormatter.getRemoteSpan(entry);
+        out.println(String.format("%016x %s %5d %s", span.traceId,
+            TraceFormatter.formatDate(new Date(span.getStart())), span.stop - span.start,
+            span.description));
+      }
     }
     return 0;
   }
@@ -104,20 +105,15 @@ private static int listSpans(Opts opts, ScannerOpts scanOpts) throws Exception {
 
   private static int dumpTrace(Opts opts, ScannerOpts scanOpts) throws Exception {
     final PrintStream out = System.out;
-    AccumuloClient client = opts.getClient();
-
     int count = 0;
-    for (String traceId : opts.traceIds) {
-      Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
-      scanner.setBatchSize(scanOpts.scanBatchSize);
-      Range range = new Range(new Text(traceId.toString()));
-      scanner.setRange(range);
-      count = printTrace(scanner, new Printer() {
-        @Override
-        public void print(String line) {
-          out.println(line);
-        }
-      });
+    try (AccumuloClient client = opts.createClient()) {
+      for (String traceId : opts.traceIds) {
+        Scanner scanner = client.createScanner(opts.getTableName(), opts.auths);
+        scanner.setBatchSize(scanOpts.scanBatchSize);
+        Range range = new Range(new Text(traceId));
+        scanner.setRange(range);
+        count = printTrace(scanner, out::println);
+      }
     }
     return count > 0 ? 0 : 1;
   }
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
index 45e630db6b..dd15da6343 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
@@ -80,7 +80,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class TraceServer implements Watcher {
+public class TraceServer implements Watcher, AutoCloseable {
 
   private static final Logger log = LoggerFactory.getLogger(TraceServer.class);
   private final ServerConfigurationFactory serverConfiguration;
@@ -97,6 +97,13 @@ private static void put(Mutation m, String cf, String cq, byte[] bytes, int len)
     m.put(new Text(cf), new Text(cq), new Value(bytes, 0, len));
   }
 
+  @Override
+  public void close() throws Exception {
+    if (accumuloClient != null) {
+      accumuloClient.close();
+    }
+  }
+
   static class ByteArrayTransport extends TTransport {
     TByteArrayOutputStream out = new TByteArrayOutputStream();
 
@@ -267,14 +274,13 @@ private AccumuloClient ensureTraceTableExists(final AccumuloConfiguration conf)
           for (Entry<String,String> entry : loginMap.entrySet()) {
             props.put(entry.getKey().substring(prefixLength), entry.getValue());
           }
-
           token.init(props);
-
           at = token;
         }
 
         accumuloClient = Accumulo.newClient().from(context.getProperties()).as(principal, at)
             .build();
+
         if (!accumuloClient.tableOperations().exists(tableName)) {
           accumuloClient.tableOperations().create(tableName);
           IteratorSetting setting = new IteratorSetting(10, "ageoff", AgeOffFilter.class.getName());
@@ -288,6 +294,10 @@ private AccumuloClient ensureTraceTableExists(final AccumuloConfiguration conf)
           | RuntimeException ex) {
         log.info("Waiting to checking/create the trace table.", ex);
         sleepUninterruptibly(1, TimeUnit.SECONDS);
+        if (accumuloClient != null) {
+          accumuloClient.close();
+          accumuloClient = null;
+        }
       }
     }
     return accumuloClient;
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceTableStats.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceTableStats.java
index 525de096b7..fd118508fa 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceTableStats.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceTableStats.java
@@ -25,8 +25,6 @@
 
 import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
@@ -72,11 +70,8 @@ public static void main(String[] args) throws Exception {
     stats.count(opts);
   }
 
-  public void count(Opts opts)
-      throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    AccumuloClient client = opts.getClient();
-    Scanner scanner = client.createScanner(opts.getTableName(), Authorizations.EMPTY);
-    scanner.setRange(new Range(null, true, "idx:", false));
+  public void count(Opts opts) throws TableNotFoundException {
+
     Map<String,SpanTypeCount> counts = new TreeMap<>();
     final SpanTypeCount hdfs = new SpanTypeCount();
     hdfs.type = "HDFS";
@@ -85,31 +80,36 @@ public void count(Opts opts)
     long numSpans = 0;
     double maxSpanLength = 0;
     double maxSpanLengthMS = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      numSpans++;
-      RemoteSpan span = TraceFormatter.getRemoteSpan(entry);
-      String id = span.getSvc() + ":" + span.getDescription().replaceAll("[0-9][0-9][0-9]+", "");
-      SpanTypeCount stc = counts.get(id);
-      if (stc == null) {
-        stc = new SpanTypeCount();
-        counts.put(id, stc);
-        if (span.description.startsWith("org.apache.hadoop") || span.svc.equals("NameNode")
-            || span.svc.equals("DataNode") || span.description.contains("DFSOutputStream")
-            || span.description.contains("DFSInputStream")
-            || span.description.contains("BlockReader")) {
-          stc.type = hdfs.type;
+
+    try (AccumuloClient client = opts.createClient()) {
+      Scanner scanner = client.createScanner(opts.getTableName(), Authorizations.EMPTY);
+      scanner.setRange(new Range(null, true, "idx:", false));
+      for (Entry<Key,Value> entry : scanner) {
+        numSpans++;
+        RemoteSpan span = TraceFormatter.getRemoteSpan(entry);
+        String id = span.getSvc() + ":" + span.getDescription().replaceAll("[0-9][0-9][0-9]+", "");
+        SpanTypeCount stc = counts.get(id);
+        if (stc == null) {
+          stc = new SpanTypeCount();
+          counts.put(id, stc);
+          if (span.description.startsWith("org.apache.hadoop") || span.svc.equals("NameNode")
+              || span.svc.equals("DataNode") || span.description.contains("DFSOutputStream")
+              || span.description.contains("DFSInputStream")
+              || span.description.contains("BlockReader")) {
+            stc.type = hdfs.type;
+          } else {
+            stc.type = accumulo.type;
+          }
+        }
+        increment(stc, span);
+        if (stc.type.equals(hdfs.type)) {
+          increment(hdfs, span);
         } else {
-          stc.type = accumulo.type;
+          increment(accumulo, span);
         }
+        maxSpanLength = Math.max(maxSpanLength, Math.log10(span.stop - span.start));
+        maxSpanLengthMS = Math.max(maxSpanLengthMS, span.stop - span.start);
       }
-      increment(stc, span);
-      if (stc.type.equals(hdfs.type)) {
-        increment(hdfs, span);
-      } else {
-        increment(accumulo, span);
-      }
-      maxSpanLength = Math.max(maxSpanLength, Math.log10(span.stop - span.start));
-      maxSpanLengthMS = Math.max(maxSpanLengthMS, span.stop - span.start);
     }
     System.out.println();
     System.out.println("log10 max span length " + maxSpanLength + " " + maxSpanLengthMS);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 4679bc1c28..bab0659c91 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -199,6 +199,7 @@
   protected int exitCode = 0;
   private String tableName;
   private AccumuloClient accumuloClient;
+  private Properties clientProperties = new Properties();
   private ClientContext context;
   protected ConsoleReader reader;
   private final Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
@@ -290,8 +291,8 @@ public boolean config(String... args) throws IOException {
     authTimeout = TimeUnit.MINUTES.toNanos(options.getAuthTimeout());
     disableAuthTimeout = options.isAuthTimeoutDisabled();
 
-    Properties properties = options.getClientProperties();
-    if (ClientProperty.SASL_ENABLED.getBoolean(properties)) {
+    clientProperties = options.getClientProperties();
+    if (ClientProperty.SASL_ENABLED.getBoolean(clientProperties)) {
       log.debug("SASL is enabled, disabling authorization timeout");
       disableAuthTimeout = true;
     }
@@ -300,12 +301,11 @@ public boolean config(String... args) throws IOException {
     this.setTableName("");
 
     if (accumuloClient == null) {
-      Properties props = options.getClientProperties();
-      if (ClientProperty.INSTANCE_ZOOKEEPERS.isEmpty(props)) {
+      if (ClientProperty.INSTANCE_ZOOKEEPERS.isEmpty(clientProperties)) {
         throw new IllegalArgumentException("ZooKeepers must be set using -z or -zh on command line"
             + " or in accumulo-client.properties");
       }
-      if (ClientProperty.INSTANCE_NAME.isEmpty(props)) {
+      if (ClientProperty.INSTANCE_NAME.isEmpty(clientProperties)) {
         throw new IllegalArgumentException("Instance name must be set using -z or -zi on command "
             + "line or in accumulo-client.properties");
       }
@@ -319,9 +319,9 @@ public boolean config(String... args) throws IOException {
       }
       String password = options.getPassword();
       AuthenticationToken token = null;
-      if (password == null && props.containsKey(ClientProperty.AUTH_TOKEN.getKey())
-          && principal.equals(ClientProperty.AUTH_PRINCIPAL.getValue(props))) {
-        token = ClientProperty.getAuthenticationToken(props);
+      if (password == null && clientProperties.containsKey(ClientProperty.AUTH_TOKEN.getKey())
+          && principal.equals(ClientProperty.AUTH_PRINCIPAL.getValue(clientProperties))) {
+        token = ClientProperty.getAuthenticationToken(clientProperties);
       }
       if (token == null) {
         Runtime.getRuntime()
@@ -338,9 +338,10 @@ public boolean config(String... args) throws IOException {
         }
       }
       try {
-        DistributedTrace.enable(InetAddress.getLocalHost().getHostName(), "shell", properties);
+        DistributedTrace.enable(InetAddress.getLocalHost().getHostName(), "shell",
+            clientProperties);
         this.setTableName("");
-        accumuloClient = Accumulo.newClient().from(props).as(principal, token).build();
+        accumuloClient = Accumulo.newClient().from(clientProperties).as(principal, token).build();
         context = new ClientContext(accumuloClient);
       } catch (Exception e) {
         printException(e);
@@ -610,6 +611,9 @@ public void shutdown() {
     if (reader != null) {
       reader.shutdown();
     }
+    if (accumuloClient != null) {
+      accumuloClient.close();
+    }
   }
 
   public void printInfo() throws IOException {
@@ -1169,8 +1173,10 @@ public ConsoleReader getReader() {
 
   public void updateUser(String principal, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
-    accumuloClient = Accumulo.newClient().from(accumuloClient.properties()).as(principal, token)
-        .build();
+    if (accumuloClient != null) {
+      accumuloClient.close();
+    }
+    accumuloClient = Accumulo.newClient().from(clientProperties).as(principal, token).build();
     accumuloClient.securityOperations().authenticateUser(principal, token);
     context = new ClientContext(accumuloClient);
   }
diff --git a/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java b/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
index 282470cdc7..30bb569de2 100644
--- a/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
+++ b/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
@@ -78,6 +78,7 @@ public boolean isDynamic() {
   private static boolean initialized = false;
 
   protected static AccumuloCluster cluster;
+  protected static AccumuloClient client;
   protected static ClusterType type;
   protected static AccumuloClusterPropertyConfiguration clusterConf;
   protected static TestingKdc krb;
@@ -102,6 +103,9 @@ public static void tearDownKdc() throws Exception {
     if (null != krb) {
       krb.stop();
     }
+    if (client != null) {
+      client.close();
+    }
   }
 
   /**
@@ -184,7 +188,7 @@ public void setupCluster() throws Exception {
           // permissions to)
           UserGroupInformation.loginUserFromKeytab(systemUser.getPrincipal(),
               systemUser.getKeytab().getAbsolutePath());
-          AccumuloClient client = cluster.getAccumuloClient(systemUser.getPrincipal(),
+          AccumuloClient client = cluster.createAccumuloClient(systemUser.getPrincipal(),
               new KerberosToken());
 
           // Then, log back in as the "root" user and do the grant
@@ -349,17 +353,19 @@ public static AccumuloClusterConfiguration getClusterConfiguration() {
   }
 
   public AccumuloClient getAccumuloClient() {
-    try {
-      String princ = getAdminPrincipal();
-      AuthenticationToken token = getAdminToken();
-      log.debug("Creating client as {} with {}", princ, token);
-      return cluster.getAccumuloClient(princ, token);
-    } catch (Exception e) {
-      log.error("Could not connect to Accumulo", e);
-      fail("Could not connect to Accumulo: " + e.getMessage());
-
-      throw new RuntimeException("Could not connect to Accumulo", e);
+    if (client == null) {
+      try {
+        String princ = getAdminPrincipal();
+        AuthenticationToken token = getAdminToken();
+        log.debug("Creating client as {} with {}", princ, token);
+        client = cluster.createAccumuloClient(princ, token);
+      } catch (Exception e) {
+        log.error("Could not connect to Accumulo", e);
+        fail("Could not connect to Accumulo: " + e.getMessage());
+        throw new RuntimeException("Could not connect to Accumulo", e);
+      }
     }
+    return client;
   }
 
   // TODO Really don't want this here. Will ultimately need to abstract configuration method away
diff --git a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
index 2f4debc18a..342f8ec53f 100644
--- a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
+++ b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
@@ -61,6 +61,7 @@
   private static String principal = "root";
   private static String rootPassword;
   private static AuthenticationToken token;
+  private static AccumuloClient client;
   private static MiniAccumuloClusterImpl cluster;
   private static TestingKdc krb;
 
@@ -117,26 +118,29 @@ public static void startMiniClusterWithConfig(
       // permissions to)
       UserGroupInformation.loginUserFromKeytab(systemUser.getPrincipal(),
           systemUser.getKeytab().getAbsolutePath());
-      AccumuloClient client = cluster.getAccumuloClient(systemUser.getPrincipal(),
-          new KerberosToken());
+
+      AuthenticationToken tempToken = new KerberosToken();
+      try (AccumuloClient c = cluster.createAccumuloClient(systemUser.getPrincipal(), tempToken)) {
+        c.securityOperations().authenticateUser(systemUser.getPrincipal(), tempToken);
+      }
 
       // Then, log back in as the "root" user and do the grant
       UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(),
           rootUser.getKeytab().getAbsolutePath());
-      client = cluster.getAccumuloClient(principal, token);
-
-      // Create the trace table
-      client.tableOperations().create(traceTable);
-
-      // Trace user (which is the same kerberos principal as the system user, but using a normal
-      // KerberosToken) needs
-      // to have the ability to read, write and alter the trace table
-      client.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
-          TablePermission.READ);
-      client.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
-          TablePermission.WRITE);
-      client.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
-          TablePermission.ALTER_TABLE);
+
+      try (AccumuloClient c = cluster.createAccumuloClient(principal, token)) {
+        // Create the trace table
+        c.tableOperations().create(traceTable);
+        // Trace user (which is the same kerberos principal as the system user, but using a normal
+        // KerberosToken) needs
+        // to have the ability to read, write and alter the trace table
+        c.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
+            TablePermission.READ);
+        c.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
+            TablePermission.WRITE);
+        c.securityOperations().grantTablePermission(systemUser.getPrincipal(), traceTable,
+            TablePermission.ALTER_TABLE);
+      }
     }
   }
 
@@ -158,6 +162,9 @@ public static void stopMiniCluster() throws Exception {
         log.error("Failed to stop KDC", e);
       }
     }
+    if (client != null) {
+      client.close();
+    }
   }
 
   public static String getRootPassword() {
@@ -189,11 +196,10 @@ public static File getMiniClusterDir() {
   }
 
   public static AccumuloClient getClient() {
-    try {
-      return getCluster().getAccumuloClient(principal, getToken());
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    if (client == null) {
+      client = getCluster().createAccumuloClient(principal, getToken());
     }
+    return client;
   }
 
   public static TestingKdc getKdc() {
diff --git a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
index 41da4096ac..aa632417ce 100644
--- a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -207,7 +207,8 @@ public void testTableOperationsAudits() throws AccumuloException, AccumuloSecuri
 
     // Connect as Audit User and do a bunch of stuff.
     // Testing activity begins here
-    auditAccumuloClient = getCluster().getAccumuloClient(AUDIT_USER_1, new PasswordToken(PASSWORD));
+    auditAccumuloClient = getCluster().createAccumuloClient(AUDIT_USER_1,
+        new PasswordToken(PASSWORD));
     auditAccumuloClient.tableOperations().create(OLD_TEST_TABLE_NAME);
     auditAccumuloClient.tableOperations().rename(OLD_TEST_TABLE_NAME, NEW_TEST_TABLE_NAME);
     Map<String,String> emptyMap = Collections.emptyMap();
@@ -247,7 +248,8 @@ public void testUserOperationsAudits() throws AccumuloSecurityException, Accumul
 
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditAccumuloClient = getCluster().getAccumuloClient(AUDIT_USER_1, new PasswordToken(PASSWORD));
+    auditAccumuloClient = getCluster().createAccumuloClient(AUDIT_USER_1,
+        new PasswordToken(PASSWORD));
     auditAccumuloClient.securityOperations().createLocalUser(AUDIT_USER_2,
         new PasswordToken(PASSWORD));
 
@@ -302,7 +304,8 @@ public void testImportExportOperationsAudits()
 
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditAccumuloClient = getCluster().getAccumuloClient(AUDIT_USER_1, new PasswordToken(PASSWORD));
+    auditAccumuloClient = getCluster().createAccumuloClient(AUDIT_USER_1,
+        new PasswordToken(PASSWORD));
     auditAccumuloClient.tableOperations().create(OLD_TEST_TABLE_NAME);
 
     // Insert some play data
@@ -390,7 +393,8 @@ public void testDataOperationsAudits() throws AccumuloSecurityException, Accumul
 
     // Connect as Audit User and do a bunch of stuff.
     // Start testing activities here
-    auditAccumuloClient = getCluster().getAccumuloClient(AUDIT_USER_1, new PasswordToken(PASSWORD));
+    auditAccumuloClient = getCluster().createAccumuloClient(AUDIT_USER_1,
+        new PasswordToken(PASSWORD));
     auditAccumuloClient.tableOperations().create(OLD_TEST_TABLE_NAME);
 
     // Insert some play data
@@ -444,7 +448,8 @@ public void testDeniedAudits() throws AccumuloSecurityException, AccumuloExcepti
     // Create our user with no privs
     client.securityOperations().createLocalUser(AUDIT_USER_1, new PasswordToken(PASSWORD));
     client.tableOperations().create(OLD_TEST_TABLE_NAME);
-    auditAccumuloClient = getCluster().getAccumuloClient(AUDIT_USER_1, new PasswordToken(PASSWORD));
+    auditAccumuloClient = getCluster().createAccumuloClient(AUDIT_USER_1,
+        new PasswordToken(PASSWORD));
 
     // Start testing activities
     // We should get denied or / failed audit messages here.
diff --git a/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java b/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java
index 93bc9ff5bf..5e2e36c80d 100644
--- a/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java
@@ -47,7 +47,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) {
   public void majorCompactionsAreRateLimited() throws Exception {
     long bytesWritten = 0;
     String tableName = getUniqueNames(1)[0];
-    AccumuloClient client = getCluster().getAccumuloClient("root",
+    AccumuloClient client = getCluster().createAccumuloClient("root",
         new PasswordToken(ROOT_PASSWORD));
     client.tableOperations().create(tableName);
     try (BatchWriter bw = client.createBatchWriter(tableName, new BatchWriterConfig())) {
diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
index c32b71f9b2..54c02cb6d2 100644
--- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -87,7 +87,7 @@ private void createEmptyConfig(File confFile) throws IOException {
   @Test
   public void testExistingInstance() throws Exception {
 
-    AccumuloClient client = getCluster().getAccumuloClient("root",
+    AccumuloClient client = getCluster().createAccumuloClient("root",
         new PasswordToken(ROOT_PASSWORD));
 
     client.tableOperations().create("table1");
@@ -142,7 +142,7 @@ public void testExistingInstance() throws Exception {
     MiniAccumuloClusterImpl accumulo2 = new MiniAccumuloClusterImpl(macConfig2);
     accumulo2.start();
 
-    client = accumulo2.getAccumuloClient("root", new PasswordToken(ROOT_PASSWORD));
+    client = accumulo2.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD));
 
     try (Scanner scanner = client.createScanner("table1", Authorizations.EMPTY)) {
       int sum = 0;
diff --git a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
index 4c4a155035..32c5ca2075 100644
--- a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java
@@ -27,8 +27,6 @@
 
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
@@ -94,65 +92,65 @@ public void run() {
     int numThreads = 1;
   }
 
-  public static void main(String[] args)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+  public static void main(String[] args) throws TableNotFoundException {
     Opts opts = new Opts();
     ScannerOpts scanOpts = new ScannerOpts();
     opts.parseArgs(QueryMetadataTable.class.getName(), args, scanOpts);
 
-    AccumuloClient accumuloClient = opts.getClient();
-    HashSet<Text> rowSet = new HashSet<>();
-
-    int count = 0;
-
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, opts.auths)) {
-      scanner.setBatchSize(scanOpts.scanBatchSize);
-      Text mdrow = new Text(TabletsSection.getRow(MetadataTable.ID, null));
-
-      for (Entry<Key,Value> entry : scanner) {
-        System.out.print(".");
-        if (count % 72 == 0) {
-          System.out.printf(" %,d%n", count);
-        }
-        if (entry.getKey().compareRow(mdrow) == 0 && entry.getKey().getColumnFamily()
-            .compareTo(TabletsSection.CurrentLocationColumnFamily.NAME) == 0) {
-          System.out.println(entry.getKey() + " " + entry.getValue());
-          location = entry.getValue().toString();
+    try (AccumuloClient accumuloClient = opts.createClient()) {
+      HashSet<Text> rowSet = new HashSet<>();
+
+      int count = 0;
+
+      try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, opts.auths)) {
+        scanner.setBatchSize(scanOpts.scanBatchSize);
+        Text mdrow = new Text(TabletsSection.getRow(MetadataTable.ID, null));
+
+        for (Entry<Key,Value> entry : scanner) {
+          System.out.print(".");
+          if (count % 72 == 0) {
+            System.out.printf(" %,d%n", count);
+          }
+          if (entry.getKey().compareRow(mdrow) == 0 && entry.getKey().getColumnFamily()
+              .compareTo(TabletsSection.CurrentLocationColumnFamily.NAME) == 0) {
+            System.out.println(entry.getKey() + " " + entry.getValue());
+            location = entry.getValue().toString();
+          }
+
+          if (!entry.getKey().getRow().toString().startsWith(MetadataTable.ID.canonicalID()))
+            rowSet.add(entry.getKey().getRow());
+          count++;
         }
-
-        if (!entry.getKey().getRow().toString().startsWith(MetadataTable.ID.canonicalID()))
-          rowSet.add(entry.getKey().getRow());
-        count++;
       }
-    }
 
-    System.out.printf(" %,d%n", count);
+      System.out.printf(" %,d%n", count);
 
-    ArrayList<Text> rows = new ArrayList<>(rowSet);
+      ArrayList<Text> rows = new ArrayList<>(rowSet);
 
-    Random r = new SecureRandom();
+      Random r = new SecureRandom();
 
-    ExecutorService tp = Executors.newFixedThreadPool(opts.numThreads);
+      ExecutorService tp = Executors.newFixedThreadPool(opts.numThreads);
 
-    long t1 = System.currentTimeMillis();
+      long t1 = System.currentTimeMillis();
 
-    for (int i = 0; i < opts.numQueries; i++) {
-      int index = r.nextInt(rows.size());
-      MDTQuery mdtq = new MDTQuery(accumuloClient, rows.get(index));
-      tp.submit(mdtq);
-    }
+      for (int i = 0; i < opts.numQueries; i++) {
+        int index = r.nextInt(rows.size());
+        MDTQuery mdtq = new MDTQuery(accumuloClient, rows.get(index));
+        tp.submit(mdtq);
+      }
 
-    tp.shutdown();
+      tp.shutdown();
 
-    try {
-      tp.awaitTermination(1, TimeUnit.HOURS);
-    } catch (InterruptedException e) {
-      log.error("Failed while awaiting the ExcecutorService to terminate.", e);
-      throw new RuntimeException(e);
-    }
+      try {
+        tp.awaitTermination(1, TimeUnit.HOURS);
+      } catch (InterruptedException e) {
+        log.error("Failed while awaiting the ExcecutorService to terminate.", e);
+        throw new RuntimeException(e);
+      }
 
-    long t2 = System.currentTimeMillis();
-    double delta = (t2 - t1) / 1000.0;
-    System.out.println("time : " + delta + "  queries per sec : " + (opts.numQueries / delta));
+      long t2 = System.currentTimeMillis();
+      double delta = (t2 - t1) / 1000.0;
+      System.out.println("time : " + delta + "  queries per sec : " + (opts.numQueries / delta));
+    }
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 209f084482..ed5394c38e 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -304,7 +304,7 @@ public static void setupMiniCluster() throws Exception {
 
     traceProcess = getCluster().exec(TraceServer.class);
 
-    AccumuloClient client = getCluster().getAccumuloClient(getPrincipal(), getToken());
+    AccumuloClient client = getCluster().createAccumuloClient(getPrincipal(), getToken());
     TableOperations tops = client.tableOperations();
 
     // give the tracer some time to start
diff --git a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
index fbac5397dc..0de6992285 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
@@ -236,8 +236,8 @@ public static void main(String[] args) {
     ScannerOpts scanOpts = new ScannerOpts();
     opts.parseArgs(TestBinaryRows.class.getName(), args, scanOpts, bwOpts);
 
-    try {
-      runTest(opts.getClient(), opts, bwOpts, scanOpts);
+    try (AccumuloClient client = opts.createClient()) {
+      runTest(client, opts, bwOpts, scanOpts);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
index 6fc7b6d05f..34df52cdb4 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -191,16 +191,14 @@ public static void main(String[] args) throws Exception {
 
     String name = TestIngest.class.getSimpleName();
     DistributedTrace.enable(name);
-
     try {
       opts.startTracing(name);
-
       if (opts.debug)
         Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
 
-      // test batch update
-
-      ingest(opts.getClient(), opts, bwOpts);
+      try (AccumuloClient client = opts.createClient()) {
+        ingest(client, opts, bwOpts);
+      }
     } catch (Exception e) {
       throw new RuntimeException(e);
     } finally {
diff --git a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
index dcf5e39dfe..7dc36c3f25 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
@@ -80,40 +80,41 @@ public static void main(String[] args) throws Exception {
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(TestMultiTableIngest.class.getName(), args, scanOpts, bwOpts);
     // create the test table within accumulo
-    AccumuloClient accumuloClient = opts.getClient();
-    for (int i = 0; i < opts.tables; i++) {
-      tableNames.add(String.format(opts.prefix + "%04d", i));
-    }
+    try (AccumuloClient accumuloClient = opts.createClient()) {
+      for (int i = 0; i < opts.tables; i++) {
+        tableNames.add(String.format(opts.prefix + "%04d", i));
+      }
 
-    if (!opts.readonly) {
-      for (String table : tableNames)
-        accumuloClient.tableOperations().create(table);
+      if (!opts.readonly) {
+        for (String table : tableNames)
+          accumuloClient.tableOperations().create(table);
 
-      MultiTableBatchWriter b;
-      try {
-        b = accumuloClient.createMultiTableBatchWriter(bwOpts.getBatchWriterConfig());
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
+        MultiTableBatchWriter b;
+        try {
+          b = accumuloClient.createMultiTableBatchWriter(bwOpts.getBatchWriterConfig());
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
 
-      // populate
-      for (int i = 0; i < opts.count; i++) {
-        Mutation m = new Mutation(new Text(String.format("%06d", i)));
-        m.put(new Text("col" + Integer.toString((i % 3) + 1)), new Text("qual"),
-            new Value("junk".getBytes(UTF_8)));
-        b.getBatchWriter(tableNames.get(i % tableNames.size())).addMutation(m);
+        // populate
+        for (int i = 0; i < opts.count; i++) {
+          Mutation m = new Mutation(new Text(String.format("%06d", i)));
+          m.put(new Text("col" + Integer.toString((i % 3) + 1)), new Text("qual"),
+              new Value("junk".getBytes(UTF_8)));
+          b.getBatchWriter(tableNames.get(i % tableNames.size())).addMutation(m);
+        }
+        try {
+          b.close();
+        } catch (MutationsRejectedException e) {
+          throw new RuntimeException(e);
+        }
       }
       try {
-        b.close();
-      } catch (MutationsRejectedException e) {
+        readBack(opts, scanOpts, accumuloClient, tableNames);
+      } catch (Exception e) {
         throw new RuntimeException(e);
       }
     }
-    try {
-      readBack(opts, scanOpts, accumuloClient, tableNames);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
   }
 
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java b/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
index 12c67c47ec..eab9096e4a 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestRandomDeletes.java
@@ -82,7 +82,8 @@ public String toString() {
   private static TreeSet<RowColumn> scanAll(ClientOnDefaultTable opts, ScannerOpts scanOpts,
       String tableName) throws Exception {
     TreeSet<RowColumn> result = new TreeSet<>();
-    try (Scanner scanner = opts.getClient().createScanner(tableName, auths)) {
+    try (AccumuloClient client = opts.createClient();
+        Scanner scanner = client.createScanner(tableName, auths)) {
       scanner.setBatchSize(scanOpts.scanBatchSize);
       for (Entry<Key,Value> entry : scanner) {
         Key key = entry.getKey();
@@ -101,22 +102,22 @@ private static long scrambleDeleteHalfAndCheck(ClientOnDefaultTable opts, Scanne
     ArrayList<RowColumn> entries = new ArrayList<>(rows);
     java.util.Collections.shuffle(entries);
 
-    AccumuloClient accumuloClient = opts.getClient();
-    BatchWriter mutations = accumuloClient.createBatchWriter(tableName,
-        bwOpts.getBatchWriterConfig());
-
-    for (int i = 0; i < (entries.size() + 1) / 2; i++) {
-      RowColumn rc = entries.get(i);
-      Mutation m = new Mutation(rc.row);
-      m.putDelete(new Text(rc.column.columnFamily), new Text(rc.column.columnQualifier),
-          new ColumnVisibility(rc.column.getColumnVisibility()), rc.timestamp + 1);
-      mutations.addMutation(m);
-      rows.remove(rc);
-      result++;
+    try (AccumuloClient accumuloClient = opts.createClient()) {
+      BatchWriter mutations = accumuloClient.createBatchWriter(tableName,
+          bwOpts.getBatchWriterConfig());
+
+      for (int i = 0; i < (entries.size() + 1) / 2; i++) {
+        RowColumn rc = entries.get(i);
+        Mutation m = new Mutation(rc.row);
+        m.putDelete(new Text(rc.column.columnFamily), new Text(rc.column.columnQualifier),
+            new ColumnVisibility(rc.column.getColumnVisibility()), rc.timestamp + 1);
+        mutations.addMutation(m);
+        rows.remove(rc);
+        result++;
+      }
+      mutations.close();
     }
 
-    mutations.close();
-
     Set<RowColumn> current = scanAll(opts, scanOpts, tableName);
     current.removeAll(rows);
     if (current.size() > 0) {
diff --git a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
index 85d8d1c30e..ff73999641 100644
--- a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java
@@ -71,7 +71,9 @@ public static void main(String[] args) throws Exception {
         Trace.data("cmdLine", Arrays.asList(args).toString());
       }
 
-      verifyIngest(opts.getClient(), opts, scanOpts);
+      try (AccumuloClient client = opts.createClient()) {
+        verifyIngest(client, opts, scanOpts);
+      }
 
     } finally {
       Trace.off();
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
index 5f5e1e8ca9..f52c382ea0 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
@@ -533,7 +533,8 @@ private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) thr
     verifyVolumesUsed(client, tableNames[0], false, v1, v2);
 
     // write to 2nd table, but do not flush data to disk before shutdown
-    writeData(tableNames[1], cluster.getAccumuloClient("root", new PasswordToken(ROOT_PASSWORD)));
+    writeData(tableNames[1],
+        cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD)));
 
     if (cleanShutdown)
       assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
index 2f8b8bba43..1d50404a74 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
@@ -69,6 +69,7 @@
   public static final Logger log = LoggerFactory.getLogger(ConfigurableMacBase.class);
 
   protected MiniAccumuloClusterImpl cluster;
+  protected AccumuloClient client;
 
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {}
 
@@ -182,12 +183,16 @@ private void createMiniAccumulo() throws Exception {
 
   @After
   public void tearDown() throws Exception {
-    if (cluster != null)
+    if (cluster != null) {
       try {
         cluster.stop();
       } catch (Exception e) {
         // ignored
       }
+    }
+    if (client != null) {
+      client.close();
+    }
   }
 
   protected MiniAccumuloClusterImpl getCluster() {
@@ -195,7 +200,10 @@ protected MiniAccumuloClusterImpl getCluster() {
   }
 
   protected AccumuloClient getClient() {
-    return getCluster().getAccumuloClient("root", new PasswordToken(ROOT_PASSWORD));
+    if (client == null) {
+      client = getCluster().createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD));
+    }
+    return client;
   }
 
   protected ClientContext getClientContext() {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 3510da1a05..98f98b8bf5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -164,7 +164,7 @@ public void testAdminUser() throws Exception {
     UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
         rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      final AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+      final AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
           new KerberosToken());
 
       // The "root" user should have all system permissions
@@ -203,7 +203,8 @@ public void testNewUser() throws Exception {
     log.info("Logged in as {}", rootUser.getPrincipal());
 
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
+          new KerberosToken());
       log.info("Created client as {}", rootUser.getPrincipal());
       assertEquals(rootUser.getPrincipal(), client.whoami());
 
@@ -219,7 +220,7 @@ public void testNewUser() throws Exception {
         newUserKeytab.getAbsolutePath());
     log.info("Logged in as {}", newQualifiedUser);
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(newQualifiedUser, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(newQualifiedUser, new KerberosToken());
       log.info("Created client as {}", newQualifiedUser);
       assertEquals(newQualifiedUser, client.whoami());
 
@@ -256,7 +257,7 @@ public void testUserPrivilegesThroughGrant() throws Exception {
     log.info("Logged in as {}", user1);
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
       // Indirectly creates this user when we use it
-      AccumuloClient client = mac.getAccumuloClient(qualifiedUser1, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(qualifiedUser1, new KerberosToken());
       log.info("Created client as {}", qualifiedUser1);
 
       // The new user should have no system permissions
@@ -270,7 +271,8 @@ public void testUserPrivilegesThroughGrant() throws Exception {
     ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(rootUser.getPrincipal(),
         rootUser.getKeytab().getAbsolutePath());
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
+          new KerberosToken());
       client.securityOperations().grantSystemPermission(qualifiedUser1,
           SystemPermission.CREATE_TABLE);
       return null;
@@ -280,7 +282,7 @@ public void testUserPrivilegesThroughGrant() throws Exception {
     ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(user1,
         user1Keytab.getAbsolutePath());
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(qualifiedUser1, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(qualifiedUser1, new KerberosToken());
 
       // Shouldn't throw an exception since we granted the create table permission
       final String table = testName.getMethodName() + "_user_table";
@@ -318,7 +320,7 @@ public void testUserPrivilegesForTable() throws Exception {
     log.info("Logged in as {}", user1);
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
       // Indirectly creates this user when we use it
-      AccumuloClient client = mac.getAccumuloClient(qualifiedUser1, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(qualifiedUser1, new KerberosToken());
       log.info("Created client as {}", qualifiedUser1);
 
       // The new user should have no system permissions
@@ -335,7 +337,8 @@ public void testUserPrivilegesForTable() throws Exception {
         rootUser.getKeytab().getAbsolutePath());
 
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
+          new KerberosToken());
       client.tableOperations().create(table);
       // Give our unprivileged user permission on the table we made for them
       client.securityOperations().grantTablePermission(qualifiedUser1, table, TablePermission.READ);
@@ -353,7 +356,7 @@ public void testUserPrivilegesForTable() throws Exception {
     ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(qualifiedUser1,
         user1Keytab.getAbsolutePath());
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(qualifiedUser1, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(qualifiedUser1, new KerberosToken());
 
       // Make sure we can actually use the table we made
 
@@ -398,7 +401,7 @@ public void testDelegationToken() throws Exception {
     // As the "root" user, open up the connection and get a delegation token
     final AuthenticationToken delegationToken = root
         .doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
-          AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+          AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
               new KerberosToken());
           log.info("Created client as {}", rootUser.getPrincipal());
           assertEquals(rootUser.getPrincipal(), client.whoami());
@@ -423,7 +426,7 @@ public void testDelegationToken() throws Exception {
     UserGroupInformation userWithoutPrivs = UserGroupInformation.createUserForTesting("fake_user",
         new String[0]);
     int recordsSeen = userWithoutPrivs.doAs((PrivilegedExceptionAction<Integer>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), delegationToken);
+      AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(), delegationToken);
 
       try (BatchScanner bs = client.createBatchScanner(tableName, Authorizations.EMPTY, 2)) {
         bs.setRanges(Collections.singleton(new Range()));
@@ -445,7 +448,8 @@ public void testDelegationTokenAsDifferentUser() throws Exception {
     try {
       delegationToken = ugi.doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
         // As the "root" user, open up the connection and get a delegation token
-        AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+        AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
+            new KerberosToken());
         log.info("Created client as {}", rootUser.getPrincipal());
         assertEquals(rootUser.getPrincipal(), client.whoami());
         return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
@@ -460,7 +464,7 @@ public void testDelegationTokenAsDifferentUser() throws Exception {
     try {
       // Use the delegation token to try to log in as a different user
       userWithoutPrivs.doAs((PrivilegedExceptionAction<Void>) () -> {
-        AccumuloClient client = mac.getAccumuloClient("some_other_user", delegationToken);
+        AccumuloClient client = mac.createAccumuloClient("some_other_user", delegationToken);
         client.securityOperations().authenticateUser("some_other_user", delegationToken);
         return null;
       });
@@ -495,7 +499,7 @@ public void testGetDelegationTokenDenied() throws Exception {
     try {
       ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
         // As the "root" user, open up the connection and get a delegation token
-        AccumuloClient client = mac.getAccumuloClient(qualifiedNewUser, new KerberosToken());
+        AccumuloClient client = mac.createAccumuloClient(qualifiedNewUser, new KerberosToken());
         log.info("Created client as {}", qualifiedNewUser);
         assertEquals(qualifiedNewUser, client.whoami());
 
@@ -517,7 +521,7 @@ public void testRestartedMasterReusesSecretKey() throws Exception {
     // As the "root" user, open up the connection and get a delegation token
     final AuthenticationToken delegationToken1 = root
         .doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
-          AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+          AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
               new KerberosToken());
           log.info("Created client as {}", rootUser.getPrincipal());
           assertEquals(rootUser.getPrincipal(), client.whoami());
@@ -526,7 +530,7 @@ public void testRestartedMasterReusesSecretKey() throws Exception {
               .getDelegationToken(new DelegationTokenConfig());
 
           assertTrue("Could not get tables with delegation token",
-              mac.getAccumuloClient(rootUser.getPrincipal(), token).tableOperations().list()
+              mac.createAccumuloClient(rootUser.getPrincipal(), token).tableOperations().list()
                   .size() > 0);
 
           return token;
@@ -540,7 +544,7 @@ public void testRestartedMasterReusesSecretKey() throws Exception {
 
     // Make sure our original token is still good
     root.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), delegationToken1);
+      AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(), delegationToken1);
 
       assertTrue("Could not get tables with delegation token",
           client.tableOperations().list().size() > 0);
@@ -551,7 +555,7 @@ public void testRestartedMasterReusesSecretKey() throws Exception {
     // Get a new token, so we can compare the keyId on the second to the first
     final AuthenticationToken delegationToken2 = root
         .doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
-          AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+          AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
               new KerberosToken());
           log.info("Created client as {}", rootUser.getPrincipal());
           assertEquals(rootUser.getPrincipal(), client.whoami());
@@ -560,7 +564,7 @@ public void testRestartedMasterReusesSecretKey() throws Exception {
               .getDelegationToken(new DelegationTokenConfig());
 
           assertTrue("Could not get tables with delegation token",
-              mac.getAccumuloClient(rootUser.getPrincipal(), token).tableOperations().list()
+              mac.createAccumuloClient(rootUser.getPrincipal(), token).tableOperations().list()
                   .size() > 0);
 
           return token;
@@ -583,7 +587,8 @@ public void testDelegationTokenWithInvalidLifetime() throws Throwable {
     // As the "root" user, open up the connection and get a delegation token
     try {
       root.doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
-        AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+        AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
+            new KerberosToken());
         log.info("Created client as {}", rootUser.getPrincipal());
         assertEquals(rootUser.getPrincipal(), client.whoami());
 
@@ -611,7 +616,7 @@ public void testDelegationTokenWithReducedLifetime() throws Throwable {
     // As the "root" user, open up the connection and get a delegation token
     final AuthenticationToken dt = root
         .doAs((PrivilegedExceptionAction<AuthenticationToken>) () -> {
-          AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+          AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
               new KerberosToken());
           log.info("Created client as {}", rootUser.getPrincipal());
           assertEquals(rootUser.getPrincipal(), client.whoami());
@@ -631,7 +636,7 @@ public void testRootUserHasIrrevocablePermissions() throws Exception {
     UserGroupInformation.loginUserFromKeytab(rootUser.getPrincipal(),
         rootUser.getKeytab().getAbsolutePath());
 
-    final AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(),
+    final AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(),
         new KerberosToken());
 
     // The server-side implementation should prevent the revocation of the 'root' user's systems
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
index 061171498e..7e2c693961 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
@@ -544,7 +544,7 @@ public void proxiedUserAccessWithoutAccumuloProxy() throws Exception {
 
     // Create a table and user, grant permission to our user to read that table.
     rootUgi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(rootUgi.getUserName(), new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(rootUgi.getUserName(), new KerberosToken());
       client.tableOperations().create(tableName);
       client.securityOperations().createLocalUser(userWithoutCredentials1,
           new PasswordToken("ignored"));
@@ -557,7 +557,7 @@ public void proxiedUserAccessWithoutAccumuloProxy() throws Exception {
       return null;
     });
     realUgi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(proxyPrincipal, new KerberosToken());
+      AccumuloClient client = mac.createAccumuloClient(proxyPrincipal, new KerberosToken());
       try (Scanner s = client.createScanner(tableName, Authorizations.EMPTY)) {
         s.iterator().hasNext();
         fail("Expected to see an exception");
@@ -572,7 +572,7 @@ public void proxiedUserAccessWithoutAccumuloProxy() throws Exception {
     });
     // Allowed to be proxied and has read permission
     proxyUser1.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(userWithoutCredentials1,
+      AccumuloClient client = mac.createAccumuloClient(userWithoutCredentials1,
           new KerberosToken(userWithoutCredentials1));
       Scanner s = client.createScanner(tableName, Authorizations.EMPTY);
       assertFalse(s.iterator().hasNext());
@@ -580,7 +580,7 @@ public void proxiedUserAccessWithoutAccumuloProxy() throws Exception {
     });
     // Allowed to be proxied but does not have read permission
     proxyUser2.doAs((PrivilegedExceptionAction<Void>) () -> {
-      AccumuloClient client = mac.getAccumuloClient(userWithoutCredentials2,
+      AccumuloClient client = mac.createAccumuloClient(userWithoutCredentials2,
           new KerberosToken(userWithoutCredentials3));
       try (Scanner s = client.createScanner(tableName, Authorizations.EMPTY)) {
         s.iterator().hasNext();
@@ -598,7 +598,7 @@ public void proxiedUserAccessWithoutAccumuloProxy() throws Exception {
     proxyUser3.doAs((PrivilegedExceptionAction<Void>) () -> {
       try {
         KerberosToken token = new KerberosToken(userWithoutCredentials3);
-        AccumuloClient client = mac.getAccumuloClient(userWithoutCredentials3, token);
+        AccumuloClient client = mac.createAccumuloClient(userWithoutCredentials3, token);
         client.securityOperations().authenticateUser(userWithoutCredentials3, token);
         fail("Should not be able to create a Connector as this user cannot be proxied");
       } catch (org.apache.accumulo.core.client.AccumuloSecurityException e) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
index d4a1b8de54..ad35191bb1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
@@ -155,7 +155,7 @@ public void testReadAndWriteThroughTicketLifetime() throws Exception {
         rootUser.getKeytab().getAbsolutePath());
     log.info("Logged in as {}", rootUser.getPrincipal());
 
-    AccumuloClient client = mac.getAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
+    AccumuloClient client = mac.createAccumuloClient(rootUser.getPrincipal(), new KerberosToken());
     log.info("Created client as {}", rootUser.getPrincipal());
     assertEquals(rootUser.getPrincipal(), client.whoami());
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
index 0fcb446ee8..d24b2f8150 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
@@ -226,7 +226,7 @@ private void runTest(ScannerBase scanner, Authorizations auths, boolean shouldFa
 
   private void runTest(Authorizations auths, boolean shouldFail) throws Exception {
     ClusterUser clusterUser = getUser(0);
-    AccumuloClient userC = getCluster().getAccumuloClient(clusterUser.getPrincipal(),
+    AccumuloClient userC = getCluster().createAccumuloClient(clusterUser.getPrincipal(),
         clusterUser.getToken());
     writeTestMutation(userC);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java b/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java
index a92e77f9fb..d5f22d3f5c 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java
@@ -89,96 +89,97 @@ public static void main(String[] args) throws Exception {
     if (opts.min < 0 || opts.max < 0 || opts.max <= opts.min) {
       throw new IllegalArgumentException("bad min and max");
     }
-    AccumuloClient client = clientOpts.getClient();
+    try (AccumuloClient client = clientOpts.createClient()) {
 
-    if (!client.tableOperations().exists(clientOpts.getTableName())) {
-      throw new TableNotFoundException(null, clientOpts.getTableName(),
-          "Consult the README and create the table before starting ingest.");
-    }
+      if (!client.tableOperations().exists(clientOpts.getTableName())) {
+        throw new TableNotFoundException(null, clientOpts.getTableName(),
+            "Consult the README and create the table before starting ingest.");
+      }
+
+      BatchWriter bw = client.createBatchWriter(clientOpts.getTableName(),
+          bwOpts.getBatchWriterConfig());
+      bw = Trace.wrapAll(bw, TraceSamplers.countSampler(1024));
 
-    BatchWriter bw = client.createBatchWriter(clientOpts.getTableName(),
-        bwOpts.getBatchWriterConfig());
-    bw = Trace.wrapAll(bw, TraceSamplers.countSampler(1024));
+      Random r = new SecureRandom();
 
-    Random r = new SecureRandom();
+      byte[] ingestInstanceId = UUID.randomUUID().toString().getBytes(UTF_8);
 
-    byte[] ingestInstanceId = UUID.randomUUID().toString().getBytes(UTF_8);
+      System.out.printf("UUID %d %s%n", System.currentTimeMillis(),
+          new String(ingestInstanceId, UTF_8));
 
-    System.out.printf("UUID %d %s%n", System.currentTimeMillis(),
-        new String(ingestInstanceId, UTF_8));
+      long count = 0;
+      final int flushInterval = 1000000;
+      final int maxDepth = 25;
 
-    long count = 0;
-    final int flushInterval = 1000000;
-    final int maxDepth = 25;
+      // always want to point back to flushed data. This way the previous item should
+      // always exist in accumulo when verifying data. To do this make insert N point
+      // back to the row from insert (N - flushInterval). The array below is used to keep
+      // track of this.
+      long prevRows[] = new long[flushInterval];
+      long firstRows[] = new long[flushInterval];
+      int firstColFams[] = new int[flushInterval];
+      int firstColQuals[] = new int[flushInterval];
 
-    // always want to point back to flushed data. This way the previous item should
-    // always exist in accumulo when verifying data. To do this make insert N point
-    // back to the row from insert (N - flushInterval). The array below is used to keep
-    // track of this.
-    long prevRows[] = new long[flushInterval];
-    long firstRows[] = new long[flushInterval];
-    int firstColFams[] = new int[flushInterval];
-    int firstColQuals[] = new int[flushInterval];
+      long lastFlushTime = System.currentTimeMillis();
 
-    long lastFlushTime = System.currentTimeMillis();
+      out: while (true) {
+        // generate first set of nodes
+        ColumnVisibility cv = getVisibility(r);
 
-    out: while (true) {
-      // generate first set of nodes
-      ColumnVisibility cv = getVisibility(r);
+        for (int index = 0; index < flushInterval; index++) {
+          long rowLong = genLong(opts.min, opts.max, r);
+          prevRows[index] = rowLong;
+          firstRows[index] = rowLong;
 
-      for (int index = 0; index < flushInterval; index++) {
-        long rowLong = genLong(opts.min, opts.max, r);
-        prevRows[index] = rowLong;
-        firstRows[index] = rowLong;
+          int cf = r.nextInt(opts.maxColF);
+          int cq = r.nextInt(opts.maxColQ);
 
-        int cf = r.nextInt(opts.maxColF);
-        int cq = r.nextInt(opts.maxColQ);
+          firstColFams[index] = cf;
+          firstColQuals[index] = cq;
 
-        firstColFams[index] = cf;
-        firstColQuals[index] = cq;
+          Mutation m = genMutation(rowLong, cf, cq, cv, ingestInstanceId, count, null, r,
+              opts.checksum);
+          count++;
+          bw.addMutation(m);
+        }
 
-        Mutation m = genMutation(rowLong, cf, cq, cv, ingestInstanceId, count, null, r,
-            opts.checksum);
-        count++;
-        bw.addMutation(m);
-      }
+        lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
+        if (count >= opts.num)
+          break out;
 
-      lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
-      if (count >= opts.num)
-        break out;
+        // generate subsequent sets of nodes that link to previous set of nodes
+        for (int depth = 1; depth < maxDepth; depth++) {
+          for (int index = 0; index < flushInterval; index++) {
+            long rowLong = genLong(opts.min, opts.max, r);
+            byte[] prevRow = genRow(prevRows[index]);
+            prevRows[index] = rowLong;
+            Mutation m = genMutation(rowLong, r.nextInt(opts.maxColF), r.nextInt(opts.maxColQ), cv,
+                ingestInstanceId, count, prevRow, r, opts.checksum);
+            count++;
+            bw.addMutation(m);
+          }
+
+          lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
+          if (count >= opts.num)
+            break out;
+        }
 
-      // generate subsequent sets of nodes that link to previous set of nodes
-      for (int depth = 1; depth < maxDepth; depth++) {
-        for (int index = 0; index < flushInterval; index++) {
-          long rowLong = genLong(opts.min, opts.max, r);
-          byte[] prevRow = genRow(prevRows[index]);
-          prevRows[index] = rowLong;
-          Mutation m = genMutation(rowLong, r.nextInt(opts.maxColF), r.nextInt(opts.maxColQ), cv,
-              ingestInstanceId, count, prevRow, r, opts.checksum);
+        // create one big linked list, this makes all of the first inserts
+        // point to something
+        for (int index = 0; index < flushInterval - 1; index++) {
+          Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], cv,
+              ingestInstanceId, count, genRow(prevRows[index + 1]), r, opts.checksum);
           count++;
           bw.addMutation(m);
         }
-
         lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
         if (count >= opts.num)
           break out;
       }
 
-      // create one big linked list, this makes all of the first inserts
-      // point to something
-      for (int index = 0; index < flushInterval - 1; index++) {
-        Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], cv,
-            ingestInstanceId, count, genRow(prevRows[index + 1]), r, opts.checksum);
-        count++;
-        bw.addMutation(m);
-      }
-      lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
-      if (count >= opts.num)
-        break out;
+      bw.close();
+      clientOpts.stopTracing();
     }
-
-    bw.close();
-    clientOpts.stopTracing();
   }
 
   private static long flush(BatchWriter bw, long count, final int flushInterval, long lastFlushTime)
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index cb3906dc2b..5dfc09860d 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -214,41 +214,32 @@ public int runTest() throws Exception {
       runTest("read tablet files w/ table iter stack", tests, opts.numThreads, threadPool);
     }
 
-    for (int i = 0; i < opts.iterations; i++) {
-
-      ArrayList<Test> tests = new ArrayList<>();
-
-      final AccumuloClient client = opts.getClient();
-
-      for (final KeyExtent ke : tabletsToTest) {
-        Test test = new Test(ke) {
-          @Override
-          public int runTest() throws Exception {
-            return scanTablet(client, opts.getTableName(), opts.auths, scanOpts.scanBatchSize,
-                ke.getPrevEndRow(), ke.getEndRow(), columns);
-          }
-        };
-
-        tests.add(test);
+    try (AccumuloClient client = opts.createClient()) {
+      for (int i = 0; i < opts.iterations; i++) {
+        ArrayList<Test> tests = new ArrayList<>();
+        for (final KeyExtent ke : tabletsToTest) {
+          Test test = new Test(ke) {
+            @Override
+            public int runTest() throws Exception {
+              return scanTablet(client, opts.getTableName(), opts.auths, scanOpts.scanBatchSize,
+                  ke.getPrevEndRow(), ke.getEndRow(), columns);
+            }
+          };
+          tests.add(test);
+        }
+        runTest("read tablet data through accumulo", tests, opts.numThreads, threadPool);
       }
 
-      runTest("read tablet data through accumulo", tests, opts.numThreads, threadPool);
-    }
-
-    for (final KeyExtent ke : tabletsToTest) {
-      final AccumuloClient client = opts.getClient();
-
-      threadPool.submit(new Runnable() {
-        @Override
-        public void run() {
+      for (final KeyExtent ke : tabletsToTest) {
+        threadPool.submit(() -> {
           try {
             calcTabletStats(client, opts.getTableName(), opts.auths, scanOpts.scanBatchSize, ke,
                 columns);
           } catch (Exception e) {
             log.error("Failed to calculate tablet stats.", e);
           }
-        }
-      });
+        });
+      }
     }
 
     threadPool.shutdown();
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index 2aaeab10dd..3df1ee8d14 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -216,9 +216,9 @@ public void dataIsNotOverReplicated() throws Exception {
     }
 
     try {
-      AccumuloClient clientMaster1 = master1Cluster.getAccumuloClient("root",
+      AccumuloClient clientMaster1 = master1Cluster.createAccumuloClient("root",
           new PasswordToken(password)),
-          clientMaster2 = master2Cluster.getAccumuloClient("root", new PasswordToken(password));
+          clientMaster2 = master2Cluster.createAccumuloClient("root", new PasswordToken(password));
 
       String master1UserName = "master1", master1Password = "foo";
       String master2UserName = "master2", master2Password = "bar";
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
index 6b953fee77..a7bed3a62f 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
@@ -167,8 +167,10 @@ public void dataReplicatedToCorrectTable() throws Exception {
     ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
       log.info("testing {}", ugi);
       final KerberosToken token = new KerberosToken();
-      try (AccumuloClient primaryclient = primary.getAccumuloClient(rootUser.getPrincipal(), token);
-          AccumuloClient peerclient = peer.getAccumuloClient(rootUser.getPrincipal(), token)) {
+      try (
+          AccumuloClient primaryclient = primary.createAccumuloClient(rootUser.getPrincipal(),
+              token);
+          AccumuloClient peerclient = peer.createAccumuloClient(rootUser.getPrincipal(), token)) {
 
         ClusterUser replicationUser = kdc.getClientPrincipal(0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 294d903bd0..66af2d33dc 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -176,7 +176,7 @@ public void dataWasReplicatedToThePeer() throws Exception {
     peerCluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peerCluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peerCluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       ReplicationTable.setOnline(clientMaster);
@@ -350,7 +350,7 @@ public void dataReplicatedToCorrectTable() throws Exception {
     peer1Cluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peer1Cluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peer1Cluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
       String peerClusterName = "peer";
       String peerUserName = "peer", peerPassword = "foo";
@@ -510,7 +510,7 @@ public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
     peerCluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peerCluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peerCluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       String peerUserName = "repl";
@@ -630,7 +630,7 @@ public void dataReplicatedToCorrectTableWithoutDrain() throws Exception {
     peer1Cluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peer1Cluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peer1Cluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       String peerClusterName = "peer";
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
index 019b1e8535..15244aa7f5 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
@@ -181,7 +181,7 @@ public void dataWasReplicatedToThePeer() throws Exception {
     peerCluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peerCluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peerCluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       ReplicationTable.setOnline(clientMaster);
@@ -346,7 +346,7 @@ public void dataReplicatedToCorrectTable() throws Exception {
     peer1Cluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peer1Cluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peer1Cluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       String peerClusterName = "peer";
@@ -521,7 +521,7 @@ public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
     peerCluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peerCluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peerCluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       String peerUserName = "repl";
@@ -637,7 +637,7 @@ public void dataReplicatedToCorrectTableWithoutDrain() throws Exception {
     peer1Cluster.start();
 
     try (AccumuloClient clientMaster = getClient();
-        AccumuloClient clientPeer = peer1Cluster.getAccumuloClient("root",
+        AccumuloClient clientPeer = peer1Cluster.createAccumuloClient("root",
             new PasswordToken(ROOT_PASSWORD))) {
 
       String peerClusterName = "peer";
diff --git a/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java b/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java
index 49524d80c4..d7fc1a95c1 100644
--- a/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java
+++ b/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java
@@ -63,7 +63,6 @@ public static void main(String[] args) throws Exception {
 
     ClientOpts opts = new ClientOpts();
     opts.parseArgs(MetadataBatchScan.class.getName(), args);
-    final AccumuloClient accumuloClient = opts.getClient();
 
     TreeSet<Long> splits = new TreeSet<>();
     Random r = new SecureRandom();
@@ -87,88 +86,87 @@ public static void main(String[] args) throws Exception {
 
     extents.add(new KeyExtent(tid, null, per));
 
-    if (args[0].equals("write")) {
+    try (AccumuloClient accumuloClient = opts.createClient()) {
+      if (args[0].equals("write")) {
 
-      BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME,
-          new BatchWriterConfig());
+        BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME,
+            new BatchWriterConfig());
 
-      for (KeyExtent extent : extents) {
-        Mutation mut = extent.getPrevRowUpdateMutation();
-        new TServerInstance(HostAndPort.fromParts("192.168.1.100", 4567), "DEADBEEF")
-            .putLocation(mut);
-        bw.addMutation(mut);
-      }
+        for (KeyExtent extent : extents) {
+          Mutation mut = extent.getPrevRowUpdateMutation();
+          new TServerInstance(HostAndPort.fromParts("192.168.1.100", 4567), "DEADBEEF")
+              .putLocation(mut);
+          bw.addMutation(mut);
+        }
 
-      bw.close();
-    } else if (args[0].equals("writeFiles")) {
-      BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME,
-          new BatchWriterConfig());
+        bw.close();
+      } else if (args[0].equals("writeFiles")) {
+        BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME,
+            new BatchWriterConfig());
 
-      for (KeyExtent extent : extents) {
+        for (KeyExtent extent : extents) {
 
-        Mutation mut = new Mutation(extent.getMetadataEntry());
+          Mutation mut = new Mutation(extent.getMetadataEntry());
 
-        String dir = "/t-" + UUID.randomUUID();
+          String dir = "/t-" + UUID.randomUUID();
 
-        TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value(dir.getBytes(UTF_8)));
+          TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut,
+              new Value(dir.getBytes(UTF_8)));
 
-        for (int i = 0; i < 5; i++) {
-          mut.put(DataFileColumnFamily.NAME, new Text(dir + "/00000_0000" + i + ".map"),
-              new DataFileValue(10000, 1000000).encodeAsValue());
-        }
-
-        bw.addMutation(mut);
-      }
+          for (int i = 0; i < 5; i++) {
+            mut.put(DataFileColumnFamily.NAME, new Text(dir + "/00000_0000" + i + ".map"),
+                new DataFileValue(10000, 1000000).encodeAsValue());
+          }
 
-      bw.close();
-    } else if (args[0].equals("scan")) {
+          bw.addMutation(mut);
+        }
 
-      int numThreads = Integer.parseInt(args[1]);
-      final int numLoop = Integer.parseInt(args[2]);
-      int numLookups = Integer.parseInt(args[3]);
+        bw.close();
+      } else if (args[0].equals("scan")) {
 
-      HashSet<Integer> indexes = new HashSet<>();
-      while (indexes.size() < numLookups) {
-        indexes.add(r.nextInt(extents.size()));
-      }
+        int numThreads = Integer.parseInt(args[1]);
+        final int numLoop = Integer.parseInt(args[2]);
+        int numLookups = Integer.parseInt(args[3]);
 
-      final List<Range> ranges = new ArrayList<>();
-      for (Integer i : indexes) {
-        ranges.add(extents.get(i).toMetadataRange());
-      }
+        HashSet<Integer> indexes = new HashSet<>();
+        while (indexes.size() < numLookups) {
+          indexes.add(r.nextInt(extents.size()));
+        }
 
-      Thread threads[] = new Thread[numThreads];
+        final List<Range> ranges = new ArrayList<>();
+        for (Integer i : indexes) {
+          ranges.add(extents.get(i).toMetadataRange());
+        }
 
-      for (int i = 0; i < threads.length; i++) {
-        threads[i] = new Thread(new Runnable() {
+        Thread threads[] = new Thread[numThreads];
 
-          @Override
-          public void run() {
+        for (int i = 0; i < threads.length; i++) {
+          threads[i] = new Thread(() -> {
             try {
               System.out.println(runScanTest(accumuloClient, numLoop, ranges));
             } catch (Exception e) {
               log.error("Exception while running scan test.", e);
             }
-          }
-        });
-      }
+          });
+        }
 
-      long t1 = System.currentTimeMillis();
+        long t1 = System.currentTimeMillis();
 
-      for (Thread thread : threads) {
-        thread.start();
-      }
+        for (Thread thread : threads) {
+          thread.start();
+        }
 
-      for (Thread thread : threads) {
-        thread.join();
-      }
+        for (Thread thread : threads) {
+          thread.join();
+        }
 
-      long t2 = System.currentTimeMillis();
+        long t2 = System.currentTimeMillis();
 
-      System.out.printf("tt : %6.2f%n", (t2 - t1) / 1000.0);
+        System.out.printf("tt : %6.2f%n", (t2 - t1) / 1000.0);
 
-    } else {
-      throw new IllegalArgumentException();
+      } else {
+        throw new IllegalArgumentException();
+      }
     }
 
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services