You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bi...@apache.org on 2015/04/21 21:40:00 UTC

[1/2] accumulo git commit: ACCUMULO-3742 add ClientConfiguration constructor that reads from a file and handles multi-valued properties correctly

Repository: accumulo
Updated Branches:
  refs/heads/1.7 b29bd921b -> 61b76b507


ACCUMULO-3742 add ClientConfiguration constructor that reads from a file and handles multi-valued properties correctly


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f5c7f05a
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f5c7f05a
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f5c7f05a

Branch: refs/heads/1.7
Commit: f5c7f05af7bdb95a2485bc0566bd2c4968f1427f
Parents: f0b3487
Author: Billie Rinaldi <bi...@apache.org>
Authored: Tue Apr 21 10:12:23 2015 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Tue Apr 21 10:12:23 2015 -0700

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |  3 +-
 .../core/client/ClientConfiguration.java        | 32 +++++++++++++++++++-
 .../core/util/shell/ShellOptionsJC.java         |  4 +--
 .../core/conf/ClientConfigurationTest.java      | 21 +++++++++++++
 .../src/test/resources/multi-valued.client.conf | 16 ++++++++++
 .../minicluster/MiniAccumuloInstance.java       |  5 ++-
 6 files changed, 74 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
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 8bb8b3f..4d00d97 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
@@ -46,7 +46,6 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
@@ -230,7 +229,7 @@ public class ClientOpts extends Help {
       if (clientConfigFile == null)
         clientConfig = ClientConfiguration.loadDefault();
       else
-        clientConfig = new ClientConfiguration(new PropertiesConfiguration(clientConfigFile));
+        clientConfig = new ClientConfiguration(clientConfigFile);
     } catch (Exception e) {
       throw new IllegalArgumentException(e);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
----------------------------------------------------------------------
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
index 17ad10b..91cda6a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@ -27,10 +27,13 @@ import java.util.UUID;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.PropertyType;
 import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.commons.configuration.AbstractConfiguration;
 import org.apache.commons.configuration.CompositeConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Contains a list of property keys recognized by the Accumulo client and convenience methods for setting them.
@@ -38,6 +41,8 @@ import org.apache.commons.configuration.PropertiesConfiguration;
  * @since 1.6.0
  */
 public class ClientConfiguration extends CompositeConfiguration {
+  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";
@@ -105,10 +110,31 @@ public class ClientConfiguration extends CompositeConfiguration {
     }
   };
 
+  public ClientConfiguration(String configFile) throws ConfigurationException {
+    this(new PropertiesConfiguration(), configFile);
+  }
+
+  private ClientConfiguration(PropertiesConfiguration propertiesConfiguration, String configFile) throws ConfigurationException {
+    super(propertiesConfiguration);
+    // Don't do list interpolation
+    propertiesConfiguration.setListDelimiter('\0');
+    propertiesConfiguration.load(configFile);
+  }
+
   public ClientConfiguration(List<? extends Configuration> configs) {
     super(configs);
     // Don't do list interpolation
     this.setListDelimiter('\0');
+    for (Configuration c : configs) {
+      if (c instanceof AbstractConfiguration) {
+        AbstractConfiguration abstractConfiguration = (AbstractConfiguration) c;
+        if (abstractConfiguration.getListDelimiter() != '\0') {
+          log.warn("Client configuration constructed with a Configuration that did not have list delimiter overridden, multi-valued config properties may " +
+              "be unavailable");
+          abstractConfiguration.setListDelimiter('\0');
+        }
+      }
+    }
   }
 
   /**
@@ -143,7 +169,10 @@ public class ClientConfiguration extends CompositeConfiguration {
       for (String path : paths) {
         File conf = new File(path);
         if (conf.canRead()) {
-          configs.add(new PropertiesConfiguration(conf));
+          PropertiesConfiguration propertiesConfiguration = new PropertiesConfiguration();
+          propertiesConfiguration.setListDelimiter('\0');
+          propertiesConfiguration.load(conf);
+          configs.add(propertiesConfiguration);
         }
       }
       return new ClientConfiguration(configs);
@@ -154,6 +183,7 @@ public class ClientConfiguration extends CompositeConfiguration {
 
   public static ClientConfiguration deserialize(String serializedConfig) {
     PropertiesConfiguration propConfig = new PropertiesConfiguration();
+    propConfig.setListDelimiter('\0');
     try {
       propConfig.load(new StringReader(serializedConfig));
     } catch (ConfigurationException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
index dfa24c5..9206c8d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.log4j.Logger;
 
 import com.beust.jcommander.DynamicParameter;
@@ -270,8 +269,7 @@ public class ShellOptionsJC {
   }
 
   public ClientConfiguration getClientConfiguration() throws ConfigurationException, FileNotFoundException {
-    ClientConfiguration clientConfig = clientConfigFile == null ? ClientConfiguration.loadDefault() : new ClientConfiguration(new PropertiesConfiguration(
-        getClientConfigFile()));
+    ClientConfiguration clientConfig = clientConfigFile == null ? ClientConfiguration.loadDefault() : new ClientConfiguration(getClientConfigFile());
     if (useSsl()) {
       clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SSL_ENABLED, "true");
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
index 40be70f..0ecc519 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
@@ -17,12 +17,14 @@
 package org.apache.accumulo.core.conf;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 
 import java.util.Arrays;
 
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.junit.Test;
 
@@ -63,4 +65,23 @@ public class ClientConfigurationTest {
     third.addProperty(ClientProperty.INSTANCE_ZK_TIMEOUT.getKey(), "123s");
     return new ClientConfiguration(Arrays.asList(first, second, third));
   }
+
+  @Test
+  public void testMultipleValues() throws ConfigurationException {
+    String val = "comma,separated,list";
+    PropertiesConfiguration propertiesConfiguration = new PropertiesConfiguration();
+    propertiesConfiguration.setListDelimiter('\0');
+    propertiesConfiguration.addProperty(ClientProperty.INSTANCE_ZK_HOST.getKey(), val);
+    ClientConfiguration conf = new ClientConfiguration(propertiesConfiguration);
+    assertEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
+    assertEquals(1, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
+
+    conf = new ClientConfiguration(new PropertiesConfiguration("multi-valued.client.conf"));
+    assertNotEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
+    assertEquals(3, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
+
+    conf = new ClientConfiguration("multi-valued.client.conf");
+    assertEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
+    assertEquals(1, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/core/src/test/resources/multi-valued.client.conf
----------------------------------------------------------------------
diff --git a/core/src/test/resources/multi-valued.client.conf b/core/src/test/resources/multi-valued.client.conf
new file mode 100644
index 0000000..457370b
--- /dev/null
+++ b/core/src/test/resources/multi-valued.client.conf
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+instance.zookeeper.host=comma,separated,list

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f5c7f05a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
----------------------------------------------------------------------
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 fb6fb0a..cc6c2ed 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
@@ -43,7 +43,10 @@ public class MiniAccumuloInstance extends ZooKeeperInstance {
 
   public static PropertiesConfiguration getConfigProperties(File directory) {
     try {
-      return new PropertiesConfiguration(new File(new File(directory, "conf"), "client.conf"));
+      PropertiesConfiguration conf = new PropertiesConfiguration();
+      conf.setListDelimiter('\0');
+      conf.load(new File(new File(directory, "conf"), "client.conf"));
+      return conf;
     } catch (ConfigurationException e) {
       // this should never happen since we wrote the config file ourselves
       throw new IllegalArgumentException(e);


[2/2] accumulo git commit: Merge branch '1.6' into 1.7

Posted by bi...@apache.org.
Merge branch '1.6' into 1.7

Conflicts:
	core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
	core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
	core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
	shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/61b76b50
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/61b76b50
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/61b76b50

Branch: refs/heads/1.7
Commit: 61b76b507356ec2d5650b011ec22bd550ccbbd56
Parents: b29bd92 f5c7f05
Author: Billie Rinaldi <bi...@apache.org>
Authored: Tue Apr 21 12:36:59 2015 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Tue Apr 21 12:36:59 2015 -0700

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |  5 +-
 .../core/client/ClientConfiguration.java        | 42 +++++++++++++++-
 .../core/client/ClientConfigurationTest.java    | 51 ++++++++++++++++++--
 .../src/test/resources/multi-valued.client.conf | 17 +++++++
 .../minicluster/MiniAccumuloInstance.java       |  5 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |  3 +-
 .../apache/accumulo/shell/ShellOptionsJC.java   |  4 +-
 .../StandaloneAccumuloClusterConfiguration.java |  3 +-
 .../org/apache/accumulo/test/ShellServerIT.java |  3 +-
 .../test/functional/ConfigurableMacIT.java      |  3 +-
 10 files changed, 116 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 0975ec8,4d00d97..54e8b53
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@@ -44,13 -43,12 +44,12 @@@ import org.apache.accumulo.core.conf.De
  import org.apache.accumulo.core.conf.Property;
  import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.security.ColumnVisibility;
 +import org.apache.accumulo.core.trace.Trace;
  import org.apache.accumulo.core.volume.VolumeConfiguration;
  import org.apache.accumulo.core.zookeeper.ZooUtil;
- import org.apache.commons.configuration.PropertiesConfiguration;
 -import org.apache.accumulo.trace.instrument.Trace;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.Path;
 -import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.security.UserGroupInformation;
  import org.apache.log4j.Level;
  import org.apache.log4j.Logger;
  
@@@ -202,50 -191,6 +201,50 @@@ public class ClientOpts extends Help 
      Trace.off();
    }
  
 +  /**
 +   * Automatically update the options to use a KerberosToken when SASL is enabled for RPCs. Don't overwrite the options if the user has provided something
 +   * specifically.
 +   */
 +  public void updateKerberosCredentials() {
 +    ClientConfiguration clientConfig;
 +    try {
 +      if (clientConfigFile == null)
 +        clientConfig = ClientConfiguration.loadDefault();
 +      else
-         clientConfig = new ClientConfiguration(new PropertiesConfiguration(clientConfigFile));
++        clientConfig = new ClientConfiguration(clientConfigFile);
 +    } catch (Exception e) {
 +      throw new IllegalArgumentException(e);
 +    }
 +    updateKerberosCredentials(clientConfig);
 +  }
 +
 +  /**
 +   * Automatically update the options to use a KerberosToken when SASL is enabled for RPCs. Don't overwrite the options if the user has provided something
 +   * specifically.
 +   */
 +  public void updateKerberosCredentials(ClientConfiguration clientConfig) {
 +    final boolean clientConfSaslEnabled = Boolean.parseBoolean(clientConfig.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED));
 +    if ((saslEnabled || clientConfSaslEnabled) && null == tokenClassName) {
 +      tokenClassName = KerberosToken.CLASS_NAME;
 +      // ACCUMULO-3701 We need to ensure we're logged in before parseArgs returns as the MapReduce Job is going to make a copy of the current user (UGI)
 +      // when it is instantiated.
 +      if (null != keytabPath) {
 +        File keytab = new File(keytabPath);
 +        if (!keytab.exists() || !keytab.isFile()) {
 +          throw new IllegalArgumentException("Keytab isn't a normal file: " + keytabPath);
 +        }
 +        if (null == principal) {
 +          throw new IllegalArgumentException("Principal must be provided if logging in via Keytab");
 +        }
 +        try {
 +          UserGroupInformation.loginUserFromKeytab(principal, keytab.getAbsolutePath());
 +        } catch (IOException e) {
 +          throw new RuntimeException("Failed to log in with keytab", e);
 +        }
 +      }
 +    }
 +  }
 +
    @Override
    public void parseArgs(String programName, String[] args, Object... others) {
      super.parseArgs(programName, args, others);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
index d37d471,91cda6a..0677365
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@@ -31,6 -26,8 +31,7 @@@ import java.util.UUID
  
  import org.apache.accumulo.core.conf.Property;
  import org.apache.accumulo.core.conf.PropertyType;
 -import org.apache.accumulo.core.util.ArgumentChecker;
+ import org.apache.commons.configuration.AbstractConfiguration;
  import org.apache.commons.configuration.CompositeConfiguration;
  import org.apache.commons.configuration.Configuration;
  import org.apache.commons.configuration.ConfigurationException;
@@@ -134,10 -110,31 +139,42 @@@ public class ClientConfiguration extend
      }
    };
  
