You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2022/11/10 15:37:40 UTC

[accumulo] branch main updated: Remove legacy ClientConfiguration code (#3074)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new a7c0575e01 Remove legacy ClientConfiguration code (#3074)
a7c0575e01 is described below

commit a7c0575e01b8cbfe13713642bd67f1684839931a
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Thu Nov 10 10:37:34 2022 -0500

    Remove legacy ClientConfiguration code (#3074)
    
    * Delete ClientConfiguration and unit test
    * Update ClientConfConverter to remove legacy client config
    * Update ClientConfConverterTest.basic to test conversion directly
      to/from client Properties and AccumuloConfiguration (which is still
      needed for some bootstrapping code and SASL configuration code)
    * Remove legacy trace props from ClientProperty
    * Remove deprecated getClientConfig() methods from minicluster
    * No longer write out client config file (only client properties file)
      and no longer set corresponding `ACCUMULO_CLIENT_CONF_PATH`
      environment variable for minicluster
    * Update SaslServerConnectionParams to remove unused overloaded
      constructors and setters (allowing secretManager field to be final);
      this wasn't strictly necessary in this patch, but helped simplify
      things to make it easier to trace the uses of the ClientConfConverter
      methods to see how they were still needed
    * Fix test failures by preserving mappings to AccumuloConfiguration
    * Simplify the AccumuloConfiguration anonymous inner class by
      computing the conversion of the kerberos primary right away, and
      only once
    * Run a flaky SASL test in separate JVM
---
 core/pom.xml                                       |   4 +-
 .../accumulo/core/client/ClientConfiguration.java  | 547 ---------------------
 .../core/clientImpl/ClientConfConverter.java       | 247 +++-------
 .../apache/accumulo/core/conf/ClientProperty.java  |  12 +-
 .../core/client/ClientConfigurationTest.java       |  98 ----
 .../core/clientImpl/ClientConfConverterTest.java   |  27 +-
 .../apache/accumulo/cluster/AccumuloCluster.java   |   8 -
 .../standalone/StandaloneAccumuloCluster.java      |   7 -
 .../accumulo/minicluster/MiniAccumuloCluster.java  |   9 -
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |  18 -
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  10 -
 .../server/rpc/SaslServerConnectionParams.java     |  20 +-
 .../rpc/SaslServerDigestCallbackHandler.java       |   2 +-
 .../apache/accumulo/server/ServerContextTest.java  |   5 +-
 .../server/rpc/SaslServerConnectionParamsTest.java |   2 +-
 15 files changed, 99 insertions(+), 917 deletions(-)

diff --git a/core/pom.xml b/core/pom.xml
index fa2525d55b..9c79518b47 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -432,6 +432,7 @@
                   <exclude>**/TestCfCqSlice.java</exclude>
                   <exclude>**/TestCfCqSliceSeekingFilter.java</exclude>
                   <exclude>**/core/constraints/DefaultKeySizeConstraintTest.java</exclude>
+                  <exclude>**/core/rpc/SaslConnectionParamsTest.java</exclude>
                 </excludes>
               </configuration>
               <executions>
@@ -451,7 +452,8 @@
                       <include>**/IdleRatioScanPrioritizerTest.java</include>
                       <include>**/TestCfCqSlice.java</include>
                       <include>**/TestCfCqSliceSeekingFilter.java</include>
-                      <exclude>**/core/constraints/DefaultKeySizeConstraintTest.java</exclude>
+                      <include>**/core/constraints/DefaultKeySizeConstraintTest.java</include>
+                      <include>**/core/rpc/SaslConnectionParamsTest.java</include>
                     </includes>
                   </configuration>
                 </execution>
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
deleted file mode 100644
index 257bc6bb57..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.PropertyType;
-import org.apache.commons.configuration2.CompositeConfiguration;
-import org.apache.commons.configuration2.Configuration;
-import org.apache.commons.configuration2.MapConfiguration;
-import org.apache.commons.configuration2.PropertiesConfiguration;
-import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * Contains a list of property keys recognized by the Accumulo client and convenience methods for
- * setting them.
- *
- * @since 1.6.0
- * @deprecated since 2.0.0, replaced by {@link Accumulo#newClient()}
- */
-@Deprecated(since = "2.0.0")
-public class ClientConfiguration {
-  private static final Logger log = LoggerFactory.getLogger(ClientConfiguration.class);
-
-  public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
-  public static final String USER_CONF_FILENAME = "config";
-  public static final String GLOBAL_CONF_FILENAME = "client.conf";
-
-  private final CompositeConfiguration compositeConfig;
-
-  public enum ClientProperty {
-    // SSL
-    RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
-    RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
-    RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
-    RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
-    RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
-    RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
-    RPC_USE_JSSE(Property.RPC_USE_JSSE),
-    GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS),
-    INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
-    INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
-
-    // ZooKeeper
-    INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
-    INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
-
-    // Instance information
-    INSTANCE_NAME("instance.name", null, PropertyType.STRING,
-        "Name of Accumulo instance to connect to"),
-    INSTANCE_ID("instance.id", null, PropertyType.STRING,
-        "UUID of Accumulo instance to connect to"),
-
-    // Tracing
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVERS(Property.TRACE_SPAN_RECEIVERS),
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVER_PREFIX(Property.TRACE_SPAN_RECEIVER_PREFIX),
-    @Deprecated(since = "2.1.0")
-    TRACE_ZK_PATH(Property.TRACE_ZK_PATH),
-
-    // SASL / GSSAPI(Kerberos)
-    /**
-     * @since 1.7.0
-     */
-    INSTANCE_RPC_SASL_ENABLED(Property.INSTANCE_RPC_SASL_ENABLED),
-    /**
-     * @since 1.7.0
-     */
-    RPC_SASL_QOP(Property.RPC_SASL_QOP),
-    /**
-     * @since 1.7.0
-     */
-    KERBEROS_SERVER_PRIMARY("kerberos.server.primary", "accumulo", PropertyType.STRING,
-        "The first component of the Kerberos principal, the 'primary', "
-            + "that Accumulo servers use to login");
-
-    private String key;
-    private String defaultValue;
-    private PropertyType type;
-    private String description;
-
-    private ClientProperty(Property prop) {
-      this(prop.getKey(), prop.getDefaultValue(), prop.getType(), prop.getDescription());
-    }
-
-    private ClientProperty(String key, String defaultValue, PropertyType type, String description) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-      this.type = type;
-      this.description = description;
-    }
-
-    public String getKey() {
-      return key;
-    }
-
-    public String getDefaultValue() {
-      return defaultValue;
-    }
-
-    private PropertyType getType() {
-      return type;
-    }
-
-    public String getDescription() {
-      return description;
-    }
-
-    public static ClientProperty getPropertyByKey(String key) {
-      for (ClientProperty prop : ClientProperty.values())
-        if (prop.getKey().equals(key))
-          return prop;
-      return null;
-    }
-  }
-
-  private ClientConfiguration(List<? extends Configuration> configs) {
-    compositeConfig = new CompositeConfiguration(configs);
-  }
-
-  /**
-   * Attempts to load a configuration file from the system using the default search paths. Uses the
-   * <em>ACCUMULO_CLIENT_CONF_PATH</em> environment variable, split on <em>File.pathSeparator</em>,
-   * for a list of target files.
-   * <p>
-   * If <em>ACCUMULO_CLIENT_CONF_PATH</em> is not set, uses the following in this order:
-   * <ul>
-   * <li>~/.accumulo/config
-   * <li><em>$ACCUMULO_CONF_DIR</em>/client.conf, if <em>$ACCUMULO_CONF_DIR</em> is defined.
-   * <li>/etc/accumulo/client.conf
-   * <li>/etc/accumulo/conf/client.conf
-   * </ul>
-   * <p>
-   * A client configuration will then be read from each location using
-   * <em>PropertiesConfiguration</em> to construct a configuration. That means the latest item will
-   * be the one in the configuration.
-   *
-   * @see PropertiesConfiguration
-   * @see File#pathSeparator
-   */
-  public static ClientConfiguration loadDefault() {
-    return loadFromSearchPath(getDefaultSearchPath());
-  }
-
-  /**
-   * Initializes an empty configuration object to be further configured with other methods on the
-   * class.
-   *
-   * @since 1.9.0
-   */
-  public static ClientConfiguration create() {
-    return new ClientConfiguration(Collections.emptyList());
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a configuration file. Currently
-   * supports Java "properties" files. The returned object can be further configured with subsequent
-   * calls to other methods on this class.
-   *
-   * @param file
-   *          the path to the configuration file
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromFile(File file) {
-    var config = new PropertiesConfiguration();
-    try (var reader = new FileReader(file, UTF_8)) {
-      config.read(reader);
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException("Bad configuration file: " + file, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(config));
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a map. The returned object can be
-   * further configured with subsequent calls to other methods on this class.
-   *
-   * @param properties
-   *          a map containing the configuration properties to use
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromMap(Map<String,String> properties) {
-    MapConfiguration mapConf = new MapConfiguration(properties);
-    return new ClientConfiguration(Collections.singletonList(mapConf));
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  private static ClientConfiguration loadFromSearchPath(List<String> paths) {
-    List<Configuration> configs = new LinkedList<>();
-    for (String path : paths) {
-      File conf = new File(path);
-      if (conf.isFile() && conf.canRead()) {
-        var config = new PropertiesConfiguration();
-        try (var reader = new FileReader(conf, UTF_8)) {
-          config.read(reader);
-        } catch (ConfigurationException | IOException e) {
-          throw new IllegalStateException("Error loading client configuration file " + conf, e);
-        }
-        configs.add(config);
-        log.info("Loaded client configuration file {}", conf);
-      }
-    }
-    // We couldn't find the client configuration anywhere
-    if (configs.isEmpty()) {
-      log.debug(
-          "Found no client.conf in default paths. Using default client configuration values.");
-    }
-    return new ClientConfiguration(configs);
-  }
-
-  public static ClientConfiguration deserialize(String serializedConfig) {
-    var propConfig = new PropertiesConfiguration();
-    try {
-      propConfig.read(new StringReader(serializedConfig));
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException(
-          "Error deserializing client configuration: " + serializedConfig, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(propConfig));
-  }
-
-  /**
-   * Muck the value of {@code clientConfPath} if it points to a directory by appending
-   * {@code client.conf} to the end of the file path. This is a no-op if the value is not a
-   * directory on the filesystem.
-   *
-   * @param clientConfPath
-   *          The value of ACCUMULO_CLIENT_CONF_PATH.
-   */
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  static String getClientConfPath(String clientConfPath) {
-    if (clientConfPath == null) {
-      return null;
-    }
-    File filePath = new File(clientConfPath);
-    // If clientConfPath is a directory, tack on the default client.conf file name.
-    if (filePath.exists() && filePath.isDirectory()) {
-      return new File(filePath, "client.conf").toString();
-    }
-    return clientConfPath;
-  }
-
-  private static List<String> getDefaultSearchPath() {
-    String clientConfSearchPath = getClientConfPath(System.getenv("ACCUMULO_CLIENT_CONF_PATH"));
-    List<String> clientConfPaths;
-    if (clientConfSearchPath != null) {
-      clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
-    } else {
-      // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
-      // ~/.accumulo/config
-      // $ACCUMULO_CONF_DIR/client.conf
-      // /etc/accumulo/client.conf
-      // /etc/accumulo/conf/client.conf
-      clientConfPaths = new LinkedList<>();
-      clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME
-          + File.separator + USER_CONF_FILENAME);
-      if (System.getenv("ACCUMULO_CONF_DIR") != null) {
-        clientConfPaths
-            .add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
-      }
-      clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
-      clientConfPaths.add("/etc/accumulo/conf/" + GLOBAL_CONF_FILENAME);
-    }
-    return clientConfPaths;
-  }
-
-  public String serialize() {
-    var propConfig = new PropertiesConfiguration();
-    propConfig.copy(compositeConfig);
-    StringWriter writer = new StringWriter();
-    try {
-      propConfig.write(writer);
-    } catch (ConfigurationException | IOException e) {
-      // this should never happen
-      throw new IllegalStateException(e);
-    }
-    return writer.toString();
-  }
-
-  /**
-   * Returns the value for prop, the default value if not present.
-   *
-   */
-  public String get(ClientProperty prop) {
-    if (compositeConfig.containsKey(prop.getKey()))
-      return compositeConfig.getString(prop.getKey());
-    else
-      return prop.getDefaultValue();
-  }
-
-  private void checkType(ClientProperty property, PropertyType type) {
-    if (!property.getType().equals(type)) {
-      String msg = "Configuration method intended for type " + type + " called with a "
-          + property.getType() + " argument (" + property.getKey() + ")";
-      throw new IllegalArgumentException(msg);
-    }
-  }
-
-  /**
-   * Gets all properties under the given prefix in this configuration.
-   *
-   * @param property
-   *          prefix property, must be of type PropertyType.PREFIX
-   * @return a map of property keys to values
-   * @throws IllegalArgumentException
-   *           if property is not a prefix
-   */
-  public Map<String,String> getAllPropertiesWithPrefix(ClientProperty property) {
-    checkType(property, PropertyType.PREFIX);
-
-    Map<String,String> propMap = new HashMap<>();
-    String prefix = property.getKey();
-    if (prefix.endsWith(".")) {
-      prefix = prefix.substring(0, prefix.length() - 1);
-    }
-    Iterator<?> iter = compositeConfig.getKeys(prefix);
-    while (iter.hasNext()) {
-      String p = (String) iter.next();
-      propMap.put(p, compositeConfig.getString(p));
-    }
-    return propMap;
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   */
-  public void setProperty(ClientProperty prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(ClientProperty, String)} but returns the ClientConfiguration for
-   * chaining purposes
-   */
-  public ClientConfiguration with(ClientProperty prop, String value) {
-    return with(prop.getKey(), value);
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   * @since 1.9.0
-   */
-  public void setProperty(String prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(String, String)} but returns the ClientConfiguration for chaining
-   * purposes
-   *
-   * @since 1.9.0
-   */
-  public ClientConfiguration with(String prop, String value) {
-    compositeConfig.setProperty(prop, value);
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_NAME
-   *
-   */
-  public ClientConfiguration withInstance(String instanceName) {
-    checkArgument(instanceName != null, "instanceName is null");
-    return with(ClientProperty.INSTANCE_NAME, instanceName);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ID
-   *
-   */
-  public ClientConfiguration withInstance(UUID instanceId) {
-    checkArgument(instanceId != null, "instanceId is null");
-    return with(ClientProperty.INSTANCE_ID, instanceId.toString());
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_HOST
-   *
-   */
-  public ClientConfiguration withZkHosts(String zooKeepers) {
-    checkArgument(zooKeepers != null, "zooKeepers is null");
-    return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_TIMEOUT
-   *
-   */
-  public ClientConfiguration withZkTimeout(int timeout) {
-    return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
-  }
-
-  /**
-   * Same as {@link #withSsl(boolean, boolean)} with useJsseConfig set to false
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled) {
-    return withSsl(sslEnabled, false);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_ENABLED and
-   * ClientProperty.RPC_USE_JSSE
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
-    return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
-        .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
-  }
-
-  /**
-   * Same as {@link #withTruststore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withTruststore(String path) {
-    return withTruststore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.RPC_SSL_TRUSTORE_PATH,
-   * ClientProperty.RPC_SSL_TRUSTORE_PASSWORD, and ClientProperty.RPC_SSL_TRUSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withTruststore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
-    if (password != null)
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
-    if (type != null)
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
-    return this;
-  }
-
-  /**
-   * Same as {@link #withKeystore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withKeystore(String path) {
-    return withKeystore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-   * ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, and
-   * ClientProperty.RPC_SSL_KEYSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withKeystore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
-    setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
-    if (password != null)
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
-    if (type != null)
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED.
-   *
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled) {
-    return with(ClientProperty.INSTANCE_RPC_SASL_ENABLED, String.valueOf(saslEnabled));
-  }
-
-  /**
-   * Show whether SASL has been set on this configuration.
-   *
-   * @since 1.9.0
-   */
-  public boolean hasSasl() {
-    return compositeConfig.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(),
-        Boolean.parseBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getDefaultValue()));
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED and
-   * ClientProperty.GENERAL_KERBEROS_PRINCIPAL.
-   *
-   * @param saslEnabled
-   *          Should SASL(kerberos) be enabled
-   * @param kerberosServerPrimary
-   *          The 'primary' component of the Kerberos principal Accumulo servers use to login (e.g.
-   *          'accumulo' in 'accumulo/_HOST@REALM')
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled, String kerberosServerPrimary) {
-    return withSasl(saslEnabled).with(ClientProperty.KERBEROS_SERVER_PRIMARY,
-        kerberosServerPrimary);
-  }
-
-  public boolean containsKey(String key) {
-    return compositeConfig.containsKey(key);
-  }
-
-  public Iterator<String> getKeys() {
-    return compositeConfig.getKeys();
-  }
-
-  public String getString(String key) {
-    return compositeConfig.getString(key);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
index f7792468bb..2ccc4402a8 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
 import java.util.function.Predicate;
@@ -38,122 +36,74 @@ import org.slf4j.LoggerFactory;
 public class ClientConfConverter {
 
   private static final Logger log = LoggerFactory.getLogger(ClientConfConverter.class);
-  private static Map<String,String> confProps = new HashMap<>();
-  private static Map<String,String> propsConf = new HashMap<>();
+  private static final Map<String,String> accumuloConfToClientProps = new HashMap<>();
+  private static final Map<String,String> clientPropsToAccumuloConf = new HashMap<>();
 
-  @SuppressWarnings("deprecation")
-  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.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT
-            .getKey());
-    propsConf.put(ClientProperty.SSL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_USE_JSSE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_USE_JSSE.getKey());
-    propsConf.put(ClientProperty.SASL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SASL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SASL_QOP.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SASL_QOP.getKey());
-    propsConf.put(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-            .getKey());
-
-    for (Map.Entry<String,String> entry : propsConf.entrySet()) {
-      confProps.put(entry.getValue(), entry.getKey());
+  static {
+    // mapping of ClientProperty equivalents in AccumuloConfiguration
+    Map<ClientProperty,Property> conversions = new HashMap<>();
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS, Property.INSTANCE_ZK_HOST);
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, Property.INSTANCE_ZK_TIMEOUT);
+
+    conversions.put(ClientProperty.SASL_ENABLED, Property.INSTANCE_RPC_SASL_ENABLED);
+    conversions.put(ClientProperty.SASL_QOP, Property.RPC_SASL_QOP);
+
+    conversions.put(ClientProperty.SSL_ENABLED, Property.INSTANCE_RPC_SSL_ENABLED);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PASSWORD, Property.RPC_SSL_KEYSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PATH, Property.RPC_SSL_KEYSTORE_PATH);
+    conversions.put(ClientProperty.SSL_KEYSTORE_TYPE, Property.RPC_SSL_KEYSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD, Property.RPC_SSL_TRUSTSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PATH, Property.RPC_SSL_TRUSTSTORE_PATH);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_TYPE, Property.RPC_SSL_TRUSTSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_USE_JSSE, Property.RPC_USE_JSSE);
+
+    for (Map.Entry<ClientProperty,Property> entry : conversions.entrySet()) {
+      accumuloConfToClientProps.put(entry.getValue().getKey(), entry.getKey().getKey());
+      clientPropsToAccumuloConf.put(entry.getKey().getKey(), entry.getValue().getKey());
     }
   }
 
-  static {
-    init();
+  public static Properties toProperties(AccumuloConfiguration config) {
+    final var propsExtractedFromConfig = new Properties();
+
+    // Extract kerberos primary from the config
+    final String serverPrincipal = config.get(Property.GENERAL_KERBEROS_PRINCIPAL);
+    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
+      var krbName = new KerberosName(serverPrincipal);
+      propsExtractedFromConfig.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
+          krbName.getServiceName());
+    }
+
+    // Extract the remaining properties from the config
+    config.stream().filter(e -> accumuloConfToClientProps.keySet().contains(e.getKey()))
+        .forEach(e -> propsExtractedFromConfig.setProperty(e.getKey(), e.getValue()));
+
+    // For all the extracted properties, convert them to their ClientProperty names
+    final var convertedProps = new Properties();
+    propsExtractedFromConfig.forEach((k, v) -> {
+      String confKey = String.valueOf(k);
+      String val = String.valueOf(v);
+      String propKey = accumuloConfToClientProps.get(confKey);
+      convertedProps.setProperty(propKey == null ? confKey : propKey, val);
+    });
+    return convertedProps;
   }
 
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(Properties properties) {
-    org.apache.accumulo.core.client.ClientConfiguration config =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-    for (Object keyObj : properties.keySet()) {
-      String propKey = (String) keyObj;
+  public static AccumuloConfiguration toAccumuloConf(Properties properties) {
+    final var convertedProps = new Properties();
+    for (String propKey : properties.stringPropertyNames()) {
       String val = properties.getProperty(propKey);
-      String confKey = propsConf.get(propKey);
-      if (confKey == null) {
-        config.setProperty(propKey, val);
-      } else {
-        config.setProperty(confKey, val);
+      String confKey = clientPropsToAccumuloConf.get(propKey);
+      if (propKey.equals(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey())) {
+        confKey = Property.GENERAL_KERBEROS_PRINCIPAL.getKey();
+        // Avoid providing a realm since we don't know what it is...
+        val += "/_HOST@" + SaslConnectionParams.getDefaultRealm();
       }
+      convertedProps.setProperty(confKey == null ? propKey : confKey, val);
       if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
-        config.setProperty(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-            "true");
-      }
-    }
-    return config;
-  }
-
-  @SuppressWarnings("deprecation")
-  public static Properties
-      toProperties(org.apache.accumulo.core.client.ClientConfiguration clientConf) {
-    Properties props = new Properties();
-    Iterator<String> clientConfIter = clientConf.getKeys();
-    while (clientConfIter.hasNext()) {
-      String confKey = clientConfIter.next();
-      String val = clientConf.getString(confKey);
-      String propKey = confProps.get(confKey);
-      if (propKey == null) {
-        if (!confKey.equals(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH
-                .getKey())) {
-          props.setProperty(confKey, val);
-        }
-      } else {
-        props.setProperty(propKey, val);
+        convertedProps.setProperty(Property.INSTANCE_RPC_SSL_CLIENT_AUTH.getKey(), "true");
       }
     }
-    return props;
-  }
-
-  public static Properties toProperties(AccumuloConfiguration config) {
-    return toProperties(toClientConf(config));
-  }
-
-  public static AccumuloConfiguration toAccumuloConf(Properties properties) {
-    return toAccumuloConf(toClientConf(properties));
-  }
-
-  /**
-   * A utility method for converting client configuration to a standard configuration object for use
-   * internally.
-   *
-   * @param config
-   *          the original config
-   * @return the client configuration presented in the form of an {@link AccumuloConfiguration}
-   */
-  @SuppressWarnings("deprecation")
-  public static AccumuloConfiguration
-      toAccumuloConf(final org.apache.accumulo.core.client.ClientConfiguration config) {
 
     final AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
 
@@ -161,7 +111,7 @@ public class ClientConfConverter {
 
       @Override
       public boolean isPropertySet(Property prop) {
-        return config.containsKey(prop.getKey());
+        return convertedProps.containsKey(prop.getKey());
       }
 
       @Override
@@ -182,51 +132,15 @@ public class ClientConfConverter {
             }
           }
         }
-
-        if (config.containsKey(key)) {
-          return config.getString(key);
-        } else {
-          // Reconstitute the server kerberos property from the client config
-          if (property == Property.GENERAL_KERBEROS_PRINCIPAL) {
-            if (config.containsKey(
-                org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                    .getKey())) {
-              // Avoid providing a realm since we don't know what it is...
-              return config.getString(
-                  org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                      .getKey())
-                  + "/_HOST@" + SaslConnectionParams.getDefaultRealm();
-            }
-          }
-          return defaults.get(property);
-        }
+        return convertedProps.getProperty(key, defaults.get(property));
       }
 
       @Override
       public void getProperties(Map<String,String> props, Predicate<String> filter) {
         defaults.getProperties(props, filter);
-
-        Iterator<String> keyIter = config.getKeys();
-        while (keyIter.hasNext()) {
-          String key = keyIter.next();
+        for (String key : convertedProps.stringPropertyNames()) {
           if (filter.test(key)) {
-            props.put(key, config.getString(key));
-          }
-        }
-
-        // Two client props that don't exist on the server config. Client doesn't need to know about
-        // the Kerberos instance from the principle, but servers do
-        // Automatically reconstruct the server property when converting a client config.
-        if (props.containsKey(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                .getKey())) {
-          final String serverPrimary = props.remove(
-              org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                  .getKey());
-          if (filter.test(Property.GENERAL_KERBEROS_PRINCIPAL.getKey())) {
-            // Use the _HOST expansion. It should be unnecessary in "client land".
-            props.put(Property.GENERAL_KERBEROS_PRINCIPAL.getKey(),
-                serverPrimary + "/_HOST@" + SaslConnectionParams.getDefaultRealm());
+            props.put(key, convertedProps.getProperty(key));
           }
         }
 
@@ -248,8 +162,8 @@ public class ClientConfConverter {
       }
 
       private org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-        String credProviderPaths =
-            config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+        String credProviderPaths = convertedProps
+            .getProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
         if (credProviderPaths != null && !credProviderPaths.isEmpty()) {
           org.apache.hadoop.conf.Configuration hConf = new org.apache.hadoop.conf.Configuration();
           HadoopCredentialProvider.setPath(hConf, credProviderPaths);
@@ -260,40 +174,7 @@ public class ClientConfConverter {
 
         return null;
       }
-    };
-  }
 
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(AccumuloConfiguration conf) {
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-
-    // Servers will only have the full principal in their configuration -- parse the
-    // primary and realm from it.
-    final String serverPrincipal = conf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
-
-    final KerberosName krbName;
-    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
-      krbName = new KerberosName(serverPrincipal);
-      clientConf.setProperty(
-          org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY,
-          krbName.getServiceName());
-    }
-
-    HashSet<String> clientKeys = new HashSet<>();
-    for (org.apache.accumulo.core.client.ClientConfiguration.ClientProperty prop : org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
-        .values()) {
-      clientKeys.add(prop.getKey());
-    }
-
-    String key;
-    for (Map.Entry<String,String> entry : conf) {
-      key = entry.getKey();
-      if (clientKeys.contains(key)) {
-        clientConf.setProperty(key, entry.getValue());
-      }
-    }
-    return clientConf;
+    };
   }
 }
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 a0c1558f39..1d2552a784 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
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.TableOperations.ImportMappingOptions;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.CredentialProviderToken;
@@ -135,16 +134,7 @@ public enum ClientProperty {
       "The maximum duration to leave idle transports open in the client's transport pool", "2.1.0",
       false),
 
-  // Trace
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TRACE_SPAN_RECEIVERS("trace.span.receivers", "org.apache.accumulo.tracer.ZooTraceClient",
-      "A list of span receiver classes to send trace spans"),
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TRACE_ZOOKEEPER_PATH("trace.zookeeper.path", Constants.ZTRACERS, PropertyType.PATH,
-      "The zookeeper node where tracers are registered", "2.0.0", false);
-
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public static final String TRACE_SPAN_RECEIVER_PREFIX = "trace.span.receiver";
+  ;
 
   private final String key;
   private final String defaultValue;
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
deleted file mode 100644
index 943d9aa922..0000000000
--- a/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class ClientConfigurationTest {
-
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_NAME =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_NAME;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_ZK_HOST =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_ZK_TIMEOUT =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty RPC_SSL_TRUSTSTORE_TYPE =
-      ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE;
-
-  @Test
-  public void testOverrides() {
-    ClientConfiguration clientConfig = createConfig();
-    assertExpectedConfig(clientConfig);
-  }
-
-  @Test
-  public void testSerialization() {
-    ClientConfiguration clientConfig = createConfig();
-    // sanity check that we're starting with what we're expecting
-    assertExpectedConfig(clientConfig);
-
-    String serialized = clientConfig.serialize();
-    ClientConfiguration deserializedClientConfig = ClientConfiguration.deserialize(serialized);
-    assertExpectedConfig(deserializedClientConfig);
-  }
-
-  private void assertExpectedConfig(ClientConfiguration clientConfig) {
-    assertEquals("firstZkHosts", clientConfig.get(INSTANCE_ZK_HOST));
-    assertEquals("secondInstanceName", clientConfig.get(INSTANCE_NAME));
-    assertEquals("123s", clientConfig.get(INSTANCE_ZK_TIMEOUT));
-    assertEquals(RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue(),
-        clientConfig.get(RPC_SSL_TRUSTSTORE_TYPE));
-  }
-
-  private ClientConfiguration createConfig() {
-    return ClientConfiguration.create().with(INSTANCE_ZK_HOST, "firstZkHosts")
-        .with(INSTANCE_NAME, "secondInstanceName").with(INSTANCE_ZK_TIMEOUT, "123s");
-  }
-
-  @Test
-  public void testConfPath() throws IOException {
-    File target = new File(System.getProperty("user.dir"), "target");
-    assertTrue(target.exists(), "'target' build directory does not exist");
-    File testDir = new File(target, getClass().getName());
-    if (!testDir.exists()) {
-      assertTrue(testDir.mkdirs(), "Failed to create test dir " + testDir);
-    }
-
-    File clientConf = new File(testDir, "client.conf");
-    if (!clientConf.exists()) {
-      assertTrue(clientConf.createNewFile(), "Failed to create file " + clientConf);
-    }
-
-    // A directory should return the path with client.conf appended.
-    assertEquals(clientConf.toString(), ClientConfiguration.getClientConfPath(testDir.toString()));
-    // A normal file should return itself
-    assertEquals(clientConf.toString(),
-        ClientConfiguration.getClientConfPath(clientConf.toString()));
-
-    // Something that doesn't exist should return itself (specifically, it shouldn't error)
-    final File missing = new File("foobarbaz12332112");
-    assertEquals(missing.toString(), ClientConfiguration.getClientConfPath(missing.toString()));
-
-    assertNull(ClientConfiguration.getClientConfPath(null));
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
index b94bbba64b..1303fbde1f 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
@@ -35,18 +35,39 @@ public class ClientConfConverterTest {
   @Test
   public void testBasic() {
     Properties before = new Properties();
+
+    // this will be dropped when converting to AccumuloConfiguration
     before.setProperty(ClientProperty.INSTANCE_NAME.getKey(), "instance");
-    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), "zookeepers");
     ClientProperty.setPassword(before, "mypass");
+    before.setProperty(ClientProperty.BATCH_WRITER_THREADS_MAX.getKey(), "5");
+
+    // these will map to equivalent in AccumuloConfiguration
+    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), "zookeepers");
+    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(), "20s");
     before.setProperty(ClientProperty.SSL_ENABLED.getKey(), "true");
+    before.setProperty(ClientProperty.SSL_USE_JSSE.getKey(), "true");
     before.setProperty(ClientProperty.SSL_KEYSTORE_PATH.getKey(), "key_path");
     before.setProperty(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(), "key_pass");
+    before.setProperty(ClientProperty.SSL_KEYSTORE_TYPE.getKey(), "jks");
     before.setProperty(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(), "trust_path");
+    before.setProperty(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(), "trust_pass");
+    before.setProperty(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(), "jks");
     before.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
     before.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(), "primary");
-    before.setProperty(ClientProperty.BATCH_WRITER_THREADS_MAX.getKey(), "5");
+    before.setProperty(ClientProperty.SASL_QOP.getKey(), "auth-int");
+
+    Properties after = ClientConfConverter.toProperties(ClientConfConverter.toAccumuloConf(before));
+
+    // some props don't have an equivalent in the AccumuloConfiguration; set them here and check
+    assertNotEquals(before, after);
+    ClientProperty.setPassword(after, "mypass");
+
+    assertNotEquals(before, after);
+    after.setProperty(ClientProperty.BATCH_WRITER_THREADS_MAX.getKey(), "5");
+
+    assertNotEquals(before, after);
+    after.setProperty(ClientProperty.INSTANCE_NAME.getKey(), "instance");
 
-    Properties after = ClientConfConverter.toProperties(ClientConfConverter.toClientConf(before));
     assertEquals(before, after);
   }
 
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 f9f7efbee6..39e79a50c3 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -57,14 +57,6 @@ public interface AccumuloCluster {
    */
   AccumuloClient createAccumuloClient(String user, AuthenticationToken token);
 
-  /**
-   * Get the client configuration for the cluster
-   *
-   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}}
-   */
-  @Deprecated(since = "2.0.0")
-  org.apache.accumulo.core.client.ClientConfiguration getClientConfig();
-
   /**
    * @return client connection info for cluster
    */
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index b0441d4db5..155b97f544 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
@@ -33,7 +33,6 @@ import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -143,12 +142,6 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
     return Accumulo.newClient().to(getInstanceName(), getZooKeepers()).as(user, token).build();
   }
 
-  @Override
-  @Deprecated(since = "2.0.0")
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return ClientConfConverter.toClientConf(info.getProperties());
-  }
-
   @Override
   public Properties getClientProperties() {
     return info.getProperties();
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 37d127d939..040aa6b92d 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -136,15 +136,6 @@ public class MiniAccumuloCluster implements AutoCloseable {
     return impl.createAccumuloClient(user, token);
   }
 
-  /**
-   * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}
-   */
-  @Deprecated(since = "2.0.0")
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return impl.getClientConfig();
-  }
-
   /**
    * @return A copy of the connection properties for the cluster
    * @since 2.0.0
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 48256c26bd..285491cd0b 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -111,7 +111,6 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Suppliers;
-import com.google.common.collect.Maps;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -157,7 +156,6 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
    * @param config
    *          initial configuration
    */
-  @SuppressWarnings("deprecation")
   public MiniAccumuloClusterImpl(MiniAccumuloConfigImpl config) throws IOException {
 
     this.config = config.initialize();
@@ -222,13 +220,6 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       dfsUri = "file:///";
     }
 
-    File clientConfFile = config.getClientConfFile();
-    // Write only the properties that correspond to ClientConfiguration properties
-    writeConfigProperties(clientConfFile,
-        Maps.filterEntries(config.getSiteConfig(),
-            v -> org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
-                .getPropertyByKey(v.getKey()) != null));
-
     Map<String,String> clientProps = config.getClientProps();
     clientProps.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), config.getZooKeepers());
     clientProps.put(ClientProperty.INSTANCE_NAME.getKey(), config.getInstanceName());
