You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2019/03/04 15:50:08 UTC

[accumulo] branch master updated: Simplified options in ClientOpts (#1000)

This is an automated email from the ASF dual-hosted git repository.

mwalch pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c0e3dc  Simplified options in ClientOpts (#1000)
8c0e3dc is described below

commit 8c0e3dccd18e8514aa20a981b4e4b056132049c7
Author: Mike Walch <mw...@apache.org>
AuthorDate: Mon Mar 4 10:49:58 2019 -0500

    Simplified options in ClientOpts (#1000)
    
    * Updated ITs given changes
    * Added -o option to allow properties to be overriden
    * Remove ContextOpts
---
 .../org/apache/accumulo/core/cli/ClientOpts.java   | 65 ++++++++--------------
 .../org/apache/accumulo/core/cli/ConfigOpts.java   |  6 +-
 .../apache/accumulo/core/cli/TestClientOpts.java   | 46 +++++++++++++++
 .../accumulo/hadoop/its/mapreduce/RowHashIT.java   |  7 +--
 .../apache/accumulo/cluster/AccumuloCluster.java   |  9 ++-
 .../standalone/StandaloneAccumuloCluster.java      |  7 +++
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |  5 ++
 .../apache/accumulo/server/cli/ContextOpts.java    | 40 -------------
 .../apache/accumulo/server/cli/ServerUtilOpts.java |  2 +-
 .../accumulo/server/util/RandomizeVolumes.java     |  4 +-
 .../accumulo/test/RewriteTabletDirectoriesIT.java  |  6 +-
 .../org/apache/accumulo/test/TestBinaryRows.java   |  4 +-
 .../java/org/apache/accumulo/test/TestIngest.java  | 23 ++------
 .../apache/accumulo/test/functional/DeleteIT.java  | 38 ++-----------
 .../test/functional/HalfDeadTServerIT.java         |  5 +-
 .../accumulo/test/functional/ReadWriteIT.java      | 52 ++---------------
 .../apache/accumulo/test/functional/RestartIT.java | 50 +++--------------
 .../accumulo/test/functional/RestartStressIT.java  | 24 +-------
 .../accumulo/test/functional/ShutdownIT.java       | 24 ++++----
 .../apache/accumulo/test/functional/SplitIT.java   | 25 +--------
 .../accumulo/test/mapreduce/MapReduceIT.java       | 18 +-----
 .../test/performance/RollWALPerformanceIT.java     |  7 +--
 .../test/performance/scan/CollectTabletStats.java  |  6 +-
 .../performance/scan/CollectTabletStatsTest.java   | 10 ----
 24 files changed, 152 insertions(+), 331 deletions(-)

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 bc39e00..1e66d90 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
@@ -19,6 +19,10 @@ package org.apache.accumulo.core.cli;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
@@ -39,16 +43,10 @@ import org.apache.log4j.Logger;
 
 import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
+import com.beust.jcommander.converters.IParameterSplitter;
 
 public class ClientOpts extends Help {
 
-  public static class TimeConverter implements IStringConverter<Long> {
-    @Override
-    public Long convert(String value) {
-      return ConfigurationTypeHelper.getTimeInMillis(value);
-    }
-  }
-
   public static class MemoryConverter implements IStringConverter<Long> {
     @Override
     public Long convert(String value) {
@@ -90,12 +88,16 @@ public class ClientOpts extends Help {
     }
   }
 
+  public static class NullSplitter implements IParameterSplitter {
+    @Override
+    public List<String> split(String value) {
+      return Collections.singletonList(value);
+    }
+  }
+
   @Parameter(names = {"-u", "--user"}, description = "Connection user")
   private String principal = null;
 
-  @Parameter(names = "-p", converter = PasswordConverter.class, description = "Connection password")
-  private Password password = null;
-
   @Parameter(names = "--password", converter = PasswordConverter.class,
       description = "Enter the connection password", password = true)
   private Password securePassword = null;
@@ -104,13 +106,6 @@ public class ClientOpts extends Help {
     return ClientProperty.getAuthenticationToken(getClientProperties());
   }
 
-  @Parameter(names = {"-z", "--keepers"},
-      description = "Comma separated list of zookeeper hosts (host:port,host:port)")
-  protected String zookeepers = null;
-
-  @Parameter(names = {"-i", "--instance"}, description = "The name of the accumulo instance")
-  protected String instance = null;
-
   @Parameter(names = {"-auths", "--auths"}, converter = AuthConverter.class,
       description = "the authorizations to use when reading or writing")
   public Authorizations auths = Authorizations.EMPTY;
@@ -118,16 +113,14 @@ public class ClientOpts extends Help {
   @Parameter(names = "--debug", description = "turn on TRACE-level log messages")
   public boolean debug = false;
 
-  @Parameter(names = "--ssl", description = "Connect to accumulo over SSL")
-  private boolean sslEnabled = false;
-
-  @Parameter(names = "--sasl", description = "Connecto to Accumulo using SASL (supports Kerberos)")
-  private boolean saslEnabled = false;
-
-  @Parameter(names = "--config-file", description = "Read the given client config file. "
+  @Parameter(names = {"-c", "--config-file"}, description = "Read the given client config file. "
       + "If omitted, the classpath will be searched for file named accumulo-client.properties")
   private String clientConfigFile = null;
 
+  @Parameter(names = "-o", splitter = NullSplitter.class, description = "Overrides property in "
+      + "accumulo-client.properties. Expected format: -o <key>=<value>")
+  private List<String> overrides = new ArrayList<>();
+
   public void startDebugLogging() {
     if (debug)
       Logger.getLogger(Constants.CORE_PACKAGE_NAME).setLevel(Level.TRACE);
@@ -136,8 +129,9 @@ public class ClientOpts extends Help {
   @Parameter(names = "--trace", description = "turn on distributed tracing")
   public boolean trace = false;
 
-  @Parameter(names = "--keytab", description = "Kerberos keytab on the local filesystem")
-  private String keytabPath = null;
+  public Map<String,String> getOverrides() {
+    return ConfigOpts.getOverrides(overrides);
+  }
 
   public TraceScope parseArgsAndTrace(String programName, String[] args, Object... others) {
     parseArgs(programName, args, others);
@@ -161,6 +155,7 @@ public class ClientOpts extends Help {
   }
 
   public void setClientProperties(Properties clientProps) {
+    ClientProperty.validate(clientProps);
     this.cachedProps = clientProps;
   }
 
@@ -188,28 +183,14 @@ public class ClientOpts extends Help {
       if (getClientConfigFile() != null) {
         cachedProps = ClientInfoImpl.toProperties(getClientConfigFile());
       }
-      if (saslEnabled) {
-        cachedProps.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
-      }
-      if (sslEnabled) {
-        cachedProps.setProperty(ClientProperty.SSL_ENABLED.getKey(), "true");
-      }
       if (principal != null) {
         cachedProps.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
       }
-      if (zookeepers != null) {
-        cachedProps.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), zookeepers);
-      }
-      if (instance != null) {
-        cachedProps.setProperty(ClientProperty.INSTANCE_NAME.getKey(), instance);
-      }
       if (securePassword != null) {
         ClientProperty.setPassword(cachedProps, securePassword.toString());
-      } else if (password != null) {
-        ClientProperty.setPassword(cachedProps, password.toString());
-      } else if (keytabPath != null) {
-        ClientProperty.setKerberosKeytab(cachedProps, keytabPath);
       }
+      getOverrides().forEach((k, v) -> cachedProps.put(k, v));
+      ClientProperty.validate(cachedProps);
     }
     return cachedProps;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
index 3ae2fce..b0d627e 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
@@ -72,8 +72,12 @@ public class ConfigOpts extends Help {
   }
 
   public Map<String,String> getOverrides() {
+    return getOverrides(overrides);
+  }
+
+  public static Map<String,String> getOverrides(List<String> args) {
     Map<String,String> config = new HashMap<>();
-    for (String prop : overrides) {
+    for (String prop : args) {
       String[] propArgs = prop.split("=", 2);
       if (propArgs.length == 2) {
         String key = propArgs[0].trim();
diff --git a/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java b/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java
new file mode 100644
index 0000000..f1e55ba
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.cli;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.junit.Test;
+
+public class TestClientOpts {
+
+  @Test
+  public void testBasic() {
+    ClientOpts opts = new ClientOpts();
+    String[] args = new String[] {"-u", "userabc", "-o", "instance.name=myinst", "-o",
+        "instance.zookeepers=zoo1,zoo2", "-o", "auth.type=password", "-o", "auth.principal=user123",
+        "-o", "auth.token=mypass"};
+    opts.parseArgs("test", args);
+    assertEquals("user123", opts.getPrincipal());
+    assertTrue(opts.getToken() instanceof PasswordToken);
+    assertEquals("myinst", opts.getClientProperties().getProperty("instance.name"));
+
+    opts = new ClientOpts();
+    args = new String[] {"-o", "instance.name=myinst", "-o", "instance.zookeepers=zoo1,zoo2", "-o",
+        "auth.type=password", "-o", "auth.token=mypass", "-u", "userabc"};
+    opts.parseArgs("test", args);
+    assertEquals("userabc", opts.getPrincipal());
+    assertTrue(opts.getToken() instanceof PasswordToken);
+    assertEquals("myinst", opts.getClientProperties().getProperty("instance.name"));
+  }
+}
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
index 414a4c0..a8823ba 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
@@ -35,7 +35,6 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -94,10 +93,8 @@ public class RowHashIT extends ConfigurableMacBase {
       bw.addMutation(m);
     }
     bw.close();
-    ClientInfo info = ClientInfo.from(c.properties());
-    Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-i",
-        info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "-t",
-        tablename, "--column", input_cfcq).getProcess();
+    Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-c",
+        cluster.getClientPropsPath(), "-t", tablename, "--column", input_cfcq).getProcess();
     assertEquals(0, hash.waitFor());
 
     try (Scanner s = c.createScanner(tablename, Authorizations.EMPTY)) {
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 cb32ca5..cc8c893 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -75,13 +75,18 @@ public interface AccumuloCluster {
   AccumuloConfiguration getSiteConfiguration();
 
   /**
-   * Get path to accumulo.properties
-   *
+   * @return Path to accumulo.properties
    * @since 2.0.0
    */
   String getAccumuloPropertiesPath();
 
   /**
+   * @return Path to accumulo-client.properties
+   * @since 2.0.0
+   */
+  String getClientPropsPath();
+
+  /**
    * Get an object that can manage a cluster
    *
    * @return Manage the state of 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 b93a716..f99f499 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
@@ -223,4 +223,11 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
   public String getAccumuloPropertiesPath() {
     return new File(serverAccumuloConfDir, "accumulo.properties").toString();
   }
+
+  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
+      justification = "code runs in same security context as user who provided input file name")
+  @Override
+  public String getClientPropsPath() {
+    return new File(clientAccumuloConfDir, "accumulo-client.properties").toString();
+  }
 }
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 e53e487..c9226c6 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -798,4 +798,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   public String getAccumuloPropertiesPath() {
     return new File(config.getConfDir(), "accumulo.properties").toString();
   }
+
+  @Override
+  public String getClientPropsPath() {
+    return config.getClientPropsFile().getAbsolutePath();
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ContextOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ContextOpts.java
deleted file mode 100644
index c596d23..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ContextOpts.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.cli;
-
-import org.apache.accumulo.core.cli.ClientOpts;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.server.ServerContext;
-
-public class ContextOpts extends ClientOpts {
-  {
-    setPrincipal("root");
-  }
-
-  private ServerContext context;
-
-  public synchronized ServerContext getServerContext() {
-    if (context == null) {
-      if (instance == null) {
-        context = new ServerContext(new SiteConfiguration());
-      } else {
-        context = new ServerContext(new SiteConfiguration(), getClientProperties());
-      }
-    }
-    return context;
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
index 1113505..0085093 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
@@ -29,7 +29,7 @@ public class ServerUtilOpts extends ClientOpts {
 
   public synchronized ServerContext getServerContext() {
     if (context == null) {
-      if (instance == null) {
+      if (getClientConfigFile() == null) {
         context = new ServerContext(new SiteConfiguration());
       } else {
         context = new ServerContext(new SiteConfiguration(), getClientProperties());
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 242687f..1387f85 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
@@ -40,7 +40,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.cli.ContextOpts;
+import org.apache.accumulo.server.cli.ServerUtilOpts;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -55,7 +55,7 @@ import com.beust.jcommander.Parameter;
 public class RandomizeVolumes {
   private static final Logger log = LoggerFactory.getLogger(RandomizeVolumes.class);
 
-  static class RandomizeOpts extends ContextOpts {
+  static class RandomizeOpts extends ServerUtilOpts {
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     String tableName;
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java b/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java
index 5068e0e..be5c404 100644
--- a/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java
@@ -124,8 +124,10 @@ public class RewriteTabletDirectoriesIT extends ConfigurableMacBase {
         assertEquals(splits.size() + 1, count);
 
         // This should fail: only one volume
-        assertEquals(1, cluster.exec(RandomizeVolumes.class, "-z", cluster.getZooKeepers(), "-i",
-            getClientInfo().getInstanceName(), "-t", tableName).getProcess().waitFor());
+        assertEquals(1,
+            cluster
+                .exec(RandomizeVolumes.class, "-c", cluster.getClientPropsPath(), "-t", tableName)
+                .getProcess().waitFor());
 
         cluster.stop();
 
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 9009fa5..fe214b1 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.server.cli.ContextOpts;
+import org.apache.accumulo.server.cli.ServerUtilOpts;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -73,7 +73,7 @@ public class TestBinaryRows {
     return l;
   }
 
-  public static class Opts extends ContextOpts {
+  public static class Opts extends ServerUtilOpts {
     @Parameter(names = "--mode",
         description = "either 'ingest', 'delete', 'randomLookups', 'split',"
             + " 'verify', 'verifyDeleted'",
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 79025fb..b75889a 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.IOException;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
@@ -48,12 +47,10 @@ import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
-import org.apache.htrace.TraceScope;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
@@ -175,7 +172,6 @@ public class TestIngest {
     random.setSeed((row ^ seed) ^ col);
     random.nextBytes(dest);
     toPrintableChars(dest);
-
     return dest;
   }
 
@@ -186,23 +182,16 @@ public class TestIngest {
     }
   }
 
-  public static void main(String[] args) {
+  public static void main(String[] args) throws Exception {
 
     Opts opts = new Opts();
+    opts.parseArgs(TestIngest.class.getSimpleName(), args);
 
-    String name = TestIngest.class.getSimpleName();
-    TraceUtil.enableClientTraces(null, name, new Properties());
-    try (TraceScope clientSpan = opts.parseArgsAndTrace(name, args)) {
-      if (opts.debug)
-        Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
+    if (opts.debug)
+      Logger.getLogger(TabletServerBatchWriter.class.getName()).setLevel(Level.TRACE);
 
-      try (AccumuloClient client = opts.createClient()) {
-        ingest(client, opts);
-      }
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    } finally {
-      TraceUtil.disable();
+    try (AccumuloClient client = opts.createClient()) {
+      ingest(client, opts);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
index 0d30547..483b0de 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
@@ -16,16 +16,10 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestRandomDeletes;
@@ -44,20 +38,12 @@ public class DeleteIT extends AccumuloClusterHarness {
     try (AccumuloClient c = createAccumuloClient()) {
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
-      AuthenticationToken token = getAdminToken();
-      if (token instanceof KerberosToken) {
-        deleteTest(c, getCluster(), getAdminPrincipal(), null, tableName,
-            getAdminUser().getKeytab().getAbsolutePath());
-      } else if (token instanceof PasswordToken) {
-        PasswordToken passwdToken = (PasswordToken) token;
-        deleteTest(c, getCluster(), getAdminPrincipal(),
-            new String(passwdToken.getPassword(), UTF_8), tableName, null);
-      }
+      deleteTest(c, getCluster(), tableName);
     }
   }
 
-  public static void deleteTest(AccumuloClient c, AccumuloCluster cluster, String user,
-      String password, String tableName, String keytab) throws Exception {
+  public static void deleteTest(AccumuloClient c, AccumuloCluster cluster, String tableName)
+      throws Exception {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     TestIngest.Opts opts = new TestIngest.Opts();
     vopts.setTableName(tableName);
@@ -66,27 +52,13 @@ public class DeleteIT extends AccumuloClusterHarness {
     vopts.cols = opts.cols = 1;
     vopts.random = opts.random = 56;
 
-    assertTrue("Expected one of password or keytab", password != null || keytab != null);
     opts.setClientProperties(getClientProperties());
     vopts.setClientProperties(getClientProperties());
 
     TestIngest.ingest(c, opts);
 
-    String[] args = null;
-
-    assertTrue("Expected one of password or keytab", password != null || keytab != null);
-    if (password != null) {
-      assertNull("Given password, expected null keytab", keytab);
-      args = new String[] {"-u", user, "-p", password, "-i", cluster.getInstanceName(), "-z",
-          cluster.getZooKeepers(), "--table", tableName};
-    }
-    if (keytab != null) {
-      assertNull("Given keytab, expect null password", password);
-      args = new String[] {"-u", user, "-i", cluster.getInstanceName(), "-z",
-          cluster.getZooKeepers(), "--table", tableName, "--keytab", keytab};
-    }
-
-    assertEquals(0, cluster.getClusterControl().exec(TestRandomDeletes.class, args));
+    assertEquals(0, cluster.getClusterControl().exec(TestRandomDeletes.class,
+        new String[] {"-c", cluster.getClientPropsPath(), "--table", tableName}));
     TestIngest.ingest(c, opts);
     VerifyIngest.verifyIngest(c, vopts);
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index 17bdb2e..27be097 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -156,8 +156,9 @@ public class HalfDeadTServerIT extends ConfigurableMacBase {
         c.tableOperations().create("test_ingest");
         assertEquals(1, c.instanceOperations().getTabletServers().size());
         int rows = 100 * 1000;
-        ingest = cluster.exec(TestIngest.class, "-u", "root", "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "-p", ROOT_PASSWORD, "--rows", rows + "").getProcess();
+        ingest = cluster
+            .exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--rows", rows + "")
+            .getProcess();
         sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
 
         // block I/O with some side-channel trickiness
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 85f5dd0..dc90fc2 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -16,10 +16,8 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
@@ -62,9 +60,6 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.Property;
@@ -250,33 +245,13 @@ public class ReadWriteIT extends AccumuloClusterHarness {
   @Test
   public void multiTableTest() throws Exception {
     // Write to multiple tables
-    final String instance = cluster.getInstanceName();
-    final String keepers = cluster.getZooKeepers();
     final ClusterControl control = cluster.getClusterControl();
     final String prefix = getClass().getSimpleName() + "_" + testName.getMethodName();
     ExecutorService svc = Executors.newFixedThreadPool(2);
     Future<Integer> p1 = svc.submit(() -> {
       try {
-        // Invocation is different for SASL. We're only logged in via this processes memory (not
-        // via some credentials cache on disk)
-        // Need to pass along the keytab because of that.
-        if (saslEnabled()) {
-          String principal = getAdminPrincipal();
-          AuthenticationToken token = getAdminToken();
-          assertTrue("Expected KerberosToken, but was " + token.getClass(),
-              token instanceof KerberosToken);
-          KerberosToken kt = (KerberosToken) token;
-          assertNotNull("Expected keytab in token", kt.getKeytab());
-          return control.exec(TestMultiTableIngest.class,
-              args("--count", Integer.toString(ROWS), "-i", instance, "-z", keepers,
-                  "--tablePrefix", prefix, "--keytab", kt.getKeytab().getAbsolutePath(), "-u",
-                  principal));
-        }
-
-        return control.exec(TestMultiTableIngest.class,
-            args("--count", Integer.toString(ROWS), "-u", getAdminPrincipal(), "-i", instance, "-z",
-                keepers, "-p", new String(((PasswordToken) getAdminToken()).getPassword(), UTF_8),
-                "--tablePrefix", prefix));
+        return control.exec(TestMultiTableIngest.class, args("--count", Integer.toString(ROWS),
+            "-c", cluster.getClientPropsPath(), "--tablePrefix", prefix));
       } catch (IOException e) {
         log.error("Error running MultiTableIngest", e);
         return -1;
@@ -284,27 +259,8 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     });
     Future<Integer> p2 = svc.submit(() -> {
       try {
-        // Invocation is different for SASL. We're only logged in via this processes memory (not
-        // via some credentials cache on disk)
-        // Need to pass along the keytab because of that.
-        if (saslEnabled()) {
-          String principal = getAdminPrincipal();
-          AuthenticationToken token = getAdminToken();
-          assertTrue("Expected KerberosToken, but was " + token.getClass(),
-              token instanceof KerberosToken);
-          KerberosToken kt = (KerberosToken) token;
-          assertNotNull("Expected keytab in token", kt.getKeytab());
-          return control.exec(TestMultiTableIngest.class,
-              args("--count", Integer.toString(ROWS), "--readonly", "-i", instance, "-z", keepers,
-                  "--tablePrefix", prefix, "--keytab", kt.getKeytab().getAbsolutePath(), "-u",
-                  principal));
-        }
-
-        return control.exec(TestMultiTableIngest.class,
-            args("--count", Integer.toString(ROWS), "--readonly", "-u", getAdminPrincipal(), "-i",
-                instance, "-z", keepers, "-p",
-                new String(((PasswordToken) getAdminToken()).getPassword(), UTF_8), "--tablePrefix",
-                prefix));
+        return control.exec(TestMultiTableIngest.class, args("--count", Integer.toString(ROWS),
+            "--readonly", "-c", cluster.getClientPropsPath(), "--tablePrefix", prefix));
       } catch (IOException e) {
         log.error("Error running MultiTableIngest", e);
         return -1;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
index 1fababc..83204e8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -29,12 +28,8 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
-import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
@@ -105,31 +100,15 @@ public class RestartIT extends AccumuloClusterHarness {
   public void restartMaster() throws Exception {
     try (AccumuloClient c = createAccumuloClient()) {
       final String tableName = getUniqueNames(1)[0];
-      OPTS.setTableName(tableName);
-      VOPTS.setTableName(tableName);
       c.tableOperations().create(tableName);
-      final AuthenticationToken token = getAdminToken();
       final ClusterControl control = getCluster().getClusterControl();
-
-      final String[] args;
-      if (token instanceof PasswordToken) {
-        byte[] password = ((PasswordToken) token).getPassword();
-        args = new String[] {"-u", getAdminPrincipal(), "-p", new String(password, UTF_8), "-i",
-            cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows", "" + OPTS.rows,
-            "--table", tableName};
-      } else if (token instanceof KerberosToken) {
-        ClusterUser rootUser = getAdminUser();
-        args = new String[] {"-u", getAdminPrincipal(), "--keytab",
-            rootUser.getKeytab().getAbsolutePath(), "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "--rows", "" + OPTS.rows, "--table", tableName};
-      } else {
-        throw new RuntimeException("Unknown token");
-      }
-      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setTableName(tableName);
+      VOPTS.setClientProperties(getClientProperties());
 
       Future<Integer> ret = svc.submit(() -> {
         try {
-          return control.exec(TestIngest.class, args);
+          return control.exec(TestIngest.class, new String[] {"-c", cluster.getClientPropsPath(),
+              "--rows", "" + OPTS.rows, "--table", tableName});
         } catch (IOException e) {
           log.error("Error running TestIngest", e);
           return -1;
@@ -198,32 +177,17 @@ public class RestartIT extends AccumuloClusterHarness {
   public void restartMasterSplit() throws Exception {
     try (AccumuloClient c = createAccumuloClient()) {
       final String tableName = getUniqueNames(1)[0];
-      final AuthenticationToken token = getAdminToken();
       final ClusterControl control = getCluster().getClusterControl();
-      VOPTS.setTableName(tableName);
       c.tableOperations().create(tableName);
       c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "5K");
 
-      final String[] args;
-      if (token instanceof PasswordToken) {
-        byte[] password = ((PasswordToken) token).getPassword();
-        args = new String[] {"-u", getAdminPrincipal(), "-p", new String(password, UTF_8), "-i",
-            cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows",
-            Integer.toString(VOPTS.rows), "--table", tableName};
-      } else if (token instanceof KerberosToken) {
-        ClusterUser rootUser = getAdminUser();
-        args = new String[] {"-u", getAdminPrincipal(), "--keytab",
-            rootUser.getKeytab().getAbsolutePath(), "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "--rows", Integer.toString(VOPTS.rows), "--table", tableName};
-      } else {
-        throw new RuntimeException("Unknown token");
-      }
-      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setTableName(tableName);
       VOPTS.setClientProperties(getClientProperties());
 
       Future<Integer> ret = svc.submit(() -> {
         try {
-          return control.exec(TestIngest.class, args);
+          return control.exec(TestIngest.class, new String[] {"-c", cluster.getClientPropsPath(),
+              "--rows", "" + VOPTS.rows, "--table", tableName});
         } catch (Exception e) {
           log.error("Error running TestIngest", e);
           return -1;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
index 08f1141..41bb33a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.Assert.assertEquals;
 
@@ -27,11 +26,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
-import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
@@ -98,28 +93,14 @@ public class RestartStressIT extends AccumuloClusterHarness {
   public void test() throws Exception {
     try (AccumuloClient c = createAccumuloClient()) {
       final String tableName = getUniqueNames(1)[0];
-      final AuthenticationToken token = getAdminToken();
       c.tableOperations().create(tableName);
       c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "500K");
       final ClusterControl control = getCluster().getClusterControl();
-      final String[] args;
-      if (token instanceof PasswordToken) {
-        byte[] password = ((PasswordToken) token).getPassword();
-        args = new String[] {"-u", getAdminPrincipal(), "-p", new String(password, UTF_8), "-i",
-            cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows", "" + VOPTS.rows,
-            "--table", tableName};
-      } else if (token instanceof KerberosToken) {
-        ClusterUser rootUser = getAdminUser();
-        args = new String[] {"-u", getAdminPrincipal(), "--keytab",
-            rootUser.getKeytab().getAbsolutePath(), "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "--rows", "" + VOPTS.rows, "--table", tableName};
-      } else {
-        throw new RuntimeException("Unrecognized token");
-      }
 
       Future<Integer> retCode = svc.submit(() -> {
         try {
-          return control.exec(TestIngest.class, args);
+          return control.exec(TestIngest.class, new String[] {"-c", cluster.getClientPropsPath(),
+              "--rows", "" + VOPTS.rows, "--table", tableName});
         } catch (Exception e) {
           log.error("Error running TestIngest", e);
           return -1;
@@ -137,5 +118,4 @@ public class RestartStressIT extends AccumuloClusterHarness {
       VerifyIngest.verifyIngest(c, VOPTS);
     }
   }
-
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
index ccf9597..6db093a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
@@ -42,8 +42,8 @@ public class ShutdownIT extends ConfigurableMacBase {
 
   @Test
   public void shutdownDuringIngest() throws Exception {
-    Process ingest = cluster.exec(TestIngest.class, "-i", cluster.getInstanceName(), "-z",
-        cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "--createTable").getProcess();
+    Process ingest = cluster
+        .exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--createTable").getProcess();
     sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     ingest.destroy();
@@ -52,11 +52,10 @@ public class ShutdownIT extends ConfigurableMacBase {
   @Test
   public void shutdownDuringQuery() throws Exception {
     assertEquals(0,
-        cluster.exec(TestIngest.class, "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "--createTable")
+        cluster.exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--createTable")
             .getProcess().waitFor());
-    Process verify = cluster.exec(VerifyIngest.class, "-i", cluster.getInstanceName(), "-z",
-        cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD).getProcess();
+    Process verify = cluster.exec(VerifyIngest.class, "-c", cluster.getClientPropsPath())
+        .getProcess();
     sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     verify.destroy();
@@ -65,11 +64,10 @@ public class ShutdownIT extends ConfigurableMacBase {
   @Test
   public void shutdownDuringDelete() throws Exception {
     assertEquals(0,
-        cluster.exec(TestIngest.class, "-i", cluster.getInstanceName(), "-z",
-            cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "--createTable")
+        cluster.exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--createTable")
             .getProcess().waitFor());
-    Process deleter = cluster.exec(TestRandomDeletes.class, "-i", cluster.getInstanceName(), "-z",
-        cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD).getProcess();
+    Process deleter = cluster.exec(TestRandomDeletes.class, "-c", cluster.getClientPropsPath())
+        .getProcess();
     sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     deleter.destroy();
@@ -112,9 +110,8 @@ public class ShutdownIT extends ConfigurableMacBase {
 
   static void runAdminStopTest(AccumuloClient c, MiniAccumuloClusterImpl cluster)
       throws InterruptedException, IOException {
-    String confPath = cluster.getConfig().getClientPropsFile().getAbsolutePath();
-    int x = cluster.exec(TestIngest.class, "--config-file", confPath, "--createTable").getProcess()
-        .waitFor();
+    int x = cluster.exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--createTable")
+        .getProcess().waitFor();
     assertEquals(0, x);
     List<String> tabletServers = c.instanceOperations().getTabletServers();
     assertEquals(2, tabletServers.size());
@@ -125,5 +122,4 @@ public class ShutdownIT extends ConfigurableMacBase {
     assertEquals(1, tabletServers.size());
     assertNotEquals(tabletServers.get(0), doomed);
   }
-
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
index c6e7118..c23cb9f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -25,11 +24,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -159,18 +156,8 @@ public class SplitIT extends AccumuloClusterHarness {
         assertTrue("Count should be cgreater than 10: " + count, count > 10);
       }
 
-      String[] args;
-      if (saslEnabled()) {
-        ClusterUser rootUser = getAdminUser();
-        args = new String[] {"-i", cluster.getInstanceName(), "-u", rootUser.getPrincipal(),
-            "--keytab", rootUser.getKeytab().getAbsolutePath(), "-z", cluster.getZooKeepers()};
-      } else {
-        PasswordToken token = (PasswordToken) getAdminToken();
-        args = new String[] {"-i", cluster.getInstanceName(), "-u", "root", "-p",
-            new String(token.getPassword(), UTF_8), "-z", cluster.getZooKeepers()};
-      }
-
-      assertEquals(0, getCluster().getClusterControl().exec(CheckForMetadataProblems.class, args));
+      assertEquals(0, getCluster().getClusterControl().exec(CheckForMetadataProblems.class,
+          new String[] {"-c", cluster.getClientPropsPath()}));
     }
   }
 
@@ -201,13 +188,7 @@ public class SplitIT extends AccumuloClusterHarness {
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
       c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
-      String password = null, keytab = null;
-      if (saslEnabled()) {
-        keytab = getAdminUser().getKeytab().getAbsolutePath();
-      } else {
-        password = new String(((PasswordToken) getAdminToken()).getPassword(), UTF_8);
-      }
-      DeleteIT.deleteTest(c, getCluster(), getAdminPrincipal(), password, tableName, keytab);
+      DeleteIT.deleteTest(c, getCluster(), tableName);
       c.tableOperations().flush(tableName, null, null, true);
       for (int i = 0; i < 5; i++) {
         sleepUninterruptibly(10, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
index ac0b420..e49ac7e 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
@@ -18,10 +18,7 @@ package org.apache.accumulo.test.mapreduce;
 
 import static org.junit.Assert.assertEquals;
 
-import java.io.File;
 import java.io.IOException;
-import java.io.PrintStream;
-import java.nio.file.Paths;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.Base64;
@@ -37,7 +34,6 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -90,17 +86,9 @@ public class MapReduceIT extends ConfigurableMacBase {
       bw.addMutation(m);
     }
     bw.close();
-    String filename = MapReduceIT.class.getName() + "-client.properties";
-    File configFile = Paths.get(System.getProperty("user.dir"), "target", filename).toFile();
-    try (PrintStream out = new PrintStream(configFile)) {
-      c.properties().store(out, "Config file for " + MapReduceIT.class.getName());
-    }
-    ClientInfo info = ClientInfo.from(c.properties());
-    Process hash = cluster
-        .exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-i",
-            info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD,
-            "-t", tablename, "--column", input_cfcq, "--config-file", configFile.getAbsolutePath())
-        .getProcess();
+
+    Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-c",
+        cluster.getClientPropsPath(), "-t", tablename, "--column", input_cfcq).getProcess();
     assertEquals(0, hash.waitFor());
 
     try (Scanner s = c.createScanner(tablename, Authorizations.EMPTY)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java b/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
index 8a504a1..ab78f28 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
@@ -23,7 +23,6 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -82,11 +81,9 @@ public class RollWALPerformanceIT extends ConfigurableMacBase {
 
     log.info("Starting ingest");
     final long start = System.nanoTime();
-    ClientInfo info = ClientInfo.from(c.properties());
     // Load 50K 100 byte entries
-    final String[] args = {"-i", info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root",
-        "-p", ROOT_PASSWORD, "--table", tableName, "--num", Long.toString(50 * 1000)};
-    ContinuousIngest.main(args);
+    ContinuousIngest.main(new String[] {"-c", cluster.getClientPropsPath(), "--table", tableName,
+        "--num", Long.toString(50 * 1000)});
     final long result = System.nanoTime() - start;
     log.debug(String.format("Finished in %,d ns", result));
     log.debug("Dropping table");
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 e46a3bc..e00cc0a 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
@@ -67,7 +67,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.cli.ContextOpts;
+import org.apache.accumulo.server.cli.ServerUtilOpts;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
@@ -88,7 +88,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 public class CollectTabletStats {
   private static final Logger log = LoggerFactory.getLogger(CollectTabletStats.class);
 
-  static class CollectOptions extends ContextOpts {
+  static class CollectOptions extends ServerUtilOpts {
     @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
     String tableName;
     @Parameter(names = "--iterations", description = "number of iterations")
@@ -97,7 +97,7 @@ public class CollectTabletStats {
     int numThreads = 1;
     @Parameter(names = "-f", description = "select far tablets, default is to use local tablets")
     boolean selectFarTablets = false;
-    @Parameter(names = "-c", description = "comma separated list of columns")
+    @Parameter(names = "--columns", description = "comma separated list of columns")
     String columns;
   }
 
diff --git a/test/src/test/java/org/apache/accumulo/test/performance/scan/CollectTabletStatsTest.java b/test/src/test/java/org/apache/accumulo/test/performance/scan/CollectTabletStatsTest.java
index 1ac72c7..69b8eac 100644
--- a/test/src/test/java/org/apache/accumulo/test/performance/scan/CollectTabletStatsTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/performance/scan/CollectTabletStatsTest.java
@@ -20,16 +20,11 @@ import static org.junit.Assert.assertEquals;
 
 import org.junit.Test;
 
-/**
- * Created by etcoleman on 10/11/16.
- */
 public class CollectTabletStatsTest {
 
   @Test
   public void paramsDefaulThreadTest() {
-
     String tablename = "aTable";
-
     String[] args = {"-t", tablename, "--iterations", "2"};
 
     final CollectTabletStats.CollectOptions opts = new CollectTabletStats.CollectOptions();
@@ -38,14 +33,11 @@ public class CollectTabletStatsTest {
     assertEquals("Check iterations is set, default is 3", 2, opts.iterations);
     assertEquals("Check tablename is set", 0, tablename.compareTo(opts.tableName));
     assertEquals("Check default numThreads", 1, opts.numThreads);
-
   }
 
   @Test
   public void paramsSetThreadsTest() {
-
     String tablename = "aTable";
-
     String[] args = {"-t", tablename, "--iterations", "2", "--numThreads", "99"};
 
     final CollectTabletStats.CollectOptions opts = new CollectTabletStats.CollectOptions();
@@ -54,7 +46,5 @@ public class CollectTabletStatsTest {
     assertEquals("Check iterations is set, default is 3", 2, opts.iterations);
     assertEquals("Check tablename is set", 0, tablename.compareTo(opts.tableName));
     assertEquals("Check numThreads is set", 99, opts.numThreads);
-
   }
-
 }