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

[GitHub] ctubbsii closed pull request #528: Refactored ClientContext & reduced use of Instance

ctubbsii closed pull request #528: Refactored ClientContext & reduced use of Instance
URL: https://github.com/apache/accumulo/pull/528
 
 
   

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

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

diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 8485df7c28..56a59a7bfb 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -27,7 +27,6 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
@@ -130,19 +129,6 @@ public static void setClientInfo(JobConf job, ClientInfo info) {
     InputConfigurator.setClientInfo(CLASS, job, inputInfo);
   }
 
-  /**
-   * Set Accumulo client properties used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo client properties
-   * @since 2.0.0
-   */
-  public static void setClientProperties(JobConf job, Properties clientProps) {
-    InputConfigurator.setClientProperties(CLASS, job, clientProps);
-  }
-
   /**
    * Set Accumulo client properties file used to connect to Accumulo
    *
@@ -234,7 +220,7 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
-   * @deprecated since 2.0.0, use {@link #setClientInfo(JobConf, ClientInfo)} instead
+   * @deprecated since 2.0.0, use {@link #setClientPropertiesFile(JobConf, String)} instead
    */
   @Deprecated
   public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
@@ -554,9 +540,7 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
             scanner = new OfflineScanner(instance, new Credentials(principal, token),
                 Table.ID.of(baseSplit.getTableId()), authorizations);
           } else {
-            Properties props = getClientInfo(job).getProperties();
-            ClientContext context = new ClientContext(instance, new Credentials(principal, token),
-                props);
+            ClientContext context = new ClientContext(getClientInfo(job));
             scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()), authorizations);
           }
           if (isIsolated) {
@@ -662,11 +646,11 @@ public float getProgress() throws IOException {
       String tableName = tableConfigEntry.getKey();
       InputTableConfig tableConfig = tableConfigEntry.getValue();
 
-      Instance instance = getInstance(job);
+      ClientContext context = new ClientContext(getClientInfo(job));
       Table.ID tableId;
       // resolve table name to id once, and use id from this point forward
       try {
-        tableId = Tables.getTableId(instance, tableName);
+        tableId = Tables.getTableId(context, tableName);
       } catch (TableNotFoundException e) {
         throw new IOException(e);
       }
@@ -708,13 +692,12 @@ public float getProgress() throws IOException {
           // tablets... so clear it
           tl.invalidateCache();
 
-          ClientContext context = new ClientContext(getClientInfo(job));
           while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
             String tableIdStr = tableId.canonicalID();
-            if (!Tables.exists(instance, tableId))
+            if (!Tables.exists(context, tableId))
               throw new TableDeletedException(tableIdStr);
-            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-              throw new TableOfflineException(instance, tableIdStr);
+            if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
+              throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
             binnedRanges.clear();
             log.warn("Unable to locate bins for specified ranges. Retrying.");
             // sleep randomly between 100 and 200 ms
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index 8105857fc5..4f5c7f2dc7 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -20,7 +20,6 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -93,19 +92,6 @@ public static void setClientInfo(JobConf job, ClientInfo info) {
     OutputConfigurator.setClientInfo(CLASS, job, outInfo);
   }
 
-  /**
-   * Set Accumulo client properties used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo client properties
-   * @since 2.0.0
-   */
-  public static void setClientProperties(JobConf job, Properties clientProps) {
-    OutputConfigurator.setClientProperties(CLASS, job, clientProps);
-  }
-
   /**
    * Set Accumulo client properties file used to connect to Accumulo
    *
@@ -187,7 +173,7 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
    * @param tokenFile
    *          the path to the password file
    * @since 1.6.0
-   * @deprecated since 2.0.0, use {@link #setClientInfo(JobConf, ClientInfo)} instead
+   * @deprecated since 2.0.0, use {@link #setClientPropertiesFile(JobConf, String)} instead
    */
   @Deprecated
   public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 508f4ae71e..6d7ef48883 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -27,7 +27,6 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
@@ -132,19 +131,6 @@ public static void setClientInfo(Job job, ClientInfo info) {
     InputConfigurator.setClientInfo(CLASS, job.getConfiguration(), inputInfo);
   }
 
-  /**
-   * Set Accumulo client properties used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo client properties
-   * @since 2.0.0
-   */
-  public static void setClientProperties(Job job, Properties clientProps) {
-    InputConfigurator.setClientProperties(CLASS, job.getConfiguration(), clientProps);
-  }
-
   /**
    * Set Accumulo client properties file used to connect to Accumulo
    *
@@ -236,7 +222,9 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
+   * @deprecated since 2.0.0, use {{@link #setClientPropertiesFile(Job, String)}}
    */
+  @Deprecated
   public static void setConnectorInfo(Job job, String principal, String tokenFile)
       throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
@@ -557,9 +545,7 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
             scanner = new OfflineScanner(instance, new Credentials(principal, token),
                 Table.ID.of(split.getTableId()), authorizations);
           } else {
-            Properties props = getClientInfo(attempt).getProperties();
-            ClientContext context = new ClientContext(instance, new Credentials(principal, token),
-                props);
+            ClientContext context = new ClientContext(getClientInfo(attempt));
             // Not using public API to create scanner so that we can use table ID
             // Table ID is used in case of renames during M/R job
             scanner = new ScannerImpl(context, Table.ID.of(split.getTableId()), authorizations);
@@ -685,11 +671,11 @@ public V getCurrentValue() throws IOException, InterruptedException {
       String tableName = tableConfigEntry.getKey();
       InputTableConfig tableConfig = tableConfigEntry.getValue();
 
-      Instance instance = getInstance(context);
+      ClientContext clientContext = new ClientContext(getClientInfo(context));
       Table.ID tableId;
       // resolve table name to id once, and use id from this point forward
       try {
-        tableId = Tables.getTableId(instance, tableName);
+        tableId = Tables.getTableId(clientContext, tableName);
       } catch (TableNotFoundException e) {
         throw new IOException(e);
       }
@@ -732,13 +718,12 @@ public V getCurrentValue() throws IOException, InterruptedException {
           // tablets... so clear it
           tl.invalidateCache();
 
-          ClientContext clientContext = new ClientContext(getClientInfo(context));
           while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
             String tableIdStr = tableId.canonicalID();
-            if (!Tables.exists(instance, tableId))
+            if (!Tables.exists(clientContext, tableId))
               throw new TableDeletedException(tableIdStr);
-            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-              throw new TableOfflineException(instance, tableIdStr);
+            if (Tables.getTableState(clientContext, tableId) == TableState.OFFLINE)
+              throw new TableOfflineException(Tables.getTableOfflineMsg(clientContext, tableId));
             binnedRanges.clear();
             log.warn("Unable to locate bins for specified ranges. Retrying.");
             // sleep randomly between 100 and 200 ms
diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index c4ff7b57b0..9777509bc8 100644
--- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -20,7 +20,6 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -93,19 +92,6 @@ public static void setClientInfo(Job job, ClientInfo info) {
     OutputConfigurator.setClientInfo(CLASS, job.getConfiguration(), info);
   }
 
-  /**
-   * Set Accumulo client properties used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo client properties
-   * @since 2.0.0
-   */
-  public static void setClientProperties(Job job, Properties clientProps) {
-    OutputConfigurator.setClientProperties(CLASS, job.getConfiguration(), clientProps);
-  }
-
   /**
    * Set Accumulo client properties file used to connect to Accumulo
    *
@@ -187,7 +173,7 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #setClientInfo(Job, ClientInfo)}
+   * @deprecated since 2.0.0, replaced by {@link #setClientPropertiesFile(Job, String)}
    */
   @Deprecated
   public static void setConnectorInfo(Job job, String principal, String tokenFile)
diff --git a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
index eb3607582a..6de701d885 100644
--- a/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
+++ b/client/mapreduce/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
@@ -97,7 +97,7 @@ public void testSetZooKeeperInstance() {
     ConfiguratorBase.setZooKeeperInstance(this.getClass(), conf,
         org.apache.accumulo.core.client.ClientConfiguration.create()
             .withInstance("testInstanceName").withZkHosts("testZooKeepers").withSsl(true)
-            .withZkTimeout(1234));
+            .withZkTimeout(15000));
 
     org.apache.accumulo.core.client.ClientConfiguration clientConf = ConfiguratorBase
         .getClientConfiguration(this.getClass(), conf);
@@ -108,8 +108,7 @@ public void testSetZooKeeperInstance() {
     assertEquals("testInstanceName", props.getProperty(ClientProperty.INSTANCE_NAME.getKey()));
     assertEquals("testZooKeepers", props.getProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey()));
     assertEquals("true", props.getProperty(ClientProperty.SSL_ENABLED.getKey()));
-    assertEquals("1234",
-        props.getProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT_SEC.getKey()));
+    assertEquals("15000", props.getProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey()));
   }
 
   @Test
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java b/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java
index 8c829e2ded..933a101ac6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java
@@ -37,6 +37,11 @@
    */
   String getZooKeepers();
 
+  /**
+   * @return ZooKeeper connection timeout
+   */
+  int getZooKeepersSessionTimeOut();
+
   /**
    * @return Accumulo principal/username
    */
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 2c4e16655e..673da77fd4 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
@@ -275,6 +275,13 @@ public abstract ConditionalWriter createConditionalWriter(String tableName,
    */
   public abstract String whoami();
 
+  /**
+   * Returns a unique string that identifies this instance of accumulo.
+   *
+   * @return a UUID
+   */
+  public abstract String getInstanceID();
+
   /**
    * Retrieves a TableOperations object to perform table functions, such as create and delete.
    *
@@ -537,7 +544,7 @@ public abstract ConditionalWriter createConditionalWriter(String tableName,
      * Build using Zookeeper timeout
      *
      * @param timeout
-     *          Zookeeper timeout
+     *          Zookeeper timeout (in milliseconds)
      * @return this builder
      */
     ConnectionOptions withZkTimeout(int timeout);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
index 2677a569be..33f65766b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
@@ -23,18 +23,15 @@
 
   private static final long serialVersionUID = 1L;
 
-  private static String getTableName(Instance instance, String tableId) {
-    if (tableId == null)
-      return " <unknown table> ";
-    try {
-      String tableName = Tables.getTableName(instance, Table.ID.of(tableId));
-      return tableName + " (" + tableId + ")";
-    } catch (TableNotFoundException e) {
-      return " <unknown table> (" + tableId + ")";
-    }
+  /**
+   * @deprecated since 2.0.0, replaced by {@link #TableOfflineException(String)}
+   */
+  @Deprecated
+  public TableOfflineException(Instance instance, String tableId) {
+    super(Tables.getTableOfflineMsg(instance, Table.ID.of(tableId)));
   }
 
-  public TableOfflineException(Instance instance, String tableId) {
-    super("Table " + getTableName(instance, tableId) + " is offline");
+  public TableOfflineException(String msg) {
+    super(msg);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 3b0392abbd..fbb0a8a342 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -22,17 +22,19 @@
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.ClientConfConverter;
 import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.ClientInfoImpl;
 import org.apache.accumulo.core.client.impl.ConnectorImpl;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -284,8 +286,10 @@ public Connector getConnector(String user, ByteBuffer pass)
   @Override
   public Connector getConnector(String principal, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(new ClientContext(this, new Credentials(principal, token),
-        ClientConfConverter.toProperties(clientConf)));
+    Properties properties = ClientConfConverter.toProperties(clientConf);
+    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
+    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
+    return new ConnectorImpl(new ClientContext(new ClientInfoImpl(properties, token)));
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
index ccd055e3e2..3721e9c68d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
@@ -20,7 +20,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.ActiveCompaction;
@@ -35,17 +34,17 @@
 public class ActiveCompactionImpl extends ActiveCompaction {
 
   private org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac;
-  private Instance instance;
+  private ClientContext context;
 
-  ActiveCompactionImpl(Instance instance,
+  ActiveCompactionImpl(ClientContext context,
       org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac) {
     this.tac = tac;
-    this.instance = instance;
+    this.context = context;
   }
 
   @Override
   public String getTable() throws TableNotFoundException {
-    return Tables.getTableName(instance, new KeyExtent(tac.getExtent()).getTableId());
+    return Tables.getTableName(context, new KeyExtent(tac.getExtent()).getTableId());
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
index 388b5b5f93..a4c6d8a60c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
@@ -20,7 +20,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.ScanState;
@@ -54,7 +53,7 @@
   private String user;
   private Authorizations authorizations;
 
-  ActiveScanImpl(Instance instance,
+  ActiveScanImpl(ClientContext context,
       org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan)
       throws TableNotFoundException {
     this.scanId = activeScan.scanId;
@@ -62,7 +61,7 @@
     this.user = activeScan.user;
     this.age = activeScan.age;
     this.idle = activeScan.idleTime;
-    this.tableName = Tables.getTableName(instance, Table.ID.of(activeScan.tableId));
+    this.tableName = Tables.getTableName(context, Table.ID.of(activeScan.tableId));
     this.type = ScanType.valueOf(activeScan.getType().name());
     this.state = ScanState.valueOf(activeScan.state.name());
     this.extent = new KeyExtent(activeScan.extent);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java
index 0c19cb0c7e..d8decd574e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/BulkImport.java
@@ -101,7 +101,7 @@ public ImportSourceOptions settingLogicalTime() {
   public void load()
       throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException {
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     Map<String,String> props = context.getConnector().instanceOperations().getSystemConfiguration();
     AccumuloConfiguration conf = new ConfigurationCopy(props);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfConverter.java
index 4d1306bd0f..c70a65ba4b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfConverter.java
@@ -24,6 +24,7 @@
 import java.util.Properties;
 import java.util.function.Predicate;
 
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
@@ -45,7 +46,7 @@ private static void init() {
     propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(),
         org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST
             .getKey());
-    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT_SEC.getKey(),
+    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(),
         org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT
             .getKey());
     propsConf.put(ClientProperty.SSL_ENABLED.getKey(),
@@ -134,6 +135,16 @@ public static Properties toProperties(
     return props;
   }
 
+  public static Properties toProperties(AccumuloConfiguration config, Instance instance,
+      Credentials credentials) {
+    Properties properties = toProperties(toClientConf(config));
+    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), instance.getInstanceName());
+    properties.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), instance.getZooKeepers());
+    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), credentials.getPrincipal());
+    ClientProperty.setAuthenticationToken(properties, credentials.getToken());
+    return properties;
+  }
+
   public static Properties toProperties(AccumuloConfiguration config) {
     return toProperties(toClientConf(config));
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
index 70762208bd..b5d407a495 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
@@ -17,9 +17,8 @@
 package org.apache.accumulo.core.client.impl;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.Objects.requireNonNull;
 
-import java.util.Properties;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
@@ -50,11 +49,11 @@
  */
 public class ClientContext {
 
-  protected final Instance inst;
+  private ClientInfo info;
+  protected Instance inst;
   private Credentials creds;
-  private Properties clientProps;
-  private BatchWriterConfig batchWriterConfig = new BatchWriterConfig();
-  private final AccumuloConfiguration rpcConf;
+  private BatchWriterConfig batchWriterConfig;
+  private AccumuloConfiguration serverConf;
   protected Connector conn;
 
   // These fields are very frequently accessed (each time a connection is created) and expensive to
@@ -69,21 +68,9 @@
     return () -> Suppliers.memoizeWithExpiration(() -> s.get(), 100, TimeUnit.MILLISECONDS).get();
   }
 
-  public ClientContext(ClientInfo clientInfo) {
-    this(ClientInfoFactory.getInstance(clientInfo), ClientInfoFactory.getCredentials(clientInfo),
-        clientInfo.getProperties(), ClientInfoFactory.getBatchWriterConfig(clientInfo));
-  }
-
-  public ClientContext(Instance instance, Credentials credentials, Properties clientProps) {
-    this(instance, credentials, clientProps, new BatchWriterConfig());
-  }
-
-  public ClientContext(Instance instance, Credentials credentials, Properties clientProps,
-      BatchWriterConfig batchWriterConfig) {
-    this(instance, credentials,
-        ClientConfConverter.toAccumuloConf(requireNonNull(clientProps, "clientProps is null")));
-    this.clientProps = clientProps;
-    this.batchWriterConfig = batchWriterConfig;
+  public ClientContext(ClientInfo info) {
+    this(info, ClientInfoFactory.getInstance(info), ClientInfoFactory.getCredentials(info),
+        ClientConfConverter.toAccumuloConf(info.getProperties()));
   }
 
   /**
@@ -92,18 +79,22 @@ public ClientContext(Instance instance, Credentials credentials, Properties clie
    */
   public ClientContext(Instance instance, Credentials credentials,
       AccumuloConfiguration serverConf) {
-    inst = requireNonNull(instance, "instance is null");
-    creds = requireNonNull(credentials, "credentials is null");
-    rpcConf = requireNonNull(serverConf, "serverConf is null");
-    clientProps = null;
+    this(null, instance, credentials, serverConf);
+  }
 
+  public ClientContext(ClientInfo info, Instance instance, Credentials credentials,
+      AccumuloConfiguration serverConf) {
+    this.info = info;
+    inst = instance;
+    creds = credentials;
+    this.serverConf = serverConf;
     saslSupplier = () -> {
       // Use the clientProps if we have it
-      if (null != clientProps) {
-        if (!ClientProperty.SASL_ENABLED.getBoolean(clientProps)) {
+      if (info != null) {
+        if (!ClientProperty.SASL_ENABLED.getBoolean(info.getProperties())) {
           return null;
         }
-        return new SaslConnectionParams(clientProps, getCredentials().getToken());
+        return new SaslConnectionParams(info.getProperties(), getCredentials().getToken());
       }
       AccumuloConfiguration conf = getConfiguration();
       if (!conf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
@@ -126,7 +117,10 @@ public Instance getInstance() {
   }
 
   public ClientInfo getClientInfo() {
-    return new ClientInfoImpl(clientProps);
+    if (info == null) {
+      info = new ClientInfoImpl(ClientConfConverter.toProperties(serverConf, inst, creds));
+    }
+    return info;
   }
 
   /**
@@ -150,7 +144,7 @@ public synchronized void setCredentials(Credentials newCredentials) {
    * Retrieve the configuration used to construct this context
    */
   public AccumuloConfiguration getConfiguration() {
-    return rpcConf;
+    return serverConf;
   }
 
   /**
@@ -192,6 +186,9 @@ public Connector getConnector() throws AccumuloException, AccumuloSecurityExcept
   }
 
   public BatchWriterConfig getBatchWriterConfig() {
+    if (batchWriterConfig == null) {
+      batchWriterConfig = ClientInfoFactory.getBatchWriterConfig(getClientInfo());
+    }
     return batchWriterConfig;
   }
 
@@ -209,4 +206,58 @@ public synchronized TCredentials rpcCreds() {
 
     return rpcCreds;
   }
+
+  /**
+   * Returns the location of the tablet server that is serving the root tablet.
+   *
+   * @return location in "hostname:port" form
+   */
+  public String getRootTabletLocation() {
+    return inst.getRootTabletLocation();
+  }
+
+  /**
+   * Returns the location(s) of the accumulo master and any redundant servers.
+   *
+   * @return a list of locations in "hostname:port" form
+   */
+  public List<String> getMasterLocations() {
+    return inst.getMasterLocations();
+  }
+
+  /**
+   * Returns a unique string that identifies this instance of accumulo.
+   *
+   * @return a UUID
+   */
+  public String getInstanceID() {
+    return inst.getInstanceID();
+  }
+
+  /**
+   * Returns the instance name given at system initialization time.
+   *
+   * @return current instance name
+   */
+  public String getInstanceName() {
+    return inst.getInstanceName();
+  }
+
+  /**
+   * Returns a comma-separated list of zookeeper servers the instance is using.
+   *
+   * @return the zookeeper servers this instance is using in "hostname:port" form
+   */
+  public String getZooKeepers() {
+    return inst.getZooKeepers();
+  }
+
+  /**
+   * Returns the zookeeper connection timeout.
+   *
+   * @return the configured timeout to connect to zookeeper
+   */
+  public int getZooKeepersSessionTimeOut() {
+    return inst.getZooKeepersSessionTimeOut();
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientInfoImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientInfoImpl.java
index 752f09f12b..5a640e7fee 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientInfoImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientInfoImpl.java
@@ -21,13 +21,20 @@
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 
 public class ClientInfoImpl implements ClientInfo {
 
   private Properties properties;
+  private AuthenticationToken token;
 
   public ClientInfoImpl(Properties properties) {
+    this(properties, null);
+  }
+
+  public ClientInfoImpl(Properties properties, AuthenticationToken token) {
     this.properties = properties;
+    this.token = token;
   }
 
   @Override
@@ -40,6 +47,12 @@ public String getZooKeepers() {
     return getString(ClientProperty.INSTANCE_ZOOKEEPERS);
   }
 
+  @Override
+  public int getZooKeepersSessionTimeOut() {
+    return (int) ConfigurationTypeHelper
+        .getTimeInMillis(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getValue(properties));
+  }
+
   @Override
   public String getPrincipal() {
     return getString(ClientProperty.AUTH_PRINCIPAL);
@@ -56,7 +69,10 @@ public Properties getProperties() {
 
   @Override
   public AuthenticationToken getAuthenticationToken() {
-    return ClientProperty.getAuthenticationToken(properties);
+    if (token == null) {
+      token = ClientProperty.getAuthenticationToken(properties);
+    }
+    return token;
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index fb941ebeeb..847331c64a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -301,10 +301,10 @@ private void queue(List<QCMutation> mutations) {
       locator.binMutations(context, mutations, binnedMutations, failures);
 
       if (failures.size() == mutations.size())
-        if (!Tables.exists(context.getInstance(), tableId))
+        if (!Tables.exists(context, tableId))
           throw new TableDeletedException(tableId.canonicalID());
         else if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-          throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+          throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
     } catch (Exception e) {
       for (QCMutation qcm : mutations)
@@ -692,12 +692,13 @@ private void invalidateSession(SessionID sessionId, HostAndPort location)
     long startTime = System.currentTimeMillis();
 
     Instance instance = context.getInstance();
-    LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
+    LockID lid = new LockID(ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS,
+        sessionId.lockId);
 
     ZooCacheFactory zcf = new ZooCacheFactory();
     while (true) {
       if (!ZooLock.isLockHeld(
-          zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), lid)) {
+          zcf.getZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut()), lid)) {
         // ACCUMULO-1152 added a tserver lock check to the tablet location cache, so this
         // invalidation prevents future attempts to contact the
         // tserver even its gone zombie and is still running w/o a lock
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 96954b1605..bf1cac53d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -88,9 +88,9 @@ public ConnectorImpl(final ClientContext context)
   }
 
   private Table.ID getTableId(String tableName) throws TableNotFoundException {
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-      throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+      throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
     return tableId;
   }
 
@@ -199,6 +199,11 @@ public String whoami() {
     return context.getCredentials().getPrincipal();
   }
 
+  @Override
+  public String getInstanceID() {
+    return getInstance().getInstanceID();
+  }
+
   @Override
   public synchronized TableOperations tableOperations() {
     return tableops;
@@ -244,16 +249,23 @@ public ClientInfo info() {
       ConnectionOptions, SslOptions, SaslOptions, ConnectorFactory, FromOptions {
 
     private Properties properties = new Properties();
+    private AuthenticationToken token = null;
+
+    private ClientInfo getClientInfo() {
+      if (token != null) {
+        return new ClientInfoImpl(properties, token);
+      }
+      return new ClientInfoImpl(properties);
+    }
 
     @Override
     public Connector build() throws AccumuloException, AccumuloSecurityException {
-      return org.apache.accumulo.core.client.impl.ClientInfoFactory
-          .getConnector(new ClientInfoImpl(properties));
+      return org.apache.accumulo.core.client.impl.ClientInfoFactory.getConnector(getClientInfo());
     }
 
     @Override
     public ClientInfo info() {
-      return new ClientInfoImpl(properties);
+      return getClientInfo();
     }
 
     @Override
@@ -299,7 +311,7 @@ public SslOptions useJsse() {
 
     @Override
     public ConnectionOptions withZkTimeout(int timeout) {
-      setProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT_SEC, Integer.toString(timeout));
+      setProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, Integer.toString(timeout) + "ms");
       return this;
     }
 
@@ -375,7 +387,7 @@ public ConnectionOptions usingKerberos(String principal, String keyTabFile) {
     @Override
     public ConnectionOptions usingToken(String principal, AuthenticationToken token) {
       setProperty(ClientProperty.AUTH_PRINCIPAL, principal);
-      ClientProperty.setAuthenticationToken(this.properties, token);
+      this.token = token;
       return this;
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 157c7c9225..41dc026ab2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -28,7 +28,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.ActiveCompaction;
 import org.apache.accumulo.core.client.admin.ActiveScan;
@@ -92,10 +91,9 @@ public void removeProperty(final String property)
 
   @Override
   public List<String> getTabletServers() {
-    Instance instance = context.getInstance();
-    ZooCache cache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(),
-        instance.getZooKeepersSessionTimeOut());
-    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
+    ZooCache cache = new ZooCacheFactory().getZooCache(context.getZooKeepers(),
+        context.getZooKeepersSessionTimeOut());
+    String path = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS;
     List<String> results = new ArrayList<>();
     for (String candidate : cache.getChildren(path)) {
       List<String> children = cache.getChildren(path + "/" + candidate);
@@ -123,7 +121,7 @@ public void removeProperty(final String property)
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client
           .getActiveScans(Tracer.traceInfo(), context.rpcCreds())) {
         try {
-          as.add(new ActiveScanImpl(context.getInstance(), activeScan));
+          as.add(new ActiveScanImpl(context, activeScan));
         } catch (TableNotFoundException e) {
           throw new AccumuloException(e);
         }
@@ -159,7 +157,7 @@ public boolean testClassLoad(final String className, final String asTypeName)
       List<ActiveCompaction> as = new ArrayList<>();
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client
           .getActiveCompactions(Tracer.traceInfo(), context.rpcCreds())) {
-        as.add(new ActiveCompactionImpl(context.getInstance(), activeCompaction));
+        as.add(new ActiveCompactionImpl(context, activeCompaction));
       }
       return as;
     } catch (TTransportException e) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
index 63d721ac02..e979b31f90 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
@@ -54,7 +54,7 @@
   public static MasterClientService.Client getConnection(ClientContext context) {
     checkArgument(context != null, "context is null");
 
-    List<String> locations = context.getInstance().getMasterLocations();
+    List<String> locations = context.getMasterLocations();
 
     if (locations.size() == 0) {
       log.debug("No masters...");
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java
index e14240ba30..149943936e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MultiTableBatchWriterImpl.java
@@ -123,7 +123,7 @@ protected void finalize() {
    */
   private Table.ID getId(String tableName) throws TableNotFoundException {
     try {
-      return Tables.getTableId(context.inst, tableName);
+      return Tables.getTableId(context, tableName);
     } catch (UncheckedExecutionException e) {
       Throwable cause = e.getCause();
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
index ef26037b09..d33296abe7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
@@ -49,7 +49,6 @@
   public static ReplicationCoordinator.Client getCoordinatorConnectionWithRetry(
       ClientContext context) throws AccumuloException {
     requireNonNull(context);
-    Instance instance = context.getInstance();
 
     for (int attempts = 1; attempts <= 10; attempts++) {
 
@@ -57,7 +56,7 @@
       if (result != null)
         return result;
       log.debug("Could not get ReplicationCoordinator connection to {}, will retry",
-          instance.getInstanceName());
+          context.getInstanceName());
       try {
         Thread.sleep(attempts * 250);
       } catch (InterruptedException e) {
@@ -66,22 +65,22 @@
     }
 
     throw new AccumuloException(
-        "Timed out trying to communicate with master from " + instance.getInstanceName());
+        "Timed out trying to communicate with master from " + context.getInstanceName());
   }
 
   public static ReplicationCoordinator.Client getCoordinatorConnection(ClientContext context) {
     Instance instance = context.getInstance();
-    List<String> locations = instance.getMasterLocations();
+    List<String> locations = context.getMasterLocations();
 
     if (locations.size() == 0) {
-      log.debug("No masters for replication to instance {}", instance.getInstanceName());
+      log.debug("No masters for replication to instance {}", context.getInstanceName());
       return null;
     }
 
     // This is the master thrift service, we just want the hostname, not the port
     String masterThriftService = locations.get(0);
     if (masterThriftService.endsWith(":0")) {
-      log.warn("Master found for {} did not have real location {}", instance.getInstanceName(),
+      log.warn("Master found for {} did not have real location {}", context.getInstanceName(),
           masterThriftService);
       return null;
     }
@@ -94,8 +93,8 @@
 
     // Get the coordinator port for the master we're trying to connect to
     try {
-      ZooReader reader = new ZooReader(instance.getZooKeepers(),
-          instance.getZooKeepersSessionTimeOut());
+      ZooReader reader = new ZooReader(context.getZooKeepers(),
+          context.getZooKeepersSessionTimeOut());
       replCoordinatorAddr = new String(reader.getData(zkPath, null), UTF_8);
     } catch (KeeperException | InterruptedException e) {
       log.error("Could not fetch remote coordinator port", e);
@@ -189,8 +188,8 @@ public static void close(ReplicationServicer.Iface iface) {
       }
     }
 
-    throw new AccumuloException("Could not connect to ReplicationCoordinator at "
-        + context.getInstance().getInstanceName());
+    throw new AccumuloException(
+        "Could not connect to ReplicationCoordinator at " + context.getInstanceName());
   }
 
   public static <T> T executeServicerWithReturn(ClientContext context, HostAndPort tserver,
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index ecd6b78a1a..132261c8a7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -106,10 +106,10 @@ public void invalidateCache(Instance instance, String server) {
   public void invalidateCache() {}
 
   protected TabletLocation getRootTabletLocation(ClientContext context) {
-    Instance instance = context.getInstance();
-    String zRootLocPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_LOCATION;
-    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(),
-        instance.getZooKeepersSessionTimeOut());
+    String zRootLocPath = ZooUtil.getRoot(context.getInstanceID())
+        + RootTable.ZROOT_TABLET_LOCATION;
+    ZooCache zooCache = zcf.getZooCache(context.getZooKeepers(),
+        context.getZooKeepersSessionTimeOut());
 
     Logger log = LoggerFactory.getLogger(this.getClass());
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index c0edd5aedf..a94c71b07d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -26,7 +26,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -161,11 +160,11 @@ public static void executeRawVoid(ClientContext context, ClientExec<ClientServic
     ArrayList<ThriftTransportKey> servers = new ArrayList<>();
 
     // add tservers
-    Instance instance = context.getInstance();
-    ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(),
-        instance.getZooKeepersSessionTimeOut());
-    for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
-      String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver;
+    ZooCache zc = new ZooCacheFactory().getZooCache(context.getZooKeepers(),
+        context.getZooKeepersSessionTimeOut());
+    for (String tserver : zc
+        .getChildren(ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS)) {
+      String path = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS + "/" + tserver;
       byte[] data = ZooUtil.getLockData(zc, path);
       if (data != null) {
         String strData = new String(data, UTF_8);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index 1f33e3ab4e..928dc8c173 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -378,8 +378,8 @@ String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,Strin
         case NAMESPACE_NOTFOUND:
           throw new NamespaceNotFoundException(e);
         case OFFLINE:
-          throw new TableOfflineException(context.getInstance(),
-              Tables.getTableId(context.getInstance(), tableOrNamespaceName).canonicalID());
+          throw new TableOfflineException(
+              Tables.getTableOfflineMsg(context, Tables.getTableId(context, tableOrNamespaceName)));
         default:
           throw new AccumuloException(e.description, e);
       }
@@ -458,7 +458,7 @@ public void run() {
   @Override
   public void addSplits(String tableName, SortedSet<Text> partitionKeys)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     List<Text> splits = new ArrayList<>(partitionKeys);
     // should be sorted because we copied from a sorted set, but that makes assumptions about
@@ -488,7 +488,7 @@ public void addSplits(String tableName, SortedSet<Text> partitionKeys)
           } else if (excep instanceof TableOfflineException) {
             log.debug("TableOfflineException occurred in background thread. Throwing new exception",
                 excep);
-            throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+            throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
           } else if (excep instanceof AccumuloSecurityException) {
             // base == background accumulo security exception
             AccumuloSecurityException base = (AccumuloSecurityException) excep;
@@ -530,10 +530,10 @@ private void addSplits(String tableName, SortedSet<Text> partitionKeys, Table.ID
         TabletLocation tl = tabLocator.locateTablet(context, split, false, false);
 
         if (tl == null) {
-          if (!Tables.exists(context.getInstance(), tableId))
+          if (!Tables.exists(context, tableId))
             throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
           else if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-            throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+            throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
           continue;
         }
 
@@ -574,7 +574,7 @@ else if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFL
           continue;
         } catch (ThriftSecurityException e) {
           Tables.clearCache(context.getInstance());
-          if (!Tables.exists(context.getInstance(), tableId))
+          if (!Tables.exists(context, tableId))
             throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
           throw new AccumuloSecurityException(e.user, e.code, e);
         } catch (NotServingTabletException e) {
@@ -641,7 +641,7 @@ public void deleteRows(String tableName, Text start, Text end)
 
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
 
@@ -654,7 +654,7 @@ public void deleteRows(String tableName, Text start, Text end)
       } catch (AccumuloSecurityException ase) {
         throw ase;
       } catch (Exception e) {
-        if (!Tables.exists(context.getInstance(), tableId)) {
+        if (!Tables.exists(context, tableId)) {
           throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
         }
 
@@ -749,7 +749,7 @@ public void clone(String srcTableName, String newTableName, boolean flush,
     checkArgument(srcTableName != null, "srcTableName is null");
     checkArgument(newTableName != null, "newTableName is null");
 
-    Table.ID srcTableId = Tables.getTableId(context.getInstance(), srcTableName);
+    Table.ID srcTableId = Tables.getTableId(context, srcTableName);
 
     if (flush)
       _flush(srcTableId, null, null, true);
@@ -803,7 +803,7 @@ public void flush(String tableName, Text start, Text end, boolean wait)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     _flush(tableId, start, end, wait);
   }
 
@@ -847,7 +847,7 @@ public void compact(String tableName, CompactionConfig config)
       }
     }
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     Text start = config.getStartRow();
     Text end = config.getEndRow();
@@ -875,7 +875,7 @@ public void compact(String tableName, CompactionConfig config)
   @Override
   public void cancelCompaction(String tableName)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()));
 
@@ -1081,16 +1081,16 @@ public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups)
 
     Random random = new Random();
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     TabletLocator tl = TabletLocator.getLocator(context, tableId);
     // its possible that the cache could contain complete, but old information about a tables
     // tablets... so clear it
     tl.invalidateCache();
     while (!tl.binRanges(context, Collections.singletonList(range), binnedRanges).isEmpty()) {
-      if (!Tables.exists(context.getInstance(), tableId))
+      if (!Tables.exists(context, tableId))
         throw new TableDeletedException(tableId.canonicalID());
       if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-        throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+        throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
       log.warn("Unable to locate bins for specified range. Retrying.");
       // sleep randomly between 100 and 200ms
@@ -1175,7 +1175,7 @@ public void importDirectory(String tableName, String dir, String failureDir, boo
     checkArgument(dir != null, "dir is null");
     checkArgument(failureDir != null, "failureDir is null");
     // check for table existance
-    Tables.getTableId(context.getInstance(), tableName);
+    Tables.getTableId(context, tableName);
 
     Path dirPath = checkPath(dir, "Bulk", "");
     Path failPath = checkPath(failureDir, "Bulk", "failure");
@@ -1207,7 +1207,7 @@ private void waitForTableStateTransition(Table.ID tableId, TableState expectedSt
         Tables.clearCache(context.getInstance());
         TableState currentState = Tables.getTableState(context.getInstance(), tableId);
         if (currentState != expectedState) {
-          if (!Tables.exists(context.getInstance(), tableId))
+          if (!Tables.exists(context, tableId))
             throw new TableDeletedException(tableId.canonicalID());
           if (currentState == TableState.DELETING)
             throw new TableNotFoundException(tableId.canonicalID(), "", "Table is being deleted.");
@@ -1339,7 +1339,7 @@ public void offline(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 
     checkArgument(tableName != null, "tableName is null");
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()));
     Map<String,String> opts = new HashMap<>();
 
@@ -1366,7 +1366,7 @@ public void online(String tableName, boolean wait)
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     /**
      * ACCUMULO-4574 if table is already online return without executing fate operation.
@@ -1398,7 +1398,7 @@ public void online(String tableName, boolean wait)
   public void clearLocatorCache(String tableName) throws TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
     TabletLocator tabLocator = TabletLocator.getLocator(context,
-        Tables.getTableId(context.getInstance(), tableName));
+        Tables.getTableId(context, tableName));
     tabLocator.invalidateCache();
   }
 
@@ -1741,7 +1741,7 @@ public Locations locate(String tableName, Collection<Range> ranges)
     requireNonNull(tableName, "tableName must be non null");
     requireNonNull(ranges, "ranges must be non null");
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     TabletLocator locator = TabletLocator.getLocator(context, tableId);
 
     List<Range> rangeList = null;
@@ -1761,10 +1761,10 @@ public Locations locate(String tableName, Collection<Range> ranges)
 
     while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
 
-      if (!Tables.exists(context.getInstance(), tableId))
+      if (!Tables.exists(context, tableId))
         throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
       if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-        throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+        throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
       binnedRanges.clear();
 
@@ -1810,9 +1810,9 @@ public SummaryRetriever startRow(CharSequence startRow) {
       @Override
       public List<Summary> retrieve()
           throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-        Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+        Table.ID tableId = Tables.getTableId(context, tableName);
         if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-          throw new TableOfflineException(context.getInstance(), tableId.canonicalID());
+          throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
         TRowRange range = new TRowRange(TextUtil.getByteBuffer(startRow),
             TextUtil.getByteBuffer(endRow));
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 48c0eb06d5..b1a58f30a5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -56,6 +56,12 @@
    * Lookup table ID in ZK. Throw TableNotFoundException if not found. Also wraps
    * NamespaceNotFoundException in TableNotFoundException if namespace is not found.
    */
+
+  public static Table.ID getTableId(ClientContext context, String tableName)
+      throws TableNotFoundException {
+    return getTableId(context.getInstance(), tableName);
+  }
+
   public static Table.ID getTableId(Instance instance, String tableName)
       throws TableNotFoundException {
     try {
@@ -111,6 +117,11 @@ private static ZooCache getZooCache(final Instance instance) {
     return tableId;
   }
 
+  public static String getTableName(ClientContext context, Table.ID tableId)
+      throws TableNotFoundException {
+    return getTableName(context.getInstance(), tableId);
+  }
+
   public static String getTableName(Instance instance, Table.ID tableId)
       throws TableNotFoundException {
     String tableName = getIdToNameMap(instance).get(tableId);
@@ -119,6 +130,21 @@ public static String getTableName(Instance instance, Table.ID tableId)
     return tableName;
   }
 
+  public static String getTableOfflineMsg(ClientContext context, Table.ID tableId) {
+    return getTableOfflineMsg(context.getInstance(), tableId);
+  }
+
+  public static String getTableOfflineMsg(Instance instance, Table.ID tableId) {
+    if (tableId == null)
+      return "Table <unknown table> is offline";
+    try {
+      String tableName = Tables.getTableName(instance, tableId);
+      return "Table " + tableName + " (" + tableId.canonicalID() + ") is offline";
+    } catch (TableNotFoundException e) {
+      return "Table <unknown table> (" + tableId.canonicalID() + ") is offline";
+    }
+  }
+
   public static Map<String,Table.ID> getNameToIdMap(Instance instance) {
     return getTableMap(instance).getNameToIdMap();
   }
@@ -143,6 +169,10 @@ private static TableMap getTableMap(final Instance instance) {
     return map;
   }
 
+  public static boolean exists(ClientContext context, Table.ID tableId) {
+    return exists(context.getInstance(), tableId);
+  }
+
   public static boolean exists(Instance instance, Table.ID tableId) {
     ZooCache zc = getZooCache(instance);
     List<String> tableIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES);
@@ -191,6 +221,10 @@ public static String getPrintableTableInfoFromName(Instance instance, String tab
         : String.format("%s(ID:%s)", tableName, tableId.canonicalID());
   }
 
+  public static TableState getTableState(ClientContext context, Table.ID tableId) {
+    return getTableState(context.getInstance(), tableId);
+  }
+
   public static TableState getTableState(Instance instance, Table.ID tableId) {
     return getTableState(instance, tableId, false);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
index e4567523c2..80c7609111 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
@@ -113,7 +113,7 @@ public static synchronized void clearLocators() {
 
   public static synchronized TabletLocator getLocator(ClientContext context, Table.ID tableId) {
     Instance instance = context.getInstance();
-    LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId);
+    LocatorKey key = new LocatorKey(context.getInstanceID(), tableId);
     TabletLocator tl = locators.get(key);
     if (tl == null) {
       MetadataLocationObtainer mlo = new MetadataLocationObtainer();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index 0ccf8653b9..db47f6037e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -249,7 +249,7 @@ private void binRanges(TabletLocator tabletLocator, List<Range> ranges,
         // cleared... so
         // need to always do the check when failures occur
         if (failures.size() >= lastFailureSize)
-          if (!Tables.exists(instance, tableId))
+          if (!Tables.exists(context, tableId))
             throw new TableDeletedException(tableId.canonicalID());
           else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
             throw new TableOfflineException(instance, tableId.canonicalID());
@@ -387,7 +387,7 @@ public void run() {
         log.debug("AccumuloSecurityException thrown", e);
 
         Tables.clearCache(instance);
-        if (!Tables.exists(instance, tableId))
+        if (!Tables.exists(context, tableId))
           fatalException = new TableDeletedException(tableId.canonicalID());
         else
           fatalException = e;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index b3dc9dd68c..61dc81bee4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -512,7 +512,7 @@ private void updateAuthorizationFailures(Map<KeyExtent,SecurityErrorCode> author
 
       Tables.clearCache(context.getInstance());
       for (Table.ID tableId : tableIds)
-        if (!Tables.exists(context.getInstance(), tableId))
+        if (!Tables.exists(context, tableId))
           throw new TableDeletedException(tableId.canonicalID());
 
       synchronized (this) {
@@ -696,11 +696,11 @@ private void binMutations(MutationSet mutationsToProcess,
               failedMutations.add(tableId, tableFailures);
 
               if (tableFailures.size() == tableMutations.size())
-                if (!Tables.exists(context.getInstance(), entry.getKey()))
+                if (!Tables.exists(context, entry.getKey()))
                   throw new TableDeletedException(entry.getKey().canonicalID());
                 else if (Tables.getTableState(context.getInstance(), tableId) == TableState.OFFLINE)
-                  throw new TableOfflineException(context.getInstance(),
-                      entry.getKey().canonicalID());
+                  throw new TableOfflineException(
+                      Tables.getTableOfflineMsg(context, entry.getKey()));
             }
           }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index a0cd9d121e..3d7d48fa2d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -260,7 +260,8 @@ static long pause(long millis, long maxSleep) throws InterruptedException {
               if (!Tables.exists(instance, scanState.tableId))
                 throw new TableDeletedException(scanState.tableId.canonicalID());
               else if (Tables.getTableState(instance, scanState.tableId) == TableState.OFFLINE)
-                throw new TableOfflineException(instance, scanState.tableId.canonicalID());
+                throw new TableOfflineException(
+                    Tables.getTableOfflineMsg(context, scanState.tableId));
 
               error = "Failed to locate tablet for table : " + scanState.tableId + " row : "
                   + scanState.startRow;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 7966ae7ee9..b831110712 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client.mock;
 
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -36,9 +37,11 @@
 import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.impl.ClientInfoImpl;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
@@ -155,6 +158,11 @@ public String whoami() {
     return username;
   }
 
+  @Override
+  public String getInstanceID() {
+    return instance.getInstanceID();
+  }
+
   @Override
   public TableOperations tableOperations() {
     return new MockTableOperations(acu, username);
@@ -190,6 +198,10 @@ public ReplicationOperations replicationOperations() {
 
   @Override
   public ClientInfo info() {
-    throw new UnsupportedOperationException();
+    Properties props = new Properties();
+    props.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), instance.getZooKeepers());
+    props.setProperty(ClientProperty.INSTANCE_NAME.getKey(), instance.getInstanceName());
+    props.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), username);
+    return new ClientInfoImpl(props);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
index d9c71c4d5c..785296db1b 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ClientProperty.java
@@ -37,8 +37,7 @@
   INSTANCE_NAME("instance.name", "", "Name of Accumulo instance to connect to", "", true),
   INSTANCE_ZOOKEEPERS("instance.zookeepers", "localhost:2181",
       "Zookeeper connection information for Accumulo instance", "", true),
-  INSTANCE_ZOOKEEPERS_TIMEOUT_SEC("instance.zookeepers.timeout.sec", "30",
-      "Zookeeper session timeout (in seconds)"),
+  INSTANCE_ZOOKEEPERS_TIMEOUT("instance.zookeepers.timeout", "30s", "Zookeeper session timeout"),
 
   // Authentication
   AUTH_TYPE("auth.type", "password",
@@ -217,7 +216,6 @@ public static void setKerberosKeytab(Properties properties, String keytabPath) {
   }
 
   public static AuthenticationToken getAuthenticationToken(Properties properties) {
-    String principal = ClientProperty.AUTH_PRINCIPAL.getValue(properties);
     String authType = ClientProperty.AUTH_TYPE.getValue(properties);
     String token = ClientProperty.AUTH_TOKEN.getValue(properties);
     switch (authType) {
@@ -227,6 +225,7 @@ public static AuthenticationToken getAuthenticationToken(Properties properties)
         return decodeToken(PasswordToken.class.getName(), token);
       case "kerberos":
         try {
+          String principal = ClientProperty.AUTH_PRINCIPAL.getValue(properties);
           return new KerberosToken(principal, new File(token));
         } catch (IOException e) {
           throw new IllegalArgumentException(e);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
index 226fa6d814..61c5103535 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
@@ -20,7 +20,6 @@
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
@@ -32,10 +31,10 @@
  */
 class ServicerForRootTable extends MetadataServicer {
 
-  private final Instance instance;
+  private final ClientContext context;
 
   public ServicerForRootTable(ClientContext context) {
-    this.instance = context.getInstance();
+    this.context = context;
   }
 
   @Override
@@ -46,6 +45,6 @@ public ServicerForRootTable(ClientContext context) {
   @Override
   public void getTabletLocations(SortedMap<KeyExtent,String> tablets)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    tablets.put(RootTable.EXTENT, instance.getRootTabletLocation());
+    tablets.put(RootTable.EXTENT, context.getRootTabletLocation());
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java b/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java
index d65f32be1e..73e3f3de5c 100644
--- a/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java
+++ b/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java
@@ -97,7 +97,7 @@ public static void enable(String hostname, String service, Properties properties
     String spanReceivers = ClientProperty.TRACE_SPAN_RECEIVERS.getValue(properties);
     String zookeepers = ClientProperty.INSTANCE_ZOOKEEPERS.getValue(properties);
     long timeout = ConfigurationTypeHelper
-        .getTimeInMillis(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT_SEC.getValue(properties));
+        .getTimeInMillis(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getValue(properties));
     String zkPath = ClientProperty.TRACE_ZOOKEEPER_PATH.getValue(properties);
     Map<String,String> props = ClientProperty
         .toMap(ClientProperty.getPrefix(properties, ClientProperty.TRACE_SPAN_RECEIVER_PREFIX));
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
index 129ed306c9..fddffa2807 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
@@ -38,7 +38,8 @@ public void waitForStoreTransitionScannerConfiguredCorrectly() throws Exception
     Instance instance = EasyMock.createMock(Instance.class);
     Credentials credentials = EasyMock.createMock(Credentials.class);
 
-    ClientContext context = new ClientContext(instance, credentials, new Properties());
+    ClientContext context = new ClientContext(new ClientInfoImpl(new Properties(), null), instance,
+        credentials, null);
     TableOperationsImpl topsImpl = new TableOperationsImpl(context);
 
     Connector connector = EasyMock.createMock(Connector.class);
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 30908f924f..616785d10c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -175,7 +175,8 @@ static TabletLocatorImpl createLocators(String table, Object... data) {
   @Before
   public void setUp() {
     testInstance = new TestInstance("instance1", "tserver1");
-    context = new ClientContext(testInstance, new Credentials("test", null), new Properties());
+    context = new ClientContext(new ClientInfoImpl(new Properties(), null), testInstance,
+        new Credentials("test", null), null);
   }
 
   private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache,
@@ -591,8 +592,7 @@ public void invalidateCache(String server) {}
 
     @Override
     protected TabletLocation getRootTabletLocation(ClientContext context) {
-      return new TabletLocation(RootTable.EXTENT, context.getInstance().getRootTabletLocation(),
-          "1");
+      return new TabletLocation(RootTable.EXTENT, context.getRootTabletLocation(), "1");
     }
 
     @Override
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
index bdc796eff9..2ed3bef0c5 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
@@ -376,7 +376,7 @@ public void testMockConnectorReturnsCorrectInstance()
     assertEquals(mockInstance,
         mockInstance.getConnector("foo", new PasswordToken("bar")).getInstance());
     assertEquals(name,
-        mockInstance.getConnector("foo", new PasswordToken("bar")).getInstance().getInstanceName());
+        mockInstance.getConnector("foo", new PasswordToken("bar")).info().getInstanceName());
   }
 
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 5e1cab99be..b60bf77fe0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -58,15 +58,11 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.thrift.ThriftNotActiveServiceException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -175,6 +171,7 @@ public void run() {
 
   private File zooCfgFile;
   private String dfsUri;
+  private ClientInfo clientInfo;
 
   public List<LogWriter> getLogWriters() {
     return logWriters;
@@ -771,8 +768,7 @@ public MiniAccumuloConfigImpl getConfig() {
   @Override
   public Connector getConnector(String user, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
-    Instance instance = new ZooKeeperInstance(getClientConfig());
-    return instance.getConnector(user, token);
+    return Connector.builder().usingClientInfo(getClientInfo()).usingToken(user, token).build();
   }
 
   @SuppressWarnings("deprecation")
@@ -784,8 +780,11 @@ public Connector getConnector(String user, AuthenticationToken token)
 
   @Override
   public ClientInfo getClientInfo() {
-    return Connector.builder().forInstance(getInstanceName(), getZooKeepers())
-        .usingPassword(config.getRootUserName(), config.getRootPassword()).info();
+    if (clientInfo == null) {
+      clientInfo = Connector.builder()
+          .usingProperties(config.getClientPropsFile().getAbsolutePath()).info();
+    }
+    return clientInfo;
   }
 
   @Override
@@ -831,10 +830,7 @@ public MasterMonitorInfo getMasterMonitorInfo()
     MasterClientService.Iface client = null;
     while (true) {
       try {
-        Instance instance = new ZooKeeperInstance(getClientConfig());
-        ClientContext context = new ClientContext(instance,
-            new Credentials("root", new PasswordToken("unchecked")),
-            getClientInfo().getProperties());
+        ClientContext context = new ClientContext(getClientInfo());
         client = MasterClient.getConnectionWithRetry(context);
         return client.getMasterStats(Tracer.traceInfo(), context.rpcCreds());
       } catch (ThriftSecurityException exception) {
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 0de59a07c5..6ac37bd327 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -649,6 +650,14 @@ public void setProperty(Property p, String value) {
     this.siteConfig.put(p.getKey(), value);
   }
 
+  public void setClientProperty(ClientProperty property, String value) {
+    setClientProperty(property.getKey(), value);
+  }
+
+  public void setClientProperty(String key, String value) {
+    this.clientProps.put(key, value);
+  }
+
   /**
    * @return the useCredentialProvider
    */
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
index ef7d184d4b..711023b4ed 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
@@ -25,7 +25,7 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.commons.io.FileUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -90,15 +90,15 @@ public void teardownTestCluster() {
   @Test
   public void canConnectViaExistingZooKeeper() throws Exception {
     Connector conn = accumulo.getConnector("root", SECRET);
-    Instance instance = conn.getInstance();
-    assertEquals(zooKeeper.getConnectString(), instance.getZooKeepers());
+    ClientContext context = new ClientContext(accumulo.getClientInfo());
+    assertEquals(zooKeeper.getConnectString(), context.getZooKeepers());
 
     String tableName = "foo";
     conn.tableOperations().create(tableName);
     Map<String,String> tableIds = conn.tableOperations().tableIdMap();
     assertTrue(tableIds.containsKey(tableName));
 
-    String zkTablePath = String.format("/accumulo/%s/tables/%s/name", instance.getInstanceID(),
+    String zkTablePath = String.format("/accumulo/%s/tables/%s/name", context.getInstanceID(),
         tableIds.get(tableName));
     try (CuratorFramework client = CuratorFrameworkFactory.newClient(zooKeeper.getConnectString(),
         new RetryOneTime(1))) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index d63e5fc2f3..3ca9f6dbe3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -120,17 +120,17 @@ public ClientServiceHandler(AccumuloServerContext context, TransactionWatcher tr
 
   @Override
   public String getInstanceId() {
-    return instance.getInstanceID();
+    return context.getInstanceID();
   }
 
   @Override
   public String getRootTabletLocation() {
-    return instance.getRootTabletLocation();
+    return context.getRootTabletLocation();
   }
 
   @Override
   public String getZooKeepers() {
-    return instance.getZooKeepers();
+    return context.getZooKeepers();
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
index 24a7ea65bb..469052945e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
@@ -80,7 +80,7 @@ public TServerConnection(HostAndPort addr) throws TException {
 
     private String lockString(ZooLock mlock) {
       return mlock.getLockID()
-          .serialize(ZooUtil.getRoot(context.getInstance()) + Constants.ZMASTER_LOCK);
+          .serialize(ZooUtil.getRoot(context.getInstanceID()) + Constants.ZMASTER_LOCK);
     }
 
     private void loadTablet(TabletClientService.Client client, ZooLock lock, KeyExtent extent)
@@ -269,7 +269,7 @@ public synchronized void scanServers() {
       final Set<TServerInstance> updates = new HashSet<>();
       final Set<TServerInstance> doomed = new HashSet<>();
 
-      final String path = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS;
+      final String path = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS;
 
       HashSet<String> all = new HashSet<>(current.keySet());
       all.addAll(getZooCache().getChildren(path));
@@ -364,7 +364,7 @@ public void process(WatchedEvent event) {
           final Set<TServerInstance> updates = new HashSet<>();
           final Set<TServerInstance> doomed = new HashSet<>();
 
-          final String path = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS;
+          final String path = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS;
 
           try {
             checkServer(updates, doomed, path, server);
@@ -437,7 +437,7 @@ public synchronized void remove(TServerInstance server) {
     currentInstances.remove(server);
 
     log.info("Removing zookeeper lock for {}", server);
-    String fullpath = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS + "/" + zPath;
+    String fullpath = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS + "/" + zPath;
     try {
       ZooReaderWriter.getInstance().recursiveDelete(fullpath, SKIP);
     } catch (Exception e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index 2605a9d524..c6911880f2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -209,7 +209,7 @@ private static boolean isMeta(Table.ID tableId) {
               List<String> children;
               if (table == null || isMeta(table)) {
                 children = zoo
-                    .getChildren(ZooUtil.getRoot(context.getInstance()) + Constants.ZPROBLEMS);
+                    .getChildren(ZooUtil.getRoot(context.getInstanceID()) + Constants.ZPROBLEMS);
               } else {
                 children = Collections.emptyList();
               }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
index 44ab47eddf..f585f58334 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
@@ -71,7 +71,7 @@ public static synchronized SecurityOperation getInstance(AccumuloServerContext c
   public static synchronized SecurityOperation getInstance(AccumuloServerContext context,
       boolean initialize) {
     if (instance == null) {
-      String instanceId = context.getInstance().getInstanceID();
+      String instanceId = context.getInstanceID();
       instance = new AuditedSecurityOperation(context, getAuthorizor(instanceId, initialize),
           getAuthenticator(instanceId, initialize), getPermHandler(instanceId, initialize));
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index ae9d0cf9f5..92b55da227 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -83,7 +83,7 @@
   public static synchronized SecurityOperation getInstance(AccumuloServerContext context,
       boolean initialize) {
     if (instance == null) {
-      String instanceId = context.getInstance().getInstanceID();
+      String instanceId = context.getInstanceID();
       instance = new SecurityOperation(context, getAuthorizor(instanceId, initialize),
           getAuthenticator(instanceId, initialize), getPermHandler(instanceId, initialize));
     }
@@ -118,7 +118,7 @@ protected static PermissionHandler getPermHandler(String instanceId, boolean ini
 
   protected SecurityOperation(AccumuloServerContext context) {
     this.context = context;
-    ZKUserPath = Constants.ZROOT + "/" + context.getInstance().getInstanceID() + "/users";
+    ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
     zooCache = new ZooCache();
   }
 
@@ -169,7 +169,7 @@ public boolean isSystemUser(TCredentials credentials) {
   }
 
   protected void authenticate(TCredentials credentials) throws ThriftSecurityException {
-    if (!credentials.getInstanceId().equals(context.getInstance().getInstanceID()))
+    if (!credentials.getInstanceId().equals(context.getInstanceID()))
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.INVALID_INSTANCEID);
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index 4436f08fa1..521d2df938 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -226,7 +226,7 @@ public void execute(final String[] args) {
       int rc = 0;
 
       if (cl.getParsedCommand().equals("listInstances")) {
-        ListInstances.listInstances(instance.getZooKeepers(), listIntancesOpts.printAll,
+        ListInstances.listInstances(context.getZooKeepers(), listIntancesOpts.printAll,
             listIntancesOpts.printErrors);
       } else if (cl.getParsedCommand().equals("ping")) {
         if (ping(context, pingCommand.args) != 0)
@@ -369,14 +369,14 @@ private static void stopServer(final ClientContext context, final boolean tablet
 
   private static void stopTabletServer(final ClientContext context, List<String> servers,
       final boolean force) throws AccumuloException, AccumuloSecurityException {
-    if (context.getInstance().getMasterLocations().size() == 0) {
+    if (context.getMasterLocations().size() == 0) {
       log.info("No masters running. Not attempting safe unload of tserver.");
       return;
     }
     final Instance instance = context.getInstance();
     final String zTServerRoot = getTServersZkPath(instance);
-    final ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(),
-        instance.getZooKeepersSessionTimeOut());
+    final ZooCache zc = new ZooCacheFactory().getZooCache(context.getZooKeepers(),
+        context.getZooKeepersSessionTimeOut());
     for (String server : servers) {
       for (int port : context.getConfiguration().getPort(Property.TSERV_CLIENTPORT)) {
         HostAndPort address = AddressUtil.parseAddress(server, port);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
index a4d645faa7..675d2bac1d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
@@ -106,7 +106,7 @@ public void update(LiveTServerSet current, Set<TServerInstance> deleted,
 
     Range range = MetadataSchema.TabletsSection.getRange();
     if (tableName != null) {
-      Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+      Table.ID tableId = Tables.getTableId(context, tableName);
       range = new KeyExtent(tableId, null, null).toMetadataRange();
     }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index 3d568be45a..585c72b92c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -33,7 +33,6 @@
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.Key;
@@ -199,7 +198,7 @@ static int checkTable(ClientContext context, String tableName, boolean fix) thro
     } else if (tableName.equals(MetadataTable.NAME)) {
       return checkTable(context, RootTable.NAME, MetadataSchema.TabletsSection.getRange(), fix);
     } else {
-      Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+      Table.ID tableId = Tables.getTableId(context, tableName);
       Range range = new KeyExtent(tableId, null, null).toMetadataRange();
       return checkTable(context, MetadataTable.NAME, range, fix);
     }
@@ -211,8 +210,7 @@ public static void main(String[] args) throws Exception {
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(RemoveEntriesForMissingFiles.class.getName(), args, scanOpts, bwOpts);
 
-    checkAllTables(new ClientContext(opts.getInstance(),
-        new Credentials(opts.getPrincipal(), opts.getToken()), opts.getClientProperties()),
-        opts.fix);
+    Connector conn = opts.getConnector();
+    checkAllTables(new ClientContext(conn.info()), opts.fix);
   }
 }
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 d8df68b167..e039ca8754 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
@@ -32,7 +32,6 @@
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -74,9 +73,8 @@ public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(VerifyTabletAssignments.class.getName(), args);
 
-    ClientContext context = new ClientContext(opts.getInstance(),
-        new Credentials(opts.getPrincipal(), opts.getToken()), opts.getClientProperties());
     Connector conn = opts.getConnector();
+    ClientContext context = new ClientContext(conn.info());
     for (String table : conn.tableOperations().list())
       checkTable(context, opts, table, null);
 
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index fef1d349b3..714324d4c4 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@ -234,7 +234,7 @@ protected void closeWal(BatchWriter bw, Key k) throws MutationsRejectedException
 
   private HostAndPort getMasterAddress() {
     try {
-      List<String> locations = context.getInstance().getMasterLocations();
+      List<String> locations = context.getMasterLocations();
       if (locations.size() == 0)
         return null;
       return HostAndPort.fromString(locations.get(0));
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 3f34f16ccf..e88b574f3a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -282,7 +282,7 @@ public void run() {
   }
 
   private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
-    String dirZPath = ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_PATH;
+    String dirZPath = ZooUtil.getRoot(getInstanceID()) + RootTable.ZROOT_TABLET_PATH;
 
     if (!zoo.exists(dirZPath)) {
       Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
@@ -343,7 +343,7 @@ private void upgradeZookeeper() {
         log.info("Upgrading zookeeper");
 
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-        final String zooRoot = ZooUtil.getRoot(getInstance());
+        final String zooRoot = ZooUtil.getRoot(getInstanceID());
 
         log.debug("Handling updates for version {}", accumuloPersistentVersion);
 
@@ -400,7 +400,7 @@ private void upgradeZookeeper() {
         }
 
         // create initial namespaces
-        String namespaces = ZooUtil.getRoot(getInstance()) + Constants.ZNAMESPACES;
+        String namespaces = ZooUtil.getRoot(getInstanceID()) + Constants.ZNAMESPACES;
         zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
         for (Pair<String,Namespace.ID> namespace : Iterables.concat(
             Collections.singleton(new Pair<>(Namespace.ACCUMULO, Namespace.ID.ACCUMULO)),
@@ -409,27 +409,26 @@ private void upgradeZookeeper() {
           Namespace.ID id = namespace.getSecond();
           log.debug("Upgrade creating namespace \"{}\" (ID: {})", ns, id);
           if (!Namespaces.exists(getInstance(), id))
-            TableManager.prepareNewNamespaceState(getInstance().getInstanceID(), id, ns,
-                NodeExistsPolicy.SKIP);
+            TableManager.prepareNewNamespaceState(getInstanceID(), id, ns, NodeExistsPolicy.SKIP);
         }
 
         // create replication table in zk
         log.debug("Upgrade creating table {} (ID: {})", ReplicationTable.NAME, ReplicationTable.ID);
-        TableManager.prepareNewTableState(getInstance().getInstanceID(), ReplicationTable.ID,
+        TableManager.prepareNewTableState(getInstanceID(), ReplicationTable.ID,
             Namespace.ID.ACCUMULO, ReplicationTable.NAME, TableState.OFFLINE,
             NodeExistsPolicy.SKIP);
 
         // create root table
         log.debug("Upgrade creating table {} (ID: {})", RootTable.NAME, RootTable.ID);
-        TableManager.prepareNewTableState(getInstance().getInstanceID(), RootTable.ID,
-            Namespace.ID.ACCUMULO, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
+        TableManager.prepareNewTableState(getInstanceID(), RootTable.ID, Namespace.ID.ACCUMULO,
+            RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
         Initialize.initSystemTablesConfig();
         // ensure root user can flush root table
         security.grantTablePermission(rpcCreds(), security.getRootUsername(), RootTable.ID,
             TablePermission.ALTER_TABLE, Namespace.ID.ACCUMULO);
 
         // put existing tables in the correct namespaces
-        String tables = ZooUtil.getRoot(getInstance()) + Constants.ZTABLES;
+        String tables = ZooUtil.getRoot(getInstanceID()) + Constants.ZTABLES;
         for (String tableId : zoo.getChildren(tables)) {
           Namespace.ID targetNamespace = (MetadataTable.ID.canonicalID().equals(tableId)
               || RootTable.ID.canonicalID().equals(tableId)) ? Namespace.ID.ACCUMULO
@@ -452,8 +451,8 @@ private void upgradeZookeeper() {
 
         // add system namespace permissions to existing users
         ZKPermHandler perm = new ZKPermHandler();
-        perm.initialize(getInstance().getInstanceID(), true);
-        String users = ZooUtil.getRoot(getInstance()) + "/users";
+        perm.initialize(getInstanceID(), true);
+        String users = ZooUtil.getRoot(getInstanceID()) + "/users";
         for (String user : zoo.getChildren(users)) {
           zoo.putPersistentData(users + "/" + user + "/Namespaces", new byte[0],
               NodeExistsPolicy.SKIP);
@@ -463,11 +462,12 @@ private void upgradeZookeeper() {
             NamespacePermission.ALTER_TABLE);
 
         // add the currlog location for root tablet current logs
-        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS,
-            new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(
+            ZooUtil.getRoot(getInstanceID()) + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0],
+            NodeExistsPolicy.SKIP);
 
         // create tablet server wal logs node in ZK
-        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + WalStateManager.ZWALS, new byte[0],
+        zoo.putPersistentData(ZooUtil.getRoot(getInstanceID()) + WalStateManager.ZWALS, new byte[0],
             NodeExistsPolicy.SKIP);
 
         haveUpgradedZooKeeper = true;
@@ -641,7 +641,7 @@ public Master(Instance instance, ServerConfigurationFactory config, VolumeManage
     AccumuloConfiguration aconf = serverConfig.getSystemConfiguration();
 
     log.info("Version {}", Constants.VERSION);
-    log.info("Instance {}", getInstance().getInstanceID());
+    log.info("Instance {}", getInstanceID());
     timeKeeper = new MasterTime(this);
     ThriftTransportPool.getInstance()
         .setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
@@ -678,7 +678,7 @@ public Master(Instance instance, ServerConfigurationFactory config, VolumeManage
       final long tokenUpdateInterval = aconf
           .getTimeInMillis(Property.GENERAL_DELEGATION_TOKEN_UPDATE_INTERVAL);
       keyDistributor = new ZooAuthenticationKeyDistributor(ZooReaderWriter.getInstance(),
-          ZooUtil.getRoot(getInstance()) + Constants.ZDELEGATION_TOKEN_KEYS);
+          ZooUtil.getRoot(getInstanceID()) + Constants.ZDELEGATION_TOKEN_KEYS);
       authenticationTokenKeyManager = new AuthenticationTokenKeyManager(getSecretManager(),
           keyDistributor, tokenUpdateInterval, tokenLifetime);
       delegationTokensAvailable = true;
@@ -696,8 +696,8 @@ public TServerConnection getConnection(TServerInstance server) {
   public MergeInfo getMergeInfo(Table.ID tableId) {
     synchronized (mergeLock) {
       try {
-        String path = ZooUtil.getRoot(getInstance().getInstanceID()) + Constants.ZTABLES + "/"
-            + tableId + "/merge";
+        String path = ZooUtil.getRoot(getInstanceID()) + Constants.ZTABLES + "/" + tableId
+            + "/merge";
         if (!ZooReaderWriter.getInstance().exists(path))
           return new MergeInfo();
         byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
@@ -719,7 +719,7 @@ public MergeInfo getMergeInfo(Table.ID tableId) {
   public void setMergeState(MergeInfo info, MergeState state)
       throws IOException, KeeperException, InterruptedException {
     synchronized (mergeLock) {
-      String path = ZooUtil.getRoot(getInstance().getInstanceID()) + Constants.ZTABLES + "/"
+      String path = ZooUtil.getRoot(getInstanceID()) + Constants.ZTABLES + "/"
           + info.getExtent().getTableId() + "/merge";
       info.setState(state);
       if (state.equals(MergeState.NONE)) {
@@ -743,8 +743,7 @@ public void setMergeState(MergeInfo info, MergeState state)
   public void clearMergeState(Table.ID tableId)
       throws IOException, KeeperException, InterruptedException {
     synchronized (mergeLock) {
-      String path = ZooUtil.getRoot(getInstance().getInstanceID()) + Constants.ZTABLES + "/"
-          + tableId + "/merge";
+      String path = ZooUtil.getRoot(getInstanceID()) + Constants.ZTABLES + "/" + tableId + "/merge";
       ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
       mergeLock.notifyAll();
     }
@@ -754,7 +753,7 @@ public void clearMergeState(Table.ID tableId)
   void setMasterGoalState(MasterGoalState state) {
     try {
       ZooReaderWriter.getInstance().putPersistentData(
-          ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
+          ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
           NodeExistsPolicy.OVERWRITE);
     } catch (Exception ex) {
       log.error("Unable to set master goal state in zookeeper");
@@ -765,7 +764,7 @@ MasterGoalState getMasterGoalState() {
     while (true)
       try {
         byte[] data = ZooReaderWriter.getInstance()
-            .getData(ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_GOAL_STATE, null);
+            .getData(ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_GOAL_STATE, null);
         return MasterGoalState.valueOf(new String(data));
       } catch (Exception e) {
         log.error("Problem getting real goal state from zookeeper: ", e);
@@ -1206,7 +1205,7 @@ public void run() {
   }
 
   public void run() throws IOException, InterruptedException, KeeperException {
-    final String zroot = ZooUtil.getRoot(getInstance());
+    final String zroot = ZooUtil.getRoot(getInstanceID());
 
     // ACCUMULO-4424 Put up the Thrift servers before getting the lock as a sign of process health
     // when a hot-standby
@@ -1313,8 +1312,8 @@ boolean canSuspendTablets() {
 
     try {
       final AgeOffStore<Master> store = new AgeOffStore<>(
-          new org.apache.accumulo.fate.ZooStore<>(ZooUtil.getRoot(getInstance()) + Constants.ZFATE,
-              ZooReaderWriter.getInstance()),
+          new org.apache.accumulo.fate.ZooStore<>(
+              ZooUtil.getRoot(getInstanceID()) + Constants.ZFATE, ZooReaderWriter.getInstance()),
           1000 * 60 * 60 * 8);
 
       int threads = getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
@@ -1377,7 +1376,7 @@ public void run() {
 
     // Advertise that port we used so peers don't have to be told what it is
     ZooReaderWriter.getInstance().putPersistentData(
-        ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
+        ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
         replAddress.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
 
     // Register replication metrics
@@ -1547,7 +1546,7 @@ public static void main(String[] args) throws Exception {
   public void update(LiveTServerSet current, Set<TServerInstance> deleted,
       Set<TServerInstance> added) {
     DeadServerList obit = new DeadServerList(
-        ZooUtil.getRoot(getInstance()) + Constants.ZDEADTSERVERS);
+        ZooUtil.getRoot(getInstanceID()) + Constants.ZDEADTSERVERS);
     if (added.size() > 0) {
       log.info("New servers: {}", added);
       for (TServerInstance up : added)
@@ -1735,7 +1734,7 @@ public MasterMonitorInfo getMasterMonitorInfo() {
         result.serversShuttingDown.add(server.hostPort());
     }
     DeadServerList obit = new DeadServerList(
-        ZooUtil.getRoot(getInstance()) + Constants.ZDEADTSERVERS);
+        ZooUtil.getRoot(getInstanceID()) + Constants.ZDEADTSERVERS);
     result.deadTabletServers = obit.getList();
     result.bulkImports = bulkImportStatus.getBulkLoadStatus();
     return result;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 3511c8287e..786331989a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -122,8 +122,8 @@ public long initiateFlush(TInfo tinfo, TCredentials c, String tableIdStr)
     Namespace.ID namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
     master.security.canFlush(c, tableId, namespaceId);
 
-    String zTablePath = Constants.ZROOT + "/" + master.getInstance().getInstanceID()
-        + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_FLUSH_ID;
+    String zTablePath = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
+        + tableId + Constants.ZTABLE_FLUSH_ID;
 
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     byte fid[];
@@ -254,7 +254,7 @@ public void waitForFlush(TInfo tinfo, TCredentials c, String tableIdStr, ByteBuf
 
         // TODO detect case of table offline AND tablets w/ logs? - ACCUMULO-1296
 
-        if (tabletCount == 0 && !Tables.exists(master.getInstance(), tableId))
+        if (tabletCount == 0 && !Tables.exists(master, tableId))
           throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.FLUSH,
               TableOperationExceptionType.NOTFOUND, null);
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java b/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
index 43845c5264..359751e8ca 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
@@ -50,7 +50,7 @@
   private long skewAmount;
 
   public MasterTime(Master master) throws IOException {
-    this.zPath = ZooUtil.getRoot(master.getInstance()) + Constants.ZMASTER_TICK;
+    this.zPath = ZooUtil.getRoot(master.getInstanceID()) + Constants.ZMASTER_TICK;
     this.zk = ZooReaderWriter.getInstance();
     this.master = master;
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index d81c221d94..fd35982cfd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -69,7 +69,7 @@ public RecoveryManager(Master master) {
     try {
       AccumuloConfiguration aconf = master.getConfiguration();
       List<String> workIDs = new DistributedWorkQueue(
-          ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY, aconf).getWorkQueued();
+          ZooUtil.getRoot(master.getInstanceID()) + Constants.ZRECOVERY, aconf).getWorkQueued();
       sortsQueued.addAll(workIDs);
     } catch (Exception e) {
       log.warn("{}", e.getMessage(), e);
@@ -120,14 +120,15 @@ public void run() {
   private void initiateSort(String sortId, String source, final String destination,
       AccumuloConfiguration aconf) throws KeeperException, InterruptedException, IOException {
     String work = source + "|" + destination;
-    new DistributedWorkQueue(ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY, aconf)
+    new DistributedWorkQueue(ZooUtil.getRoot(master.getInstanceID()) + Constants.ZRECOVERY, aconf)
         .addWork(sortId, work.getBytes(UTF_8));
 
     synchronized (this) {
       sortsQueued.add(sortId);
     }
 
-    final String path = ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY + "/" + sortId;
+    final String path = ZooUtil.getRoot(master.getInstanceID()) + Constants.ZRECOVERY + "/"
+        + sortId;
     log.info("Created zookeeper entry {} with data {}", path, work);
   }
 
@@ -161,7 +162,7 @@ public boolean recoverLogs(KeyExtent extent, Collection<Collection<String>> walo
         }
 
         if (sortQueued && zooCache.get(
-            ZooUtil.getRoot(master.getInstance()) + Constants.ZRECOVERY + "/" + sortId) == null) {
+            ZooUtil.getRoot(master.getInstanceID()) + Constants.ZRECOVERY + "/" + sortId) == null) {
           synchronized (this) {
             sortsQueued.remove(sortId);
           }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
index d6f31e7af0..53cb3fd3bb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
@@ -115,7 +115,7 @@ protected void setZooCache(ZooCache zooCache) {
    */
   protected void initializeWorkQueue(AccumuloConfiguration conf) {
     workQueue = new DistributedWorkQueue(
-        ZooUtil.getRoot(conn.getInstance()) + ReplicationConstants.ZOO_WORK_QUEUE, conf);
+        ZooUtil.getRoot(conn.getInstanceID()) + ReplicationConstants.ZOO_WORK_QUEUE, conf);
   }
 
   @Override
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java b/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java
index 3453f24e34..3c71fbba61 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java
@@ -23,7 +23,6 @@
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.replication.ReplicationConstants;
 import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
@@ -47,20 +46,17 @@
   private static final Logger log = LoggerFactory.getLogger(MasterReplicationCoordinator.class);
 
   private final Master master;
-  private final Instance inst;
   private final Random rand;
   private final ZooReader reader;
   private final SecurityOperation security;
 
   public MasterReplicationCoordinator(Master master) {
-    this(master, new ZooReader(master.getInstance().getZooKeepers(),
-        master.getInstance().getZooKeepersSessionTimeOut()));
+    this(master, new ZooReader(master.getZooKeepers(), master.getZooKeepersSessionTimeOut()));
   }
 
   protected MasterReplicationCoordinator(Master master, ZooReader reader) {
     this.master = master;
     this.rand = new Random(358923462L);
-    this.inst = master.getInstance();
     this.reader = reader;
     this.security = SecurityOperation.getInstance(master, false);
   }
@@ -87,9 +83,8 @@ public String getServicerAddress(String remoteTableId, TCredentials creds)
     TServerInstance tserver = getRandomTServer(tservers, rand.nextInt(tservers.size()));
     String replServiceAddr;
     try {
-      replServiceAddr = new String(reader.getData(
-          ZooUtil.getRoot(inst) + ReplicationConstants.ZOO_TSERVERS + "/" + tserver.hostPort(),
-          null), UTF_8);
+      replServiceAddr = new String(reader.getData(ZooUtil.getRoot(master.getInstanceID())
+          + ReplicationConstants.ZOO_TSERVERS + "/" + tserver.hostPort(), null), UTF_8);
     } catch (KeeperException | InterruptedException e) {
       log.error("Could not fetch repliation service port for tserver", e);
       throw new ReplicationCoordinatorException(
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
index fe3aca7820..5f1624c02c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
@@ -147,7 +147,7 @@ public long isReady(long tid, Master master) throws Exception {
 
     Instance instance = master.getInstance();
     Tables.clearCache(instance);
-    if (tabletCount == 0 && !Tables.exists(instance, tableId))
+    if (tabletCount == 0 && !Tables.exists(master, tableId))
       throw new AcceptableThriftTableOperationException(tableId.canonicalID(), null,
           TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
index 1e35095596..f795e9ff4d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
@@ -60,8 +60,8 @@ public RenameNamespace(Namespace.ID namespaceId, String oldName, String newName)
     try {
       Utils.checkNamespaceDoesNotExist(instance, newName, namespaceId, TableOperation.RENAME);
 
-      final String tap = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId
-          + Constants.ZNAMESPACE_NAME;
+      final String tap = ZooUtil.getRoot(master.getInstanceID()) + Constants.ZNAMESPACES + "/"
+          + namespaceId + Constants.ZNAMESPACE_NAME;
 
       zoo.mutate(tap, null, null, new Mutator() {
         @Override
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index f3f27f7e87..1599ed293c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -81,7 +81,7 @@ public RenameTable(Namespace.ID namespaceId, Table.ID tableId, String oldTableNa
       final String newName = qualifiedNewTableName.getSecond();
       final String oldName = qualifiedOldTableName.getSecond();
 
-      final String tap = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId
+      final String tap = ZooUtil.getRoot(master.getInstanceID()) + Constants.ZTABLES + "/" + tableId
           + Constants.ZTABLE_NAME;
 
       zoo.mutate(tap, null, null, new Mutator() {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
index f8871760b2..8fcf1dedf3 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
@@ -158,9 +158,9 @@ public static void exportTable(VolumeManager fs, AccumuloServerContext context,
 
       zipOut.putNextEntry(new ZipEntry(Constants.EXPORT_INFO_FILE));
       osw.append(ExportTable.EXPORT_VERSION_PROP + ":" + ExportTable.VERSION + "\n");
-      osw.append("srcInstanceName:" + context.getInstance().getInstanceName() + "\n");
-      osw.append("srcInstanceID:" + context.getInstance().getInstanceID() + "\n");
-      osw.append("srcZookeepers:" + context.getInstance().getZooKeepers() + "\n");
+      osw.append("srcInstanceName:" + context.getInstanceName() + "\n");
+      osw.append("srcInstanceID:" + context.getInstanceID() + "\n");
+      osw.append("srcZookeepers:" + context.getZooKeepers() + "\n");
       osw.append("srcTableName:" + tableName + "\n");
       osw.append("srcTableID:" + tableID.canonicalID() + "\n");
       osw.append(ExportTable.DATA_VERSION_PROP + ":" + ServerConstants.DATA_VERSION + "\n");
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
index e39654ab2d..f19d825627 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
@@ -38,6 +38,7 @@ public void randomServer() {
 
     EasyMock.expect(master.getInstance()).andReturn(inst);
     EasyMock.expect(inst.getInstanceID()).andReturn("1234");
+    EasyMock.expect(master.getInstanceID()).andReturn("1234");
 
     EasyMock.replay(master, reader, inst);
 
@@ -55,6 +56,7 @@ public void invalidOffset() {
 
     EasyMock.expect(master.getInstance()).andReturn(inst);
     EasyMock.expect(inst.getInstanceID()).andReturn("1234");
+    EasyMock.expect(master.getInstanceID()).andReturn("1234");
 
     EasyMock.replay(master, reader, inst);
 
@@ -72,6 +74,7 @@ public void randomServerFromMany() {
 
     EasyMock.expect(master.getInstance()).andReturn(inst).anyTimes();
     EasyMock.expect(inst.getInstanceID()).andReturn("1234").anyTimes();
+    EasyMock.expect(master.getInstanceID()).andReturn("1234").anyTimes();
 
     EasyMock.replay(master, reader, inst);
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 42b8959dbe..aaecf4e6fd 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -453,7 +453,7 @@ public static void main(String[] args) throws Exception {
     config = new ServerConfigurationFactory(instance);
     context = new AccumuloServerContext(instance, config);
     log.info("Version " + Constants.VERSION);
-    log.info("Instance " + instance.getInstanceID());
+    log.info("Instance " + context.getInstanceID());
     MetricsSystemHelper.configure(Monitor.class.getSimpleName());
     Accumulo.init(fs, instance, config, app);
     Monitor monitor = new Monitor();
@@ -520,7 +520,7 @@ public void run(String hostname) {
     }
 
     if (null != advertiseHost) {
-      LogService.startLogListener(Monitor.getContext().getConfiguration(), instance.getInstanceID(),
+      LogService.startLogListener(Monitor.getContext().getConfiguration(), context.getInstanceID(),
           advertiseHost);
     } else {
       log.warn("Not starting log4j listener as we could not determine address to use");
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
index 0ab0abc4ff..d28a7e0646 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/master/MasterResource.java
@@ -97,7 +97,7 @@ public static MasterInformation getTables() {
       for (DeadServer down : mmi.deadTabletServers) {
         tservers.add(down.server);
       }
-      List<String> masters = Monitor.getContext().getInstance().getMasterLocations();
+      List<String> masters = Monitor.getContext().getMasterLocations();
 
       String master = masters.size() == 0 ? "Down"
           : AddressUtil.parseAddress(masters.get(0), false).getHost();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
index 7d4f5293a9..4bde31d412 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
@@ -71,7 +71,7 @@ public StatusInformation getTables() {
       for (DeadServer down : Monitor.getMmi().deadTabletServers) {
         tservers.add(down.server);
       }
-      List<String> masters = Monitor.getContext().getInstance().getMasterLocations();
+      List<String> masters = Monitor.getContext().getMasterLocations();
 
       masterStatus = masters.size() == 0 ? Status.ERROR : Status.OK;
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
index af2fdc351f..321c62ff67 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
@@ -33,7 +33,6 @@
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.Range;
@@ -116,7 +115,7 @@ public static TableInformationList getTables() {
   @GET
   public TabletServers getParticipatingTabletServers(@PathParam("tableId") @NotNull @Pattern(
       regexp = ALPHA_NUM_REGEX_TABLE_ID) String tableIdStr) {
-    Instance instance = Monitor.getContext().getInstance();
+    String rootTabletLocation = Monitor.getContext().getRootTabletLocation();
     Table.ID tableId = Table.ID.of(tableIdStr);
 
     TabletServers tabletServers = new TabletServers(Monitor.getMmi().tServerInfo.size());
@@ -127,7 +126,7 @@ public TabletServers getParticipatingTabletServers(@PathParam("tableId") @NotNul
 
     TreeSet<String> locs = new TreeSet<>();
     if (RootTable.ID.equals(tableId)) {
-      locs.add(instance.getRootTabletLocation());
+      locs.add(rootTabletLocation);
     } else {
       String systemTableName = MetadataTable.ID.equals(tableId) ? RootTable.NAME
           : MetadataTable.NAME;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index fa5ffc4287..a7517d463f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -363,7 +363,7 @@ public TabletServer(Instance instance, ServerConfigurationFactory confFactory, V
     this.fs = fs;
     final AccumuloConfiguration aconf = getConfiguration();
     log.info("Version " + Constants.VERSION);
-    log.info("Instance " + instance.getInstanceID());
+    log.info("Instance " + getInstanceID());
     this.sessionManager = new SessionManager(aconf);
     this.logSorter = new LogSorter(instance, fs, aconf);
     this.replWorker = new ReplicationWorker(this, fs);
@@ -438,7 +438,7 @@ public void run() {
       // Watcher to notice new AuthenticationKeys which enable delegation tokens
       authKeyWatcher = new ZooAuthenticationKeyWatcher(getSecretManager(),
           ZooReaderWriter.getInstance(),
-          ZooUtil.getRoot(instance) + Constants.ZDELEGATION_TOKEN_KEYS);
+          ZooUtil.getRoot(getInstanceID()) + Constants.ZDELEGATION_TOKEN_KEYS);
     } else {
       authKeyWatcher = null;
     }
@@ -1671,7 +1671,7 @@ public void run() {
 
       if (lock != null) {
         ZooUtil.LockID lid = new ZooUtil.LockID(
-            ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_LOCK, lock);
+            ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_LOCK, lock);
 
         try {
           if (!ZooLock.isLockHeld(masterLockCache, lid)) {
@@ -2597,7 +2597,7 @@ private HostAndPort startServer(AccumuloConfiguration conf, String address, Prop
 
   private HostAndPort getMasterAddress() {
     try {
-      List<String> locations = getInstance().getMasterLocations();
+      List<String> locations = getMasterLocations();
       if (locations.size() == 0)
         return null;
       return HostAndPort.fromString(locations.get(0));
@@ -2668,9 +2668,11 @@ private HostAndPort startReplicationService() throws UnknownHostException {
       // The replication service is unique to the thrift service for a tserver, not just a host.
       // Advertise the host and port for replication service given the host and port for the
       // tserver.
-      ZooReaderWriter.getInstance().putPersistentData(
-          ZooUtil.getRoot(getInstance()) + ReplicationConstants.ZOO_TSERVERS + "/" + clientAddress,
-          sp.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
+      ZooReaderWriter.getInstance()
+          .putPersistentData(
+              ZooUtil.getRoot(getInstanceID()) + ReplicationConstants.ZOO_TSERVERS + "/"
+                  + clientAddress,
+              sp.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     } catch (Exception e) {
       log.error("Could not advertise replication service port", e);
       throw new RuntimeException(e);
@@ -2686,7 +2688,7 @@ public ZooLock getLock() {
   private void announceExistence() {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
-      String zPath = ZooUtil.getRoot(getInstance()) + Constants.ZTSERVERS + "/"
+      String zPath = ZooUtil.getRoot(getInstanceID()) + Constants.ZTSERVERS + "/"
           + getClientAddressString();
 
       try {
@@ -2735,7 +2737,7 @@ public void run() {
         if (tabletServerLock.tryLock(lw, lockContent)) {
           log.debug("Obtained tablet server lock {}", tabletServerLock.getLockPath());
           lockID = tabletServerLock.getLockID()
-              .serialize(ZooUtil.getRoot(getInstance()) + Constants.ZTSERVERS + "/");
+              .serialize(ZooUtil.getRoot(getInstanceID()) + Constants.ZTSERVERS + "/");
           return;
         }
         log.info("Waiting for tablet server lock");
@@ -2759,7 +2761,7 @@ public void run() {
     // We can just make the zookeeper paths before we try to use.
     try {
       ZooKeeperInitialization.ensureZooKeeperInitialized(ZooReaderWriter.getInstance(),
-          ZooUtil.getRoot(getInstance()));
+          ZooUtil.getRoot(getInstanceID()));
     } catch (KeeperException | InterruptedException e) {
       log.error("Could not ensure that ZooKeeper is properly initialized", e);
       throw new RuntimeException(e);
@@ -2808,7 +2810,7 @@ public void run() {
         getConfiguration().getCount(Property.TSERV_WORKQ_THREADS), "distributed work queue");
 
     bulkFailedCopyQ = new DistributedWorkQueue(
-        ZooUtil.getRoot(getInstance()) + Constants.ZBULK_FAILED_COPYQ, getConfiguration());
+        ZooUtil.getRoot(getInstanceID()) + Constants.ZBULK_FAILED_COPYQ, getConfiguration());
     try {
       bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(), distWorkQThreadPool);
     } catch (Exception e1) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index 3f9fc62e61..0eda5c7e5b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -407,7 +407,7 @@ protected Status replicateLogs(ClientContext peerContext, final HostAndPort peer
         span = Trace.start("Replicate WAL batch");
         span.data("Batch size (bytes)", Long.toString(sizeLimit));
         span.data("File", p.toString());
-        span.data("Peer instance name", peerContext.getInstance().getInstanceName());
+        span.data("Peer instance name", peerContext.getInstanceName());
         span.data("Peer tserver", peerTserver.toString());
         span.data("Remote table ID", remoteTableId);
 
@@ -419,8 +419,8 @@ protected Status replicateLogs(ClientContext peerContext, final HostAndPort peer
                   tcreds, tids),
               timeout);
         } catch (Exception e) {
-          log.error("Caught exception replicating data to {} at {}",
-              peerContext.getInstance().getInstanceName(), peerTserver, e);
+          log.error("Caught exception replicating data to {} at {}", peerContext.getInstanceName(),
+              peerTserver, e);
           throw e;
         } finally {
           span.stop();
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java
index 6def221bf6..5b192bc09a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java
@@ -59,7 +59,7 @@ public long replicateLog(String tableIdStr, WalEdits data, TCredentials tcreds)
     String tableName;
 
     try {
-      tableName = Tables.getTableName(tabletServer.getInstance(), tableId);
+      tableName = Tables.getTableName(tabletServer, tableId);
     } catch (TableNotFoundException e) {
       log.error("Could not find table with id {}", tableId);
       throw new RemoteReplicationException(RemoteReplicationErrorCode.TABLE_DOES_NOT_EXIST,
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
index 51aa7e192b..3b9c1963c7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
@@ -67,11 +67,12 @@ public void run() {
         log.debug("Configuration DistributedWorkQueue with delay and period of {} and {}", delay,
             period);
         workQueue = new DistributedWorkQueue(
-            ZooUtil.getRoot(inst) + ReplicationConstants.ZOO_WORK_QUEUE, conf, delay, period);
+            ZooUtil.getRoot(context.getInstanceID()) + ReplicationConstants.ZOO_WORK_QUEUE, conf,
+            delay, period);
       } else {
         log.debug("Configuring DistributedWorkQueue with default delay and period");
         workQueue = new DistributedWorkQueue(
-            ZooUtil.getRoot(inst) + ReplicationConstants.ZOO_WORK_QUEUE, conf);
+            ZooUtil.getRoot(context.getInstanceID()) + ReplicationConstants.ZOO_WORK_QUEUE, conf);
       }
 
       workQueue.startProcessing(new ReplicationProcessor(context, conf, fs), executor);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 6359aa77fe..1b50cc87f3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -1095,8 +1095,8 @@ private MinorCompactionTask createMinorCompactionTask(long flushId,
 
   public long getFlushID() throws NoNodeException {
     try {
-      String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstance().getInstanceID()
-          + Constants.ZTABLES + "/" + extent.getTableId() + Constants.ZTABLE_FLUSH_ID;
+      String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstanceID() + Constants.ZTABLES
+          + "/" + extent.getTableId() + Constants.ZTABLE_FLUSH_ID;
       return Long
           .parseLong(new String(ZooReaderWriter.getInstance().getData(zTablePath, null), UTF_8));
     } catch (InterruptedException | NumberFormatException e) {
@@ -1111,8 +1111,8 @@ public long getFlushID() throws NoNodeException {
   }
 
   long getCompactionCancelID() {
-    String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstance().getInstanceID()
-        + Constants.ZTABLES + "/" + extent.getTableId() + Constants.ZTABLE_COMPACT_CANCEL_ID;
+    String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstanceID() + Constants.ZTABLES
+        + "/" + extent.getTableId() + Constants.ZTABLE_COMPACT_CANCEL_ID;
 
     try {
       return Long
@@ -1124,8 +1124,8 @@ long getCompactionCancelID() {
 
   public Pair<Long,UserCompactionConfig> getCompactionID() throws NoNodeException {
     try {
-      String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstance().getInstanceID()
-          + Constants.ZTABLES + "/" + extent.getTableId() + Constants.ZTABLE_COMPACT_ID;
+      String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstanceID() + Constants.ZTABLES
+          + "/" + extent.getTableId() + Constants.ZTABLE_COMPACT_ID;
 
       String[] tokens = new String(ZooReaderWriter.getInstance().getData(zTablePath, null), UTF_8)
           .split(",");
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java
index 1aff86433f..e1b38e227e 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java
@@ -20,13 +20,9 @@
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Properties;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Table;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.replication.ReplicationTarget;
@@ -44,10 +40,8 @@
 
   @Test
   public void peerTypeExtractionFromConfiguration() {
-    Instance inst = EasyMock.createMock(Instance.class);
     VolumeManager fs = EasyMock.createMock(VolumeManager.class);
-    Credentials creds = new Credentials("foo", new PasswordToken("bar"));
-    ClientContext context = new ClientContext(inst, creds, new Properties());
+    ClientContext context = EasyMock.createMock(ClientContext.class);
 
     Map<String,String> data = new HashMap<>();
 
@@ -63,10 +57,8 @@ public void peerTypeExtractionFromConfiguration() {
 
   @Test(expected = IllegalArgumentException.class)
   public void noPeerConfigurationThrowsAnException() {
-    Instance inst = EasyMock.createMock(Instance.class);
     VolumeManager fs = EasyMock.createMock(VolumeManager.class);
-    Credentials creds = new Credentials("foo", new PasswordToken("bar"));
-    ClientContext context = new ClientContext(inst, creds, new Properties());
+    ClientContext context = EasyMock.createMock(ClientContext.class);
 
     Map<String,String> data = new HashMap<>();
     ConfigurationCopy conf = new ConfigurationCopy(data);
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 1f21ad8124..6baae6dffd 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -691,8 +691,8 @@ public void shutdown() {
 
   public void printInfo() throws IOException {
     reader.print("\n" + SHELL_DESCRIPTION + "\n" + "- \n" + "- version: " + Constants.VERSION + "\n"
-        + "- instance name: " + connector.getInstance().getInstanceName() + "\n" + "- instance id: "
-        + connector.getInstance().getInstanceID() + "\n" + "- \n"
+        + "- instance name: " + connector.info().getInstanceName() + "\n" + "- instance id: "
+        + connector.getInstanceID() + "\n" + "- \n"
         + "- type 'help' for a list of available commands\n" + "- \n");
     reader.flush();
   }
@@ -729,7 +729,7 @@ public void printVerboseInfo() throws IOException {
   }
 
   public String getDefaultPrompt() {
-    return connector.whoami() + "@" + connector.getInstance().getInstanceName()
+    return connector.whoami() + "@" + connector.info().getInstanceName()
         + (getTableName().isEmpty() ? "" : " ") + getTableName() + "> ";
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/harness/MiniClusterHarness.java b/test/src/main/java/org/apache/accumulo/harness/MiniClusterHarness.java
index 7ddf114c33..00ed99f56e 100644
--- a/test/src/main/java/org/apache/accumulo/harness/MiniClusterHarness.java
+++ b/test/src/main/java/org/apache/accumulo/harness/MiniClusterHarness.java
@@ -33,6 +33,7 @@
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
@@ -225,6 +226,13 @@ protected void configureForSsl(MiniAccumuloConfigImpl cfg, File folder) {
         publicTruststoreFile.getAbsolutePath());
     siteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
     cfg.setSiteConfig(siteConfig);
+
+    cfg.setClientProperty(ClientProperty.SSL_ENABLED, "true");
+    cfg.setClientProperty(ClientProperty.SSL_KEYSTORE_PATH, localKeystoreFile.getAbsolutePath());
+    cfg.setClientProperty(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(), cfg.getRootPassword());
+    cfg.setClientProperty(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(),
+        publicTruststoreFile.getAbsolutePath());
+    cfg.setClientProperty(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
   }
 
   protected void configureForKerberos(MiniAccumuloConfigImpl cfg, File folder,
@@ -271,5 +279,7 @@ protected void configureForKerberos(MiniAccumuloConfigImpl cfg, File folder,
     coreSite.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
 
     cfg.setRootUserName(kdc.getRootUser().getPrincipal());
+
+    cfg.setClientProperty(ClientProperty.SASL_ENABLED, "true");
   }
 }
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 3a9ecfdc97..888c1fb35c 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -93,8 +93,9 @@ public void alterConfig() throws Exception {
 
     getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
 
-    Instance instance = getConnector().getInstance();
-    ZooCache zcache = new ZooCache(instance.getZooKeepers(),
+    Connector conn = getConnector();
+    Instance instance = conn.getInstance();
+    ZooCache zcache = new ZooCache(conn.info().getZooKeepers(),
         instance.getZooKeepersSessionTimeOut());
     zcache.clear();
     String path = ZooUtil.getRoot(instance) + 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 292461d7c0..050ec47f4c 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIterator.java
@@ -100,10 +100,9 @@ public static IteratorSetting iteratorSetting(int priority, int sleepAfterFirstW
       Connector connector, AuthenticationToken token, boolean clearCacheAfterFirstWrite,
       boolean splitAfterFirstWrite) {
     return iteratorSetting(priority, sleepAfterFirstWrite, batchWriterTimeout, batchWriterMaxMemory,
-        numEntriesToWrite, tableName, connector.getInstance().getZooKeepers(),
-        connector.getInstance().getInstanceName(),
-        connector.getInstance().getZooKeepersSessionTimeOut(), connector.whoami(), token,
-        clearCacheAfterFirstWrite, splitAfterFirstWrite);
+        numEntriesToWrite, tableName, connector.info().getZooKeepers(),
+        connector.info().getInstanceName(), connector.getInstance().getZooKeepersSessionTimeOut(),
+        connector.whoami(), token, clearCacheAfterFirstWrite, splitAfterFirstWrite);
   }
 
   public static IteratorSetting iteratorSetting(int priority, int sleepAfterFirstWrite,
diff --git a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
index 922b09a516..1a888eaeb9 100644
--- a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
@@ -27,6 +27,7 @@
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -59,6 +60,7 @@ public int defaultTimeoutSeconds() {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
+    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     cfg.setNumTservers(1);
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
index e1c103d2ca..8f9e381761 100644
--- a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.thrift.ThriftNotActiveServiceException;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
@@ -45,6 +46,7 @@
   @Override
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
+    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
index 6863572321..947b67f5c1 100644
--- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -32,6 +32,7 @@
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -64,6 +65,7 @@ public int defaultTimeoutSeconds() {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
+    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     // NativeMap.java was changed to fail if native lib missing in ACCUMULO-4596
     // testExistingInstance will fail because the native path is not set in
     // MiniAccumuloConfigImpl.useExistingInstance
@@ -116,7 +118,7 @@ public void testExistingInstance() throws Exception {
         getCluster().getConfig().getSiteConfig().get(Property.INSTANCE_ZK_TIMEOUT.getKey()));
     IZooReaderWriter zrw = new ZooReaderWriterFactory().getZooReaderWriter(
         getCluster().getZooKeepers(), (int) zkTimeout, defaultConfig.get(Property.INSTANCE_SECRET));
-    final String zInstanceRoot = Constants.ZROOT + "/" + conn.getInstance().getInstanceID();
+    final String zInstanceRoot = Constants.ZROOT + "/" + conn.getInstanceID();
     while (!AccumuloStatus.isAccumuloOffline(zrw, zInstanceRoot)) {
       log.debug("Accumulo services still have their ZK locks held");
       Thread.sleep(1000);
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 9668743709..4961b6df2a 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 void test() throws Exception {
 
       // This should fail: only one volume
       assertEquals(1, cluster.exec(RandomizeVolumes.class, "-z", cluster.getZooKeepers(), "-i",
-          c.getInstance().getInstanceName(), "-t", tableName).waitFor());
+          c.info().getInstanceName(), "-t", tableName).waitFor());
 
       cluster.stop();
 
diff --git a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
index 7eadd281f6..04fe5b2e61 100644
--- a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
@@ -127,7 +127,7 @@ public void testMonitorService() throws Exception {
   @Test
   public void testMasterService() throws Exception {
     final MiniAccumuloClusterImpl cluster = (MiniAccumuloClusterImpl) getCluster();
-    final String instanceID = getConnector().getInstance().getInstanceID();
+    final String instanceID = getConnector().getInstanceID();
 
     // Wait for the Master to grab its lock
     while (true) {
@@ -190,7 +190,7 @@ public void testMasterService() throws Exception {
   @Test
   public void testGarbageCollectorPorts() throws Exception {
     final MiniAccumuloClusterImpl cluster = (MiniAccumuloClusterImpl) getCluster();
-    String instanceID = getConnector().getInstance().getInstanceID();
+    String instanceID = getConnector().getInstanceID();
 
     // Wait for the Master to grab its lock
     while (true) {
diff --git a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
index 4628f41733..c056e0f18e 100644
--- a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
@@ -61,7 +61,7 @@ public void testCachedTransport() {
     ArrayList<ThriftTransportKey> servers = new ArrayList<>();
 
     // add tservers
-    ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(),
+    ZooCache zc = new ZooCacheFactory().getZooCache(conn.info().getZooKeepers(),
         instance.getZooKeepersSessionTimeOut());
     for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
       String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver;
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 c807ddfd57..c03b6ddb84 100644
--- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -79,7 +79,7 @@ public void test() throws Exception {
     c.tableOperations().create(lilTable);
 
     Instance i = c.getInstance();
-    zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+    zk = new ZooReaderWriter(c.info().getZooKeepers(), c.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/VolumeIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
index 0433aeeec5..cc36a30a6c 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
@@ -48,6 +48,7 @@
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -110,6 +111,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
     cfg.setProperty(Property.INSTANCE_DFS_URI, v1Uri.getScheme() + v1Uri.getHost());
     cfg.setProperty(Property.INSTANCE_VOLUMES, v1 + "," + v2);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
+    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(), "15s");
 
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
@@ -278,7 +280,7 @@ public void testAddVolumes() throws Exception {
     String[] tableNames = getUniqueNames(2);
 
     // grab this before shutting down cluster
-    String uuid = getConnector().getInstance().getInstanceID();
+    String uuid = getConnector().getInstanceID();
 
     verifyVolumesUsed(tableNames[0], false, v1, v2);
 
@@ -323,7 +325,7 @@ public void testNonConfiguredVolumes() throws Exception {
     String[] tableNames = getUniqueNames(2);
 
     // grab this before shutting down cluster
-    String uuid = getConnector().getInstance().getInstanceID();
+    String uuid = getConnector().getInstanceID();
 
     verifyVolumesUsed(tableNames[0], false, v1, v2);
 
@@ -449,8 +451,8 @@ private void verifyVolumesUsed(String tableName, boolean shouldExist, Path... pa
       // keep retrying until WAL state information in ZooKeeper stabilizes or until test times out
       retry: while (true) {
         Instance i = conn.getInstance();
-        ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(),
-            "");
+        ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
+            conn.info().getZooKeepersSessionTimeOut(), "");
         WalStateManager wals = new WalStateManager(i, zk);
         try {
           outer: for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BackupMasterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BackupMasterIT.java
index fa358e364f..25e5b1b0d0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BackupMasterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BackupMasterIT.java
@@ -44,7 +44,7 @@ public void test() throws Exception {
       String secret = getCluster().getSiteConfiguration().get(Property.INSTANCE_SECRET);
       IZooReaderWriter writer = new ZooReaderWriterFactory()
           .getZooReaderWriter(cluster.getZooKeepers(), 30 * 1000, secret);
-      String root = "/accumulo/" + getConnector().getInstance().getInstanceID();
+      String root = "/accumulo/" + getConnector().getInstanceID();
       List<String> children = Collections.emptyList();
       // wait for 2 lock entries
       do {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConnectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConnectorIT.java
index 3ab36f682f..762fcffdc4 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConnectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConnectorIT.java
@@ -31,18 +31,19 @@
   @Test
   public void testConnectorBuilder() throws Exception {
     Connector c = getConnector();
-    String instanceName = c.getInstance().getInstanceName();
-    String zookeepers = c.getInstance().getZooKeepers();
+    String instanceName = c.info().getInstanceName();
+    String zookeepers = c.info().getZooKeepers();
     final String user = "testuser";
     final String password = "testpassword";
     c.securityOperations().createLocalUser(user, new PasswordToken(password));
 
     Connector conn = Connector.builder().forInstance(instanceName, zookeepers)
-        .usingPassword(user, password).build();
+        .usingPassword(user, password).withZkTimeout(1234).build();
 
-    Assert.assertEquals(instanceName, conn.getInstance().getInstanceName());
-    Assert.assertEquals(zookeepers, conn.getInstance().getZooKeepers());
+    Assert.assertEquals(instanceName, conn.info().getInstanceName());
+    Assert.assertEquals(zookeepers, conn.info().getZooKeepers());
     Assert.assertEquals(user, conn.whoami());
+    Assert.assertEquals(1234, conn.info().getZooKeepersSessionTimeOut());
 
     ClientInfo info = Connector.builder().forInstance(instanceName, zookeepers)
         .usingPassword(user, password).info();
@@ -55,12 +56,14 @@ public void testConnectorBuilder() throws Exception {
     props.put(ClientProperty.INSTANCE_NAME.getKey(), instanceName);
     props.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), zookeepers);
     props.put(ClientProperty.AUTH_PRINCIPAL.getKey(), user);
+    props.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(), "22s");
     ClientProperty.setPassword(props, password);
     conn = Connector.builder().usingProperties(props).build();
 
-    Assert.assertEquals(instanceName, conn.getInstance().getInstanceName());
-    Assert.assertEquals(zookeepers, conn.getInstance().getZooKeepers());
+    Assert.assertEquals(instanceName, conn.info().getInstanceName());
+    Assert.assertEquals(zookeepers, conn.info().getZooKeepers());
     Assert.assertEquals(user, conn.whoami());
+    Assert.assertEquals(22000, conn.info().getZooKeepersSessionTimeOut());
 
     final String user2 = "testuser2";
     final String password2 = "testpassword2";
@@ -68,8 +71,8 @@ public void testConnectorBuilder() throws Exception {
 
     Connector conn2 = Connector.builder().usingClientInfo(conn.info())
         .usingToken(user2, new PasswordToken(password2)).build();
-    Assert.assertEquals(instanceName, conn2.getInstance().getInstanceName());
-    Assert.assertEquals(zookeepers, conn2.getInstance().getZooKeepers());
+    Assert.assertEquals(instanceName, conn2.info().getInstanceName());
+    Assert.assertEquals(zookeepers, conn2.info().getZooKeepers());
     Assert.assertEquals(user2, conn2.whoami());
     info = conn2.info();
     Assert.assertEquals(instanceName, info.getInstanceName());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index f3ad5d2580..bb32aecf50 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -37,7 +37,6 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -99,7 +98,7 @@ private void killMacGc() throws ProcessNotFoundException, InterruptedException,
     getCluster().killProcess(ServerType.GARBAGE_COLLECTOR,
         getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR).iterator().next());
     // delete lock in zookeeper if there, this will allow next GC to start quickly
-    String path = ZooUtil.getRoot(getConnector().getInstance()) + Constants.ZGC_LOCK;
+    String path = ZooUtil.getRoot(getConnector().getInstanceID()) + Constants.ZGC_LOCK;
     ZooReaderWriter zk = new ZooReaderWriter(cluster.getZooKeepers(), 30000, OUR_SECRET);
     try {
       ZooLock.deleteLock(zk, path);
@@ -251,10 +250,9 @@ public void testInvalidDelete() throws Exception {
   public void testProperPortAdvertisement() throws Exception {
 
     Connector conn = getConnector();
-    Instance instance = conn.getInstance();
 
     ZooReaderWriter zk = new ZooReaderWriter(cluster.getZooKeepers(), 30000, OUR_SECRET);
-    String path = ZooUtil.getRoot(instance) + Constants.ZGC_LOCK;
+    String path = ZooUtil.getRoot(conn.getInstanceID()) + Constants.ZGC_LOCK;
     for (int i = 0; i < 5; i++) {
       List<String> locks;
       try {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
index 91b51f41e8..36a4d24490 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
@@ -33,6 +33,7 @@
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 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;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -121,8 +122,8 @@ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreS
             // Reduce the period just to make sure we trigger renewal fast
             site.put(Property.GENERAL_KERBEROS_RENEWAL_PERIOD.getKey(), "5s");
             cfg.setSiteConfig(site);
+            cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
           }
-
         });
 
     mac.getConfig().setNumTservers(1);
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 77456d138b..00fff978ff 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
@@ -77,8 +77,8 @@ static void runTest(Connector c, MiniAccumuloClusterImpl cluster) throws Accumul
     }
     bw.close();
     Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-i",
-        c.getInstance().getInstanceName(), "-z", c.getInstance().getZooKeepers(), "-u", "root",
-        "-p", ROOT_PASSWORD, "-t", tablename, "--column", input_cfcq);
+        c.info().getInstanceName(), "-z", c.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/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index e6d1c04f5c..f53af9ee69 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
@@ -179,13 +179,13 @@ public void sunnyDay() throws Exception {
     log.debug("Stopping accumulo cluster");
     ClusterControl control = cluster.getClusterControl();
     control.adminStopAll();
-    ZooReader zreader = new ZooReader(connector.getInstance().getZooKeepers(),
-        connector.getInstance().getZooKeepersSessionTimeOut());
+    ZooReader zreader = new ZooReader(connector.info().getZooKeepers(),
+        connector.info().getZooKeepersSessionTimeOut());
     ZooCache zcache = new ZooCache(zreader, null);
     byte[] masterLockData;
     do {
       masterLockData = ZooLock.getLockData(zcache,
-          ZooUtil.getRoot(connector.getInstance()) + Constants.ZMASTER_LOCK, null);
+          ZooUtil.getRoot(connector.getInstanceID()) + Constants.ZMASTER_LOCK, null);
       if (null != masterLockData) {
         log.info("Master lock is still held");
         Thread.sleep(1000);
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 473355659f..5a1c598e9e 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
@@ -163,13 +163,13 @@ public void restartMasterRecovery() throws Exception {
     control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
     control.stopAllServers(ServerType.MONITOR);
 
-    ZooReader zreader = new ZooReader(c.getInstance().getZooKeepers(),
-        c.getInstance().getZooKeepersSessionTimeOut());
+    ZooReader zreader = new ZooReader(c.info().getZooKeepers(),
+        c.info().getZooKeepersSessionTimeOut());
     ZooCache zcache = new ZooCache(zreader, null);
     byte[] masterLockData;
     do {
       masterLockData = ZooLock.getLockData(zcache,
-          ZooUtil.getRoot(c.getInstance()) + Constants.ZMASTER_LOCK, null);
+          ZooUtil.getRoot(c.getInstanceID()) + Constants.ZMASTER_LOCK, null);
       if (null != masterLockData) {
         log.info("Master lock is still held");
         Thread.sleep(1000);
@@ -183,7 +183,7 @@ public void restartMasterRecovery() throws Exception {
     masterLockData = new byte[0];
     do {
       masterLockData = ZooLock.getLockData(zcache,
-          ZooUtil.getRoot(c.getInstance()) + Constants.ZMASTER_LOCK, null);
+          ZooUtil.getRoot(c.getInstanceID()) + Constants.ZMASTER_LOCK, null);
       if (null != masterLockData) {
         log.info("Master lock is still held");
         Thread.sleep(1000);
@@ -231,13 +231,13 @@ public void restartMasterSplit() throws Exception {
 
     control.stopAllServers(ServerType.MASTER);
 
-    ZooReader zreader = new ZooReader(c.getInstance().getZooKeepers(),
-        c.getInstance().getZooKeepersSessionTimeOut());
+    ZooReader zreader = new ZooReader(c.info().getZooKeepers(),
+        c.info().getZooKeepersSessionTimeOut());
     ZooCache zcache = new ZooCache(zreader, null);
     byte[] masterLockData;
     do {
       masterLockData = ZooLock.getLockData(zcache,
-          ZooUtil.getRoot(c.getInstance()) + Constants.ZMASTER_LOCK, null);
+          ZooUtil.getRoot(c.getInstanceID()) + Constants.ZMASTER_LOCK, null);
       if (null != masterLockData) {
         log.info("Master lock is still held");
         Thread.sleep(1000);
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 fdfa97530b..000c62508d 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
@@ -585,8 +585,8 @@ public void testPermissions() throws Exception {
     PasswordToken passTok = new PasswordToken("letmesee");
     c.securityOperations().createLocalUser("user1", passTok);
 
-    String instanceName = c.getInstance().getInstanceName();
-    String zookeepers = c.getInstance().getZooKeepers();
+    String instanceName = c.info().getInstanceName();
+    String zookeepers = c.info().getZooKeepers();
     Connector c2 = new ZooKeeperInstance(instanceName, zookeepers).getConnector("user1", passTok);
     try {
       c2.tableOperations().summaries(table).retrieve();
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 5f53fa7561..8e80230a69 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,11 +35,11 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
@@ -74,10 +74,12 @@
   private static final long SLOW_SCAN_SLEEP_MS = 100L;
 
   private Connector connector;
+  private ClientContext context;
 
   @Before
   public void setup() {
     connector = getConnector();
+    context = new ClientContext(connector.info());
   }
 
   @Override
@@ -223,21 +225,22 @@ private boolean blockUntilCompactionRunning(final String tableName) {
    */
   private boolean findFate(final String tableName) {
 
-    Instance instance = connector.getInstance();
     AdminUtil<String> admin = new AdminUtil<>(false);
 
     try {
 
-      Table.ID tableId = Tables.getTableId(instance, tableName);
+      Table.ID tableId = Tables.getTableId(context, tableName);
 
       log.trace("tid: {}", tableId);
 
       String secret = cluster.getSiteConfiguration().get(Property.INSTANCE_SECRET);
       IZooReaderWriter zk = new ZooReaderWriterFactory().getZooReaderWriter(
-          instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), secret);
-      ZooStore<String> zs = new ZooStore<>(ZooUtil.getRoot(instance) + Constants.ZFATE, zk);
+          connector.info().getZooKeepers(), connector.info().getZooKeepersSessionTimeOut(), secret);
+      ZooStore<String> zs = new ZooStore<>(
+          ZooUtil.getRoot(connector.getInstanceID()) + Constants.ZFATE, zk);
       AdminUtil.FateStatus fateStatus = admin.getStatus(zs, zk,
-          ZooUtil.getRoot(instance) + Constants.ZTABLE_LOCKS + "/" + tableId, null, null);
+          ZooUtil.getRoot(connector.getInstanceID()) + Constants.ZTABLE_LOCKS + "/" + tableId, null,
+          null);
 
       for (AdminUtil.TransactionStatus tx : fateStatus.getTransactions()) {
 
@@ -265,7 +268,7 @@ private boolean findFate(final String tableName) {
    */
   private TableState getTableState(String tableName) throws TableNotFoundException {
 
-    Table.ID tableId = Tables.getTableId(connector.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
 
     TableState tstate = Tables.getTableState(connector.getInstance(), tableId);
 
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 f52c2989d4..fcc670e899 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
@@ -228,10 +228,10 @@ private void dropTables(String... tables)
       HashSet<TServerInstance> tservers = new HashSet<>();
       for (String tserver : getConnector().instanceOperations().getTabletServers()) {
         try {
-          String zPath = ZooUtil.getRoot(getConnector().getInstance()) + Constants.ZTSERVERS + "/"
+          String zPath = ZooUtil.getRoot(getConnector().getInstanceID()) + Constants.ZTSERVERS + "/"
               + tserver;
           long sessionId = ZooLock.getSessionId(new ZooCache(getCluster().getZooKeepers(),
-              getConnector().getInstance().getZooKeepersSessionTimeOut()), zPath);
+              getConnector().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/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 4eaa2d8b1d..c5c78a63ea 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -41,7 +41,6 @@
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -262,9 +261,8 @@ private int getWaitFactor() {
 
   private Map<String,Boolean> _getWals(Connector c) throws Exception {
     Map<String,Boolean> result = new HashMap<>();
-    Instance i = c.getInstance();
-    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(),
-        "");
+    ZooReaderWriter zk = new ZooReaderWriter(c.info().getZooKeepers(),
+        c.info().getZooKeepersSessionTimeOut(), "");
     WalStateManager wals = new WalStateManager(c.getInstance(), zk);
     for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
       // WALs are in use if they are not unreferenced
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 476e0610a1..9ca1d90483 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
@@ -49,7 +49,7 @@ public void test() throws Exception {
       c.tableOperations().create(tableName);
     }
     c.tableOperations().list();
-    String zooKeepers = c.getInstance().getZooKeepers();
+    String zooKeepers = c.info().getZooKeepers();
     final long MIN = 475L;
     final long MAX = 700L;
     long total = 0;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 55d3568d35..92447b9ed5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -115,7 +115,7 @@ public static void main(String[] args) throws Exception {
         10 * 1024 * 1024, null, null, -1, HostAndPort.fromParts("0.0.0.0", port));
 
     String addressString = serverPort.address.toString();
-    String zPath = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS + "/"
+    String zPath = ZooUtil.getRoot(context.getInstanceID()) + Constants.ZTSERVERS + "/"
         + addressString;
     ZooReaderWriter zoo = ZooReaderWriter.getInstance();
     zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP);
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 d52664080d..39d5cb19a9 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
@@ -56,7 +56,7 @@ public void test() throws Exception {
         public void run() {
           try {
             CacheTestReader.main(new String[] {pathName, testDir.getAbsolutePath(),
-                getConnector().getInstance().getZooKeepers()});
+                getConnector().info().getZooKeepers()});
           } catch (Exception ex) {
             ref.set(ex);
           }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
index af75b902a4..26877f8b91 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
@@ -99,7 +99,8 @@ public int run(String[] args) throws Exception {
             + " <token file> <inputtable> <outputtable>");
       }
 
-      // String tokenFile = args[0];
+      String user = getAdminPrincipal();
+      String tokenFile = args[0];
       String table1 = args[1];
       String table2 = args[2];
 
@@ -108,8 +109,8 @@ public int run(String[] args) throws Exception {
 
       job.setInputFormat(AccumuloInputFormat.class);
 
+      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
       AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setClientInfo(job, getClientInfo());
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
index a69810b80a..a4ffcd47bb 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
@@ -26,12 +26,12 @@
 import java.io.PrintStream;
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.data.Key;
@@ -89,7 +89,8 @@ public int run(String[] args) throws Exception {
             + " <token file> <inputtable> <outputtable>");
       }
 
-      // String tokenFile = args[0];
+      String user = getAdminPrincipal();
+      String tokenFile = args[0];
       String table1 = args[1];
       String table2 = args[2];
 
@@ -99,8 +100,8 @@ public int run(String[] args) throws Exception {
 
       job.setInputFormatClass(AccumuloInputFormat.class);
 
+      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
       AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setClientInfo(job, getClientInfo());
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
@@ -109,7 +110,7 @@ public int run(String[] args) throws Exception {
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.setClientInfo(job, getClientInfo());
+      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
 
@@ -150,10 +151,13 @@ public void testMR() throws Exception {
     }
     bw.close();
 
-    File tf = folder.newFile("root_test.pw");
+    File tf = folder.newFile("client.properties");
     PrintStream out = new PrintStream(tf);
-    String outString = new Credentials(getAdminPrincipal(), getAdminToken()).serialize();
-    out.println(outString);
+    Properties props = getClientInfo().getProperties();
+    for (Object keyObj : props.keySet()) {
+      String key = (String) keyObj;
+      out.println(key + " = " + props.getProperty(key));
+    }
     out.close();
 
     MRTokenFileTester.main(new String[] {tf.getAbsolutePath(), table1, table2});
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index dbe945f666..7c103d4ee5 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -325,7 +325,7 @@ public static void main(String[] args) throws Exception {
 
     HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port);
 
-    Table.ID tableId = Tables.getTableId(zki, opts.tableName);
+    Table.ID tableId = Tables.getTableId(context, opts.tableName);
 
     // read the locations for the table
     Range tableRange = new KeyExtent(tableId, null, null).toMetadataRange();
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 e76de6b668..3a5cfa28a1 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
@@ -23,7 +23,6 @@
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -82,12 +81,10 @@ private long ingest() throws Exception {
     log.info("Waiting for balance");
     c.instanceOperations().waitForBalance();
 
-    final Instance inst = c.getInstance();
-
     log.info("Starting ingest");
     final long start = System.nanoTime();
-    final String args[] = {"-i", inst.getInstanceName(), "-z", inst.getZooKeepers(), "-u", "root",
-        "-p", ROOT_PASSWORD, "--batchThreads", "2", "--table", tableName, "--num",
+    final String args[] = {"-i", c.info().getInstanceName(), "-z", c.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/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 9cdad9891e..cbeb1e4fc8 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -114,7 +114,7 @@ public static void main(String[] args) throws Exception {
     Credentials creds = new Credentials(opts.getPrincipal(), opts.getToken());
     ClientContext context = new ClientContext(instance, creds, sconf.getSystemConfiguration());
 
-    Table.ID tableId = Tables.getTableId(instance, opts.getTableName());
+    Table.ID tableId = Tables.getTableId(context, opts.getTableName());
     if (tableId == null) {
       log.error("Unable to find table named {}", opts.getTableName());
       System.exit(-1);
@@ -359,7 +359,7 @@ private static void runTest(String desc, List<Test> tests, int numThreads,
   private static List<KeyExtent> findTablets(ClientContext context, boolean selectLocalTablets,
       String tableName, SortedMap<KeyExtent,String> tabletLocations) throws Exception {
 
-    Table.ID tableId = Tables.getTableId(context.getInstance(), tableName);
+    Table.ID tableId = Tables.getTableId(context, tableName);
     MetadataServicer.forTableId(context, tableId).getTabletLocations(tabletLocations);
 
     InetAddress localaddress = InetAddress.getLocalHost();
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 23b5f7c425..fe35e37cc7 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,7 +47,6 @@
 
 import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Namespace;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -178,14 +177,13 @@ public static void setUpProxy() throws Exception {
     assertNotNull("Implementations must initialize the TProtocolFactory", factory);
 
     Connector c = SharedMiniClusterBase.getConnector();
-    Instance inst = c.getInstance();
     waitForAccumulo(c);
 
     hostname = InetAddress.getLocalHost().getCanonicalHostName();
 
     Properties props = new Properties();
-    props.put("instance", inst.getInstanceName());
-    props.put("zookeepers", inst.getZooKeepers());
+    props.put("instance", c.info().getInstanceName());
+    props.put("zookeepers", c.info().getZooKeepers());
 
     final String tokenClass;
     if (isKerberosEnabled()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 79bad796d3..c5635ad6f0 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -26,7 +26,6 @@
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.MasterClient;
@@ -104,9 +103,8 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
 
     Assert.assertNotNull("Could not determine table ID for " + tableName, tableId);
 
-    Instance i = conn.getInstance();
-    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(),
-        "");
+    ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
+        conn.info().getZooKeepersSessionTimeOut(), "");
     WalStateManager wals = new WalStateManager(conn.getInstance(), zk);
 
     Set<String> result = new HashSet<>();
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 e2b4f165c1..045303686f 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
@@ -178,8 +178,8 @@ public void dataReplicatedToCorrectTable() throws Exception {
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
       primaryConn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + PEER_NAME,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peerConn.getInstance().getInstanceName(),
-                  peerConn.getInstance().getZooKeepers())));
+              AccumuloReplicaSystem.buildConfiguration(peerConn.info().getInstanceName(),
+                  peerConn.info().getZooKeepers())));
 
       String primaryTable1 = "primary", peerTable1 = "peer";
 
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
index d722be66ab..6e133e172c 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
@@ -23,8 +23,8 @@
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.replication.ReplicationConstants;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.HostAndPort;
@@ -52,23 +52,25 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
   public void tserverReplicationServicePortsAreAdvertised() throws Exception {
     // Wait for the cluster to be up
     Connector conn = getConnector();
-    Instance inst = conn.getInstance();
+    ClientContext context = new ClientContext(getClientInfo());
 
     // Wait for a tserver to come up to fulfill this request
     conn.tableOperations().create("foo");
     try (Scanner s = conn.createScanner("foo", Authorizations.EMPTY)) {
       Assert.assertEquals(0, Iterables.size(s));
 
-      ZooReader zreader = new ZooReader(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      ZooReader zreader = new ZooReader(context.getZooKeepers(),
+          context.getZooKeepersSessionTimeOut());
       Set<String> tserverHost = new HashSet<>();
-      tserverHost.addAll(zreader.getChildren(ZooUtil.getRoot(inst) + Constants.ZTSERVERS));
+      tserverHost
+          .addAll(zreader.getChildren(ZooUtil.getRoot(conn.getInstanceID()) + Constants.ZTSERVERS));
 
       Set<HostAndPort> replicationServices = new HashSet<>();
 
       for (String tserver : tserverHost) {
         try {
-          byte[] portData = zreader.getData(
-              ZooUtil.getRoot(inst) + ReplicationConstants.ZOO_TSERVERS + "/" + tserver, null);
+          byte[] portData = zreader.getData(ZooUtil.getRoot(conn.getInstanceID())
+              + ReplicationConstants.ZOO_TSERVERS + "/" + tserver, null);
           HostAndPort replAddress = HostAndPort.fromString(new String(portData, UTF_8));
           replicationServices.add(replAddress);
         } catch (Exception e) {
@@ -87,24 +89,26 @@ public void tserverReplicationServicePortsAreAdvertised() throws Exception {
   public void masterReplicationServicePortsAreAdvertised() throws Exception {
     // Wait for the cluster to be up
     Connector conn = getConnector();
-    Instance inst = conn.getInstance();
+    ClientContext context = new ClientContext(getClientInfo());
 
     // Wait for a tserver to come up to fulfill this request
     conn.tableOperations().create("foo");
     try (Scanner s = conn.createScanner("foo", Authorizations.EMPTY)) {
       Assert.assertEquals(0, Iterables.size(s));
 
-      ZooReader zreader = new ZooReader(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      ZooReader zreader = new ZooReader(context.getZooKeepers(),
+          context.getZooKeepersSessionTimeOut());
 
       // Should have one master instance
-      Assert.assertEquals(1, inst.getMasterLocations().size());
+      Assert.assertEquals(1, context.getMasterLocations().size());
 
       // Get the master thrift service addr
-      String masterAddr = Iterables.getOnlyElement(inst.getMasterLocations());
+      String masterAddr = Iterables.getOnlyElement(context.getMasterLocations());
 
       // Get the master replication coordinator addr
       String replCoordAddr = new String(zreader.getData(
-          ZooUtil.getRoot(inst) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR, null), UTF_8);
+          ZooUtil.getRoot(conn.getInstanceID()) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
+          null), UTF_8);
 
       // They shouldn't be the same
       Assert.assertNotEquals(masterAddr, replCoordAddr);
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 411b8f1144..8f365412f5 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
@@ -50,6 +50,7 @@
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -126,6 +127,7 @@ public int defaultTimeoutSeconds() {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // Run the master replication loop run frequently
+    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
@@ -155,8 +157,8 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
       // Map of logs to tableId
       Multimap<String,Table.ID> logs = HashMultimap.create();
       Instance i = conn.getInstance();
-      ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(),
-          "");
+      ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
+          conn.info().getZooKeepersSessionTimeOut(), "");
       WalStateManager wals = new WalStateManager(conn.getInstance(), zk);
       for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
         for (UUID id : entry.getValue()) {
@@ -197,8 +199,9 @@ private void waitForGCLock(Connector conn) throws InterruptedException {
     // Check if the GC process has the lock before wasting our retry attempts
     ZooKeeperInstance zki = (ZooKeeperInstance) conn.getInstance();
     ZooCacheFactory zcf = new ZooCacheFactory();
-    ZooCache zcache = zcf.getZooCache(zki.getZooKeepers(), zki.getZooKeepersSessionTimeOut());
-    String zkPath = ZooUtil.getRoot(conn.getInstance()) + Constants.ZGC_LOCK;
+    ZooCache zcache = zcf.getZooCache(conn.info().getZooKeepers(),
+        conn.info().getZooKeepersSessionTimeOut());
+    String zkPath = ZooUtil.getRoot(conn.getInstanceID()) + Constants.ZGC_LOCK;
     log.info("Looking for GC lock at {}", zkPath);
     byte[] data = ZooLock.getLockData(zcache, zkPath, null);
     while (null == data) {
@@ -341,8 +344,8 @@ public void correctRecordsCompleteFile() throws Exception {
     Set<String> wals = new HashSet<>();
     attempts = 5;
     Instance i = conn.getInstance();
-    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(),
-        "");
+    ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
+        conn.info().getZooKeepersSessionTimeOut(), "");
     while (wals.isEmpty() && attempts > 0) {
       WalStateManager markers = new WalStateManager(i, zk);
       for (Entry<Path,WalState> entry : markers.getAllState().entrySet()) {


 

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


With regards,
Apache Git Services