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

[GitHub] mikewalch closed pull request #799: Removed more ClientInfo references from API

mikewalch closed pull request #799: Removed more ClientInfo references from API
URL: https://github.com/apache/accumulo/pull/799
 
 
   

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

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

diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 3137db7866..21e15612d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -24,7 +24,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
 import org.apache.accumulo.core.conf.ClientProperty;
@@ -99,7 +98,7 @@ public ColumnVisibility convert(String value) {
   private Password securePassword = null;
 
   public AuthenticationToken getToken() {
-    return getClientInfo().getAuthenticationToken();
+    return ClientProperty.getAuthenticationToken(getClientProperties());
   }
 
   @Parameter(names = {"-z", "--keepers"},
@@ -154,27 +153,19 @@ public void parseArgs(String programName, String[] args, Object... others) {
     startTracing(programName);
   }
 
-  private ClientInfo cachedInfo = null;
   private AccumuloClient cachedAccumuloClient = null;
   private Properties cachedProps = null;
 
   public String getPrincipal() {
-    return getClientInfo().getPrincipal();
+    return ClientProperty.AUTH_PRINCIPAL.getValue(getClientProperties());
   }
 
   public void setPrincipal(String principal) {
     this.principal = principal;
   }
 
-  public void setClientInfo(ClientInfo info) {
-    this.cachedInfo = info;
-  }
-
-  public ClientInfo getClientInfo() {
-    if (cachedInfo == null) {
-      cachedInfo = ClientInfo.from(getClientProperties());
-    }
-    return cachedInfo;
+  public void setClientProperties(Properties clientProps) {
+    this.cachedProps = clientProps;
   }
 
   public AccumuloClient getClient() {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 9fbc547711..5a242f0af1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -26,7 +26,8 @@
  * This class represents the information a client needs to know to connect to an instance of
  * accumulo.
  *
- * @deprecated since 2.0.0, use {@link Accumulo#newClient()} and {@link ClientInfo} instead
+ * @deprecated since 2.0.0, use {@link Accumulo#newClient()} and {@link java.util.Properties}
+ *             instead
  */
 @Deprecated
 public interface Instance {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index f06b0d2dac..2e670bc8db 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -22,6 +22,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
@@ -55,7 +56,7 @@
    *
    * @since 1.7.0
    * @deprecated since 2.0.0, replaced by
-   *             {@link #MutationsRejectedException(ClientInfo, List, Map, Collection, int, Throwable)}
+   *             {@link #MutationsRejectedException(Properties, List, Map, Collection, int, Throwable)}
    */
   @Deprecated
   public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList,
@@ -74,8 +75,8 @@ public MutationsRejectedException(Instance instance, List<ConstraintViolationSum
   /**
    * Creates Mutations rejected exception
    *
-   * @param info
-   *          Client info
+   * @param clientProps
+   *          Client props
    * @param cvsList
    *          list of constraint violations
    * @param hashMap
@@ -87,12 +88,12 @@ public MutationsRejectedException(Instance instance, List<ConstraintViolationSum
    *
    * @since 2.0.0
    */
-  public MutationsRejectedException(ClientInfo info, List<ConstraintViolationSummary> cvsList,
-      Map<TabletId,Set<SecurityErrorCode>> hashMap, Collection<String> serverSideErrors,
-      int unknownErrors, Throwable cause) {
+  public MutationsRejectedException(Properties clientProps,
+      List<ConstraintViolationSummary> cvsList, Map<TabletId,Set<SecurityErrorCode>> hashMap,
+      Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
     super("# constraint violations : " + cvsList.size() + "  security codes: "
-        + format(hashMap, new ClientContext(info)) + "  # server errors " + serverSideErrors.size()
-        + " # exceptions " + unknownErrors, cause);
+        + format(hashMap, new ClientContext(clientProps)) + "  # server errors "
+        + serverSideErrors.size() + " # exceptions " + unknownErrors, cause);
     this.cvsl = cvsList;
     this.af = hashMap;
     this.es = serverSideErrors;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 73f380e3ab..d80ddbaa14 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -97,6 +97,10 @@ public ClientContext(AccumuloClient client) {
     this(ClientInfo.from(client.properties(), ((AccumuloClientImpl) client).token()));
   }
 
+  public ClientContext(Properties clientProperties) {
+    this(ClientInfo.from(clientProperties));
+  }
+
   public ClientContext(ClientInfo info) {
     this(info, ClientConfConverter.toAccumuloConf(info.getProperties()));
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index ce2aa1c3d5..21391200ae 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -568,7 +568,7 @@ private void checkForFailures() throws MutationsRejectedException {
         af.put(new TabletIdImpl(entry.getKey()), codes);
       }
 
-      throw new MutationsRejectedException(context.getClientInfo(), cvsList, af, serverSideErrors,
+      throw new MutationsRejectedException(context.getProperties(), cvsList, af, serverSideErrors,
           unknownErrors, lastUnknownError);
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
index 0183fadd08..31635171e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
@@ -40,7 +40,7 @@ public String getTableName() {
   public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
     super.setAccumuloConfigs(job);
     final String tableName = getTableName();
-    final ClientInfo info = getClientInfo();
+    final ClientInfo info = ClientInfo.from(getClientProperties());
     AccumuloInputFormat.setClientInfo(job, info);
     AccumuloInputFormat.setInputTableName(job, tableName);
     AccumuloInputFormat.setScanAuthorizations(job, auths);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
index 27c60225e8..9d6ab72e6f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
@@ -20,6 +20,7 @@
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
@@ -38,8 +39,9 @@
   private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
 
   public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    AccumuloInputFormat.setClientInfo(job, this.getClientInfo());
-    AccumuloOutputFormat.setClientInfo(job, this.getClientInfo());
+    ClientInfo info = ClientInfo.from(this.getClientProperties());
+    AccumuloInputFormat.setClientInfo(job, info);
+    AccumuloOutputFormat.setClientInfo(job, info);
   }
 
   @Override
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
index 84bc0bb972..3d7eeab025 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
@@ -16,11 +16,10 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
-import java.io.IOException;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -40,13 +39,12 @@ public String getTableName() {
   }
 
   @Override
-  public void setAccumuloConfigs(Job job)
-      throws IOException, AccumuloException, AccumuloSecurityException {
+  public void setAccumuloConfigs(Job job) throws AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
-    final ClientInfo info = getClientInfo();
-    AccumuloInputFormat.configure().clientProperties(info.getProperties()).table(tableName)
-        .auths(auths).store(job);
-    AccumuloOutputFormat.configure().clientProperties(info.getProperties()).defaultTable(tableName)
+    final Properties clientProps = getClientProperties();
+    AccumuloInputFormat.configure().clientProperties(clientProps).table(tableName).auths(auths)
+        .store(job);
+    AccumuloOutputFormat.configure().clientProperties(clientProps).defaultTable(tableName)
         .createTables().store(job);
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
index 1a63d59af0..a2bb14ffc4 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
@@ -16,11 +16,10 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
-import java.io.IOException;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.hadoop.mapreduce.AccumuloOutputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -33,13 +32,12 @@
   private String tableName;
 
   @Override
-  public void setAccumuloConfigs(Job job)
-      throws IOException, AccumuloException, AccumuloSecurityException {
+  public void setAccumuloConfigs(Job job) throws AccumuloException, AccumuloSecurityException {
     final String tableName = getTableName();
-    final ClientInfo info = getClientInfo();
-    AccumuloInputFormat.configure().clientProperties(info.getProperties()).table(tableName)
-        .auths(auths).store(job);
-    AccumuloOutputFormat.configure().clientProperties(info.getProperties()).defaultTable(tableName)
+    final Properties clientProps = getClientProperties();
+    AccumuloInputFormat.configure().clientProperties(clientProps).table(tableName).auths(auths)
+        .store(job);
+    AccumuloOutputFormat.configure().clientProperties(clientProps).defaultTable(tableName)
         .createTables().store(job);
   }
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
index 0b40cdf6e1..aa64aa86f5 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/RowHashIT.java
@@ -151,7 +151,7 @@ public int run(String[] args) throws Exception {
       Text cf = new Text(idx < 0 ? col : col.substring(0, idx));
       Text cq = idx < 0 ? null : new Text(col.substring(idx + 1));
       if (cf.getLength() > 0)
-        AccumuloInputFormat.configure().clientProperties(opts.getClientInfo().getProperties())
+        AccumuloInputFormat.configure().clientProperties(opts.getClientProperties())
             .table(opts.getTableName()).auths(Authorizations.EMPTY)
             .fetchColumns(Collections.singleton(new IteratorSetting.Column(cf, cq))).store(job);
 
@@ -162,8 +162,7 @@ public int run(String[] args) throws Exception {
       job.setNumReduceTasks(0);
 
       job.setOutputFormatClass(AccumuloOutputFormat.class);
-      AccumuloOutputFormat.configure().clientProperties(opts.getClientInfo().getProperties())
-          .store(job);
+      AccumuloOutputFormat.configure().clientProperties(opts.getClientProperties()).store(job);
 
       job.waitForCompletion(true);
       return job.isSuccessful() ? 0 : 1;
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
index 663c97e6bf..0be85ade24 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -17,11 +17,11 @@
 package org.apache.accumulo.cluster;
 
 import java.io.IOException;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.server.ServerContext;
@@ -63,15 +63,15 @@ AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
   /**
    * Get the client configuration for the cluster
    *
-   * @deprecated since 2.0.0, replaced by {@link #getClientInfo()}
+   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}}
    */
   @Deprecated
   org.apache.accumulo.core.client.ClientConfiguration getClientConfig();
 
   /**
-   * @return Connection Info for cluster
+   * @return client connection info for cluster
    */
-  ClientInfo getClientInfo();
+  Properties getClientProperties();
 
   /**
    * Get server side config derived from accumulo.properties
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index fed0ce1a4e..5870535188 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -23,6 +23,7 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.cluster.ClusterUser;
@@ -132,7 +133,7 @@ public String getZooKeepers() {
   @Override
   public synchronized ServerContext getServerContext() {
     if (context == null) {
-      context = new ServerContext(siteConfig, getClientInfo());
+      context = new ServerContext(siteConfig, getClientProperties());
     }
     return context;
   }
@@ -149,8 +150,8 @@ public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
   }
 
   @Override
-  public ClientInfo getClientInfo() {
-    return info;
+  public Properties getClientProperties() {
+    return info.getProperties();
   }
 
   @Override
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 72a69559b4..e830551ae7 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -18,14 +18,15 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -137,7 +138,7 @@ public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
 
   /**
    * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #getClientInfo()}
+   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}
    */
   @Deprecated
   public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
@@ -145,24 +146,24 @@ public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
   }
 
   /**
-   * @return Connection info for cluster
+   * @return Connection properties for cluster
    * @since 2.0.0
    */
-  public ClientInfo getClientInfo() {
-    return impl.getClientInfo();
+  public Properties getClientProperties() {
+    return impl.getClientProperties();
   }
 
   /**
-   * Construct a {@link ClientInfo} using a {@link MiniAccumuloCluster} directory
+   * Construct client {@link Properties} using a {@link MiniAccumuloCluster} directory
    *
    * @param directory
    *          MiniAccumuloCluster directory
-   * @return {@link ClientInfo} for that directory
+   * @return {@link Properties} for that directory
    * @since 2.0.0
    */
-  public static ClientInfo getClientInfo(File directory) {
+  public static Properties getClientProperties(File directory) {
     File clientProps = new File(new File(directory, "conf"), "accumulo-client.properties");
     Preconditions.checkArgument(clientProps.exists());
-    return ClientInfo.from(clientProps.toPath());
+    return ClientInfoImpl.toProperties(clientProps.toPath());
   }
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
index bb758d88db..eeacdb18fb 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
@@ -24,7 +24,7 @@
 
 /**
  * @since 1.6.0
- * @deprecated since 2.0.0, Use {@link MiniAccumuloCluster#getClientInfo(File)} instead
+ * @deprecated since 2.0.0, Use {@link MiniAccumuloCluster#getClientProperties(File)} instead
  */
 @Deprecated
 public class MiniAccumuloInstance extends org.apache.accumulo.core.client.ZooKeeperInstance {
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 6ac09f8729..2c840ce7ba 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -58,7 +58,6 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.MasterClient;
@@ -175,7 +174,7 @@ public void run() {
   private String dfsUri;
   private SiteConfiguration siteConfig;
   private ServerContext context;
-  private ClientInfo clientInfo;
+  private Properties clientProperties;
 
   public List<LogWriter> getLogWriters() {
     return logWriters;
@@ -782,7 +781,7 @@ public MiniAccumuloConfigImpl getConfig() {
 
   @Override
   public AccumuloClient getAccumuloClient(String user, AuthenticationToken token) {
-    return Accumulo.newClient().from(getClientInfo().getProperties()).as(user, token).build();
+    return Accumulo.newClient().from(getClientProperties()).as(user, token).build();
   }
 
   @SuppressWarnings("deprecation")
@@ -793,11 +792,12 @@ public AccumuloClient getAccumuloClient(String user, AuthenticationToken token)
   }
 
   @Override
-  public ClientInfo getClientInfo() {
-    if (clientInfo == null) {
-      clientInfo = ClientInfo.from(config.getClientPropsFile().toPath());
+  public synchronized Properties getClientProperties() {
+    if (clientProperties == null) {
+      clientProperties = Accumulo.newClientProperties().from(config.getClientPropsFile().toPath())
+          .build();
     }
-    return clientInfo;
+    return clientProperties;
   }
 
   @Override
@@ -843,7 +843,7 @@ public MasterMonitorInfo getMasterMonitorInfo()
     MasterClientService.Iface client = null;
     while (true) {
       try {
-        ClientContext context = new ClientContext(getClientInfo());
+        ClientContext context = new ClientContext(getClientProperties());
         client = MasterClient.getConnectionWithRetry(context);
         return client.getMasterStats(Tracer.traceInfo(), context.rpcCreds());
       } catch (ThriftSecurityException exception) {
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 a275563b3a..605ac9ac03 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
@@ -93,7 +93,7 @@ public void teardownTestCluster() {
   @Test
   public void canConnectViaExistingZooKeeper() throws Exception {
     org.apache.accumulo.core.client.Connector conn = accumulo.getConnector("root", SECRET);
-    ClientContext context = new ClientContext(accumulo.getClientInfo());
+    ClientContext context = new ClientContext(accumulo.getClientProperties());
     assertEquals(zooKeeper.getConnectString(), context.getZooKeepers());
 
     String tableName = "foo";
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index e1fdfba674..8106baf10b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -20,6 +20,7 @@
 
 import java.io.IOException;
 import java.util.Objects;
+import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
@@ -81,6 +82,10 @@ public ServerContext(SiteConfiguration siteConfig, String instanceName, String z
     this(new ServerInfo(siteConfig, instanceName, zooKeepers, zooKeepersSessionTimeOut));
   }
 
+  public ServerContext(SiteConfiguration siteConfig, Properties clientProps) {
+    this(siteConfig, ClientInfo.from(clientProps));
+  }
+
   public ServerContext(SiteConfiguration siteConfig, ClientInfo info) {
     this(new ServerInfo(siteConfig, info.getInstanceName(), info.getZooKeepers(),
         info.getZooKeepersSessionTimeOut()));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOnRequiredTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOnRequiredTable.java
index 305bbcb799..c44210d27a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOnRequiredTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOnRequiredTable.java
@@ -32,7 +32,7 @@ public synchronized ServerContext getServerContext() {
       if (instance == null) {
         context = new ServerContext(new SiteConfiguration());
       } else {
-        context = new ServerContext(new SiteConfiguration(), getClientInfo());
+        context = new ServerContext(new SiteConfiguration(), getClientProperties());
       }
     }
     return context;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
index 0d0b07edc2..598f23608c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
@@ -27,7 +27,7 @@
   }
 
   public ClientContext getClientContext() {
-    return new ClientContext(getClientInfo());
+    return new ClientContext(getClientProperties());
   }
 
   private ServerContext context;
@@ -37,7 +37,7 @@ public synchronized ServerContext getServerContext() {
       if (instance == null) {
         context = new ServerContext(new SiteConfiguration());
       } else {
-        context = new ServerContext(new SiteConfiguration(), getClientInfo());
+        context = new ServerContext(new SiteConfiguration(), getClientProperties());
       }
     }
     return context;
diff --git a/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java b/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
index bb9628a336..bb585de3ee 100644
--- a/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
+++ b/test/src/main/java/org/apache/accumulo/harness/AccumuloClusterHarness.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.Properties;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.cluster.ClusterControl;
@@ -266,9 +267,14 @@ public static String getAdminPrincipal() {
     return clusterConf.getAdminPrincipal();
   }
 
+  public static Properties getClientProperties() {
+    checkState(initialized);
+    return getCluster().getClientProperties();
+  }
+
   public static ClientInfo getClientInfo() {
     checkState(initialized);
-    return getCluster().getClientInfo();
+    return ClientInfo.from(getCluster().getClientProperties());
   }
 
   public static ClientContext getClientContext() {
diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
index a74e7de404..2b65f1b6d7 100644
--- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -1547,7 +1547,7 @@ public void testTrace() throws Exception {
       String tableName = getUniqueNames(1)[0];
       client.tableOperations().create(tableName);
 
-      DistributedTrace.enable("localhost", "testTrace", mac.getClientInfo().getProperties());
+      DistributedTrace.enable("localhost", "testTrace", mac.getClientProperties());
       sleepUninterruptibly(1, TimeUnit.SECONDS);
       Span root = Trace.on("traceTest");
       try (ConditionalWriter cw = client.createConditionalWriter(tableName,
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index a09a32d29b..17b4160fc3 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -370,7 +370,8 @@ public void exporttableImporttable() throws Exception {
     ts.exec("exporttable -t " + table + " " + exportUri, true);
     DistCp cp = newDistCp(new Configuration(false));
     String import_ = "file://" + new File(rootPath, "ShellServerIT.import");
-    if (getCluster().getClientInfo().saslEnabled()) {
+    ClientInfo info = ClientInfo.from(getCluster().getClientProperties());
+    if (info.saslEnabled()) {
       // DistCp bugs out trying to get a fs delegation token to perform the cp. Just copy it
       // ourselves by hand.
       FileSystem fs = getCluster().getFileSystem();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index b99739f79d..5050d26fc0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -131,8 +131,8 @@ public void test() throws Exception {
 
     TestIngest.Opts opts = new TestIngest.Opts();
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-    opts.setClientInfo(getClientInfo());
-    vopts.setClientInfo(getClientInfo());
+    opts.setClientProperties(getClientProperties());
+    vopts.setClientProperties(getClientProperties());
     vopts.rows = opts.rows = 200000;
     opts.setTableName(TEST_TABLE);
     TestIngest.ingest(accumuloClient, opts, new BatchWriterOpts());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
index df88c83284..9cb7679c5e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
@@ -99,7 +99,7 @@ static void runTest(AccumuloClient c, ClientInfo info, FileSystem fs, Path baseP
     opts.rows = N;
     opts.cols = 1;
     opts.setTableName(tableName);
-    opts.setClientInfo(info);
+    opts.setClientProperties(info.getProperties());
     opts.conf = new Configuration(false);
     opts.fs = fs;
     String fileFormat = filePrefix + "rf%02d";
@@ -118,7 +118,7 @@ static void runTest(AccumuloClient c, ClientInfo info, FileSystem fs, Path baseP
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.setTableName(tableName);
     vopts.random = 56;
-    vopts.setClientInfo(info);
+    vopts.setClientProperties(info.getProperties());
     for (int i = 0; i < COUNT; i++) {
       vopts.startRow = i * N;
       vopts.rows = N;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 1372e003d6..340b827b69 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -118,7 +118,7 @@ public void testBulkSplitOptimization() throws Exception {
       opts.cols = 1;
       opts.setTableName(tableName);
 
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
       VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
 
       // ensure each tablet does not have all map files, should be ~2.5 files per tablet
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
index 8c2ee60651..21b300e42d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
@@ -68,8 +68,8 @@ public void test() throws Exception {
       vopts.rows = opts.rows = 20000;
       opts.setTableName(tableName);
       vopts.setTableName(tableName);
-      opts.setClientInfo(getClientInfo());
-      vopts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
+      vopts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       c.tableOperations().flush(tableName, null, null, true);
       VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index 30a221a83b..f26f0b8df3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -139,7 +139,7 @@ public void run() {
                 opts.dataSize = 50;
                 opts.cols = 1;
                 opts.setTableName(tableName);
-                opts.setClientInfo(getClientInfo());
+                opts.setClientProperties(getClientProperties());
                 VerifyIngest.verifyIngest(c, opts, new ScannerOpts());
               } catch (Exception ex) {
                 log.warn("Got exception verifying data", ex);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
index f524b02faf..246fff935b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
@@ -24,6 +24,7 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Map;
+import java.util.Properties;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -201,6 +202,10 @@ protected ClientContext getClientContext() {
     return new ClientContext(getClientInfo());
   }
 
+  protected Properties getClientProperties() {
+    return getClientInfo().getProperties();
+  }
+
   protected ClientInfo getClientInfo() {
     return ClientInfo.from(Accumulo.newClientProperties()
         .to(getCluster().getInstanceName(), getCluster().getZooKeepers()).as("root", ROOT_PASSWORD)
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
index 3c09d0581b..7e10be2876 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteIT.java
@@ -69,8 +69,8 @@ public static void deleteTest(AccumuloClient c, AccumuloCluster cluster, String
     vopts.random = opts.random = 56;
 
     assertTrue("Expected one of password or keytab", null != password || null != keytab);
-    opts.setClientInfo(getClientInfo());
-    vopts.setClientInfo(getClientInfo());
+    opts.setClientProperties(getClientProperties());
+    vopts.setClientProperties(getClientProperties());
 
     BatchWriterOpts BWOPTS = new BatchWriterOpts();
     TestIngest.ingest(c, opts, BWOPTS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
index ab0e193593..ed98fd2cf1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
@@ -88,7 +88,7 @@ public void test() throws Exception {
       opts.rows = 500 * 1000;
       opts.createTable = true;
       opts.setTableName(firstTable);
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       c.tableOperations().flush(firstTable, null, null, true);
       for (int i = 1; i < tables.length; i++)
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
index 4bbbb1ab64..e1ca4e5038 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
@@ -53,7 +53,7 @@ public void run() throws Exception {
       opts.rows = 100000;
       opts.cols = 1;
       opts.setTableName(tableName);
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
 
       c.tableOperations().flush(tableName, null, null, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 0ce534c133..c8101eefad 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
@@ -122,8 +122,8 @@ public void gcTest() throws Exception {
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       vopts.rows = opts.rows = 10000;
       vopts.cols = opts.cols = 1;
-      opts.setClientInfo(getClientInfo());
-      vopts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
+      vopts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, cluster.getFileSystem(), opts, new BatchWriterOpts());
       c.tableOperations().compact("test_ingest", null, null, true, true);
       int before = countFiles();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index b6a7df9808..1c20a30300 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -173,7 +173,7 @@ public String test(int seconds, boolean expectTserverDied) throws Exception {
           assertEquals(0, ingest.waitFor());
           VerifyIngest.Opts vopts = new VerifyIngest.Opts();
           vopts.rows = rows;
-          vopts.setClientInfo(getClientInfo());
+          vopts.setClientProperties(getClientProperties());
           VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
         } else {
           sleepUninterruptibly(5, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
index 6b8661e459..ee50b5c005 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
@@ -54,7 +54,7 @@ public void test() throws Exception {
       c.tableOperations().create(names[0]);
       TestIngest.Opts opts = new TestIngest.Opts();
       opts.setTableName(names[0]);
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
 
       ClusterControl control = cluster.getClusterControl();
@@ -65,7 +65,7 @@ public void test() throws Exception {
       c.tableOperations().rename(names[0], names[1]);
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       vopts.setTableName(names[1]);
-      vopts.setClientInfo(getClientInfo());
+      vopts.setClientProperties(getClientProperties());
       VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
index 458606488f..05ae5ddb27 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
@@ -111,7 +111,7 @@ public void run() throws Exception {
         opts.cols = 1;
         opts.random = i;
         opts.setTableName(tableName);
-        opts.setClientInfo(getClientInfo());
+        opts.setClientProperties(getClientProperties());
         TestIngest.ingest(c, opts, new BatchWriterOpts());
 
         c.tableOperations().flush(tableName, null, null, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index dc4696ea4a..59b3143733 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
@@ -220,7 +220,7 @@ public static void ingest(AccumuloClient accumuloClient, ClientInfo info, int ro
     opts.columnFamily = colf;
     opts.createTable = true;
     opts.setTableName(tableName);
-    opts.setClientInfo(info);
+    opts.setClientProperties(info.getProperties());
 
     TestIngest.ingest(accumuloClient, opts, new BatchWriterOpts());
   }
@@ -240,7 +240,7 @@ private static void verify(AccumuloClient accumuloClient, ClientInfo info, int r
     opts.startRow = offset;
     opts.columnFamily = colf;
     opts.setTableName(tableName);
-    opts.setClientInfo(info);
+    opts.setClientProperties(info.getProperties());
 
     VerifyIngest.verifyIngest(accumuloClient, opts, scannerOpts);
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
index 6998656fed..4f2e16ad1e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RenameIT.java
@@ -41,14 +41,14 @@ public void renameTest() throws Exception {
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.createTable = true;
     opts.setTableName(name1);
-    opts.setClientInfo(cluster.getClientInfo());
+    opts.setClientProperties(cluster.getClientProperties());
 
     try (AccumuloClient c = getAccumuloClient()) {
       TestIngest.ingest(c, opts, bwOpts);
       c.tableOperations().rename(name1, name2);
       TestIngest.ingest(c, opts, bwOpts);
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-      vopts.setClientInfo(cluster.getClientInfo());
+      vopts.setClientProperties(cluster.getClientProperties());
       vopts.setTableName(name2);
       VerifyIngest.verifyIngest(c, vopts, scanOpts);
       c.tableOperations().delete(name1);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
index 86862732ee..1345fc74b3 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
@@ -129,7 +129,7 @@ public void restartMaster() throws Exception {
       } else {
         throw new RuntimeException("Unknown token");
       }
-      OPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
 
       Future<Integer> ret = svc.submit(() -> {
         try {
@@ -154,8 +154,8 @@ public void restartMasterRecovery() throws Exception {
       c.tableOperations().create(tableName);
       OPTS.setTableName(tableName);
       VOPTS.setTableName(tableName);
-      OPTS.setClientInfo(getClientInfo());
-      VOPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setClientProperties(getClientProperties());
       TestIngest.ingest(c, OPTS, BWOPTS);
       ClusterControl control = getCluster().getClusterControl();
 
@@ -222,8 +222,8 @@ public void restartMasterSplit() throws Exception {
       } else {
         throw new RuntimeException("Unknown token");
       }
-      OPTS.setClientInfo(getClientInfo());
-      VOPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setClientProperties(getClientProperties());
 
       Future<Integer> ret = svc.submit(() -> {
         try {
@@ -262,8 +262,8 @@ public void killedTabletServer() throws Exception {
       c.tableOperations().create(tableName);
       OPTS.setTableName(tableName);
       VOPTS.setTableName(tableName);
-      OPTS.setClientInfo(getClientInfo());
-      VOPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setClientProperties(getClientProperties());
       TestIngest.ingest(c, OPTS, BWOPTS);
       VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
       cluster.getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -294,7 +294,7 @@ public void killedTabletServerDuringShutdown() throws Exception {
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
       OPTS.setTableName(tableName);
-      OPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
       TestIngest.ingest(c, OPTS, BWOPTS);
       try {
         getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
@@ -310,8 +310,8 @@ public void shutdownDuringCompactingSplitting() throws Exception {
     try (AccumuloClient c = getAccumuloClient()) {
       String tableName = getUniqueNames(1)[0];
       VOPTS.setTableName(tableName);
-      OPTS.setClientInfo(getClientInfo());
-      VOPTS.setClientInfo(getClientInfo());
+      OPTS.setClientProperties(getClientProperties());
+      VOPTS.setClientProperties(getClientProperties());
       c.tableOperations().create(tableName);
       c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
       String splitThreshold = null;
@@ -327,7 +327,7 @@ public void shutdownDuringCompactingSplitting() throws Exception {
             "20K");
         TestIngest.Opts opts = new TestIngest.Opts();
         opts.setTableName(tableName);
-        opts.setClientInfo(getClientInfo());
+        opts.setClientProperties(getClientProperties());
         TestIngest.ingest(c, opts, BWOPTS);
         c.tableOperations().flush(tableName, null, null, false);
         VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
index e6dc27d38e..8e0e26b37b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -135,7 +135,7 @@ public void test() throws Exception {
       }
       assertEquals(0, retCode.get().intValue());
       VOPTS.setTableName(tableName);
-      VOPTS.setClientInfo(getClientInfo());
+      VOPTS.setClientProperties(getClientProperties());
       VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index e278a81099..807f482305 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -74,7 +74,7 @@ public void simpleBalancerFairness() throws Exception {
       List<String> tservers = c.instanceOperations().getTabletServers();
       TestIngest.Opts opts = new TestIngest.Opts();
       opts.rows = 50000;
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       c.tableOperations().flush("test_ingest", null, null, false);
       sleepUninterruptibly(45, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
index 207a54cf7b..f59ed5a4c0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -133,12 +133,12 @@ public void tabletShouldSplit() throws Exception {
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       opts.rows = 100000;
       opts.setTableName(table);
-      opts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
 
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       vopts.rows = opts.rows;
       vopts.setTableName(table);
-      vopts.setClientInfo(getClientInfo());
+      vopts.setClientProperties(getClientProperties());
       VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
       while (c.tableOperations().listSplits(table).size() < 10) {
         sleepUninterruptibly(15, TimeUnit.SECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
index 34b9d067e4..24120ff1b0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
@@ -70,8 +70,8 @@ public void test() throws Exception {
 
       TestIngest.Opts opts = new TestIngest.Opts();
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
-      opts.setClientInfo(getClientInfo());
-      vopts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
+      vopts.setClientProperties(getClientProperties());
       opts.setTableName(tableName);
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       to.flush(tableName, null, null, true);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
index 2567fb427d..6fe46bfc6f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java
@@ -89,8 +89,8 @@ public void test() throws Exception {
       TestIngest.Opts opts = new TestIngest.Opts();
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       opts.setTableName(tableName);
-      opts.setClientInfo(getClientInfo());
-      vopts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
+      vopts.setClientProperties(getClientProperties());
 
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       vopts.setTableName(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index c60f23cda0..93cbc581b9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -56,8 +56,8 @@ public void test() throws Exception {
       TestIngest.Opts opts = new TestIngest.Opts();
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       opts.setTableName(tableName);
-      opts.setClientInfo(getClientInfo());
-      vopts.setClientInfo(getClientInfo());
+      opts.setClientProperties(getClientProperties());
+      vopts.setClientProperties(getClientProperties());
 
       TestIngest.ingest(c, opts, new BatchWriterOpts());
       vopts.setTableName(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
index 23c87452b6..029c0de854 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteLotsIT.java
@@ -55,7 +55,7 @@ public void run() {
               opts.startRow = index * 10000;
               opts.rows = 10000;
               opts.setTableName(tableName);
-              opts.setClientInfo(getClientInfo());
+              opts.setClientProperties(getClientProperties());
               BatchWriterOpts bwOpts = new BatchWriterOpts();
               bwOpts.batchMemory = 1024L * 1024;
               bwOpts.batchThreads = 2;
@@ -75,7 +75,7 @@ public void run() {
       VerifyIngest.Opts vopts = new VerifyIngest.Opts();
       vopts.rows = 10000 * THREADS;
       vopts.setTableName(tableName);
-      vopts.setClientInfo(getClientInfo());
+      vopts.setClientProperties(getClientProperties());
       VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
     }
   }
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 b1bd0228fd..f3babd9219 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
@@ -228,7 +228,7 @@ public static void setUpProxy() throws Exception {
       }
 
       props.put("tokenClass", tokenClass);
-      props.putAll(SharedMiniClusterBase.getCluster().getClientInfo().getProperties());
+      props.putAll(SharedMiniClusterBase.getCluster().getClientProperties());
       proxyPort = PortUtils.getRandomFreePort();
       proxyServer = Proxy.createProxyServer(HostAndPort.fromParts(hostname, proxyPort), factory,
           props).server;


 

----------------------------------------------------------------
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