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 2018/04/16 17:46:33 UTC

[accumulo] branch master updated: #408 - Removed ClientConfiguration from ClientOpts (#419)

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 5d216d4   #408 - Removed ClientConfiguration from ClientOpts (#419)
5d216d4 is described below

commit 5d216d414cc9ce9e79b161f12bf4c92a430c803d
Author: Mike Walch <mw...@apache.org>
AuthorDate: Mon Apr 16 13:46:29 2018 -0400

     #408 - Removed ClientConfiguration from ClientOpts (#419)
---
 assemble/bin/accumulo-cluster                      |   4 +-
 .../lib/impl/MapReduceClientOnRequiredTable.java   |  20 +-
 .../mapreduce/lib/impl/MapReduceClientOpts.java    |   4 +-
 .../org/apache/accumulo/core/cli/ClientOpts.java   | 317 ++++++---------------
 .../minicluster/impl/MiniAccumuloClusterImpl.java  |  10 +-
 .../minicluster/impl/MiniAccumuloConfigImpl.java   |  17 ++
 .../accumulo/server/cli/ClientOnDefaultTable.java  |  12 +-
 .../accumulo/server/cli/ClientOnRequiredTable.java |  12 +-
 .../org/apache/accumulo/server/cli/ClientOpts.java |  13 +-
 .../server/util/VerifyTabletAssignments.java       |   4 +-
 .../org/apache/accumulo/server/util/ZooZap.java    |  18 +-
 .../org/apache/accumulo/test/WrongTabletTest.java  |   7 +-
 .../BalanceInPresenceOfOfflineTableIT.java         |   9 +-
 .../apache/accumulo/test/functional/BulkIT.java    |  13 +-
 .../test/functional/BulkSplitOptimizationIT.java   |  16 +-
 .../test/functional/ChaoticBalancerIT.java         |  11 +-
 .../accumulo/test/functional/CompactionIT.java     |  11 +-
 .../test/functional/ConfigurableMacBase.java       |  17 +-
 .../apache/accumulo/test/functional/DeleteIT.java  |  16 +-
 .../test/functional/DynamicThreadPoolsIT.java      |   6 +-
 .../accumulo/test/functional/FateStarvationIT.java |   6 +-
 .../test/functional/GarbageCollectorIT.java        |   4 +-
 .../test/functional/HalfDeadTServerIT.java         |   2 +-
 .../accumulo/test/functional/MasterFailoverIT.java |  12 +-
 .../apache/accumulo/test/functional/MaxOpenIT.java |   6 +-
 .../accumulo/test/functional/ReadWriteIT.java      |  61 ++--
 .../test/functional/RecoveryWithEmptyRFileIT.java  |   4 +-
 .../apache/accumulo/test/functional/RenameIT.java  |  17 +-
 .../apache/accumulo/test/functional/RestartIT.java |  50 +---
 .../accumulo/test/functional/RestartStressIT.java  |  10 +-
 .../accumulo/test/functional/ShutdownIT.java       |   4 +-
 .../test/functional/SimpleBalancerFairnessIT.java  |   2 +-
 .../apache/accumulo/test/functional/SplitIT.java   |  10 +-
 .../org/apache/accumulo/test/functional/SslIT.java |   6 +-
 .../apache/accumulo/test/functional/TableIT.java   |  10 +-
 .../accumulo/test/functional/WriteAheadLogIT.java  |  10 +-
 .../accumulo/test/functional/WriteLotsIT.java      |  12 +-
 .../test/security/KerberosClientOptsTest.java      |  89 ------
 38 files changed, 237 insertions(+), 615 deletions(-)

diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster
index 77320c8..92da91b 100755
--- a/assemble/bin/accumulo-cluster
+++ b/assemble/bin/accumulo-cluster
@@ -244,7 +244,7 @@ function kill_all() {
   done
 
   echo "Cleaning all server entries in ZooKeeper"
-  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers --site-file "${conf}/accumulo-site.xml"
+  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers
 }
 
 