@@ -365,8 +356,6 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     builder.environment().put("CLASSPATH", classpath);
     builder.environment().put("ACCUMULO_HOME", config.getDir().getAbsolutePath());
     builder.environment().put("ACCUMULO_LOG_DIR", config.getLogDir().getAbsolutePath());
-    builder.environment().put("ACCUMULO_CLIENT_CONF_PATH",
-        config.getClientConfFile().getAbsolutePath());
     String ldLibraryPath = Joiner.on(File.pathSeparator).join(config.getNativeLibPaths());
     builder.environment().put("LD_LIBRARY_PATH", ldLibraryPath);
     builder.environment().put("DYLD_LIBRARY_PATH", ldLibraryPath);
@@ -861,13 +850,6 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     return Accumulo.newClient().from(clientProperties.get()).as(user, token).build();
   }
 
-  @SuppressWarnings("deprecation")
-  @Override
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return org.apache.accumulo.core.client.ClientConfiguration.fromMap(config.getSiteConfig())
-        .withInstance(this.getInstanceName()).withZkHosts(this.getZooKeepers());
-  }
-
   @Override
   public Properties getClientProperties() {
     // return a copy, without re-reading the file
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 4e3c71665c..4888449e9d 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -588,16 +588,6 @@ public class MiniAccumuloConfigImpl {
     this.useMiniDFS = useMiniDFS;
   }
 
-  /**
-   * @return location of client conf file containing connection parameters for connecting to this
-   *         minicluster
-   *
-   * @since 1.6.0
-   */
-  public File getClientConfFile() {
-    return new File(getConfDir(), "client.conf");
-  }
-
   public File getAccumuloPropsFile() {
     return new File(getConfDir(), "accumulo.properties");
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
index edf5403975..8b0e3561f8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.server.rpc;
 
-import java.util.Properties;
-
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -32,22 +30,12 @@ import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretM
  */
 public class SaslServerConnectionParams extends SaslConnectionParams {
 
-  private AuthenticationTokenSecretManager secretManager;
-
-  public SaslServerConnectionParams(AccumuloConfiguration conf, AuthenticationToken token) {
-    this(conf, token, null);
-  }
+  private final AuthenticationTokenSecretManager secretManager;
 
   public SaslServerConnectionParams(AccumuloConfiguration conf, AuthenticationToken token,
       AuthenticationTokenSecretManager secretManager) {
     super(conf, token);
-    setSecretManager(secretManager);
-  }
-
-  public SaslServerConnectionParams(Properties clientProps, AuthenticationToken token,
-      AuthenticationTokenSecretManager secretManager) {
-    super(clientProps, token);
-    setSecretManager(secretManager);
+    this.secretManager = secretManager;
   }
 
   @Override
@@ -64,8 +52,4 @@ public class SaslServerConnectionParams extends SaslConnectionParams {
   public AuthenticationTokenSecretManager getSecretManager() {
     return secretManager;
   }
-
-  public void setSecretManager(AuthenticationTokenSecretManager secretManager) {
-    this.secretManager = secretManager;
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
index 871d45258b..5cd5afd9ea 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
@@ -45,7 +45,7 @@ public class SaslServerDigestCallbackHandler extends SaslDigestCallbackHandler {
   private static final Logger log = LoggerFactory.getLogger(SaslServerDigestCallbackHandler.class);
   private static final String NAME = SaslServerDigestCallbackHandler.class.getSimpleName();
 
-  private AuthenticationTokenSecretManager secretManager;
+  private final AuthenticationTokenSecretManager secretManager;
 
   public SaslServerDigestCallbackHandler(AuthenticationTokenSecretManager secretManager) {
     this.secretManager = secretManager;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
index 674d48ec0e..97a882d435 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
@@ -115,12 +115,13 @@ public class ServerContextTest {
       }).anyTimes();
 
       expect(siteConfig.iterator()).andAnswer(conf::iterator).anyTimes();
+      expect(siteConfig.stream()).andAnswer(conf::stream).anyTimes();
 
       replay(factory, context, siteConfig);
 
       assertEquals(ThriftServerType.SASL, context.getThriftServerType());
       SaslServerConnectionParams saslParams = context.getSaslParams();
-      assertEquals(new SaslServerConnectionParams(conf, token), saslParams);
+      assertEquals(new SaslServerConnectionParams(conf, token, null), saslParams);
       assertEquals(username, saslParams.getPrincipal());
 
       verify(factory, context, siteConfig);
@@ -136,7 +137,7 @@ public class ServerContextTest {
     // decision and this check will ensure we don't overlook it
     final int oldestSupported = 8;
     final int currentVersion = AccumuloDataVersion.get();
-    IntConsumer shouldPass = v -> ServerContext.ensureDataVersionCompatible(v);
+    IntConsumer shouldPass = ServerContext::ensureDataVersionCompatible;
     IntConsumer shouldFail = v -> assertThrows(IllegalStateException.class,
         () -> ServerContext.ensureDataVersionCompatible(v));
     IntStream.rangeClosed(oldestSupported, currentVersion).forEach(shouldPass);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java b/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
index 4d30ced426..f3584d770c 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
@@ -80,7 +80,7 @@ public class SaslServerConnectionParamsTest {
       SystemToken token = new SystemToken();
       token.readFields(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
 
-      final SaslConnectionParams saslParams = new SaslServerConnectionParams(rpcConf, token);
+      final SaslConnectionParams saslParams = new SaslServerConnectionParams(rpcConf, token, null);
       assertEquals(primary, saslParams.getKerberosServerPrimary());
       assertEquals(SaslMechanism.GSSAPI, saslParams.getMechanism());
       assertNull(saslParams.getCallbackHandler());