+   public ClientConfiguration(String configFile) throws ConfigurationException {
+     this(new PropertiesConfiguration(), configFile);
+   }
+ 
+   private ClientConfiguration(PropertiesConfiguration propertiesConfiguration, String configFile) throws ConfigurationException {
+     super(propertiesConfiguration);
+     // Don't do list interpolation
 -    propertiesConfiguration.setListDelimiter('\0');
++    propertiesConfiguration.setDelimiterParsingDisabled(true);
++    propertiesConfiguration.load(configFile);
++  }
++
++  public ClientConfiguration(File configFile) throws ConfigurationException {
++    this(new PropertiesConfiguration(), configFile);
++  }
++
++  private ClientConfiguration(PropertiesConfiguration propertiesConfiguration, File configFile) throws ConfigurationException {
++    super(propertiesConfiguration);
++    // Don't do list interpolation
++    propertiesConfiguration.setDelimiterParsingDisabled(true);
+     propertiesConfiguration.load(configFile);
+   }
+ 
    public ClientConfiguration(List<? extends Configuration> configs) {
      super(configs);
      // Don't do list interpolation
--    this.setListDelimiter('\0');
++    this.setDelimiterParsingDisabled(true);
+     for (Configuration c : configs) {
+       if (c instanceof AbstractConfiguration) {
+         AbstractConfiguration abstractConfiguration = (AbstractConfiguration) c;
 -        if (abstractConfiguration.getListDelimiter() != '\0') {
 -          log.warn("Client configuration constructed with a Configuration that did not have list delimiter overridden, multi-valued config properties may " +
 -              "be unavailable");
 -          abstractConfiguration.setListDelimiter('\0');
++        if (!abstractConfiguration.isDelimiterParsingDisabled() && abstractConfiguration.getListDelimiter() != '\0') {
++          log.warn("Client configuration constructed with a Configuration that did not have list delimiter disabled or overridden, multi-valued config " +
++              "properties may be unavailable");
++          abstractConfiguration.setDelimiterParsingDisabled(true);
+         }
+       }
+     }
    }
  
    /**
@@@ -172,7 -169,10 +209,7 @@@
        for (String path : paths) {
          File conf = new File(path);
          if (conf.canRead()) {
-           configs.add(new PropertiesConfiguration(conf));
 -          PropertiesConfiguration propertiesConfiguration = new PropertiesConfiguration();
 -          propertiesConfiguration.setListDelimiter('\0');
 -          propertiesConfiguration.load(conf);
 -          configs.add(propertiesConfiguration);
++          configs.add(new ClientConfiguration(conf));
          }
        }
        return new ClientConfiguration(configs);
@@@ -183,6 -183,7 +220,7 @@@
  
    public static ClientConfiguration deserialize(String serializedConfig) {
      PropertiesConfiguration propConfig = new PropertiesConfiguration();
 -    propConfig.setListDelimiter('\0');
++    propConfig.setDelimiterParsingDisabled(true);
      try {
        propConfig.load(new StringReader(serializedConfig));
      } catch (ConfigurationException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
index 424cea1,0000000..225298b
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
@@@ -1,81 -1,0 +1,126 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.core.client;
 +
 +import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertNotEquals;
 +
++import java.io.StringReader;
 +import java.util.Arrays;
 +import java.util.Collections;
++import java.util.HashMap;
++import java.util.Map;
 +
 +import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 +import org.apache.commons.configuration.Configuration;
++import org.apache.commons.configuration.ConfigurationException;
++import org.apache.commons.configuration.MapConfiguration;
 +import org.apache.commons.configuration.PropertiesConfiguration;
 +import org.junit.Test;
 +
 +public class ClientConfigurationTest {
 +
 +  @Test
 +  public void testOverrides() throws Exception {
 +    ClientConfiguration clientConfig = createConfig();
 +    assertExpectedConfig(clientConfig);
 +  }
 +
 +  @Test
 +  public void testSerialization() throws Exception {
 +    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(ClientProperty.INSTANCE_ZK_HOST));
 +    assertEquals("secondInstanceName", clientConfig.get(ClientProperty.INSTANCE_NAME));
 +    assertEquals("123s", clientConfig.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
 +    assertEquals(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue(), clientConfig.get(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE));
 +  }
 +
 +  private ClientConfiguration createConfig() {
-     Configuration first = new PropertiesConfiguration();
++    Configuration first = new ClientConfiguration();
 +    first.addProperty(ClientProperty.INSTANCE_ZK_HOST.getKey(), "firstZkHosts");
-     Configuration second = new PropertiesConfiguration();
++    Configuration second = new ClientConfiguration();
 +    second.addProperty(ClientProperty.INSTANCE_ZK_HOST.getKey(), "secondZkHosts");
 +    second.addProperty(ClientProperty.INSTANCE_NAME.getKey(), "secondInstanceName");
-     Configuration third = new PropertiesConfiguration();
++    Configuration third = new ClientConfiguration();
 +    third.addProperty(ClientProperty.INSTANCE_ZK_HOST.getKey(), "thirdZkHosts");
 +    third.addProperty(ClientProperty.INSTANCE_NAME.getKey(), "thirdInstanceName");
 +    third.addProperty(ClientProperty.INSTANCE_ZK_TIMEOUT.getKey(), "123s");
 +    return new ClientConfiguration(Arrays.asList(first, second, third));
 +  }
 +
 +  @Test
 +  public void testSasl() {
 +    ClientConfiguration conf = new ClientConfiguration(Collections.<Configuration> emptyList());
 +    assertEquals("false", conf.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED));
 +    conf.withSasl(false);
 +    assertEquals("false", conf.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED));
 +    conf.withSasl(true);
 +    assertEquals("true", conf.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED));
 +    final String primary = "accumulo";
 +    conf.withSasl(true, primary);
 +    assertEquals(primary, conf.get(ClientProperty.KERBEROS_SERVER_PRIMARY));
 +  }
 +
++  @Test
++  public void testMultipleValues() throws ConfigurationException {
++    String val = "comma,separated,list";
++
++    // not the recommended way to construct a client configuration, but it works
++    PropertiesConfiguration propertiesConfiguration = new PropertiesConfiguration();
++    propertiesConfiguration.setDelimiterParsingDisabled(true);
++    propertiesConfiguration.addProperty(ClientProperty.INSTANCE_ZK_HOST.getKey(), val);
++    propertiesConfiguration.load(new StringReader(ClientProperty.TRACE_SPAN_RECEIVERS.getKey() + "=" + val));
++    ClientConfiguration conf = new ClientConfiguration(propertiesConfiguration);
++    assertEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
++    assertEquals(1, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
++    assertEquals(val, conf.get(ClientProperty.TRACE_SPAN_RECEIVERS));
++    assertEquals(1, conf.getList(ClientProperty.TRACE_SPAN_RECEIVERS.getKey()).size());
++
++    // incorrect usage that results in not being able to use multi-valued properties
++    conf = new ClientConfiguration(new PropertiesConfiguration("multi-valued.client.conf"));
++    assertNotEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
++    assertEquals(3, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
++    assertNotEquals(val, conf.get(ClientProperty.TRACE_SPAN_RECEIVERS));
++    assertEquals(3, conf.getList(ClientProperty.TRACE_SPAN_RECEIVERS.getKey()).size());
++
++    // recommended usage
++    conf = new ClientConfiguration("multi-valued.client.conf");
++    assertEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
++    assertEquals(1, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
++    assertEquals(val, conf.get(ClientProperty.TRACE_SPAN_RECEIVERS));
++    assertEquals(1, conf.getList(ClientProperty.TRACE_SPAN_RECEIVERS.getKey()).size());
++
++    // only used internally
++    Map<String, String> map = new HashMap<>();
++    map.put(ClientProperty.INSTANCE_ZK_HOST.getKey(), val);
++    map.put(ClientProperty.TRACE_SPAN_RECEIVERS.getKey(), val);
++    conf = new ClientConfiguration(new MapConfiguration(map));
++    assertEquals(val, conf.get(ClientProperty.INSTANCE_ZK_HOST));
++    assertEquals(1, conf.getList(ClientProperty.INSTANCE_ZK_HOST.getKey()).size());
++    assertEquals(val, conf.get(ClientProperty.TRACE_SPAN_RECEIVERS));
++    assertEquals(1, conf.getList(ClientProperty.TRACE_SPAN_RECEIVERS.getKey()).size());
++  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/core/src/test/resources/multi-valued.client.conf
----------------------------------------------------------------------
diff --cc core/src/test/resources/multi-valued.client.conf
index 0000000,457370b..b04a42e
mode 000000,100644..100644
--- a/core/src/test/resources/multi-valued.client.conf
+++ b/core/src/test/resources/multi-valued.client.conf
@@@ -1,0 -1,16 +1,17 @@@
+ # Licensed to the Apache Software Foundation (ASF) under one or more
+ # contributor license agreements.  See the NOTICE file distributed with
+ # this work for additional information regarding copyright ownership.
+ # The ASF licenses this file to You under the Apache License, Version 2.0
+ # (the "License"); you may not use this file except in compliance with
+ # the License.  You may obtain a copy of the License at
+ #
+ #     http://www.apache.org/licenses/LICENSE-2.0
+ #
+ # Unless required by applicable law or agreed to in writing, software
+ # distributed under the License is distributed on an "AS IS" BASIS,
+ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ # See the License for the specific language governing permissions and
+ # limitations under the License.
+ 
+ instance.zookeeper.host=comma,separated,list
++trace.span.receivers=comma,separated,list

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
----------------------------------------------------------------------
diff --cc minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
index 095bae2,cc6c2ed..80546bd
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
@@@ -43,7 -43,10 +43,10 @@@ public class MiniAccumuloInstance exten
  
    public static PropertiesConfiguration getConfigProperties(File directory) {
      try {
-       return new PropertiesConfiguration(new File(new File(directory, "conf"), "client.conf"));
+       PropertiesConfiguration conf = new PropertiesConfiguration();
 -      conf.setListDelimiter('\0');
++      conf.setDelimiterParsingDisabled(true);
+       conf.load(new File(new File(directory, "conf"), "client.conf"));
+       return conf;
      } catch (ConfigurationException e) {
        // this should never happen since we wrote the config file ourselves
        throw new IllegalArgumentException(e);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --cc proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index b3e7639,83e8af4..0519d84
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@@ -104,14 -100,9 +104,13 @@@ import org.apache.accumulo.proxy.thrift
  import org.apache.accumulo.proxy.thrift.UnknownScanner;
  import org.apache.accumulo.proxy.thrift.UnknownWriter;
  import org.apache.accumulo.proxy.thrift.WriterOptions;
 +import org.apache.accumulo.server.rpc.ThriftServerType;
 +import org.apache.accumulo.server.rpc.UGIAssumingProcessor;
 +import org.apache.commons.configuration.ConfigurationException;
- import org.apache.commons.configuration.PropertiesConfiguration;
  import org.apache.hadoop.io.Text;
 -import org.apache.log4j.Logger;
  import org.apache.thrift.TException;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
  
  import com.google.common.cache.Cache;
  import com.google.common.cache.CacheBuilder;
@@@ -188,21 -176,9 +187,21 @@@ public class ProxyServer implements Acc
      String useMock = props.getProperty("useMockInstance");
      if (useMock != null && Boolean.parseBoolean(useMock))
        instance = new MockInstance();
 -    else
 -      instance = new ZooKeeperInstance(ClientConfiguration.loadDefault().withInstance(props.getProperty("instance"))
 +    else {
 +      ClientConfiguration clientConf;
 +      if (props.containsKey("clientConfigurationFile")) {
 +        String clientConfFile = props.getProperty("clientConfigurationFile");
 +        try {
-           clientConf = new ClientConfiguration(new PropertiesConfiguration(clientConfFile));
++          clientConf = new ClientConfiguration(clientConfFile);
 +        } catch (ConfigurationException e) {
 +          throw new RuntimeException(e);
 +        }
 +      } else {
 +        clientConf = ClientConfiguration.loadDefault();
 +      }
 +      instance = new ZooKeeperInstance(clientConf.withInstance(props.getProperty("instance"))
            .withZkHosts(props.getProperty("zookeepers")));
 +    }
  
      try {
        String tokenProp = props.getProperty("tokenClass", PasswordToken.class.getName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
index 4ba6de0,0000000..7e6445d
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
@@@ -1,315 -1,0 +1,313 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.shell;
 +
 +import java.io.File;
 +import java.io.FileNotFoundException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Scanner;
 +import java.util.TreeMap;
 +
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 +import org.apache.commons.configuration.ConfigurationException;
- import org.apache.commons.configuration.PropertiesConfiguration;
 +import org.apache.hadoop.security.UserGroupInformation;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.beust.jcommander.DynamicParameter;
 +import com.beust.jcommander.IStringConverter;
 +import com.beust.jcommander.Parameter;
 +import com.beust.jcommander.ParameterException;
 +import com.beust.jcommander.converters.FileConverter;
 +
 +public class ShellOptionsJC {
 +  private static final Logger log = LoggerFactory.getLogger(ShellOptionsJC.class);
 +
 +  @Parameter(names = {"-u", "--user"}, description = "username (defaults to your OS user)")
 +  private String username = null;
 +
 +  public static class PasswordConverter implements IStringConverter<String> {
 +    public static final String STDIN = "stdin";
 +
 +    private enum KeyType {
 +      PASS("pass:"), ENV("env:") {
 +        @Override
 +        String process(String value) {
 +          return System.getenv(value);
 +        }
 +      },
 +      FILE("file:") {
 +        @Override
 +        String process(String value) {
 +          Scanner scanner = null;
 +          try {
 +            scanner = new Scanner(new File(value));
 +            return scanner.nextLine();
 +          } catch (FileNotFoundException e) {
 +            throw new ParameterException(e);
 +          } finally {
 +            if (scanner != null) {
 +              scanner.close();
 +            }
 +          }
 +        }
 +      },
 +      STDIN(PasswordConverter.STDIN) {
 +        @Override
 +        public boolean matches(String value) {
 +          return prefix.equals(value);
 +        }
 +
 +        @Override
 +        public String convert(String value) {
 +          // Will check for this later
 +          return prefix;
 +        }
 +      };
 +
 +      String prefix;
 +
 +      private KeyType(String prefix) {
 +        this.prefix = prefix;
 +      }
 +
 +      public boolean matches(String value) {
 +        return value.startsWith(prefix);
 +      }
 +
 +      public String convert(String value) {
 +        return process(value.substring(prefix.length()));
 +      }
 +
 +      String process(String value) {
 +        return value;
 +      }
 +    };
 +
 +    @Override
 +    public String convert(String value) {
 +      for (KeyType keyType : KeyType.values()) {
 +        if (keyType.matches(value)) {
 +          return keyType.convert(value);
 +        }
 +      }
 +
 +      return value;
 +    }
 +  }
 +
 +  // Note: Don't use "password = true" because then it will prompt even if we have a token
 +  @Parameter(names = {"-p", "--password"}, description = "password (can be specified as 'pass:<password>', 'file:<local file containing the password>', "
 +      + "'env:<variable containing the pass>', or stdin)", converter = PasswordConverter.class)
 +  private String password;
 +
 +  public static class TokenConverter implements IStringConverter<AuthenticationToken> {
 +    @Override
 +    public AuthenticationToken convert(String value) {
 +      try {
 +        return Class.forName(value).asSubclass(AuthenticationToken.class).newInstance();
 +      } catch (Exception e) {
 +        // Catching ClassNotFoundException, ClassCastException, InstantiationException and IllegalAccessException
 +        log.error("Could not instantiate AuthenticationToken {}", value, e);
 +        throw new ParameterException(e);
 +      }
 +    }
 +  }
 +
 +  @Parameter(names = {"-tc", "--tokenClass"}, description = "token type to create, use the -l to pass options", converter = TokenConverter.class)
 +  private AuthenticationToken authenticationToken;
 +
 +  @DynamicParameter(names = {"-l", "--tokenProperty"}, description = "login properties in the format key=value. Reuse -l for each property")
 +  private Map<String,String> tokenProperties = new TreeMap<String,String>();
 +
 +  @Parameter(names = "--disable-tab-completion", description = "disables tab completion (for less overhead when scripting)")
 +  private boolean tabCompletionDisabled;
 +
 +  @Parameter(names = "--debug", description = "enables client debugging")
 +  private boolean debugEnabled;
 +
 +  @Parameter(names = "--fake", description = "fake a connection to accumulo")
 +  private boolean fake;
 +
 +  @Parameter(names = {"-?", "--help"}, help = true, description = "display this help")
 +  private boolean helpEnabled;
 +
 +  @Parameter(names = {"-e", "--execute-command"}, description = "executes a command, and then exits")
 +  private String execCommand;
 +
 +  @Parameter(names = {"-f", "--execute-file"}, description = "executes commands from a file at startup", converter = FileConverter.class)
 +  private File execFile;
 +
 +  @Parameter(names = {"-fv", "--execute-file-verbose"}, description = "executes commands from a file at startup, with commands shown",
 +      converter = FileConverter.class)
 +  private File execFileVerbose;
 +
 +  @Parameter(names = {"-h", "--hdfsZooInstance"}, description = "use hdfs zoo instance")
 +  private boolean hdfsZooInstance;
 +
 +  @Parameter(names = {"-z", "--zooKeeperInstance"}, description = "use a zookeeper instance with the given instance name and list of zoo hosts", arity = 2)
 +  private List<String> zooKeeperInstance = new ArrayList<String>();
 +
 +  @Parameter(names = {"--ssl"}, description = "use ssl to connect to accumulo")
 +  private boolean useSsl = false;
 +
 +  @Parameter(names = "--sasl", description = "use SASL to connect to Accumulo (Kerberos)")
 +  private boolean useSasl = false;
 +
 +  @Parameter(names = "--config-file", description = "read the given client config file. "
 +      + "If omitted, the path searched can be specified with $ACCUMULO_CLIENT_CONF_PATH, "
 +      + "which defaults to ~/.accumulo/config:$ACCUMULO_CONF_DIR/client.conf:/etc/accumulo/client.conf")
 +  private String clientConfigFile = null;
 +
 +  @Parameter(names = {"-zi", "--zooKeeperInstanceName"}, description = "use a zookeeper instance with the given instance name")
 +  private String zooKeeperInstanceName;
 +
 +  @Parameter(names = {"-zh", "--zooKeeperHosts"}, description = "use a zookeeper instance with the given list of zoo hosts")
 +  private String zooKeeperHosts;
 +
 +  @Parameter(names = "--auth-timeout", description = "minutes the shell can be idle without re-entering a password")
 +  private int authTimeout = 60; // TODO Add validator for positive number
 +
 +  @Parameter(names = "--disable-auth-timeout", description = "disables requiring the user to re-type a password after being idle")
 +  private boolean authTimeoutDisabled;
 +
 +  @Parameter(hidden = true)
 +  private List<String> unrecognizedOptions;
 +
 +  public String getUsername() throws Exception {
 +    if (null == username) {
 +      final ClientConfiguration clientConf = getClientConfiguration();
 +      if (Boolean.parseBoolean(clientConf.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED))) {
 +        if (!UserGroupInformation.isSecurityEnabled()) {
 +          throw new RuntimeException("Kerberos security is not enabled");
 +        }
 +        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 +        username = ugi.getUserName();
 +      } else {
 +        username = System.getProperty("user.name", "root");
 +      }
 +    }
 +    return username;
 +  }
 +
 +  public String getPassword() {
 +    return password;
 +  }
 +
 +  public AuthenticationToken getAuthenticationToken() throws Exception {
 +    if (null == authenticationToken) {
 +      final ClientConfiguration clientConf = getClientConfiguration();
 +      // Automatically use a KerberosToken if the client conf is configured for SASL
 +      final boolean saslEnabled = Boolean.parseBoolean(clientConf.get(ClientProperty.INSTANCE_RPC_SASL_ENABLED));
 +      if (saslEnabled) {
 +        authenticationToken = new KerberosToken();
 +      }
 +    }
 +    return authenticationToken;
 +  }
 +
 +  public Map<String,String> getTokenProperties() {
 +    return tokenProperties;
 +  }
 +
 +  public boolean isTabCompletionDisabled() {
 +    return tabCompletionDisabled;
 +  }
 +
 +  public boolean isDebugEnabled() {
 +    return debugEnabled;
 +  }
 +
 +  public boolean isFake() {
 +    return fake;
 +  }
 +
 +  public boolean isHelpEnabled() {
 +    return helpEnabled;
 +  }
 +
 +  public String getExecCommand() {
 +    return execCommand;
 +  }
 +
 +  public File getExecFile() {
 +    return execFile;
 +  }
 +
 +  public File getExecFileVerbose() {
 +    return execFileVerbose;
 +  }
 +
 +  public boolean isHdfsZooInstance() {
 +    return hdfsZooInstance;
 +  }
 +
 +  public List<String> getZooKeeperInstance() {
 +    return zooKeeperInstance;
 +  }
 +
 +  public String getZooKeeperInstanceName() {
 +    return zooKeeperInstanceName;
 +  }
 +
 +  public String getZooKeeperHosts() {
 +    return zooKeeperHosts;
 +  }
 +
 +  public int getAuthTimeout() {
 +    return authTimeout;
 +  }
 +
 +  public boolean isAuthTimeoutDisabled() {
 +    if (useSasl()) {
 +      return true;
 +    }
 +    return authTimeoutDisabled;
 +  }
 +
 +  public List<String> getUnrecognizedOptions() {
 +    return unrecognizedOptions;
 +  }
 +
 +  public boolean useSsl() {
 +    return useSsl;
 +  }
 +
 +  public String getClientConfigFile() {
 +    return clientConfigFile;
 +  }
 +
 +  public ClientConfiguration getClientConfiguration() throws ConfigurationException, FileNotFoundException {
-     ClientConfiguration clientConfig = clientConfigFile == null ? ClientConfiguration.loadDefault() : new ClientConfiguration(new PropertiesConfiguration(
-         getClientConfigFile()));
++    ClientConfiguration clientConfig = clientConfigFile == null ? ClientConfiguration.loadDefault() : new ClientConfiguration(getClientConfigFile());
 +    if (useSsl()) {
 +      clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SSL_ENABLED, "true");
 +    }
 +    if (useSasl()) {
 +      clientConfig.setProperty(ClientProperty.INSTANCE_RPC_SASL_ENABLED, "true");
 +    }
 +    return clientConfig;
 +  }
 +
 +  public boolean useSasl() {
 +    return useSasl;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/test/src/test/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
index 22fe103,5fb7203..53e30c2
--- a/test/src/test/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
+++ b/test/src/test/java/org/apache/accumulo/harness/conf/StandaloneAccumuloClusterConfiguration.java
@@@ -32,14 -21,8 +32,13 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.Instance;
  import org.apache.accumulo.core.client.ZooKeeperInstance;
  import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.KerberosToken;
  import org.apache.accumulo.core.client.security.tokens.PasswordToken;
  import org.apache.accumulo.harness.AccumuloClusterIT.ClusterType;
 +import org.apache.commons.configuration.ConfigurationException;
- import org.apache.commons.configuration.PropertiesConfiguration;
 +import org.apache.hadoop.fs.Path;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
  
  /**
   * Extract connection information to a standalone Accumulo instance from Java properties
@@@ -85,48 -51,6 +84,48 @@@ public class StandaloneAccumuloClusterC
      }
  
      this.conf = getConfiguration(type);
 +    this.clientConfFile = clientConfFile;
 +    this.clientConf = ClientConfiguration.loadDefault();
 +    try {
-       clientConf.addConfiguration(new PropertiesConfiguration(clientConfFile));
++      clientConf.addConfiguration(new ClientConfiguration(clientConfFile));
 +    } catch (ConfigurationException e) {
 +      throw new RuntimeException("Failed to load client configuration from " + clientConfFile);
 +    }
 +    // Update instance name if not already set
 +    if (!clientConf.containsKey(ClientProperty.INSTANCE_NAME.getKey())) {
 +      clientConf.withInstance(getInstanceName());
 +    }
 +    // Update zookeeper hosts if not already set
 +    if (!clientConf.containsKey(ClientProperty.INSTANCE_ZK_HOST.getKey())) {
 +      clientConf.withZkHosts(getZooKeepers());
 +    }
 +
 +    // The user Accumulo is running as
 +    serverUser = conf.get(ACCUMULO_STANDALONE_SERVER_USER);
 +    if (null == serverUser) {
 +      serverUser = ACCUMULO_STANDALONE_SERVER_USER_DEFAULT;
 +    }
 +
 +    clusterUsers = new ArrayList<>();
 +    for (Entry<String,String> entry : conf.entrySet()) {
 +      String key = entry.getKey();
 +      if (key.startsWith(ACCUMULO_STANDALONE_USER_KEY)) {
 +        String suffix = key.substring(ACCUMULO_STANDALONE_USER_KEY.length());
 +        String keytab = conf.get(ACCUMULO_STANDALONE_USER_KEYTABS_KEY + suffix);
 +        if (null != keytab) {
 +          File keytabFile = new File(keytab);
 +          assertTrue("Keytab doesn't exist: " + keytabFile, keytabFile.exists() && keytabFile.isFile());
 +          clusterUsers.add(new ClusterUser(entry.getValue(), keytabFile));
 +        } else {
 +          String password = conf.get(ACCUMULO_STANDALONE_USER_PASSWORDS_KEY + suffix);
 +          if (null == password) {
 +            throw new IllegalArgumentException("Missing password or keytab configuration for user with offset " + suffix);
 +          }
 +          clusterUsers.add(new ClusterUser(entry.getValue(), password));
 +        }
 +      }
 +    }
 +    log.info("Initialized Accumulo users with Kerberos keytabs: {}", clusterUsers);
    }
  
    @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index fb725d6,914a914..8b37169
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@@ -67,8 -60,6 +67,7 @@@ import org.apache.accumulo.shell.Shell
  import org.apache.accumulo.test.functional.FunctionalTestUtils;
  import org.apache.accumulo.test.functional.SlowIterator;
  import org.apache.accumulo.tracer.TraceServer;
 +import org.apache.commons.configuration.ConfigurationException;
- import org.apache.commons.configuration.PropertiesConfiguration;
  import org.apache.commons.io.FileUtils;
  import org.apache.commons.lang.StringUtils;
  import org.apache.hadoop.conf.Configuration;
@@@ -144,13 -132,7 +143,13 @@@ public class ShellServerIT extends Shar
      public StringInputStream input;
      public Shell shell;
  
 -    TestShell(String rootPass, String instanceName, String zookeepers, String configFile) throws IOException {
 +    TestShell(String user, String rootPass, String instanceName, String zookeepers, File configFile) throws IOException {
 +      ClientConfiguration clientConf;
 +      try {
-         clientConf = new ClientConfiguration(new PropertiesConfiguration(configFile));
++        clientConf = new ClientConfiguration(configFile);
 +      } catch (ConfigurationException e) {
 +        throw new IOException(e);
 +      }
        // start the shell
        output = new TestOutputStream();
        input = new StringInputStream();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/61b76b50/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
index eced2c3,1c8b1f0..53eb8e4
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
@@@ -41,9 -38,8 +41,8 @@@ import org.apache.accumulo.minicluster.
  import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
  import org.apache.accumulo.minicluster.impl.ZooKeeperBindException;
  import org.apache.accumulo.test.util.CertUtils;
- import org.apache.commons.configuration.PropertiesConfiguration;
 +import org.apache.commons.io.FileUtils;
  import org.apache.hadoop.conf.Configuration;
 -import org.apache.log4j.Logger;
  import org.apache.zookeeper.KeeperException;
  import org.junit.After;
  import org.junit.Before;
@@@ -174,8 -162,4 +173,8 @@@ public class ConfigurableMacIT extends 
      return MonitorUtil.getLocation(instance);
    }
  
 +  protected ClientConfiguration getClientConfig() throws Exception {
-     return new ClientConfiguration(new PropertiesConfiguration(getCluster().getConfig().getClientConfFile()));
++    return new ClientConfiguration(getCluster().getConfig().getClientConfFile());
 +  }
 +
  }