@@ -284,7 +284,7 @@ function stop_all() {
   stop_tservers
 
   echo "Cleaning all server entries in ZooKeeper"
-  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers --site-file "${conf}/accumulo-site.xml"
+  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers
 }
 
 function stop_here() {
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOnRequiredTable.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOnRequiredTable.java
index ea440a9..cd1e8e0 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOnRequiredTable.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOnRequiredTable.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.client.mapreduce.lib.impl;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.hadoop.mapreduce.Job;
 
 import com.beust.jcommander.Parameter;
@@ -29,28 +28,13 @@ public class MapReduceClientOnRequiredTable extends MapReduceClientOpts {
   @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
   private String tableName;
 
-  @Parameter(names = {"-tf", "--tokenFile"},
-      description = "User's token file in HDFS created with \"bin/accumulo create-token\"")
-  private String tokenFile = "";
-
   @Override
   public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
     super.setAccumuloConfigs(job);
-
-    final String principal = getPrincipal(), tableName = getTableName();
-
-    if (tokenFile.isEmpty()) {
-      AuthenticationToken token = getToken();
-      AccumuloInputFormat.setConnectorInfo(job, principal, token);
-      AccumuloOutputFormat.setConnectorInfo(job, principal, token);
-    } else {
-      AccumuloInputFormat.setConnectorInfo(job, principal, tokenFile);
-      AccumuloOutputFormat.setConnectorInfo(job, principal, tokenFile);
-    }
-    AccumuloInputFormat.setInputTableName(job, tableName);
+    AccumuloInputFormat.setInputTableName(job, getTableName());
     AccumuloInputFormat.setScanAuthorizations(job, auths);
     AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, tableName);
+    AccumuloOutputFormat.setDefaultTableName(job, getTableName());
   }
 
   public String getTableName() {
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOpts.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOpts.java
index 6dfc84c..f879247 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOpts.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/MapReduceClientOpts.java
@@ -37,8 +37,8 @@ public class MapReduceClientOpts extends ClientOpts {
   private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
 
   public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    AccumuloInputFormat.setZooKeeperInstance(job, this.getClientConfiguration());
-    AccumuloOutputFormat.setZooKeeperInstance(job, this.getClientConfiguration());
+    AccumuloInputFormat.setConnectionInfo(job, this.getConnectionInfo());
+    AccumuloOutputFormat.setConnectionInfo(job, this.getConnectionInfo());
   }
 
   @Override
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 6fa88a3..7e78d51 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
@@ -18,49 +18,30 @@ package org.apache.accumulo.core.cli;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.UUID;
-import java.util.function.Predicate;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.accumulo.core.util.DeprecationUtil;
-import org.apache.accumulo.core.volume.VolumeConfiguration;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
-import com.beust.jcommander.DynamicParameter;
 import com.beust.jcommander.IStringConverter;
 import com.beust.jcommander.Parameter;
 
-import jline.console.ConsoleReader;
-
 public class ClientOpts extends Help {
 
   public static class TimeConverter implements IStringConverter<Long> {
@@ -95,20 +76,6 @@ public class ClientOpts extends Help {
     public String toString() {
       return new String(value, UTF_8);
     }
-
-    /**
-     * Prompts user for a password
-     *
-     * @return user entered Password object, null if no console exists
-     */
-    public static Password promptUser() throws IOException {
-      if (System.console() == null) {
-        throw new IOException("Attempted to prompt user on the console when System.console = null");
-      }
-      ConsoleReader reader = new ConsoleReader();
-      String enteredPass = reader.readLine("Enter password: ", '*');
-      return new Password(enteredPass);
-    }
   }
 
   public static class PasswordConverter implements IStringConverter<Password> {
@@ -135,55 +102,16 @@ public class ClientOpts extends Help {
       description = "Enter the connection password", password = true)
   private Password securePassword = null;
 
-  @Parameter(names = {"-tc", "--tokenClass"}, description = "Token class")
-  private String tokenClassName = null;
-
-  @DynamicParameter(names = "-l", description = "login properties in the format key=value. "
-      + "Reuse -l for each property (prompt for properties if this option is missing")
-  public Map<String,String> loginProps = new LinkedHashMap<>();
-
   public AuthenticationToken getToken() {
-    if (null != tokenClassName) {
-      final Properties props = new Properties();
-      if (!loginProps.isEmpty()) {
-        for (Entry<String,String> loginOption : loginProps.entrySet())
-          props.put(loginOption.getKey(), loginOption.getValue());
-      }
-
-      // It's expected that the user is already logged in via UserGroupInformation or external to
-      // this program (kinit).
-      try {
-        AuthenticationToken token = Class.forName(tokenClassName)
-            .asSubclass(AuthenticationToken.class).newInstance();
-        token.init(props);
-        return token;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    // other token types should have resolved by this point, so return PasswordToken
-    Password pass = null;
-    if (securePassword != null) {
-      pass = securePassword;
-    } else if (password != null) {
-      pass = password;
-    } else {
-      try {
-        pass = Password.promptUser();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return new PasswordToken(pass.value);
+    return getConnectionInfo().getAuthenticationToken();
   }
 
   @Parameter(names = {"-z", "--keepers"},
       description = "Comma separated list of zookeeper hosts (host:port,host:port)")
-  public String zookeepers = "localhost:2181";
+  private String zookeepers = null;
 
   @Parameter(names = {"-i", "--instance"}, description = "The name of the accumulo instance")
-  public String instance = null;
+  protected String instance = null;
 
   @Parameter(names = {"-auths", "--auths"}, converter = AuthConverter.class,
       description = "the authorizations to use when reading or writing")
@@ -192,23 +120,15 @@ public class ClientOpts extends Help {
   @Parameter(names = "--debug", description = "turn on TRACE-level log messages")
   public boolean debug = false;
 
-  @Parameter(names = {"-fake", "--mock"}, description = "Use a mock Instance")
-  public boolean mock = false;
-
-  @Parameter(names = "--site-file",
-      description = "Read the given accumulo site file to find the accumulo instance")
-  public String siteFile = null;
-
   @Parameter(names = "--ssl", description = "Connect to accumulo over SSL")
-  public boolean sslEnabled = false;
+  private boolean sslEnabled = false;
 
   @Parameter(names = "--sasl", description = "Connecto to Accumulo using SASL (supports Kerberos)")
-  public boolean saslEnabled = false;
+  private boolean saslEnabled = false;
 
   @Parameter(names = "--config-file", description = "Read the given client config file. "
-      + "If omitted, the path searched can be specified with $ACCUMULO_CLIENT_CONF_PATH, which "
-      + "defaults to ~/.accumulo/config:$ACCUMULO_CONF_DIR/client.conf:/etc/accumulo/client.conf")
-  public String clientConfigFile = null;
+      + "If omitted, the classpath will be searched for file named accumulo-client.properties")
+  private String clientConfigFile = null;
 
   public void startDebugLogging() {
     if (debug)
@@ -219,7 +139,7 @@ public class ClientOpts extends Help {
   public boolean trace = false;
 
   @Parameter(names = "--keytab", description = "Kerberos keytab on the local filesystem")
-  public String keytabPath = null;
+  private String keytabPath = null;
 
   public void startTracing(String applicationName) {
     if (trace) {
@@ -231,176 +151,101 @@ public class ClientOpts extends Help {
     Trace.off();
   }
 
-  /**
-   * Automatically update the options to use a KerberosToken when SASL is enabled for RPCs. Don't
-   * overwrite the options if the user has provided something specifically.
-   */
-  public void updateKerberosCredentials(String clientConfigFile) {
-    boolean saslEnabled = false;
-    if (clientConfigFile != null) {
-      saslEnabled = Connector.builder().usingProperties(clientConfigFile).info().saslEnabled();
-    }
-    updateKerberosCredentials(saslEnabled);
-  }
-
-  public void updateKerberosCredentials() {
-    updateKerberosCredentials(true);
-  }
-
-  /**
-   * Automatically update the options to use a KerberosToken when SASL is enabled for RPCs. Don't
-   * overwrite the options if the user has provided something specifically.
-   */
-  public void updateKerberosCredentials(boolean clientSaslEnabled) {
-    if ((saslEnabled || clientSaslEnabled) && null == tokenClassName) {
-      tokenClassName = KerberosToken.CLASS_NAME;
-      // ACCUMULO-3701 We need to ensure we're logged in before parseArgs returns as the MapReduce
-      // Job is going to make a copy of the current user (UGI)
-      // when it is instantiated.
-      if (null != keytabPath) {
-        File keytab = new File(keytabPath);
-        if (!keytab.exists() || !keytab.isFile()) {
-          throw new IllegalArgumentException("Keytab isn't a normal file: " + keytabPath);
-        }
-        if (null == principal) {
-          throw new IllegalArgumentException("Principal must be provided if logging in via Keytab");
-        }
-        try {
-          UserGroupInformation.loginUserFromKeytab(principal, keytab.getAbsolutePath());
-        } catch (IOException e) {
-          throw new RuntimeException("Failed to log in with keytab", e);
-        }
-      }
-    }
-  }
-
   @Override
   public void parseArgs(String programName, String[] args, Object... others) {
     super.parseArgs(programName, args, others);
     startDebugLogging();
     startTracing(programName);
-    updateKerberosCredentials(clientConfigFile);
   }
 
+  private ConnectionInfo cachedInfo = null;
+  private Connector cachedConnector = null;
   protected Instance cachedInstance = null;
-  protected ClientConfiguration cachedClientConfig = null;
+  private Properties cachedProps = null;
 
   synchronized public Instance getInstance() {
-    if (cachedInstance != null)
-      return cachedInstance;
-    if (mock)
-      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
-    return cachedInstance = new ZooKeeperInstance(this.getClientConfiguration());
-  }
-
-  public String getPrincipal() throws AccumuloSecurityException {
-    if (null == principal) {
-      AuthenticationToken token = getToken();
-      if (null == token) {
-        throw new AccumuloSecurityException("No principal or authentication token was provided",
-            SecurityErrorCode.BAD_CREDENTIALS);
-      }
-
-      // In MapReduce, if we create a DelegationToken, the principal is updated from the
-      // KerberosToken
-      // used to obtain the DelegationToken.
-      if (null != principal) {
-        return principal;
-      }
-
-      // Try to extract the principal automatically from Kerberos
-      if (token instanceof KerberosToken) {
-        principal = ((KerberosToken) token).getPrincipal();
-      } else {
-        principal = System.getProperty("user.name");
+    if (cachedInstance == null) {
+      try {
+        cachedInstance = getConnector().getInstance();
+      } catch (AccumuloSecurityException | AccumuloException e) {
+        throw new IllegalStateException(e);
       }
     }
-    return principal;
-  }
-
-  public void setPrincipal(String principal) {
-    this.principal = principal;
-  }
-
-  public Password getPassword() {
-    return password;
+    return cachedInstance;
   }
 
-  public void setPassword(Password password) {
-    this.password = password;
+  public String getPrincipal() {
+    return getConnectionInfo().getPrincipal();
   }
 
-  public Password getSecurePassword() {
-    return securePassword;
+  public void setPrincipal(String principal) {
+    this.principal = principal;
   }
 
-  public void setSecurePassword(Password securePassword) {
-    this.securePassword = securePassword;
+  public void setConnectionInfo(ConnectionInfo info) {
+    this.cachedInfo = info;
   }
 
-  public String getTokenClassName() {
-    return tokenClassName;
+  public ConnectionInfo getConnectionInfo() {
+    if (cachedInfo == null) {
+      cachedInfo = Connector.builder().usingProperties(getClientProperties()).info();
+    }
+    return cachedInfo;
   }
 
   public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
-    return getInstance().getConnector(getPrincipal(), getToken());
+    if (cachedConnector == null) {
+      cachedConnector = Connector.builder().usingConnectionInfo(getConnectionInfo()).build();
+    }
+    return cachedConnector;
   }
 
-  public ClientConfiguration getClientConfiguration() throws IllegalArgumentException {
-    if (cachedClientConfig != null)
-      return cachedClientConfig;
-
-    ClientConfiguration clientConfig;
-    try {
-      if (clientConfigFile == null)
-        clientConfig = ClientConfiguration.loadDefault();
-      else
-        clientConfig = ClientConfiguration.fromFile(new File(clientConfigFile));
-    } catch (Exception e) {
-      throw new IllegalArgumentException(e);
+  public String getClientConfigFile() {
+    if (clientConfigFile == null) {
+      URL clientPropsUrl = ClientOpts.class.getClassLoader()
+          .getResource("accumulo-client.properties");
+      if (clientPropsUrl != null) {
+        clientConfigFile = clientPropsUrl.getFile();
+      }
     }
-    if (sslEnabled)
-      clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SSL_ENABLED, "true");
-
-    if (saslEnabled)
-      clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SASL_ENABLED, "true");
-
-    if (siteFile != null) {
-      AccumuloConfiguration config = new AccumuloConfiguration() {
-        Configuration xml = new Configuration();
-        {
-          xml.addResource(new Path(siteFile));
-        }
-
-        @Override
-        public void getProperties(Map<String,String> props, Predicate<String> filter) {
-          for (Entry<String,String> prop : DefaultConfiguration.getInstance())
-            if (filter.test(prop.getKey()))
-              props.put(prop.getKey(), prop.getValue());
-          for (Entry<String,String> prop : xml)
-            if (filter.test(prop.getKey()))
-              props.put(prop.getKey(), prop.getValue());
-        }
+    return clientConfigFile;
+  }
 
-        @Override
-        public String get(Property property) {
-          String value = xml.get(property.getKey());
-          if (value != null)
-            return value;
-          return DefaultConfiguration.getInstance().get(property);
+  public Properties getClientProperties() {
+    if (cachedProps == null) {
+      cachedProps = new Properties();
+      if (getClientConfigFile() != null) {
+        try (InputStream is = new FileInputStream(getClientConfigFile())) {
+          cachedProps.load(is);
+        } catch (IOException e) {
+          throw new IllegalArgumentException(
+              "Failed to load properties from " + getClientConfigFile());
         }
-      };
-      this.zookeepers = config.get(Property.INSTANCE_ZK_HOST);
-
-      String volDir = VolumeConfiguration.getVolumeUris(config)[0];
-      Path instanceDir = new Path(volDir, "instance_id");
-      String instanceIDFromFile = ZooUtil.getInstanceIDFromHdfs(instanceDir, config);
-      if (config.getBoolean(Property.INSTANCE_RPC_SSL_ENABLED))
-        clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SSL_ENABLED, "true");
-      return cachedClientConfig = clientConfig.withInstance(UUID.fromString(instanceIDFromFile))
-          .withZkHosts(zookeepers);
+      }
+      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_USERNAME.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) {
+        cachedProps.setProperty(ClientProperty.AUTH_PASSWORD.getKey(), securePassword.toString());
+      } else if (password != null) {
+        cachedProps.setProperty(ClientProperty.AUTH_PASSWORD.getKey(), password.toString());
+      } else if (keytabPath != null) {
+        cachedProps.setProperty(ClientProperty.AUTH_METHOD.getKey(), "kerberos");
+        cachedProps.setProperty(ClientProperty.AUTH_KERBEROS_KEYTAB_PATH.getKey(), keytabPath);
+      }
     }
-    return cachedClientConfig = clientConfig.withInstance(instance).withZkHosts(zookeepers);
+    return cachedProps;
   }
-
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 3864186..f78eeec 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -455,12 +455,14 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     writeConfigProperties(clientConfFile, Maps.filterEntries(config.getSiteConfig(),
         v -> ClientConfiguration.ClientProperty.getPropertyByKey(v.getKey()) != null));
 
-    Map<String,String> clientProps = new HashMap<>();
+    Map<String,String> clientProps = config.getClientProps();
     clientProps.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), config.getZooKeepers());
     clientProps.put(ClientProperty.INSTANCE_NAME.getKey(), config.getInstanceName());
-    clientProps.put(ClientProperty.AUTH_METHOD.getKey(), "password");
-    clientProps.put(ClientProperty.AUTH_USERNAME.getKey(), config.getRootUserName());
-    clientProps.put(ClientProperty.AUTH_PASSWORD.getKey(), config.getRootPassword());
+    if (!clientProps.containsKey(ClientProperty.AUTH_METHOD.getKey())) {
+      clientProps.put(ClientProperty.AUTH_METHOD.getKey(), "password");
+      clientProps.put(ClientProperty.AUTH_USERNAME.getKey(), config.getRootUserName());
+      clientProps.put(ClientProperty.AUTH_PASSWORD.getKey(), config.getRootPassword());
+    }
 
     File clientPropsFile = config.getClientPropsFile();
     writeConfigProperties(clientPropsFile, clientProps);
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 01e4e77..0de59a0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -50,6 +50,7 @@ public class MiniAccumuloConfigImpl {
   private String rootPassword = null;
   private Map<String,String> siteConfig = new HashMap<>();
   private Map<String,String> configuredSiteConig = new HashMap<>();
+  private Map<String,String> clientProps = new HashMap<>();
   private int numTservers = 2;
   private Map<ServerType,Long> memoryConfig = new HashMap<>();
   private boolean jdwpEnabled = false;
@@ -292,6 +293,15 @@ public class MiniAccumuloConfigImpl {
     return _setSiteConfig(siteConfig);
   }
 
+  public MiniAccumuloConfigImpl setClientProps(Map<String,String> clientProps) {
+    if (existingInstance != null && existingInstance)
+      throw new UnsupportedOperationException(
+          "Cannot set zookeeper info when using an existing instance.");
+    this.existingInstance = Boolean.FALSE;
+    this.clientProps = clientProps;
+    return this;
+  }
+
   private MiniAccumuloConfigImpl _setSiteConfig(Map<String,String> siteConfig) {
     this.siteConfig = new HashMap<>(siteConfig);
     this.configuredSiteConig = new HashMap<>(siteConfig);
@@ -397,6 +407,13 @@ public class MiniAccumuloConfigImpl {
     return new HashMap<>(siteConfig);
   }
 
+  /**
+   * @return a copy of client props
+   */
+  public Map<String,String> getClientProps() {
+    return new HashMap<>(clientProps);
+  }
+
   public Map<String,String> getConfiguredSiteConfig() {
     return new HashMap<>(configuredSiteConig);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
index a058660..ebf1ad5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
@@ -16,9 +16,9 @@
  */
 package org.apache.accumulo.server.cli;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnDefaultTable extends org.apache.accumulo.core.cli.ClientOnDefaultTable {
@@ -31,12 +31,14 @@ public class ClientOnDefaultTable extends org.apache.accumulo.core.cli.ClientOnD
     if (cachedInstance != null)
       return cachedInstance;
 
-    if (mock)
-      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }
-    return cachedInstance = new ZooKeeperInstance(this.getClientConfiguration());
+    try {
+      return cachedInstance = getConnector().getInstance();
+    } catch (AccumuloSecurityException | AccumuloException e) {
+      throw new IllegalStateException(e);
+    }
   }
 
   public ClientOnDefaultTable(String table) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
index e02dd93..9c147d5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
@@ -16,9 +16,9 @@
  */
 package org.apache.accumulo.server.cli;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnRequiredTable extends org.apache.accumulo.core.cli.ClientOnRequiredTable {
@@ -31,11 +31,13 @@ public class ClientOnRequiredTable extends org.apache.accumulo.core.cli.ClientOn
     if (cachedInstance != null)
       return cachedInstance;
 
-    if (mock)
-      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }
-    return cachedInstance = new ZooKeeperInstance(getClientConfiguration());
+    try {
+      return cachedInstance = getConnector().getInstance();
+    } catch (AccumuloSecurityException | AccumuloException e) {
+      throw new IllegalStateException(e);
+    }
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
index c91471e..9d67c4d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
@@ -16,24 +16,25 @@
  */
 package org.apache.accumulo.server.cli;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOpts extends org.apache.accumulo.core.cli.ClientOpts {
-
   {
     setPrincipal("root");
   }
 
   @Override
   public Instance getInstance() {
-    if (mock)
-      return DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return HdfsZooInstance.getInstance();
     }
-    return new ZooKeeperInstance(this.getClientConfiguration());
+    try {
+      return getConnector().getInstance();
+    } catch (AccumuloSecurityException | AccumuloException e) {
+      throw new IllegalStateException(e);
+    }
   }
 }
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 2d6664d..94f214f 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
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ClientConfConverter;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Table;
@@ -75,7 +76,8 @@ public class VerifyTabletAssignments {
     opts.parseArgs(VerifyTabletAssignments.class.getName(), args);
 
     ClientContext context = new ClientContext(opts.getInstance(),
-        new Credentials(opts.getPrincipal(), opts.getToken()), opts.getClientConfiguration());
+        new Credentials(opts.getPrincipal(), opts.getToken()),
+        ClientConfConverter.toClientConf(opts.getClientProperties()));
     Connector conn = opts.getConnector();
     for (String table : conn.tableOperations().list())
       checkTable(context, opts, table, null);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
index be82a83..f1a6bce 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
@@ -19,16 +19,18 @@ package org.apache.accumulo.server.util;
 import java.util.List;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.hadoop.fs.Path;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,7 +46,7 @@ public class ZooZap {
       System.out.println(msg);
   }
 
-  static class Opts extends ClientOpts {
+  static class Opts extends Help {
     @Parameter(names = "-master", description = "remove master locks")
     boolean zapMaster = false;
     @Parameter(names = "-tservers", description = "remove tablet server locks")
@@ -53,10 +55,6 @@ public class ZooZap {
     boolean zapTracers = false;
     @Parameter(names = "-verbose", description = "print out messages about progress")
     boolean verbose = false;
-
-    String getTraceZKPath() {
-      return super.getClientConfiguration().get(ClientProperty.TRACE_ZK_PATH);
-    }
   }
 
   public static void main(String[] args) {
@@ -74,7 +72,9 @@ public class ZooZap {
       SecurityUtil.serverLogin(siteConf);
     }
 
-    String iid = opts.getInstance().getInstanceID();
+    String volDir = VolumeConfiguration.getVolumeUris(siteConf)[0];
+    Path instanceDir = new Path(volDir, "instance_id");
+    String iid = ZooUtil.getInstanceIDFromHdfs(instanceDir, siteConf);
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 
     if (opts.zapMaster) {
@@ -112,7 +112,7 @@ public class ZooZap {
     }
 
     if (opts.zapTracers) {
-      String path = opts.getTraceZKPath();
+      String path = siteConf.get(Property.TRACE_ZK_PATH);
       try {
         zapDirectory(zoo, path, opts);
       } catch (Exception e) {
diff --git a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
index c14616e..f41cdcb 100644
--- a/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/WrongTabletTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test;
 
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
@@ -52,11 +51,7 @@ public class WrongTabletTest {
     final ClientContext context = new AccumuloServerContext(inst, conf) {
       @Override
       public synchronized Credentials getCredentials() {
-        try {
-          return new Credentials(opts.getPrincipal(), opts.getToken());
-        } catch (AccumuloSecurityException e) {
-          throw new RuntimeException(e);
-        }
+        return new Credentials(opts.getPrincipal(), opts.getToken());
       }
     };
     try {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index b03b528..24fad5f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -127,13 +127,8 @@ public class BalanceInPresenceOfOfflineTableIT extends AccumuloClusterHarness {
 
     TestIngest.Opts opts = new TestIngest.Opts();
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-      vopts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal("root");
-      vopts.setPrincipal("root");
-    }
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
     vopts.rows = opts.rows = 200000;
     opts.setTableName(TEST_TABLE);
     TestIngest.ingest(connector, opts, new BatchWriterOpts());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
index f51ae1b..0455f6b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.test.functional;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
+import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -59,13 +60,13 @@ public class BulkIT extends AccumuloClusterHarness {
 
   @Test
   public void test() throws Exception {
-    runTest(getConnector(), getCluster().getFileSystem(), getCluster().getTemporaryPath(),
-        getAdminPrincipal(), getUniqueNames(1)[0], this.getClass().getName(),
+    runTest(getConnector(), getConnectionInfo(), getCluster().getFileSystem(),
+        getCluster().getTemporaryPath(), getUniqueNames(1)[0], this.getClass().getName(),
         testName.getMethodName());
   }
 
-  static void runTest(Connector c, FileSystem fs, Path basePath, String principal, String tableName,
-      String filePrefix, String dirSuffix) throws Exception {
+  static void runTest(Connector c, ConnectionInfo info, FileSystem fs, Path basePath,
+      String tableName, String filePrefix, String dirSuffix) throws Exception {
     c.tableOperations().create(tableName);
 
     Path base = new Path(basePath, "testBulkFail_" + dirSuffix);
@@ -80,9 +81,9 @@ public class BulkIT extends AccumuloClusterHarness {
     opts.timestamp = 1;
     opts.random = 56;
     opts.rows = N;
-    opts.instance = c.getInstance().getInstanceName();
     opts.cols = 1;
     opts.setTableName(tableName);
+    opts.setConnectionInfo(info);
     opts.conf = new Configuration(false);
     opts.fs = fs;
     String fileFormat = filePrefix + "rf%02d";
@@ -103,7 +104,7 @@ public class BulkIT extends AccumuloClusterHarness {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.setTableName(tableName);
     vopts.random = 56;
-    vopts.setPrincipal(principal);
+    vopts.setConnectionInfo(info);
     for (int i = 0; i < COUNT; i++) {
       vopts.startRow = i * N;
       vopts.rows = N;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index dbc7aa4..405eade 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -16,17 +16,13 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 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;
@@ -37,7 +33,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -125,16 +120,7 @@ public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
     opts.setTableName(tableName);
 
     AuthenticationToken adminToken = getAdminToken();
-    if (adminToken instanceof PasswordToken) {
-      PasswordToken token = (PasswordToken) getAdminToken();
-      opts.setPassword(new Password(new String(token.getPassword(), UTF_8)));
-      opts.setPrincipal(getAdminPrincipal());
-    } else if (adminToken instanceof KerberosToken) {
-      opts.updateKerberosCredentials(saslEnabled());
-    } else {
-      Assert.fail("Unknown token type");
-    }
-
+    opts.setConnectionInfo(getConnectionInfo());
     VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
 
     // ensure each tablet does not have all map files, should be ~2.5 files per tablet
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
index 05314ae..795a0fd 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
@@ -22,7 +22,6 @@ import java.util.stream.Stream;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
-import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -69,14 +68,8 @@ public class ChaoticBalancerIT extends AccumuloClusterHarness {
     vopts.rows = opts.rows = 20000;
     opts.setTableName(tableName);
     vopts.setTableName(tableName);
-    ConnectionInfo connectionInfo = getCluster().getConnectionInfo();
-    if (connectionInfo.saslEnabled()) {
-      opts.updateKerberosCredentials(connectionInfo.saslEnabled());
-      vopts.updateKerberosCredentials(connectionInfo.saslEnabled());
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-      vopts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     c.tableOperations().flush(tableName, null, null, true);
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index 4323856..8815838 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -26,12 +25,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 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.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -138,13 +135,7 @@ public class CompactionIT extends AccumuloClusterHarness {
               opts.dataSize = 50;
               opts.cols = 1;
               opts.setTableName(tableName);
-              if (saslEnabled()) {
-                opts.updateKerberosCredentials();
-              } else {
-                opts.setPrincipal(getAdminPrincipal());
-                PasswordToken passwordToken = (PasswordToken) getAdminToken();
-                opts.setPassword(new Password(new String(passwordToken.getPassword(), UTF_8)));
-              }
+              opts.setConnectionInfo(getConnectionInfo());
               VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
             } catch (Exception ex) {
               log.warn("Got exception verifying data", ex);
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 1882a2c..cf94f21 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
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.MonitorUtil;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -80,7 +81,7 @@ public class ConfigurableMacBase extends AccumuloITBase {
   }
 
   protected static void configureForSsl(MiniAccumuloConfigImpl cfg, File sslDir) {
-    Map<String,String> siteConfig = cfg.getSiteConfig();
+    Map<String, String> siteConfig = cfg.getSiteConfig();
     if ("true".equals(siteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
       // already enabled; don't mess with it
       return;
@@ -99,9 +100,9 @@ public class ConfigurableMacBase extends AccumuloITBase {
     try {
       new CertUtils(Property.RPC_SSL_KEYSTORE_TYPE.getDefaultValue(),
           "o=Apache Accumulo,cn=MiniAccumuloCluster", "RSA", 2048, "sha1WithRSAEncryption")
-              .createAll(rootKeystoreFile, localKeystoreFile, publicTruststoreFile,
-                  cfg.getInstanceName(), rootKeystorePassword, cfg.getRootPassword(),
-                  truststorePassword);
+          .createAll(rootKeystoreFile, localKeystoreFile, publicTruststoreFile,
+              cfg.getInstanceName(), rootKeystorePassword, cfg.getRootPassword(),
+              truststorePassword);
     } catch (Exception e) {
       throw new RuntimeException("error creating MAC keystore", e);
     }
@@ -113,6 +114,14 @@ public class ConfigurableMacBase extends AccumuloITBase {
         publicTruststoreFile.getAbsolutePath());
     siteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
     cfg.setSiteConfig(siteConfig);
+
+    Map<String, String> clientProps = cfg.getClientProps();
+    clientProps.put(ClientProperty.SSL_ENABLED.getKey(), "true");
+    clientProps.put(ClientProperty.SSL_KEYSTORE_PATH.getKey(), localKeystoreFile.getAbsolutePath());
+    clientProps.put(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(), cfg.getRootPassword());
+    clientProps.put(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(), publicTruststoreFile.getAbsolutePath());
+    clientProps.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
+    cfg.setClientProps(clientProps);
   }
 
   @Before
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 3305e29..9e91f87 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
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -69,19 +68,8 @@ public class DeleteIT extends AccumuloClusterHarness {
     vopts.random = opts.random = 56;
 
     assertTrue("Expected one of password or keytab", null != password || null != keytab);
-    if (null != password) {
-      assertNull("Given password, expected null keytab", keytab);
-      Password passwd = new Password(password);
-      opts.setPassword(passwd);
-      opts.setPrincipal(user);
-      vopts.setPassword(passwd);
-      vopts.setPrincipal(user);
-    }
-    if (null != keytab) {
-      assertNull("Given keytab, expect null password", password);
-      opts.updateKerberosCredentials(saslEnabled());
-      vopts.updateKerberosCredentials(saslEnabled());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
 
     BatchWriterOpts BWOPTS = new BatchWriterOpts();
     TestIngest.ingest(c, opts, BWOPTS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
index 93cf491..56a412c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
@@ -87,11 +87,7 @@ public class DynamicThreadPoolsIT extends AccumuloClusterHarness {
     opts.rows = 500 * 1000;
     opts.createTable = true;
     opts.setTableName(firstTable);
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     c.tableOperations().flush(firstTable, null, null, true);
     for (int i = 1; i < tables.length; i++)
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
index 2d88f3b..f851e18 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
@@ -52,11 +52,7 @@ public class FateStarvationIT extends AccumuloClusterHarness {
     opts.rows = 100000;
     opts.cols = 1;
     opts.setTableName(tableName);
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, opts, new BatchWriterOpts());
 
     c.tableOperations().flush(tableName, null, null, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index aecab52..f95eb6c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -120,8 +120,8 @@ public class GarbageCollectorIT extends ConfigurableMacBase {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.rows = opts.rows = 10000;
     vopts.cols = opts.cols = 1;
-    opts.setPrincipal("root");
-    vopts.setPrincipal("root");
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, cluster.getFileSystem(), opts, new BatchWriterOpts());
     c.tableOperations().compact("test_ingest", null, null, true, true);
     int before = countFiles();
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 003f16f..dd8175c 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
@@ -169,7 +169,7 @@ public class HalfDeadTServerIT extends ConfigurableMacBase {
         assertEquals(0, ingest.waitFor());
         VerifyIngest.Opts vopts = new VerifyIngest.Opts();
         vopts.rows = rows;
-        vopts.setPrincipal("root");
+        vopts.setConnectionInfo(getConnectionInfo());
         VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
       } else {
         sleepUninterruptibly(5, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
index a4159d0..bc43349 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
@@ -52,11 +52,7 @@ public class MasterFailoverIT extends AccumuloClusterHarness {
     c.tableOperations().create(names[0]);
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.setTableName(names[0]);
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, opts, new BatchWriterOpts());
 
     ClusterControl control = cluster.getClusterControl();
@@ -67,11 +63,7 @@ public class MasterFailoverIT extends AccumuloClusterHarness {
     c.tableOperations().rename(names[0], names[1]);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.setTableName(names[1]);
-    if (saslEnabled()) {
-      vopts.updateKerberosCredentials();
-    } else {
-      vopts.setPrincipal(getAdminPrincipal());
-    }
+    vopts.setConnectionInfo(getConnectionInfo());
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
index 402cb5a..2aa2113 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
@@ -106,11 +106,7 @@ public class MaxOpenIT extends AccumuloClusterHarness {
       opts.cols = 1;
       opts.random = i;
       opts.setTableName(tableName);
-      if (saslEnabled()) {
-        opts.updateKerberosCredentials();
-      } else {
-        opts.setPrincipal(getAdminPrincipal());
-      }
+      opts.setConnectionInfo(getConnectionInfo());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
 
       c.tableOperations().flush(tableName, null, null, true);
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 cc0d3fb..bc81e37 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
@@ -64,6 +64,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -136,8 +137,8 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     cluster.getClusterControl().startAllServers(ServerType.MONITOR);
     Connector connector = getConnector();
     String tableName = getUniqueNames(1)[0];
-    ingest(connector, getAdminPrincipal(), ROWS, COLS, 50, 0, tableName);
-    verify(connector, getAdminPrincipal(), ROWS, COLS, 50, 0, tableName);
+    ingest(connector, getConnectionInfo(), ROWS, COLS, 50, 0, tableName);
+    verify(connector, getConnectionInfo(), ROWS, COLS, 50, 0, tableName);
     String monitorLocation = null;
     while (null == monitorLocation) {
       monitorLocation = MonitorUtil.getLocation(getConnector().getInstance());
@@ -200,12 +201,12 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     cluster.start();
   }
 
-  public static void ingest(Connector connector, String principal, int rows, int cols, int width,
-      int offset, String tableName) throws Exception {
-    ingest(connector, principal, rows, cols, width, offset, COLF, tableName);
+  public static void ingest(Connector connector, ConnectionInfo info, int rows, int cols, int width,
+                            int offset, String tableName) throws Exception {
+    ingest(connector, info, rows, cols, width, offset, COLF, tableName);
   }
 
-  public static void ingest(Connector connector, String principal, int rows, int cols, int width,
+  public static void ingest(Connector connector, ConnectionInfo info, int rows, int cols, int width,
       int offset, String colf, String tableName) throws Exception {
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.rows = rows;
@@ -215,21 +216,17 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     opts.columnFamily = colf;
     opts.createTable = true;
     opts.setTableName(tableName);
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(principal);
-    }
+    opts.setConnectionInfo(info);
 
     TestIngest.ingest(connector, opts, new BatchWriterOpts());
   }
 
-  public static void verify(Connector connector, String principal, int rows, int cols, int width,
+  public static void verify(Connector connector, ConnectionInfo info, int rows, int cols, int width,
       int offset, String tableName) throws Exception {
-    verify(connector, principal, rows, cols, width, offset, COLF, tableName);
+    verify(connector, info, rows, cols, width, offset, COLF, tableName);
   }
 
-  private static void verify(Connector connector, String principal, int rows, int cols, int width,
+  private static void verify(Connector connector, ConnectionInfo info, int rows, int cols, int width,
       int offset, String colf, String tableName) throws Exception {
     ScannerOpts scannerOpts = new ScannerOpts();
     VerifyIngest.Opts opts = new VerifyIngest.Opts();
@@ -239,11 +236,7 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     opts.startRow = offset;
     opts.columnFamily = colf;
     opts.setTableName(tableName);
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(principal);
-    }
+    opts.setConnectionInfo(info);
 
     VerifyIngest.verifyIngest(connector, opts, scannerOpts);
   }
@@ -335,8 +328,8 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     // write a few large values
     Connector connector = getConnector();
     String table = getUniqueNames(1)[0];
-    ingest(connector, getAdminPrincipal(), 2, 1, 500000, 0, table);
-    verify(connector, getAdminPrincipal(), 2, 1, 500000, 0, table);
+    ingest(connector, getConnectionInfo(), 2, 1, 500000, 0, table);
+    verify(connector, getConnectionInfo(), 2, 1, 500000, 0, table);
   }
 
   @Test
@@ -350,7 +343,7 @@ public class ReadWriteIT extends AccumuloClusterHarness {
   static void interleaveTest(final Connector connector, final String tableName) throws Exception {
     final AtomicBoolean fail = new AtomicBoolean(false);
     final int CHUNKSIZE = ROWS / 10;
-    ingest(connector, getAdminPrincipal(), CHUNKSIZE, 1, 50, 0, tableName);
+    ingest(connector, getConnectionInfo(), CHUNKSIZE, 1, 50, 0, tableName);
     int i;
     for (i = 0; i < ROWS; i += CHUNKSIZE) {
       final int start = i;
@@ -358,18 +351,18 @@ public class ReadWriteIT extends AccumuloClusterHarness {
         @Override
         public void run() {
           try {
-            verify(connector, getAdminPrincipal(), CHUNKSIZE, 1, 50, start, tableName);
+            verify(connector, getConnectionInfo(), CHUNKSIZE, 1, 50, start, tableName);
           } catch (Exception ex) {
             fail.set(true);
           }
         }
       };
       verify.start();
-      ingest(connector, getAdminPrincipal(), CHUNKSIZE, 1, 50, i + CHUNKSIZE, tableName);
+      ingest(connector, getConnectionInfo(), CHUNKSIZE, 1, 50, i + CHUNKSIZE, tableName);
       verify.join();
       assertFalse(fail.get());
     }
-    verify(connector, getAdminPrincipal(), CHUNKSIZE, 1, 50, i, tableName);
+    verify(connector, getConnectionInfo(), CHUNKSIZE, 1, 50, i, tableName);
   }
 
   public static Text t(String s) {
@@ -390,7 +383,7 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     connector.tableOperations().create(tableName);
     connector.tableOperations().setProperty(tableName, "table.group.g1", "colf");
     connector.tableOperations().setProperty(tableName, "table.groups.enabled", "g1");
-    ingest(connector, getAdminPrincipal(), 2000, 1, 50, 0, tableName);
+    ingest(connector, getConnectionInfo(), 2000, 1, 50, 0, tableName);
     connector.tableOperations().compact(tableName, null, null, true, true);
     BatchWriter bw = connector.createBatchWriter(tableName, new BatchWriterConfig());
     bw.addMutation(m("zzzzzzzzzzz", "colf2", "cq", "value"));
@@ -445,8 +438,8 @@ public class ReadWriteIT extends AccumuloClusterHarness {
 
   private void verifyLocalityGroupsInRFile(final Connector connector, final String tableName)
       throws Exception, AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    ingest(connector, getAdminPrincipal(), 2000, 1, 50, 0, tableName);
-    verify(connector, getAdminPrincipal(), 2000, 1, 50, 0, tableName);
+    ingest(connector, getConnectionInfo(), 2000, 1, 50, 0, tableName);
+    verify(connector, getConnectionInfo(), 2000, 1, 50, 0, tableName);
     connector.tableOperations().flush(tableName, null, null, true);
     try (BatchScanner bscanner = connector.createBatchScanner(MetadataTable.NAME,
         Authorizations.EMPTY, 1)) {
@@ -495,9 +488,9 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     int i = 0;
     for (String cfg : config) {
       to.setLocalityGroups(table, getGroups(cfg));
-      ingest(connector, getAdminPrincipal(), ROWS * (i + 1), 1, 50, ROWS * i, table);
+      ingest(connector, getConnectionInfo(), ROWS * (i + 1), 1, 50, ROWS * i, table);
       to.flush(table, null, null, true);
-      verify(connector, getAdminPrincipal(), 0, 1, 50, ROWS * (i + 1), table);
+      verify(connector, getConnectionInfo(), 0, 1, 50, ROWS * (i + 1), table);
       i++;
     }
     to.delete(table);
@@ -505,12 +498,12 @@ public class ReadWriteIT extends AccumuloClusterHarness {
     config = new String[] {"lg1:colf", null, "lg1:colf,xyz", "lg1:colf;lg2:colf",};
     i = 1;
     for (String cfg : config) {
-      ingest(connector, getAdminPrincipal(), ROWS * i, 1, 50, 0, table);
-      ingest(connector, getAdminPrincipal(), ROWS * i, 1, 50, 0, "xyz", table);
+      ingest(connector, getConnectionInfo(), ROWS * i, 1, 50, 0, table);
+      ingest(connector, getConnectionInfo(), ROWS * i, 1, 50, 0, "xyz", table);
       to.setLocalityGroups(table, getGroups(cfg));
       to.flush(table, null, null, true);
-      verify(connector, getAdminPrincipal(), ROWS * i, 1, 50, 0, table);
-      verify(connector, getAdminPrincipal(), ROWS * i, 1, 50, 0, "xyz", table);
+      verify(connector, getConnectionInfo(), ROWS * i, 1, 50, 0, table);
+      verify(connector, getConnectionInfo(), ROWS * i, 1, 50, 0, "xyz", table);
       i++;
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
index 9c5b57f..1eeebb9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
@@ -71,8 +71,8 @@ public class RecoveryWithEmptyRFileIT extends ConfigurableMacBase {
         + " underlying rfile with an empty one and verify we can scan.");
     Connector connector = getConnector();
     String tableName = getUniqueNames(1)[0];
-    ReadWriteIT.ingest(connector, "root", ROWS, COLS, 50, 0, tableName);
-    ReadWriteIT.verify(connector, "root", ROWS, COLS, 50, 0, tableName);
+    ReadWriteIT.ingest(connector, getConnectionInfo(), ROWS, COLS, 50, 0, tableName);
+    ReadWriteIT.verify(connector, getConnectionInfo(), ROWS, COLS, 50, 0, tableName);
 
     connector.tableOperations().flush(tableName, null, null, true);
     connector.tableOperations().offline(tableName, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
index 68be450..654e6f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.test.functional;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
-import org.apache.accumulo.core.client.ConnectionInfo;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.TestIngest;
@@ -42,26 +41,14 @@ public class RenameIT extends AccumuloClusterHarness {
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.createTable = true;
     opts.setTableName(name1);
-
-    final ConnectionInfo connectionInfo = cluster.getConnectionInfo();
-    if (connectionInfo.saslEnabled()) {
-      opts.updateKerberosCredentials(connectionInfo.saslEnabled());
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(cluster.getConnectionInfo());
 
     Connector c = getConnector();
     TestIngest.ingest(c, opts, bwOpts);
     c.tableOperations().rename(name1, name2);
     TestIngest.ingest(c, opts, bwOpts);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-
-    if (connectionInfo.saslEnabled()) {
-      vopts.updateKerberosCredentials(connectionInfo.saslEnabled());
-    } else {
-      vopts.setPrincipal(getAdminPrincipal());
-    }
-
+    vopts.setConnectionInfo(cluster.getConnectionInfo());
     vopts.setTableName(name2);
     VerifyIngest.verifyIngest(c, vopts, scanOpts);
     c.tableOperations().delete(name1);
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 72951b1..d048610 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
@@ -119,18 +119,15 @@ public class RestartIT extends AccumuloClusterHarness {
       args = new String[] {"-u", getAdminPrincipal(), "-p", new String(password, UTF_8), "-i",
           cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows", "" + OPTS.rows,
           "--table", tableName};
-      OPTS.setPrincipal(getAdminPrincipal());
-      VOPTS.setPrincipal(getAdminPrincipal());
     } 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};
-      OPTS.updateKerberosCredentials(saslEnabled());
-      VOPTS.updateKerberosCredentials(saslEnabled());
     } else {
       throw new RuntimeException("Unknown token");
     }
+    OPTS.setConnectionInfo(getConnectionInfo());
 
     Future<Integer> ret = svc.submit(new Callable<Integer>() {
       @Override
@@ -157,13 +154,8 @@ public class RestartIT extends AccumuloClusterHarness {
     c.tableOperations().create(tableName);
     OPTS.setTableName(tableName);
     VOPTS.setTableName(tableName);
-    if (saslEnabled()) {
-      OPTS.updateKerberosCredentials();
-      VOPTS.updateKerberosCredentials();
-    } else {
-      OPTS.setPrincipal(getAdminPrincipal());
-      VOPTS.setPrincipal(getAdminPrincipal());
-    }
+    OPTS.setConnectionInfo(getConnectionInfo());
+    VOPTS.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, OPTS, BWOPTS);
     ClusterControl control = getCluster().getClusterControl();
 
@@ -221,18 +213,16 @@ public class RestartIT extends AccumuloClusterHarness {
       args = new String[] {"-u", getAdminPrincipal(), "-p", new String(password, UTF_8), "-i",
           cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows",
           Integer.toString(VOPTS.rows), "--table", tableName};
-      OPTS.setPrincipal(getAdminPrincipal());
-      VOPTS.setPrincipal(getAdminPrincipal());
     } 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};
-      OPTS.updateKerberosCredentials(saslEnabled());
-      VOPTS.updateKerberosCredentials(saslEnabled());
     } else {
       throw new RuntimeException("Unknown token");
     }
+    OPTS.setConnectionInfo(getConnectionInfo());
+    VOPTS.setConnectionInfo(getConnectionInfo());
 
     Future<Integer> ret = svc.submit(new Callable<Integer>() {
       @Override
@@ -273,13 +263,8 @@ public class RestartIT extends AccumuloClusterHarness {
     c.tableOperations().create(tableName);
     OPTS.setTableName(tableName);
     VOPTS.setTableName(tableName);
-    if (saslEnabled()) {
-      OPTS.updateKerberosCredentials();
-      VOPTS.updateKerberosCredentials();
-    } else {
-      OPTS.setPrincipal(getAdminPrincipal());
-      VOPTS.setPrincipal(getAdminPrincipal());
-    }
+    OPTS.setConnectionInfo(getConnectionInfo());
+    VOPTS.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, OPTS, BWOPTS);
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
     cluster.getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -308,11 +293,7 @@ public class RestartIT extends AccumuloClusterHarness {
     String tableName = getUniqueNames(1)[0];
     c.tableOperations().create(tableName);
     OPTS.setTableName(tableName);
-    if (saslEnabled()) {
-      OPTS.updateKerberosCredentials();
-    } else {
-      OPTS.setPrincipal(getAdminPrincipal());
-    }
+    OPTS.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, OPTS, BWOPTS);
     try {
       getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -327,13 +308,8 @@ public class RestartIT extends AccumuloClusterHarness {
     Connector c = getConnector();
     String tableName = getUniqueNames(1)[0];
     VOPTS.setTableName(tableName);
-    if (saslEnabled()) {
-      OPTS.updateKerberosCredentials();
-      VOPTS.updateKerberosCredentials();
-    } else {
-      OPTS.setPrincipal(getAdminPrincipal());
-      VOPTS.setPrincipal(getAdminPrincipal());
-    }
+    OPTS.setConnectionInfo(getConnectionInfo());
+    VOPTS.setConnectionInfo(getConnectionInfo());
     c.tableOperations().create(tableName);
     c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
     String splitThreshold = null;
@@ -349,11 +325,7 @@ public class RestartIT extends AccumuloClusterHarness {
           "20K");
       TestIngest.Opts opts = new TestIngest.Opts();
       opts.setTableName(tableName);
-      if (saslEnabled()) {
-        opts.updateKerberosCredentials();
-      } else {
-        opts.setPrincipal(getAdminPrincipal());
-      }
+      opts.setConnectionInfo(getConnectionInfo());
       TestIngest.ingest(c, opts, BWOPTS);
       c.tableOperations().flush(tableName, null, null, false);
       VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
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 06a2bfe..3ab9dfc 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
@@ -139,15 +139,7 @@ public class RestartStressIT extends AccumuloClusterHarness {
     }
     assertEquals(0, retCode.get().intValue());
     VOPTS.setTableName(tableName);
-
-    if (token instanceof PasswordToken) {
-      VOPTS.setPrincipal(getAdminPrincipal());
-    } else if (token instanceof KerberosToken) {
-      VOPTS.updateKerberosCredentials(saslEnabled());
-    } else {
-      throw new RuntimeException("Unrecognized token");
-    }
-
+    VOPTS.setConnectionInfo(getConnectionInfo());
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
   }
 
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 a3baa68..efcfb95 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
@@ -108,8 +108,8 @@ public class ShutdownIT extends ConfigurableMacBase {
 
   static void runAdminStopTest(Connector c, MiniAccumuloClusterImpl cluster)
       throws InterruptedException, IOException {
-    assertEquals(0, cluster.exec(TestIngest.class, "-i", cluster.getInstanceName(), "-z",
-        cluster.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "--createTable").waitFor());
+    assertEquals(0, cluster.exec(TestIngest.class, "--config-file",
+        cluster.getConfig().getClientPropsFile().getAbsolutePath(), "--createTable").waitFor());
     List<String> tabletServers = c.instanceOperations().getTabletServers();
     assertEquals(2, tabletServers.size());
     String doomed = tabletServers.get(0);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index 7ed7549..728603c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -73,7 +73,7 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacBase {
     List<String> tservers = c.instanceOperations().getTabletServers();
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.rows = 50000;
-    opts.setPrincipal("root");
+    opts.setConnectionInfo(getConnectionInfo());
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     c.tableOperations().flush("test_ingest", null, null, false);
     sleepUninterruptibly(45, TimeUnit.SECONDS);
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 0dbfca3..661cfab 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
@@ -131,18 +131,12 @@ public class SplitIT extends AccumuloClusterHarness {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     opts.rows = 100000;
     opts.setTableName(table);
-
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-      vopts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-      vopts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
 
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     vopts.rows = opts.rows;
     vopts.setTableName(table);
+    vopts.setConnectionInfo(getConnectionInfo());
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
     while (c.tableOperations().listSplits(table).size() < 10) {
       sleepUninterruptibly(15, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
index f98645a..085a93e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
@@ -61,9 +61,9 @@ public class SslIT extends ConfigurableMacBase {
 
   @Test
   public void bulk() throws Exception {
-    BulkIT.runTest(getConnector(), cluster.getFileSystem(),
-        new Path(getCluster().getConfig().getDir().getAbsolutePath(), "tmp"), "root",
-        getUniqueNames(1)[0], this.getClass().getName(), testName.getMethodName());
+    BulkIT.runTest(getConnector(), getConnectionInfo(), cluster.getFileSystem(),
+        new Path(getCluster().getConfig().getDir().getAbsolutePath(), "tmp"), getUniqueNames(1)[0],
+        this.getClass().getName(), testName.getMethodName());
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
index b8a07c2..14021f5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
@@ -70,14 +70,8 @@ public class TableIT extends AccumuloClusterHarness {
 
     TestIngest.Opts opts = new TestIngest.Opts();
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-      vopts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-      vopts.setPrincipal(getAdminPrincipal());
-    }
-
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
     opts.setTableName(tableName);
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     to.flush(tableName, null, null, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index fdb29bc..7df99f3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -56,14 +56,8 @@ public class WriteAheadLogIT extends AccumuloClusterHarness {
     TestIngest.Opts opts = new TestIngest.Opts();
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     opts.setTableName(tableName);
-
-    if (saslEnabled()) {
-      opts.updateKerberosCredentials();
-      vopts.updateKerberosCredentials();
-    } else {
-      opts.setPrincipal(getAdminPrincipal());
-      vopts.setPrincipal(getAdminPrincipal());
-    }
+    opts.setConnectionInfo(getConnectionInfo());
+    vopts.setConnectionInfo(getConnectionInfo());
 
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     vopts.setTableName(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
index 20b4cbb..bf86907 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
@@ -55,11 +55,7 @@ public class WriteLotsIT extends AccumuloClusterHarness {
             opts.startRow = index * 10000;
             opts.rows = 10000;
             opts.setTableName(tableName);
-            if (saslEnabled()) {
-              opts.updateKerberosCredentials();
-            } else {
-              opts.setPrincipal(getAdminPrincipal());
-            }
+            opts.setConnectionInfo(getConnectionInfo());
             BatchWriterOpts bwOpts = new BatchWriterOpts();
             bwOpts.batchMemory = 1024L * 1024;
             bwOpts.batchThreads = 2;
@@ -79,11 +75,7 @@ public class WriteLotsIT extends AccumuloClusterHarness {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.rows = 10000 * THREADS;
     vopts.setTableName(tableName);
-    if (saslEnabled()) {
-      vopts.updateKerberosCredentials();
-    } else {
-      vopts.setPrincipal(getAdminPrincipal());
-    }
+    vopts.setConnectionInfo(getConnectionInfo());
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
   }
 
diff --git a/test/src/test/java/org/apache/accumulo/test/security/KerberosClientOptsTest.java b/test/src/test/java/org/apache/accumulo/test/security/KerberosClientOptsTest.java
deleted file mode 100644
index 69c5811..0000000
--- a/test/src/test/java/org/apache/accumulo/test/security/KerberosClientOptsTest.java
+++ /dev/null
@@ -1,89 +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.test.security;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-
-import org.apache.accumulo.core.cli.ClientOpts;
-import org.apache.accumulo.harness.TestingKdc;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test that kerberos features work properly in {@link ClientOpts}
- */
-public class KerberosClientOptsTest {
-  private static final Logger log = LoggerFactory.getLogger(KerberosClientOptsTest.class);
-
-  @Rule
-  public TestName testName = new TestName();
-
-  private static TestingKdc kdc;
-
-  @BeforeClass
-  public static void startKdc() throws Exception {
-    kdc = new TestingKdc();
-    kdc.start();
-  }
-
-  @AfterClass
-  public static void stopKdc() throws Exception {
-    if (null != kdc) {
-      kdc.stop();
-    }
-  }
-
-  @Before
-  public void resetUgiForKrb() {
-    Configuration conf = new Configuration(false);
-    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-  }
-
-  @Test
-  public void testParseArgsPerformsLogin() throws Exception {
-    String user = testName.getMethodName();
-    File userKeytab = new File(kdc.getKeytabDir(), user + ".keytab");
-    if (userKeytab.exists() && !userKeytab.delete()) {
-      log.warn("Unable to delete {}", userKeytab);
-    }
-
-    kdc.createPrincipal(userKeytab, user);
-
-    user = kdc.qualifyUser(user);
-
-    ClientOpts opts = new ClientOpts();
-    String[] args = new String[] {"--sasl", "--keytab", userKeytab.getAbsolutePath(), "-u", user};
-    opts.parseArgs(testName.getMethodName(), args);
-
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    assertEquals(user, ugi.getUserName());
-    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
-  }
-}

-- 
To stop receiving notification emails like this one, please contact
mwalch@apache.org.