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/12/04 18:33:56 UTC

[accumulo] branch master updated: Updates to client builder API (#792)

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 d3a5a45  Updates to client builder API (#792)
d3a5a45 is described below

commit d3a5a45ea7530945301884bc7541adb168b05644
Author: Mike Walch <mw...@apache.org>
AuthorDate: Tue Dec 4 13:33:51 2018 -0500

    Updates to client builder API (#792)
    
    * Removed use ClientInfo in AccumuloClient
    * Removed changeUser method from AccumuloClient
    * Created Accumulo.newClientProperties() for building client properties
    * Removed client verification from AccumuloClient constructor but kept
      it for Connector
    * Added properties() method to AccumuloClient
---
 .../org/apache/accumulo/core/cli/ClientOpts.java   |   8 +-
 .../org/apache/accumulo/core/client/Accumulo.java  |  23 +++-
 .../accumulo/core/client/AccumuloClient.java       | 111 ++++++------------
 .../org/apache/accumulo/core/client/Connector.java |   2 +-
 .../core/client/mapred/AbstractInputFormat.java    |   4 +-
 .../core/client/mapred/AccumuloOutputFormat.java   |   4 +-
 .../core/client/mapreduce/AbstractInputFormat.java |   4 +-
 .../client/mapreduce/AccumuloOutputFormat.java     |   4 +-
 .../core/clientImpl/AccumuloClientImpl.java        | 126 +++++++++++----------
 .../accumulo/core/clientImpl/ClientContext.java    |  10 +-
 .../accumulo/core/clientImpl/ConnectorImpl.java    |   5 +-
 .../clientImpl/mapreduce/lib/ConfiguratorBase.java |  19 ++--
 .../mapreduce/lib/MapReduceClientOpts.java         |   2 +-
 .../core/replication/ReplicationTable.java         |   2 +-
 .../java/org/apache/accumulo/core/util/Merge.java  |   2 +-
 .../mapreduce/lib/ConfiguratorBaseTest.java        |   4 +-
 .../hadoop/mapred/AccumuloOutputFormat.java        |   2 +-
 .../hadoop/mapreduce/AccumuloOutputFormat.java     |   2 +-
 .../mapred/AccumuloOutputFormatImpl.java           |   2 +-
 .../mapreduce/AccumuloOutputFormatImpl.java        |   2 +-
 .../mapreduce/InputFormatBuilderImpl.java          |   2 +-
 .../hadoopImpl/mapreduce/lib/ConfiguratorBase.java |  13 +--
 .../mapreduce/lib/MapReduceClientOpts.java         |   2 +-
 .../hadoop/its/mapred/AccumuloOutputFormatIT.java  |   3 +-
 .../accumulo/hadoop/its/mapreduce/RowHashIT.java   |   6 +-
 .../hadoop/mapred/AccumuloOutputFormatTest.java    |  11 +-
 .../hadoop/mapreduce/AccumuloOutputFormatTest.java |  12 +-
 .../mapreduce/lib/ConfiguratorBaseTest.java        |   4 +-
 .../standalone/StandaloneAccumuloCluster.java      |   5 +-
 .../accumulo/minicluster/MiniAccumuloCluster.java  |   3 +-
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |   5 +-
 .../org/apache/accumulo/server/ServerContext.java  |   5 +-
 .../accumulo/server/util/TableDiskUsage.java       |   4 +-
 .../server/util/VerifyTabletAssignments.java       |   2 +-
 .../org/apache/accumulo/tracer/TraceServer.java    |   2 +-
 .../main/java/org/apache/accumulo/shell/Shell.java |  18 +--
 .../accumulo/shell/commands/CreateUserCommand.java |   3 +-
 .../accumulo/shell/commands/ScriptCommand.java     |   8 +-
 .../StandaloneAccumuloClusterConfiguration.java    |   4 +-
 .../accumulo/test/ArbitraryTablePropertiesIT.java  |   7 +-
 .../accumulo/test/BadDeleteMarkersCreatedIT.java   |   2 +-
 .../apache/accumulo/test/BatchWriterIterator.java  |   7 +-
 .../apache/accumulo/test/ConditionalWriterIT.java  |  14 ++-
 .../org/apache/accumulo/test/NamespacesIT.java     |   4 +-
 .../accumulo/test/RewriteTabletDirectoriesIT.java  |   2 +-
 .../java/org/apache/accumulo/test/SampleIT.java    |   2 +-
 .../apache/accumulo/test/TestMultiTableIngest.java |   9 +-
 .../java/org/apache/accumulo/test/UnusedWALIT.java |   4 +-
 .../accumulo/test/functional/AccumuloClientIT.java |  61 +++++-----
 .../test/functional/ConfigurableMacBase.java       |  12 +-
 .../accumulo/test/functional/CredentialsIT.java    |  13 ++-
 .../accumulo/test/functional/KerberosIT.java       |   3 +-
 .../accumulo/test/functional/KerberosProxyIT.java  |   4 +-
 .../accumulo/test/functional/MapReduceIT.java      |   6 +-
 .../accumulo/test/functional/PermissionsIT.java    |   7 +-
 .../accumulo/test/functional/ReadWriteIT.java      |   4 +-
 .../apache/accumulo/test/functional/RestartIT.java |   9 +-
 .../apache/accumulo/test/functional/SummaryIT.java |   4 +-
 .../test/functional/TableChangeStateIT.java        |   9 +-
 .../functional/TabletStateChangeIteratorIT.java    |   6 +-
 .../test/functional/WatchTheWatchCountIT.java      |   2 +-
 .../accumulo/test/functional/ZooCacheIT.java       |   4 +-
 .../test/mapred/AccumuloOutputFormatIT.java        |   3 +-
 .../test/performance/RollWALPerformanceIT.java     |   6 +-
 .../accumulo/test/proxy/SimpleProxyBase.java       |   6 +-
 .../test/replication/KerberosReplicationIT.java    |   6 +-
 .../accumulo/test/replication/ReplicationIT.java   |   5 +-
 .../test/server/security/SystemCredentialsIT.java  |  18 ++-
 68 files changed, 353 insertions(+), 336 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 b74b115..3137db7 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
@@ -24,8 +24,6 @@ import java.util.Properties;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
@@ -174,14 +172,14 @@ public class ClientOpts extends Help {
 
   public ClientInfo getClientInfo() {
     if (cachedInfo == null) {
-      cachedInfo = Accumulo.newClient().from(getClientProperties()).info();
+      cachedInfo = ClientInfo.from(getClientProperties());
     }
     return cachedInfo;
   }
 
-  public AccumuloClient getClient() throws AccumuloException, AccumuloSecurityException {
+  public AccumuloClient getClient() {
     if (cachedAccumuloClient == null) {
-      cachedAccumuloClient = Accumulo.newClient().from(getClientInfo()).build();
+      cachedAccumuloClient = Accumulo.newClient().from(getClientProperties()).build();
     }
     return cachedAccumuloClient;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Accumulo.java b/core/src/main/java/org/apache/accumulo/core/client/Accumulo.java
index e2fff71..331889c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Accumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Accumulo.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.util.Properties;
+
 import org.apache.accumulo.core.client.lexicoder.Lexicoder;
 import org.apache.accumulo.core.client.rfile.RFile;
 import org.apache.accumulo.core.clientImpl.AccumuloClientImpl;
@@ -62,7 +64,24 @@ public final class Accumulo {
    *
    * @return a builder object for Accumulo clients
    */
-  public static AccumuloClient.ClientInfoOptions newClient() {
-    return new AccumuloClientImpl.AccumuloClientBuilderImpl();
+  public static AccumuloClient.PropertyOptions<AccumuloClient> newClient() {
+    return new AccumuloClientImpl.ClientBuilderImpl<>(AccumuloClientImpl.ClientBuilderImpl::buildClient);
+  }
+
+  /**
+   * Fluent entry point for creating client {@link Properties}. For example:
+   *
+   * <pre>
+   * <code>
+   * Properties clientProperties = Accumulo.newClientProperties()
+   *              .to(instanceName, zookeepers)
+   *              .as(user, password).build())
+   * </code>
+   * </pre>
+   *
+   * @return a builder object for client Properties
+   */
+  public static AccumuloClient.PropertyOptions<Properties> newClientProperties() {
+    return new AccumuloClientImpl.ClientBuilderImpl<>(AccumuloClientImpl.ClientBuilderImpl::buildProps);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index d84a0db..efaaac7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -307,21 +307,9 @@ public interface AccumuloClient extends AutoCloseable {
   ReplicationOperations replicationOperations();
 
   /**
-   * @return {@link ClientInfo} which contains information about client connection to Accumulo
+   * @return All {@link Properties} used to create client except 'auth.token'
    */
-  ClientInfo info();
-
-  /**
-   * Change user
-   *
-   * @param principal
-   *          Principal/username
-   * @param token
-   *          Authentication token
-   * @return {@link AccumuloClient} for new user
-   */
-  AccumuloClient changeUser(String principal, AuthenticationToken token)
-      throws AccumuloSecurityException, AccumuloException;
+  Properties properties();
 
   /**
    * Cleans up any resources created by an AccumuloClient like threads and sockets. Anything created
@@ -332,34 +320,18 @@ public interface AccumuloClient extends AutoCloseable {
   void close();
 
   /**
-   * Builds ClientInfo after all options have been specified
+   * Builds AccumuloClient or client Properties after all options have been specified
    *
    * @since 2.0.0
    */
-  interface ClientInfoFactory {
+  interface ClientFactory<T> {
 
     /**
-     * Builds ClientInfo after all options have been specified
+     * Builds AccumuloClient or client Properties
      *
-     * @return ClientInfo
+     * @return AccumuloClient or Properties
      */
-    ClientInfo info();
-  }
-
-  /**
-   * Builds AccumuloClient
-   *
-   * @since 2.0.0
-   */
-  interface AccumuloClientFactory extends ClientInfoFactory {
-
-    /**
-     * Builds AccumuloClient after all options have been specified
-     *
-     * @return AccumuloClient
-     */
-    AccumuloClient build() throws AccumuloException, AccumuloSecurityException;
-
+    T build();
   }
 
   /**
@@ -367,8 +339,8 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface InstanceArgs {
-    AuthenticationArgs to(CharSequence instanceName, CharSequence zookeepers);
+  interface InstanceArgs<T> {
+    AuthenticationArgs<T> to(CharSequence instanceName, CharSequence zookeepers);
   }
 
   /**
@@ -376,7 +348,7 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface PropertyOptions extends InstanceArgs {
+  interface PropertyOptions<T> extends InstanceArgs<T> {
 
     /**
      * Build using properties file. An example properties file can be found at
@@ -386,7 +358,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Path to properties file
      * @return this builder
      */
-    AccumuloClientFactory from(String propertiesFilePath);
+    FromOptions<T> from(String propertiesFilePath);
 
     /**
      * Build using properties file. An example properties file can be found at
@@ -396,7 +368,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Path to properties file
      * @return this builder
      */
-    AccumuloClientFactory from(Path propertiesFile);
+    FromOptions<T> from(Path propertiesFile);
 
     /**
      * Build using Java properties object. A list of available properties can be found in the
@@ -406,22 +378,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Properties object
      * @return this builder
      */
-    AccumuloClientFactory from(Properties properties);
-  }
-
-  /**
-   * @since 2.0.0
-   */
-  interface ClientInfoOptions extends PropertyOptions {
-
-    /**
-     * Build using Accumulo client information
-     *
-     * @param clientInfo
-     *          ClientInfo object
-     * @return this builder
-     */
-    FromOptions from(ClientInfo clientInfo);
+    FromOptions<T> from(Properties properties);
   }
 
   /**
@@ -429,7 +386,7 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface AuthenticationArgs {
+  interface AuthenticationArgs<T> {
 
     /**
      * Build using password-based credentials
@@ -440,7 +397,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Password
      * @return this builder
      */
-    ConnectionOptions as(CharSequence username, CharSequence password);
+    ConnectionOptions<T> as(CharSequence username, CharSequence password);
 
     /**
      * Build using Kerberos credentials
@@ -451,7 +408,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Path to keytab file
      * @return this builder
      */
-    ConnectionOptions as(CharSequence principal, Path keyTabFile);
+    ConnectionOptions<T> as(CharSequence principal, Path keyTabFile);
 
     /**
      * Build using specified credentials
@@ -462,7 +419,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Authentication token
      * @return this builder
      */
-    ConnectionOptions as(CharSequence principal, AuthenticationToken token);
+    ConnectionOptions<T> as(CharSequence principal, AuthenticationToken token);
   }
 
   /**
@@ -470,7 +427,7 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface SslOptions extends AccumuloClientFactory {
+  interface SslOptions<T> extends ClientFactory<T> {
 
     /**
      * Build with SSL trust store
@@ -479,7 +436,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Path to trust store
      * @return this builder
      */
-    SslOptions truststore(CharSequence path);
+    SslOptions<T> truststore(CharSequence path);
 
     /**
      * Build with SSL trust store
@@ -492,7 +449,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Trust store type
      * @return this builder
      */
-    SslOptions truststore(CharSequence path, CharSequence password, CharSequence type);
+    SslOptions<T> truststore(CharSequence path, CharSequence password, CharSequence type);
 
     /**
      * Build with SSL key store
@@ -501,7 +458,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Path to SSL key store
      * @return this builder
      */
-    SslOptions keystore(CharSequence path);
+    SslOptions<T> keystore(CharSequence path);
 
     /**
      * Build with SSL key store
@@ -514,14 +471,14 @@ public interface AccumuloClient extends AutoCloseable {
      *          Key store type
      * @return this builder
      */
-    SslOptions keystore(CharSequence path, CharSequence password, CharSequence type);
+    SslOptions<T> keystore(CharSequence path, CharSequence password, CharSequence type);
 
     /**
      * Use JSSE system properties to configure SSL
      *
      * @return this builder
      */
-    SslOptions useJsse();
+    SslOptions<T> useJsse();
   }
 
   /**
@@ -529,7 +486,7 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface SaslOptions extends AccumuloClientFactory {
+  interface SaslOptions<T> extends ClientFactory<T> {
 
     /**
      * Build with Kerberos Server Primary
@@ -538,7 +495,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Kerberos server primary
      * @return this builder
      */
-    SaslOptions primary(CharSequence kerberosServerPrimary);
+    SaslOptions<T> primary(CharSequence kerberosServerPrimary);
 
     /**
      * Build with SASL quality of protection
@@ -547,7 +504,7 @@ public interface AccumuloClient extends AutoCloseable {
      *          Quality of protection
      * @return this builder
      */
-    SaslOptions qop(CharSequence qualityOfProtection);
+    SaslOptions<T> qop(CharSequence qualityOfProtection);
   }
 
   /**
@@ -555,7 +512,7 @@ public interface AccumuloClient extends AutoCloseable {
    *
    * @since 2.0.0
    */
-  interface ConnectionOptions extends AccumuloClientFactory {
+  interface ConnectionOptions<T> extends ClientFactory<T> {
 
     /**
      * Build using Zookeeper timeout
@@ -564,21 +521,21 @@ public interface AccumuloClient extends AutoCloseable {
      *          Zookeeper timeout (in milliseconds)
      * @return this builder
      */
-    ConnectionOptions zkTimeout(int timeout);
+    ConnectionOptions<T> zkTimeout(int timeout);
 
     /**
      * Build with SSL/TLS options
      *
      * @return this builder
      */
-    SslOptions useSsl();
+    SslOptions<T> useSsl();
 
     /**
      * Build with SASL options
      *
      * @return this builder
      */
-    SaslOptions useSasl();
+    SaslOptions<T> useSasl();
 
     /**
      * Build with BatchWriterConfig defaults for BatchWriter, MultiTableBatchWriter &amp;
@@ -588,23 +545,23 @@ public interface AccumuloClient extends AutoCloseable {
      *          BatchWriterConfig
      * @return this builder
      */
-    ConnectionOptions batchWriterConfig(BatchWriterConfig batchWriterConfig);
+    ConnectionOptions<T> batchWriterConfig(BatchWriterConfig batchWriterConfig);
 
     /**
      * Build with default number of query threads for BatchScanner
      */
-    ConnectionOptions batchScannerQueryThreads(int numQueryThreads);
+    ConnectionOptions<T> batchScannerQueryThreads(int numQueryThreads);
 
     /**
      * Build with default batch size for Scanner
      */
-    ConnectionOptions scannerBatchSize(int batchSize);
+    ConnectionOptions<T> scannerBatchSize(int batchSize);
   }
 
   /**
    * @since 2.0.0
    */
-  interface FromOptions extends ConnectionOptions, PropertyOptions, AuthenticationArgs {
+  interface FromOptions<T> extends ConnectionOptions<T>, AuthenticationArgs<T> {
 
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index d3dc64b..9aa75aa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -278,7 +278,7 @@ public abstract class Connector {
    *
    * @since 2.0
    */
-  public static Connector from(AccumuloClient client) {
+  public static Connector from(AccumuloClient client) throws AccumuloSecurityException, AccumuloException {
     return new ConnectorImpl((AccumuloClientImpl) client);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index d25c22f..e1546f4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -181,8 +181,8 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
     if (token instanceof KerberosToken) {
       log.info("Received KerberosToken, attempting to fetch DelegationToken");
       try {
-        AccumuloClient client = Accumulo.newClient().from(getClientInfo(job)).as(principal, token)
-            .build();
+        AccumuloClient client = Accumulo.newClient().from(getClientInfo(job).getProperties())
+            .as(principal, token).build();
         token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
       } catch (Exception e) {
         log.warn("Failed to automatically obtain DelegationToken, Mappers/Reducers will likely"
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index ef3441a..82cf95f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -427,7 +427,7 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(job)).build();
+        this.client = Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
         mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(job));
       }
     }
@@ -564,7 +564,7 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
       throw new IOException("Connector info has not been set.");
     try {
       // if the instance isn't configured, it will complain here
-      AccumuloClient c = Accumulo.newClient().from(getClientInfo(job)).build();
+      AccumuloClient c = Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
       String principal = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
       if (!c.securityOperations().authenticateUser(principal, token))
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 7e27774..235ac32 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -183,8 +183,8 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     if (token instanceof KerberosToken) {
       log.info("Received KerberosToken, attempting to fetch DelegationToken");
       try {
-        AccumuloClient client = Accumulo.newClient().from(getClientInfo(job)).as(principal, token)
-            .build();
+        AccumuloClient client = Accumulo.newClient().from(getClientInfo(job).getProperties())
+            .as(principal, token).build();
         token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
       } catch (Exception e) {
         log.warn("Failed to automatically obtain DelegationToken, "
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 34e0d7e..3572acb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -430,7 +430,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(context)).build();
+        this.client = Accumulo.newClient().from(getClientInfo(context).getProperties()).build();
         mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(context));
       }
     }
@@ -569,7 +569,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
       // if the instance isn't configured, it will complain here
       String principal = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
-      AccumuloClient c = Accumulo.newClient().from(getClientInfo(job)).build();
+      AccumuloClient c = Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException | AccumuloSecurityException e) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
index 6cac797..fc9cfde 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
@@ -22,8 +22,8 @@ import java.nio.file.Path;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 
-import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -71,19 +71,30 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
   }
 
-  public AccumuloClientImpl(SingletonReservation reservation, final ClientContext context)
-      throws AccumuloSecurityException, AccumuloException {
+  public AccumuloClientImpl(SingletonReservation reservation, final ClientContext context) {
     checkArgument(context != null, "Context is null");
     checkArgument(context.getCredentials() != null, "Credentials are null");
     checkArgument(context.getCredentials().getToken() != null, "Authentication token is null");
-    if (context.getCredentials().getToken().isDestroyed())
-      throw new AccumuloSecurityException(context.getCredentials().getPrincipal(),
-          SecurityErrorCode.TOKEN_EXPIRED);
 
     this.singletonReservation = Objects.requireNonNull(reservation);
     this.context = context;
     instanceID = context.getInstanceID();
+    this.tableops = new TableOperationsImpl(context);
+    this.namespaceops = new NamespaceOperationsImpl(context, tableops);
+  }
+
+  Table.ID getTableId(String tableName) throws TableNotFoundException {
+    Table.ID tableId = Tables.getTableId(context, tableName);
+    if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
+      throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
+    return tableId;
+  }
 
+
+  void authenticate() throws AccumuloSecurityException, AccumuloException {
+    if (context.getCredentials().getToken().isDestroyed())
+      throw new AccumuloSecurityException(context.getCredentials().getPrincipal(),
+          SecurityErrorCode.TOKEN_EXPIRED);
     // Skip fail fast for system services; string literal for class name, to avoid dependency on
     // server jar
     final String tokenClassName = context.getCredentials().getToken().getClass().getName();
@@ -94,16 +105,6 @@ public class AccumuloClientImpl implements AccumuloClient {
               SecurityErrorCode.BAD_CREDENTIALS);
       });
     }
-
-    this.tableops = new TableOperationsImpl(context);
-    this.namespaceops = new NamespaceOperationsImpl(context, tableops);
-  }
-
-  Table.ID getTableId(String tableName) throws TableNotFoundException {
-    Table.ID tableId = Tables.getTableId(context, tableName);
-    if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
-      throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
-    return tableId;
   }
 
   @Override
@@ -261,16 +262,20 @@ public class AccumuloClientImpl implements AccumuloClient {
   }
 
   @Override
-  public ClientInfo info() {
+  public Properties properties() {
     ensureOpen();
-    return this.context.getClientInfo();
+    Properties result = new Properties();
+    this.context.getProperties().forEach((key, value) -> {
+      if (!key.equals(ClientProperty.AUTH_TOKEN.getKey())) {
+        result.setProperty((String) key, (String) value);
+      }
+    });
+    return result;
   }
 
-  @Override
-  public AccumuloClient changeUser(String principal, AuthenticationToken token)
-      throws AccumuloSecurityException, AccumuloException {
+  public AuthenticationToken token() {
     ensureOpen();
-    return Accumulo.newClient().from(info()).as(principal, token).build();
+    return this.context.getAuthenticationToken();
   }
 
   @Override
@@ -283,12 +288,17 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
   }
 
-  public static class AccumuloClientBuilderImpl
-      implements InstanceArgs, PropertyOptions, ClientInfoOptions, AuthenticationArgs,
-      ConnectionOptions, SslOptions, SaslOptions, AccumuloClientFactory, FromOptions {
+  public static class ClientBuilderImpl<T>
+      implements InstanceArgs<T>, PropertyOptions<T>, AuthenticationArgs<T>, ConnectionOptions<T>,
+      SslOptions<T>, SaslOptions<T>, ClientFactory<T>, FromOptions<T> {
 
     private Properties properties = new Properties();
     private AuthenticationToken token = null;
+    private Function<ClientBuilderImpl, T> builderFunction;
+
+    public ClientBuilderImpl(Function<ClientBuilderImpl, T> builderFunction) {
+      this.builderFunction = builderFunction;
+    }
 
     private ClientInfo getClientInfo() {
       if (token != null) {
@@ -298,36 +308,40 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
 
     @Override
-    public AccumuloClient build() throws AccumuloException, AccumuloSecurityException {
+    public T build() {
+      return builderFunction.apply(this);
+    }
+
+    public static AccumuloClient buildClient(ClientBuilderImpl<AccumuloClient> cbi) {
       SingletonReservation reservation = SingletonManager.getClientReservation();
       try {
-        return new AccumuloClientImpl(reservation, new ClientContext(getClientInfo()));
-      } catch (AccumuloException | AccumuloSecurityException | RuntimeException e) {
+        // AccumuloClientImpl closes reservation unless a RuntimeException is thrown
+        return new AccumuloClientImpl(reservation, new ClientContext(cbi.getClientInfo()));
+      } catch (RuntimeException e) {
         reservation.close();
         throw e;
       }
     }
 
-    @Override
-    public ClientInfo info() {
-      return getClientInfo();
+    public static Properties buildProps(ClientBuilderImpl<Properties> cbi) {
+      return cbi.properties;
     }
 
     @Override
-    public AuthenticationArgs to(CharSequence instanceName, CharSequence zookeepers) {
+    public AuthenticationArgs<T> to(CharSequence instanceName, CharSequence zookeepers) {
       setProperty(ClientProperty.INSTANCE_NAME, instanceName);
       setProperty(ClientProperty.INSTANCE_ZOOKEEPERS, zookeepers);
       return this;
     }
 
     @Override
-    public SslOptions truststore(CharSequence path) {
+    public SslOptions<T> truststore(CharSequence path) {
       setProperty(ClientProperty.SSL_TRUSTSTORE_PATH, path);
       return this;
     }
 
     @Override
-    public SslOptions truststore(CharSequence path, CharSequence password, CharSequence type) {
+    public SslOptions<T> truststore(CharSequence path, CharSequence password, CharSequence type) {
       setProperty(ClientProperty.SSL_TRUSTSTORE_PATH, path);
       setProperty(ClientProperty.SSL_TRUSTSTORE_PASSWORD, password);
       setProperty(ClientProperty.SSL_TRUSTSTORE_TYPE, type);
@@ -335,13 +349,13 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
 
     @Override
-    public SslOptions keystore(CharSequence path) {
+    public SslOptions<T> keystore(CharSequence path) {
       setProperty(ClientProperty.SSL_KEYSTORE_PATH, path);
       return this;
     }
 
     @Override
-    public SslOptions keystore(CharSequence path, CharSequence password, CharSequence type) {
+    public SslOptions<T> keystore(CharSequence path, CharSequence password, CharSequence type) {
       setProperty(ClientProperty.SSL_KEYSTORE_PATH, path);
       setProperty(ClientProperty.SSL_KEYSTORE_PASSWORD, password);
       setProperty(ClientProperty.SSL_KEYSTORE_TYPE, type);
@@ -349,31 +363,31 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
 
     @Override
-    public SslOptions useJsse() {
+    public SslOptions<T> useJsse() {
       setProperty(ClientProperty.SSL_USE_JSSE, "true");
       return this;
     }
 
     @Override
-    public ConnectionOptions zkTimeout(int timeout) {
+    public ConnectionOptions<T> zkTimeout(int timeout) {
       ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.setTimeInMillis(properties, (long) timeout);
       return this;
     }
 
     @Override
-    public SslOptions useSsl() {
+    public SslOptions<T> useSsl() {
       setProperty(ClientProperty.SSL_ENABLED, "true");
       return this;
     }
 
     @Override
-    public SaslOptions useSasl() {
+    public SaslOptions<T> useSasl() {
       setProperty(ClientProperty.SASL_ENABLED, "true");
       return this;
     }
 
     @Override
-    public ConnectionOptions batchWriterConfig(BatchWriterConfig batchWriterConfig) {
+    public ConnectionOptions<T> batchWriterConfig(BatchWriterConfig batchWriterConfig) {
       ClientProperty.BATCH_WRITER_MEMORY_MAX.setBytes(properties, batchWriterConfig.getMaxMemory());
       ClientProperty.BATCH_WRITER_LATENCY_MAX.setTimeInMillis(properties,
           batchWriterConfig.getMaxLatency(TimeUnit.MILLISECONDS));
@@ -386,72 +400,70 @@ public class AccumuloClientImpl implements AccumuloClient {
     }
 
     @Override
-    public ConnectionOptions batchScannerQueryThreads(int numQueryThreads) {
+    public ConnectionOptions<T> batchScannerQueryThreads(int numQueryThreads) {
       setProperty(ClientProperty.BATCH_SCANNER_NUM_QUERY_THREADS, numQueryThreads);
       return this;
     }
 
     @Override
-    public ConnectionOptions scannerBatchSize(int batchSize) {
+    public ConnectionOptions<T> scannerBatchSize(int batchSize) {
       setProperty(ClientProperty.SCANNER_BATCH_SIZE, batchSize);
       return this;
     }
 
     @Override
-    public SaslOptions primary(CharSequence kerberosServerPrimary) {
+    public SaslOptions<T> primary(CharSequence kerberosServerPrimary) {
       setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY, kerberosServerPrimary);
       return this;
     }
 
     @Override
-    public SaslOptions qop(CharSequence qualityOfProtection) {
+    public SaslOptions<T> qop(CharSequence qualityOfProtection) {
       setProperty(ClientProperty.SASL_QOP, qualityOfProtection);
       return this;
     }
 
     @Override
-    public AccumuloClientFactory from(String propertiesFilePath) {
+    public FromOptions<T> from(String propertiesFilePath) {
       return from(ClientInfoImpl.toProperties(propertiesFilePath));
     }
 
     @Override
-    public AccumuloClientFactory from(Path propertiesFile) {
+    public FromOptions<T> from(Path propertiesFile) {
       return from(ClientInfoImpl.toProperties(propertiesFile));
     }
 
     @Override
-    public AccumuloClientFactory from(Properties properties) {
+    public FromOptions<T> from(Properties properties) {
       this.properties = properties;
       return this;
     }
 
     @Override
-    public ConnectionOptions as(CharSequence username, CharSequence password) {
+    public ConnectionOptions<T> as(CharSequence username, CharSequence password) {
       setProperty(ClientProperty.AUTH_PRINCIPAL, username);
       ClientProperty.setPassword(properties, password);
       return this;
     }
 
     @Override
-    public ConnectionOptions as(CharSequence principal, Path keyTabFile) {
+    public ConnectionOptions<T> as(CharSequence principal, Path keyTabFile) {
       setProperty(ClientProperty.AUTH_PRINCIPAL, principal);
       ClientProperty.setKerberosKeytab(properties, keyTabFile.toString());
       return this;
     }
 
     @Override
-    public ConnectionOptions as(CharSequence principal, AuthenticationToken token) {
+    public ConnectionOptions<T> as(CharSequence principal, AuthenticationToken token) {
+      if (token.isDestroyed()) {
+        throw new IllegalArgumentException("AuthenticationToken has been destroyed");
+      }
       setProperty(ClientProperty.AUTH_PRINCIPAL, principal.toString());
+      ClientProperty.setAuthenticationToken(properties, token);
       this.token = token;
       return this;
     }
 
-    @Override
-    public FromOptions from(ClientInfo clientInfo) {
-      this.properties = clientInfo.getProperties();
-      return this;
-    }
-
     public void setProperty(ClientProperty property, CharSequence value) {
       properties.setProperty(property.getKey(), value.toString());
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index e41e81e..73f380e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -92,6 +93,10 @@ public class ClientContext {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, TimeUnit.MILLISECONDS).get();
   }
 
+  public ClientContext(AccumuloClient client) {
+    this(ClientInfo.from(client.properties(), ((AccumuloClientImpl) client).token()));
+  }
+
   public ClientContext(ClientInfo info) {
     this(info, ClientConfConverter.toAccumuloConf(info.getProperties()));
   }
@@ -150,8 +155,9 @@ public class ClientContext {
       @Override
       public org.apache.accumulo.core.client.Connector getConnector(String principal,
           AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-        return org.apache.accumulo.core.client.Connector
-            .from(context.getClient().changeUser(principal, token));
+        AccumuloClient client = Accumulo.newClient().from(context.getProperties())
+            .as(principal, token).build();
+        return org.apache.accumulo.core.client.Connector.from(client);
       }
     };
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
index 748e52c..03f60c6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
@@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -47,9 +49,10 @@ public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
 
   private final AccumuloClientImpl impl;
 
-  public ConnectorImpl(AccumuloClientImpl impl) {
+  public ConnectorImpl(AccumuloClientImpl impl) throws AccumuloSecurityException, AccumuloException {
     this.impl = impl;
     SingletonManager.setMode(Mode.CONNECTOR);
+    impl.authenticate();
   }
 
   public AccumuloClientImpl getAccumuloClient() {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
index a091cd9..0b77d59 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
@@ -119,10 +119,11 @@ public class ConfiguratorBase {
     if (info.getAuthenticationToken() instanceof KerberosToken) {
       log.info("Received KerberosToken, attempting to fetch DelegationToken");
       try {
-        AccumuloClient client = Accumulo.newClient().from(info).build();
+        AccumuloClient client = Accumulo.newClient().from(info.getProperties()).build();
         AuthenticationToken token = client.securityOperations()
             .getDelegationToken(new DelegationTokenConfig());
-        result = Accumulo.newClient().from(info).as(info.getPrincipal(), token).info();
+        result = ClientInfo.from(Accumulo.newClientProperties().from(info.getProperties())
+            .as(info.getPrincipal(), token).build());
       } catch (Exception e) {
         log.warn("Failed to automatically obtain DelegationToken, "
             + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
@@ -336,8 +337,12 @@ public class ConfiguratorBase {
   @Deprecated
   public static org.apache.accumulo.core.client.Instance getInstance(Class<?> implementingClass,
       Configuration conf) {
-    return org.apache.accumulo.core.client.Connector.from(getClient(implementingClass, conf))
-        .getInstance();
+    try {
+      return org.apache.accumulo.core.client.Connector.from(getClient(implementingClass, conf))
+          .getInstance();
+    } catch (AccumuloSecurityException|AccumuloException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   /**
@@ -351,11 +356,7 @@ public class ConfiguratorBase {
    * @since 2.0.0
    */
   public static AccumuloClient getClient(Class<?> implementingClass, Configuration conf) {
-    try {
-      return Accumulo.newClient().from(getClientInfo(implementingClass, conf)).build();
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IllegalStateException(e);
-    }
+    return Accumulo.newClient().from(getClientProperties(implementingClass, conf)).build();
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
index e5f98cf..27c6022 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
@@ -61,7 +61,7 @@ public class MapReduceClientOpts extends ClientOpts {
         log.info("Obtaining delegation token for {}", newPrincipal);
 
         setPrincipal(newPrincipal);
-        AccumuloClient client = Accumulo.newClient().from(getClientInfo())
+        AccumuloClient client = Accumulo.newClient().from(getClientProperties())
             .as(newPrincipal, krbToken).build();
 
         // Do the explicit check to see if the user has the permission to get a delegation token
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
index b8500b0..9d69f24 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
@@ -91,7 +91,7 @@ public class ReplicationTable {
   }
 
   public static boolean isOnline(AccumuloClient client) {
-    return TableState.ONLINE == Tables.getTableState(new ClientContext(client.info()), ID);
+    return TableState.ONLINE == Tables.getTableState(new ClientContext(client), ID);
   }
 
   public static void setOnline(AccumuloClient client)
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index 68aa3d2..64400df 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -221,7 +221,7 @@ public class Merge {
     Table.ID tableId;
     Scanner scanner;
     try {
-      ClientContext context = new ClientContext(client.info());
+      ClientContext context = new ClientContext(client);
       tableId = Tables.getTableId(context, tablename);
       scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     } catch (Exception e) {
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
index 39d8fe9..6f65a6f 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
@@ -79,8 +79,8 @@ public class ConfiguratorBaseTest {
   @Test
   public void testSetClientInfo() {
     Configuration conf = new Configuration();
-    ClientInfo info = Accumulo.newClient().to("myinstance", "myzookeepers").as("user", "pass")
-        .info();
+    ClientInfo info = ClientInfo.from(
+        Accumulo.newClientProperties().to("myinstance", "myzookeepers").as("user", "pass").build());
     ConfiguratorBase.setClientInfo(this.getClass(), conf, info);
     ClientInfo info2 = ConfiguratorBase.getClientInfo(this.getClass(), conf);
     assertEquals("myinstance", info2.getInstanceName());
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
index 47864fa..86892f0 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormat.java
@@ -51,7 +51,7 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
       ClientInfo clientInfo = getClientInfo(job);
       String principal = clientInfo.getPrincipal();
       AuthenticationToken token = clientInfo.getAuthenticationToken();
-      AccumuloClient c = Accumulo.newClient().from(clientInfo).build();
+      AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException | AccumuloSecurityException e) {
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
index 565de0e..e6d4204 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormat.java
@@ -62,7 +62,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
       ClientInfo clientInfo = getClientInfo(job);
       String principal = clientInfo.getPrincipal();
       AuthenticationToken token = clientInfo.getAuthenticationToken();
-      AccumuloClient c = Accumulo.newClient().from(clientInfo).build();
+      AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
 
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
index 49890ff..2b6942d 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloOutputFormatImpl.java
@@ -219,7 +219,7 @@ public class AccumuloOutputFormatImpl {
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(job)).build();
+        this.client = Accumulo.newClient().from(getClientInfo(job).getProperties()).build();
         mtbw = client.createMultiTableBatchWriter();
       }
     }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
index 696f7f3..50b8925 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloOutputFormatImpl.java
@@ -222,7 +222,7 @@ public class AccumuloOutputFormatImpl {
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientInfo(context)).build();
+        this.client = Accumulo.newClient().from(getClientInfo(context).getProperties()).build();
         mtbw = client.createMultiTableBatchWriter();
       }
     }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
index e6b3a51..6350c9e 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/InputFormatBuilderImpl.java
@@ -245,7 +245,7 @@ public class InputFormatBuilderImpl<T>
       throws AccumuloSecurityException, AccumuloException {
     if (scanAuths != null)
       return scanAuths;
-    AccumuloClient c = Accumulo.newClient().from(clientInfo).build();
+    AccumuloClient c = Accumulo.newClient().from(clientInfo.getProperties()).build();
     return c.securityOperations().getUserAuthorizations(clientInfo.getPrincipal());
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
index 36544bd..54bdb58 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
@@ -32,8 +32,6 @@ import java.util.Scanner;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -118,10 +116,11 @@ public class ConfiguratorBase {
     if (info.getAuthenticationToken() instanceof KerberosToken) {
       log.info("Received KerberosToken, attempting to fetch DelegationToken");
       try {
-        AccumuloClient client = Accumulo.newClient().from(info).build();
+        AccumuloClient client = Accumulo.newClient().from(info.getProperties()).build();
         AuthenticationToken token = client.securityOperations()
             .getDelegationToken(new DelegationTokenConfig());
-        result = Accumulo.newClient().from(info).as(info.getPrincipal(), token).info();
+        result = ClientInfo.from(Accumulo.newClientProperties().from(info.getProperties())
+            .as(info.getPrincipal(), token).build());
       } catch (Exception e) {
         log.warn("Failed to automatically obtain DelegationToken, "
             + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
@@ -306,11 +305,7 @@ public class ConfiguratorBase {
    * @since 2.0.0
    */
   public static AccumuloClient getClient(Class<?> implementingClass, Configuration conf) {
-    try {
-      return Accumulo.newClient().from(getClientInfo(implementingClass, conf)).build();
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IllegalStateException(e);
-    }
+    return Accumulo.newClient().from(getClientProperties(implementingClass, conf)).build();
   }
 
   /**
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
index 96a516c..4eb339b 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOpts.java
@@ -60,7 +60,7 @@ public abstract class MapReduceClientOpts extends ClientOpts {
         log.info("Obtaining delegation token for {}", newPrincipal);
 
         setPrincipal(newPrincipal);
-        AccumuloClient client = Accumulo.newClient().from(getClientInfo())
+        AccumuloClient client = Accumulo.newClient().from(getClientProperties())
             .as(newPrincipal, krbToken).build();
 
         // Do the explicit check to see if the user has the permission to get a delegation token
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
index 973ebc3..98eadd4 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
@@ -168,7 +168,8 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
-      ClientInfo info = Accumulo.newClient().to(instanceName, zooKeepers).as(user, pass).info();
+      ClientInfo info = ClientInfo
+          .from(Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass).build());
 
       AccumuloInputFormat.configure().clientInfo(info).table(table1).auths(Authorizations.EMPTY)
           .store(job);
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 75c3efb..801bb81 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
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
@@ -93,9 +94,10 @@ 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",
-        c.info().getInstanceName(), "-z", c.info().getZooKeepers(), "-u", "root", "-p",
-        ROOT_PASSWORD, "-t", tablename, "--column", input_cfcq);
+        info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "-t",
+        tablename, "--column", input_cfcq);
     assertEquals(0, hash.waitFor());
 
     try (Scanner s = c.createScanner(tablename, Authorizations.EMPTY)) {
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
index 0f9f777..46d86e7 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapred/AccumuloOutputFormatTest.java
@@ -23,8 +23,8 @@ import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.JobConf;
@@ -35,9 +35,6 @@ public class AccumuloOutputFormatTest {
   public void testBWSettings() throws IOException {
     JobConf job = new JobConf();
 
-    AccumuloClient.ConnectionOptions opts = Accumulo.newClient().to("test", "zk").as("blah",
-        "blah");
-
     // make sure we aren't testing defaults
     final BatchWriterConfig bwDefaults = new BatchWriterConfig();
     assertNotEquals(7654321L, bwDefaults.getMaxLatency(TimeUnit.MILLISECONDS));
@@ -50,8 +47,10 @@ public class AccumuloOutputFormatTest {
     bwConfig.setTimeout(9898989L, TimeUnit.MILLISECONDS);
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321L);
-    opts.batchWriterConfig(bwConfig);
-    AccumuloOutputFormat.configure().clientInfo(opts.info()).store(job);
+
+    ClientInfo info = ClientInfo.from(Accumulo.newClientProperties().to("test", "zk")
+        .as("blah", "blah").batchWriterConfig(bwConfig).build());
+    AccumuloOutputFormat.configure().clientInfo(info).store(job);
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
index 52aace9..f841183 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/mapreduce/AccumuloOutputFormatTest.java
@@ -23,8 +23,8 @@ import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoopImpl.mapreduce.lib.OutputConfigurator;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -36,9 +36,6 @@ public class AccumuloOutputFormatTest {
   public void testBWSettings() throws IOException {
     Job job = Job.getInstance();
 
-    AccumuloClient.ConnectionOptions opts = Accumulo.newClient().to("test", "zk").as("blah",
-        "blah");
-
     // make sure we aren't testing defaults
     final BatchWriterConfig bwDefaults = new BatchWriterConfig();
     assertNotEquals(7654321L, bwDefaults.getMaxLatency(TimeUnit.MILLISECONDS));
@@ -51,8 +48,11 @@ public class AccumuloOutputFormatTest {
     bwConfig.setTimeout(9898989L, TimeUnit.MILLISECONDS);
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321L);
-    opts.batchWriterConfig(bwConfig);
-    AccumuloOutputFormat.configure().clientInfo(opts.info()).store(job);
+
+    ClientInfo info = ClientInfo.from(Accumulo.newClientProperties().to("test", "zk")
+        .as("blah", "blah").batchWriterConfig(bwConfig).build());
+
+    AccumuloOutputFormat.configure().clientInfo(info).store(job);
 
     AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
       @Override
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
index 3dcf05a..2c16354 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBaseTest.java
@@ -74,8 +74,8 @@ public class ConfiguratorBaseTest {
   @Test
   public void testSetClientInfo() {
     Configuration conf = new Configuration();
-    ClientInfo info = Accumulo.newClient().to("myinstance", "myzookeepers").as("user", "pass")
-        .info();
+    ClientInfo info = ClientInfo.from(
+        Accumulo.newClientProperties().to("myinstance", "myzookeepers").as("user", "pass").build());
     ConfiguratorBase.setClientInfo(this.getClass(), conf, info);
     ClientInfo info2 = ConfiguratorBase.getClientInfo(this.getClass(), conf);
     assertEquals("myinstance", info2.getInstanceName());
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 353897e..fed0ce1 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
@@ -28,8 +28,6 @@ import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientConfConverter;
@@ -140,8 +138,7 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
   }
 
   @Override
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
+  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
     return Accumulo.newClient().to(getInstanceName(), getZooKeepers()).as(user, token).build();
   }
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 775ea55..72a6955 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -131,8 +131,7 @@ public class MiniAccumuloCluster {
    *
    * @since 2.0.0
    */
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
+  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
     return impl.getAccumuloClient(user, token);
   }
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 5483c01..6ac09f8 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -781,9 +781,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   }
 
   @Override
-  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
-    return Accumulo.newClient().from(getClientInfo()).as(user, token).build();
+  public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
+    return Accumulo.newClient().from(getClientInfo().getProperties()).as(user, token).build();
   }
 
   @SuppressWarnings("deprecation")
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index f18a68a..e1fdfba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -245,9 +245,8 @@ public class ServerContext extends ClientContext {
     return client;
   }
 
-  public AccumuloClient getClient(String principal, AuthenticationToken token)
-      throws AccumuloSecurityException, AccumuloException {
-    return Accumulo.newClient().from(info).as(principal, token).build();
+  public AccumuloClient getClient(String principal, AuthenticationToken token) {
+    return Accumulo.newClient().from(info.getProperties()).as(principal, token).build();
   }
 
   public synchronized TableManager getTableManager() {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
index 4c9f3cc..b03765a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
@@ -216,7 +216,7 @@ public class TableDiskUsage {
       }
     }
 
-    ClientContext context = new ClientContext(client.info());
+    ClientContext context = new ClientContext(client);
     Map<Table.ID,String> reverseTableIdMap = Tables.getIdToNameMap(context);
 
     TreeMap<TreeSet<String>,Long> usage = new TreeMap<>((o1, o2) -> {
@@ -271,7 +271,7 @@ public class TableDiskUsage {
       throws TableNotFoundException, IOException {
 
     HashSet<Table.ID> tableIds = new HashSet<>();
-    ClientContext context = new ClientContext(client.info());
+    ClientContext context = new ClientContext(client);
 
     // Get table IDs for all tables requested to be 'du'
     for (String tableName : tableNames) {
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 d98bcdb..68941bc 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
@@ -74,7 +74,7 @@ public class VerifyTabletAssignments {
     opts.parseArgs(VerifyTabletAssignments.class.getName(), args);
 
     AccumuloClient client = opts.getClient();
-    ClientContext context = new ClientContext(client.info());
+    ClientContext context = new ClientContext(client);
     for (String table : client.tableOperations().list())
       checkTable(context, opts, table, null);
 
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
index fd4c130..45e630d 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
@@ -273,7 +273,7 @@ public class TraceServer implements Watcher {
           at = token;
         }
 
-        accumuloClient = Accumulo.newClient().from(context.getClientInfo()).as(principal, at)
+        accumuloClient = Accumulo.newClient().from(context.getProperties()).as(principal, at)
             .build();
         if (!accumuloClient.tableOperations().exists(tableName)) {
           accumuloClient.tableOperations().create(tableName);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 5e27864..c141ce2 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -337,12 +337,11 @@ public class Shell extends ShellOptions implements KeywordExecutable {
           token = new PasswordToken(password);
         }
       }
-      ClientInfo info = ClientInfo.from(props);
       try {
         DistributedTrace.enable(InetAddress.getLocalHost().getHostName(), "shell", properties);
         this.setTableName("");
-        accumuloClient = Accumulo.newClient().from(info).as(principal, token).build();
-        context = new ClientContext(accumuloClient.info());
+        accumuloClient = Accumulo.newClient().from(props).as(principal, token).build();
+        context = new ClientContext(accumuloClient);
       } catch (Exception e) {
         printException(e);
         exitCode = 1;
@@ -614,8 +613,9 @@ public class Shell extends ShellOptions implements KeywordExecutable {
   }
 
   public void printInfo() throws IOException {
+    ClientInfo info = ClientInfo.from(accumuloClient.properties());
     reader.print("\n" + SHELL_DESCRIPTION + "\n" + "- \n" + "- version: " + Constants.VERSION + "\n"
-        + "- instance name: " + accumuloClient.info().getInstanceName() + "\n" + "- instance id: "
+        + "- instance name: " + info.getInstanceName() + "\n" + "- instance id: "
         + accumuloClient.getInstanceID() + "\n" + "- \n"
         + "- type 'help' for a list of available commands\n" + "- \n");
     reader.flush();
@@ -654,8 +654,8 @@ public class Shell extends ShellOptions implements KeywordExecutable {
 
   public String getDefaultPrompt() {
     Objects.nonNull(accumuloClient);
-    Objects.nonNull(accumuloClient.info());
-    return accumuloClient.whoami() + "@" + accumuloClient.info().getInstanceName()
+    ClientInfo info = ClientInfo.from(accumuloClient.properties());
+    return accumuloClient.whoami() + "@" + info.getInstanceName()
         + (getTableName().isEmpty() ? "" : " ") + getTableName() + "> ";
   }
 
@@ -1169,8 +1169,10 @@ public class Shell extends ShellOptions implements KeywordExecutable {
 
   public void updateUser(String principal, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
-    accumuloClient = accumuloClient.changeUser(principal, token);
-    context = new ClientContext(accumuloClient.info());
+    accumuloClient = Accumulo.newClient().from(accumuloClient.properties()).as(principal, token)
+        .build();
+    accumuloClient.securityOperations().authenticateUser(principal, token);
+    context = new ClientContext(accumuloClient);
   }
 
   public ClientContext getContext() {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateUserCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateUserCommand.java
index 182ebfe..b6a21a3 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CreateUserCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CreateUserCommand.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 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.AccumuloClientImpl;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -37,7 +38,7 @@ public class CreateUserCommand extends Command {
       TableExistsException, IOException {
     final String user = cl.getArgs()[0];
 
-    AuthenticationToken userToken = shellState.getAccumuloClient().info().getAuthenticationToken();
+    AuthenticationToken userToken = ((AccumuloClientImpl) shellState.getAccumuloClient()).token();
     PasswordToken passwordToken;
     if (userToken instanceof KerberosToken) {
       passwordToken = new PasswordToken();
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java
index 9af9f3d..51da57a 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java
@@ -39,6 +39,8 @@ import javax.script.ScriptException;
 import javax.script.SimpleScriptContext;
 
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -182,7 +184,11 @@ public class ScriptCommand extends Command {
 
   @SuppressWarnings("deprecation")
   private void putConnector(Bindings b, AccumuloClient client) {
-    b.put("connection", org.apache.accumulo.core.client.Connector.from(client));
+    try {
+      b.put("connection", org.apache.accumulo.core.client.Connector.from(client));
+    } catch (AccumuloSecurityException|AccumuloException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Override
diff --git a/test/src/main/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java b/test/src/main/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
index d228c5f..15fa6aa 100644
--- a/test/src/main/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
+++ b/test/src/main/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
@@ -104,8 +104,8 @@ public class StandaloneAccumuloClusterConfiguration extends AccumuloClusterPrope
 
     this.conf = getConfiguration(type);
     this.clientPropsFile = clientPropsFile;
-    clientInfo = Accumulo.newClient().to(getInstanceName(), getZooKeepers())
-        .as(getAdminPrincipal(), getAdminToken()).info();
+    clientInfo = ClientInfo.from(Accumulo.newClientProperties()
+        .to(getInstanceName(), getZooKeepers()).as(getAdminPrincipal(), getAdminToken()).build());
 
     // The user Accumulo is running as
     serverUser = conf.get(ACCUMULO_STANDALONE_SERVER_USER);
diff --git a/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java b/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
index 49aaf42..3160b07 100644
--- a/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ArbitraryTablePropertiesIT.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -134,7 +135,8 @@ public class ArbitraryTablePropertiesIT extends SharedMiniClusterBase {
       assertTrue(Property.isValidPropertyKey(propertyName));
 
       // Getting a fresh token will ensure we're logged in as this user (if necessary)
-      try (AccumuloClient testclient = c.changeUser(testUser, user.getToken())) {
+      try (AccumuloClient testclient = Accumulo.newClient().from(c.properties())
+          .as(testUser, user.getToken()).build()) {
         // Set the property to the desired value
         testclient.tableOperations().setProperty(tableName, propertyName, description1);
 
@@ -203,7 +205,8 @@ public class ArbitraryTablePropertiesIT extends SharedMiniClusterBase {
       assertTrue(Property.isValidPropertyKey(propertyName));
 
       // Getting a fresh token will ensure we're logged in as this user (if necessary)
-      try (AccumuloClient testclient = c.changeUser(testUser, user.getToken())) {
+      try (AccumuloClient testclient = Accumulo.newClient().from(c.properties())
+          .as(testUser, user.getToken()).build()) {
 
         // Try to set the property to the desired value.
         // If able to set it, the test fails, since permission was never granted
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index e2d70e8..b442b70 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -97,7 +97,7 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
     getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
 
     try (AccumuloClient client = getAccumuloClient()) {
-      ClientInfo info = client.info();
+      ClientInfo info = ClientInfo.from(client.properties());
       ZooCache zcache = new ZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       zcache.clear();
       String path = ZooUtil.getRoot(client.getInstanceID()) + Constants.ZGC_LOCK;
diff --git a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
index 15d5044..7424160 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -100,10 +101,10 @@ public class BatchWriterIterator extends WrappingIterator {
       long batchWriterTimeout, long batchWriterMaxMemory, int numEntriesToWrite, String tableName,
       AccumuloClient accumuloClient, AuthenticationToken token, boolean clearCacheAfterFirstWrite,
       boolean splitAfterFirstWrite) {
+    ClientInfo info = ClientInfo.from(accumuloClient.properties());
     return iteratorSetting(priority, sleepAfterFirstWrite, batchWriterTimeout, batchWriterMaxMemory,
-        numEntriesToWrite, tableName, accumuloClient.info().getZooKeepers(),
-        accumuloClient.info().getInstanceName(),
-        accumuloClient.info().getZooKeepersSessionTimeOut(), accumuloClient.whoami(), token,
+        numEntriesToWrite, tableName, info.getZooKeepers(), info.getInstanceName(),
+        info.getZooKeepersSessionTimeOut(), accumuloClient.whoami(), token,
         clearCacheAfterFirstWrite, splitAfterFirstWrite);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
index 99c08f1..a74e7de 100644
--- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -47,6 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -261,7 +262,8 @@ public class ConditionalWriterIT extends AccumuloClusterHarness {
       client1.securityOperations().changeUserAuthorizations(user, auths);
       client1.securityOperations().grantSystemPermission(user, SystemPermission.CREATE_TABLE);
 
-      try (AccumuloClient client2 = client1.changeUser(user, user1.getToken())) {
+      try (AccumuloClient client2 = Accumulo.newClient().from(client1.properties())
+          .as(user, user1.getToken()).build()) {
         client2.tableOperations().create(tableName);
 
         try (
@@ -1323,12 +1325,14 @@ public class ConditionalWriterIT extends AccumuloClusterHarness {
       cm1.put("tx", "seq", "1");
       cm1.put("data", "x", "a");
 
-      try (AccumuloClient conn2 = client.changeUser(user, user1.getToken());
-          ConditionalWriter cw1 = conn2.createConditionalWriter(table1,
+      try (
+          AccumuloClient client2 = Accumulo.newClient().from(client.properties())
+              .as(user, user1.getToken()).build();
+          ConditionalWriter cw1 = client2.createConditionalWriter(table1,
               new ConditionalWriterConfig());
-          ConditionalWriter cw2 = conn2.createConditionalWriter(table2,
+          ConditionalWriter cw2 = client2.createConditionalWriter(table2,
               new ConditionalWriterConfig());
-          ConditionalWriter cw3 = conn2.createConditionalWriter(table3,
+          ConditionalWriter cw3 = client2.createConditionalWriter(table3,
               new ConditionalWriterConfig())) {
 
         // Should be able to conditional-update a table we have R/W on
diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
index 155d678..29fdbea 100644
--- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
@@ -37,6 +37,7 @@ import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -679,7 +680,8 @@ public class NamespacesIT extends AccumuloClusterHarness {
     c.securityOperations().createLocalUser(u1, pass);
 
     loginAs(user1);
-    try (AccumuloClient user1Con = c.changeUser(u1, user1.getToken())) {
+    try (AccumuloClient user1Con = Accumulo.newClient().from(c.properties())
+        .as(u1, user1.getToken()).build()) {
 
       try {
         user1Con.tableOperations().create(t2);
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 2944e0f..2f1428c 100644
--- a/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/RewriteTabletDirectoriesIT.java
@@ -125,7 +125,7 @@ public class RewriteTabletDirectoriesIT extends ConfigurableMacBase {
 
         // This should fail: only one volume
         assertEquals(1, cluster.exec(RandomizeVolumes.class, "-z", cluster.getZooKeepers(), "-i",
-            c.info().getInstanceName(), "-t", tableName).waitFor());
+            getClientInfo().getInstanceName(), "-t", tableName).waitFor());
 
         cluster.stop();
 
diff --git a/test/src/main/java/org/apache/accumulo/test/SampleIT.java b/test/src/main/java/org/apache/accumulo/test/SampleIT.java
index 92e54ca..1169df7 100644
--- a/test/src/main/java/org/apache/accumulo/test/SampleIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SampleIT.java
@@ -216,7 +216,7 @@ public class SampleIT extends AccumuloClusterHarness {
     client.tableOperations().clone(tableName, clone, false, em, es);
     client.tableOperations().offline(clone, true);
     Table.ID cloneID = Table.ID.of(client.tableOperations().tableIdMap().get(clone));
-    ClientContext context = new ClientContext(client.info());
+    ClientContext context = new ClientContext(client);
     OfflineScanner oScanner = new OfflineScanner(context, cloneID, Authorizations.EMPTY);
     if (sc != null) {
       oScanner.setSamplerConfiguration(sc);
diff --git a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
index 0844c3e..dcf5e39 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
@@ -26,8 +26,6 @@ import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ClientOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
@@ -82,12 +80,7 @@ public class TestMultiTableIngest {
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(TestMultiTableIngest.class.getName(), args, scanOpts, bwOpts);
     // create the test table within accumulo
-    AccumuloClient accumuloClient;
-    try {
-      accumuloClient = opts.getClient();
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
+    AccumuloClient accumuloClient = opts.getClient();
     for (int i = 0; i < opts.tables; i++) {
       tableNames.add(String.format(opts.prefix + "%04d", i));
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
index 9d23692..2799772 100644
--- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -82,7 +83,8 @@ public class UnusedWALIT extends ConfigurableMacBase {
       c.tableOperations().create(lilTable);
 
       ServerContext context = getServerContext();
-      new ZooReaderWriter(c.info().getZooKeepers(), c.info().getZooKeepersSessionTimeOut(), "");
+      ClientInfo info = getClientInfo();
+      new ZooReaderWriter(info.getZooKeepers(), info.getZooKeepersSessionTimeOut(), "");
 
       // put some data in a log that should be replayed for both tables
       writeSomeData(c, bigTable, 0, 10, 0, 10);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
index e006d7c..d0da323 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.test.functional;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -75,8 +76,8 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
   @Test
   public void testAccumuloClientBuilder() throws Exception {
     AccumuloClient c = getAccumuloClient();
-    String instanceName = c.info().getInstanceName();
-    String zookeepers = c.info().getZooKeepers();
+    String instanceName = getClientInfo().getInstanceName();
+    String zookeepers = getClientInfo().getZooKeepers();
     final String user = "testuser";
     final String password = "testpassword";
     c.securityOperations().createLocalUser(user, new PasswordToken(password));
@@ -84,18 +85,26 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
     AccumuloClient client = Accumulo.newClient().to(instanceName, zookeepers).as(user, password)
         .zkTimeout(1234).build();
 
-    assertEquals(instanceName, client.info().getInstanceName());
-    assertEquals(zookeepers, client.info().getZooKeepers());
+    Properties props = client.properties();
+    assertFalse(props.containsKey(ClientProperty.AUTH_TOKEN.getKey()));
+    ClientInfo info = ClientInfo.from(client.properties());
+    assertEquals(instanceName, info.getInstanceName());
+    assertEquals(zookeepers, info.getZooKeepers());
     assertEquals(user, client.whoami());
-    assertEquals(1234, client.info().getZooKeepersSessionTimeOut());
-
-    ClientInfo info = Accumulo.newClient().to(instanceName, zookeepers).as(user, password).info();
+    assertEquals(1234, info.getZooKeepersSessionTimeOut());
+
+    props = Accumulo.newClientProperties().to(instanceName, zookeepers).as(user, password).build();
+    assertTrue(props.containsKey(ClientProperty.AUTH_TOKEN.getKey()));
+    assertEquals(password, props.get(ClientProperty.AUTH_TOKEN.getKey()));
+    assertEquals("password", props.get(ClientProperty.AUTH_TYPE.getKey()));
+    assertEquals(instanceName, props.getProperty(ClientProperty.INSTANCE_NAME.getKey()));
+    info = ClientInfo.from(props);
     assertEquals(instanceName, info.getInstanceName());
     assertEquals(zookeepers, info.getZooKeepers());
     assertEquals(user, info.getPrincipal());
     assertTrue(info.getAuthenticationToken() instanceof PasswordToken);
 
-    Properties props = new Properties();
+    props = new Properties();
     props.put(ClientProperty.INSTANCE_NAME.getKey(), instanceName);
     props.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), zookeepers);
     props.put(ClientProperty.AUTH_PRINCIPAL.getKey(), user);
@@ -104,42 +113,27 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
     client.close();
     client = Accumulo.newClient().from(props).build();
 
-    assertEquals(instanceName, client.info().getInstanceName());
-    assertEquals(zookeepers, client.info().getZooKeepers());
+    info = ClientInfo.from(client.properties());
+    assertEquals(instanceName, info.getInstanceName());
+    assertEquals(zookeepers, info.getZooKeepers());
     assertEquals(user, client.whoami());
-    assertEquals(22000, client.info().getZooKeepersSessionTimeOut());
+    assertEquals(22000, info.getZooKeepersSessionTimeOut());
 
     final String user2 = "testuser2";
     final String password2 = "testpassword2";
     c.securityOperations().createLocalUser(user2, new PasswordToken(password2));
 
-    AccumuloClient client2 = Accumulo.newClient().from(client.info())
+    AccumuloClient client2 = Accumulo.newClient().from(client.properties())
         .as(user2, new PasswordToken(password2)).build();
-    assertEquals(instanceName, client2.info().getInstanceName());
-    assertEquals(zookeepers, client2.info().getZooKeepers());
-    assertEquals(user2, client2.whoami());
-    info = client2.info();
+    info = ClientInfo.from(client2.properties());
     assertEquals(instanceName, info.getInstanceName());
     assertEquals(zookeepers, info.getZooKeepers());
+    assertEquals(user2, client2.whoami());
     assertEquals(user2, info.getPrincipal());
 
-    final String user3 = "testuser3";
-    final String password3 = "testpassword3";
-    c.securityOperations().createLocalUser(user3, new PasswordToken(password3));
-
-    AccumuloClient client3 = client.changeUser(user3, new PasswordToken(password3));
-    assertEquals(instanceName, client3.info().getInstanceName());
-    assertEquals(zookeepers, client3.info().getZooKeepers());
-    assertEquals(user3, client3.whoami());
-    info = client3.info();
-    assertEquals(instanceName, info.getInstanceName());
-    assertEquals(zookeepers, info.getZooKeepers());
-    assertEquals(user3, info.getPrincipal());
-
     c.close();
     client.close();
     client2.close();
-    client3.close();
   }
 
   @Test
@@ -151,7 +145,7 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
     assertEquals(0, SingletonManager.getReservationCount());
     assertEquals(Mode.CLIENT, SingletonManager.getMode());
 
-    try (AccumuloClient c = Accumulo.newClient().from(getClientInfo()).build()) {
+    try (AccumuloClient c = Accumulo.newClient().from(getClientInfo().getProperties()).build()) {
       assertEquals(1, SingletonManager.getReservationCount());
 
       c.tableOperations().create(tableName);
@@ -170,7 +164,7 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
 
     assertEquals(0, SingletonManager.getReservationCount());
 
-    AccumuloClient c = Accumulo.newClient().from(getClientInfo()).build();
+    AccumuloClient c = Accumulo.newClient().from(getClientInfo().getProperties()).build();
     assertEquals(1, SingletonManager.getReservationCount());
 
     // ensure client created after everything was closed works
@@ -195,9 +189,8 @@ public class AccumuloClientIT extends AccumuloClusterHarness {
     expectClosed(() -> c.instanceOperations());
     expectClosed(() -> c.securityOperations());
     expectClosed(() -> c.namespaceOperations());
-    expectClosed(() -> c.info());
+    expectClosed(() -> c.properties());
     expectClosed(() -> c.getInstanceID());
-    expectClosed(() -> c.changeUser("root", new PasswordToken("secret")));
 
     // check a few table ops to ensure they fail
     expectClosed(() -> tops.create("expectFail"));
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 ba0fb85..f524b02 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
@@ -193,7 +193,7 @@ public class ConfigurableMacBase extends AccumuloITBase {
     return cluster;
   }
 
-  protected AccumuloClient getClient() throws AccumuloException, AccumuloSecurityException {
+  protected AccumuloClient getClient() {
     return getCluster().getAccumuloClient("root", new PasswordToken(ROOT_PASSWORD));
   }
 
@@ -202,13 +202,15 @@ public class ConfigurableMacBase extends AccumuloITBase {
   }
 
   protected ClientInfo getClientInfo() {
-    return Accumulo.newClient().to(getCluster().getInstanceName(), getCluster().getZooKeepers())
-        .as("root", ROOT_PASSWORD).info();
+    return ClientInfo.from(Accumulo.newClientProperties()
+        .to(getCluster().getInstanceName(), getCluster().getZooKeepers()).as("root", ROOT_PASSWORD)
+        .build());
   }
 
   protected ClientInfo getClientInfo(BatchWriterConfig bwConfig) {
-    return Accumulo.newClient().to(getCluster().getInstanceName(), getCluster().getZooKeepers())
-        .as("root", ROOT_PASSWORD).batchWriterConfig(bwConfig).info();
+    return ClientInfo.from(Accumulo.newClientProperties()
+        .to(getCluster().getInstanceName(), getCluster().getZooKeepers()).as("root", ROOT_PASSWORD)
+        .batchWriterConfig(bwConfig).build());
   }
 
   protected ServerContext getServerContext() {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
index 7ab59e2..ba6dde0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
@@ -26,6 +26,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -91,11 +92,11 @@ public class CredentialsIT extends AccumuloClusterHarness {
     assertFalse(token.isDestroyed());
     token.destroy();
     assertTrue(token.isDestroyed());
-    try (AccumuloClient client = getAccumuloClient()) {
-      client.changeUser("non_existent_user", token);
+    try (AccumuloClient ignored = Accumulo.newClient().from(getClientInfo().getProperties())
+        .as("non_existent_user", token).build()) {
       fail();
-    } catch (AccumuloSecurityException e) {
-      assertEquals(e.getSecurityErrorCode(), SecurityErrorCode.TOKEN_EXPIRED);
+    } catch (IllegalArgumentException e) {
+      assertEquals(e.getMessage(), "AuthenticationToken has been destroyed");
     }
   }
 
@@ -103,7 +104,9 @@ public class CredentialsIT extends AccumuloClusterHarness {
   public void testDestroyTokenBeforeRPC() throws Exception {
     try (AccumuloClient client = getAccumuloClient()) {
       AuthenticationToken token = getUser(0).getToken();
-      try (AccumuloClient userAccumuloClient = client.changeUser(username, token);
+      try (
+          AccumuloClient userAccumuloClient = Accumulo.newClient().from(client.properties())
+              .as(username, token).build();
           Scanner scanner = userAccumuloClient.createScanner(MetadataTable.NAME,
               Authorizations.EMPTY)) {
         assertFalse(token.isDestroyed());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 06af586..3510da1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -460,7 +460,8 @@ public class KerberosIT extends AccumuloITBase {
     try {
       // Use the delegation token to try to log in as a different user
       userWithoutPrivs.doAs((PrivilegedExceptionAction<Void>) () -> {
-        mac.getAccumuloClient("some_other_user", delegationToken);
+        AccumuloClient client = mac.getAccumuloClient("some_other_user", delegationToken);
+        client.securityOperations().authenticateUser("some_other_user", delegationToken);
         return null;
       });
       fail("Using a delegation token as a different user should throw an exception");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
index 0286960..0611714 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
@@ -597,7 +597,9 @@ public class KerberosProxyIT extends AccumuloITBase {
     // Has read permission but is not allowed to be proxied
     proxyUser3.doAs((PrivilegedExceptionAction<Void>) () -> {
       try {
-        mac.getAccumuloClient(userWithoutCredentials3, new KerberosToken(userWithoutCredentials3));
+        KerberosToken token = new KerberosToken(userWithoutCredentials3);
+        AccumuloClient client = mac.getAccumuloClient(userWithoutCredentials3, token);
+        client.securityOperations().authenticateUser(userWithoutCredentials3, token);
         fail("Should not be able to create a Connector as this user cannot be proxied");
       } catch (org.apache.accumulo.core.client.AccumuloSecurityException e) {
         // Expected, this user cannot be proxied
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java
index 73c74c4..e3fb5c5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -81,9 +82,10 @@ public class MapReduceIT extends ConfigurableMacBase {
       bw.addMutation(m);
     }
     bw.close();
+    ClientInfo info = ClientInfo.from(c.properties());
     Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-i",
-        c.info().getInstanceName(), "-z", c.info().getZooKeepers(), "-u", "root", "-p",
-        ROOT_PASSWORD, "-t", tablename, "--column", input_cfcq);
+        info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root", "-p", ROOT_PASSWORD, "-t",
+        tablename, "--column", input_cfcq);
     assertEquals(0, hash.waitFor());
 
     try (Scanner s = c.createScanner(tablename, Authorizations.EMPTY)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
index c8ae720..61dd99e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -30,6 +30,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.cluster.ClusterUser;
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -109,7 +110,8 @@ public class PermissionsIT extends AccumuloClusterHarness {
       loginAs(rootUser);
       c.securityOperations().createLocalUser(principal, passwordToken);
       loginAs(testUser);
-      try (AccumuloClient test_user_client = c.changeUser(principal, token)) {
+      try (AccumuloClient test_user_client = Accumulo.newClient().from(c.properties())
+          .as(principal, token).build()) {
         loginAs(rootUser);
         verifyHasNoSystemPermissions(c, principal, SystemPermission.values());
 
@@ -570,7 +572,8 @@ public class PermissionsIT extends AccumuloClusterHarness {
     try (AccumuloClient c = getAccumuloClient()) {
       c.securityOperations().createLocalUser(principal, passwordToken);
       loginAs(testUser);
-      try (AccumuloClient test_user_client = c.changeUser(principal, token)) {
+      try (AccumuloClient test_user_client = Accumulo.newClient().from(c.properties())
+          .as(principal, token).build()) {
 
         // check for read-only access to metadata table
         loginAs(rootUser);
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 5e2a678..dc4696e 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
@@ -183,8 +183,8 @@ public class ReadWriteIT extends AccumuloClusterHarness {
       log.debug("Stopping accumulo cluster");
       ClusterControl control = cluster.getClusterControl();
       control.adminStopAll();
-      ZooReader zreader = new ZooReader(accumuloClient.info().getZooKeepers(),
-          accumuloClient.info().getZooKeepersSessionTimeOut());
+      ClientInfo info = ClientInfo.from(accumuloClient.properties());
+      ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
       byte[] masterLockData;
       do {
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 9c9bce2..8686273 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
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.ClientInfo;
 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;
@@ -166,8 +167,8 @@ public class RestartIT extends AccumuloClusterHarness {
       control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
       control.stopAllServers(ServerType.MONITOR);
 
-      ZooReader zreader = new ZooReader(c.info().getZooKeepers(),
-          c.info().getZooKeepersSessionTimeOut());
+      ClientInfo info = ClientInfo.from(c.properties());
+      ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
       byte[] masterLockData;
       do {
@@ -235,8 +236,8 @@ public class RestartIT extends AccumuloClusterHarness {
 
       control.stopAllServers(ServerType.MASTER);
 
-      ZooReader zreader = new ZooReader(c.info().getZooKeepers(),
-          c.info().getZooKeepersSessionTimeOut());
+      ClientInfo info = ClientInfo.from(c.properties());
+      ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
       byte[] masterLockData;
       do {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
index fe2e0b0..4d6a8e8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
@@ -51,6 +51,7 @@ import java.util.regex.PatternSyntaxException;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -593,7 +594,8 @@ public class SummaryIT extends AccumuloClusterHarness {
       PasswordToken passTok = new PasswordToken("letmesee");
       c.securityOperations().createLocalUser("user1", passTok);
 
-      try (AccumuloClient c2 = c.changeUser("user1", passTok)) {
+      try (AccumuloClient c2 = Accumulo.newClient().from(c.properties()).as("user1", passTok)
+          .build()) {
         try {
           c2.tableOperations().summaries(table).retrieve();
           fail("Expected operation to fail because user does not have permssion to get summaries");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TableChangeStateIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TableChangeStateIT.java
index a4f1f33..c7d570e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TableChangeStateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TableChangeStateIT.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -80,7 +81,7 @@ public class TableChangeStateIT extends AccumuloClusterHarness {
   @Before
   public void setup() {
     accumuloClient = getAccumuloClient();
-    context = new ClientContext(accumuloClient.info());
+    context = new ClientContext(accumuloClient);
   }
 
   @After
@@ -241,9 +242,9 @@ public class TableChangeStateIT extends AccumuloClusterHarness {
       log.trace("tid: {}", tableId);
 
       String secret = cluster.getSiteConfiguration().get(Property.INSTANCE_SECRET);
-      IZooReaderWriter zk = new ZooReaderWriterFactory().getZooReaderWriter(
-          accumuloClient.info().getZooKeepers(),
-          accumuloClient.info().getZooKeepersSessionTimeOut(), secret);
+      ClientInfo info = ClientInfo.from(accumuloClient.properties());
+      IZooReaderWriter zk = new ZooReaderWriterFactory().getZooReaderWriter(info.getZooKeepers(),
+          info.getZooKeepersSessionTimeOut(), secret);
       ZooStore<String> zs = new ZooStore<>(
           ZooUtil.getRoot(accumuloClient.getInstanceID()) + Constants.ZFATE, zk);
       AdminUtil.FateStatus fateStatus = admin.getStatus(zs, zk,
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index 3286ae8..88df7e8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchDeleter;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -242,8 +243,9 @@ public class TabletStateChangeIteratorIT extends AccumuloClusterHarness {
         try {
           String zPath = ZooUtil.getRoot(client.getInstanceID()) + Constants.ZTSERVERS + "/"
               + tserver;
-          long sessionId = ZooLock.getSessionId(new ZooCache(getCluster().getZooKeepers(),
-              client.info().getZooKeepersSessionTimeOut()), zPath);
+          ClientInfo info = getClientInfo();
+          long sessionId = ZooLock.getSessionId(
+              new ZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut()), zPath);
           tservers.add(new TServerInstance(tserver, sessionId));
         } catch (Exception e) {
           throw new RuntimeException(e);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index 40d35d0..4b6738e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@ -53,7 +53,7 @@ public class WatchTheWatchCountIT extends ConfigurableMacBase {
         c.tableOperations().create(tableName);
       }
       c.tableOperations().list();
-      String zooKeepers = c.info().getZooKeepers();
+      String zooKeepers = getClientInfo().getZooKeepers();
       final long MIN = 475L;
       final long MAX = 700L;
       long total = 0;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
index 9556e0d..19dc57b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
@@ -59,8 +59,8 @@ public class ZooCacheIT extends ConfigurableMacBase {
         @Override
         public void run() {
           try (AccumuloClient client = getClient()) {
-            CacheTestReader.main(
-                new String[] {pathName, testDir.getAbsolutePath(), client.info().getZooKeepers()});
+            CacheTestReader.main(new String[] {pathName, testDir.getAbsolutePath(),
+                getClientInfo().getZooKeepers()});
           } catch (Exception ex) {
             ref.set(ex);
           }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
index 8f1b6e5..0d94a67 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
@@ -169,7 +169,8 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
-      ClientInfo info = Accumulo.newClient().to(instanceName, zooKeepers).as(user, pass).info();
+      ClientInfo info = ClientInfo
+          .from(Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass).build());
 
       AccumuloInputFormat.setClientInfo(job, info);
       AccumuloInputFormat.setInputTableName(job, table1);
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 ca16769..0a150d6 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,6 +23,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -81,8 +82,9 @@ public class RollWALPerformanceIT extends ConfigurableMacBase {
 
     log.info("Starting ingest");
     final long start = System.nanoTime();
-    final String args[] = {"-i", c.info().getInstanceName(), "-z", c.info().getZooKeepers(), "-u",
-        "root", "-p", ROOT_PASSWORD, "--batchThreads", "2", "--table", tableName, "--num",
+    ClientInfo info = ClientInfo.from(c.properties());
+    final String args[] = {"-i", info.getInstanceName(), "-z", info.getZooKeepers(), "-u", "root",
+        "-p", ROOT_PASSWORD, "--batchThreads", "2", "--table", tableName, "--num",
         Long.toString(50 * 1000), // 50K 100 byte entries
     };
 
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index eee6f58..b1bd022 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -47,6 +47,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.Namespace;
@@ -183,8 +184,9 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
       hostname = InetAddress.getLocalHost().getCanonicalHostName();
 
       Properties props = new Properties();
-      props.put("instance", c.info().getInstanceName());
-      props.put("zookeepers", c.info().getZooKeepers());
+      ClientInfo info = ClientInfo.from(c.properties());
+      props.put("instance", info.getInstanceName());
+      props.put("zookeepers", info.getZooKeepers());
 
       final String tokenClass;
       if (isKerberosEnabled()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
index 8fce8ac..f967a65 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.ClientProperty;
@@ -181,11 +182,12 @@ public class KerberosReplicationIT extends AccumuloITBase {
             replicationUser.getKeytab().getAbsolutePath());
 
         // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
+        ClientInfo info = ClientInfo.from(peerclient.properties());
         primaryclient.instanceOperations().setProperty(
             Property.REPLICATION_PEERS.getKey() + PEER_NAME,
             ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-                AccumuloReplicaSystem.buildConfiguration(peerclient.info().getInstanceName(),
-                    peerclient.info().getZooKeepers())));
+                AccumuloReplicaSystem.buildConfiguration(info.getInstanceName(),
+                    info.getZooKeepers())));
 
         String primaryTable1 = "primary", peerTable1 = "peer";
 
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
index d3454e4..ab251c9 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.client.Scanner;
@@ -200,8 +201,8 @@ public class ReplicationIT extends ConfigurableMacBase {
   private void waitForGCLock(AccumuloClient client) throws InterruptedException {
     // Check if the GC process has the lock before wasting our retry attempts
     ZooCacheFactory zcf = new ZooCacheFactory();
-    ZooCache zcache = zcf.getZooCache(client.info().getZooKeepers(),
-        client.info().getZooKeepersSessionTimeOut());
+    ClientInfo info = ClientInfo.from(client.properties());
+    ZooCache zcache = zcf.getZooCache(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
     String zkPath = ZooUtil.getRoot(client.getInstanceID()) + Constants.ZGC_LOCK;
     log.info("Looking for GC lock at {}", zkPath);
     byte[] data = ZooLock.getLockData(zcache, zkPath, null);
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index f11571c..e461fe5 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -40,7 +40,7 @@ import org.junit.Test;
 
 public class SystemCredentialsIT extends ConfigurableMacBase {
 
-  private static final int FAIL_CODE = 7, BAD_PASSWD_FAIL_CODE = 8;
+  private static final int SCAN_FAILED = 7, AUTHENICATION_FAILED = 8;
 
   @Override
   protected int defaultTimeoutSeconds() {
@@ -51,9 +51,9 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
   public void testSystemCredentials() throws Exception {
     assertEquals(0,
         exec(SystemCredentialsIT.class, "good", getCluster().getZooKeepers()).waitFor());
-    assertEquals(FAIL_CODE,
+    assertEquals(AUTHENICATION_FAILED,
         exec(SystemCredentialsIT.class, "bad", getCluster().getZooKeepers()).waitFor());
-    assertEquals(BAD_PASSWD_FAIL_CODE,
+    assertEquals(AUTHENICATION_FAILED,
         exec(SystemCredentialsIT.class, "bad_password", getCluster().getZooKeepers()).waitFor());
   }
 
@@ -77,9 +77,10 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
     AccumuloClient client;
     try {
       client = context.getClient(creds.getPrincipal(), creds.getToken());
+      client.securityOperations().authenticateUser(creds.getPrincipal(), creds.getToken());
     } catch (AccumuloSecurityException e) {
       e.printStackTrace(System.err);
-      System.exit(BAD_PASSWD_FAIL_CODE);
+      System.exit(AUTHENICATION_FAILED);
       return;
     }
     try (Scanner scan = client.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
@@ -87,13 +88,8 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
         e.hashCode();
       }
     } catch (RuntimeException e) {
-      // catch the runtime exception from the scanner iterator
-      if (e.getCause() instanceof AccumuloSecurityException
-          && ((AccumuloSecurityException) e.getCause())
-              .getSecurityErrorCode() == SecurityErrorCode.BAD_CREDENTIALS) {
-        e.printStackTrace(System.err);
-        System.exit(FAIL_CODE);
-      }
+      e.printStackTrace(System.err);
+      System.exit(SCAN_FAILED);
     }
   }
 }