You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2018/05/01 14:06:36 UTC

[accumulo] branch master updated: #408 - Removing ClientConfiguration from Proxy code (#451)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9408af7   #408 - Removing ClientConfiguration from Proxy code (#451)
9408af7 is described below

commit 9408af7b9338e9b37d28ac56ae829f9868e94fd8
Author: Mike Walch <mw...@apache.org>
AuthorDate: Tue May 1 10:06:33 2018 -0400

     #408 - Removing ClientConfiguration from Proxy code (#451)
    
    * Updated proxy.properties
    * Proxy command now takes proxy.properties & accumulo-client.properties
    * Removed proxy from accumulo-service
---
 assemble/bin/accumulo-service                      |   3 +-
 proxy/proxy.properties                             |   4 -
 .../main/java/org/apache/accumulo/proxy/Proxy.java | 113 ++++++++-------------
 .../org/apache/accumulo/proxy/ProxyServer.java     |  13 +--
 .../accumulo/test/functional/KerberosProxyIT.java  |  34 ++++++-
 .../accumulo/test/proxy/ProxyDurabilityIT.java     |  12 +--
 .../accumulo/test/proxy/SimpleProxyBase.java       |  11 +-
 .../test/proxy/TestProxyInstanceOperations.java    |   8 +-
 8 files changed, 84 insertions(+), 114 deletions(-)

diff --git a/assemble/bin/accumulo-service b/assemble/bin/accumulo-service
index 54161af..d44ab37 100755
--- a/assemble/bin/accumulo-service
+++ b/assemble/bin/accumulo-service
@@ -23,7 +23,6 @@ Services:
   gc          Accumulo garbage collector
   monitor     Accumulo monitor
   master      Accumulo master
-  proxy       Accumulo proxy
   tserver     Accumulo tserver
   tracer      Accumulo tracter
 
@@ -136,7 +135,7 @@ function main() {
   service="$1"
   pid_file="${basedir}/run/accumulo-${service}.pid"
   case "$service" in
-    gc|master|monitor|proxy|tserver|tracer)
+    gc|master|monitor|tserver|tracer)
       if [[ -z $2 ]]; then
         invalid_args "<command> cannot be empty"
       fi
diff --git a/proxy/proxy.properties b/proxy/proxy.properties
index e6f3d2d..ad5b77e 100644
--- a/proxy/proxy.properties
+++ b/proxy/proxy.properties
@@ -13,10 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Accumulo instance name
-instance=test
-# List of Zookeepers
-zookeepers=localhost:2181
 # Port to run proxy on
 port=42424
 # Set to true if you wish to an in-memory Mock instance
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java b/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
index 898ea0b..fa4af9c 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
@@ -24,10 +24,9 @@ import java.nio.file.Files;
 import java.util.Properties;
 
 import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.impl.ClientConfConverter;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
@@ -64,8 +63,6 @@ public class Proxy implements KeywordExecutable {
   public static final String USE_MINI_ACCUMULO_DEFAULT = "false";
   public static final String USE_MOCK_INSTANCE_KEY = "useMockInstance";
   public static final String USE_MOCK_INSTANCE_DEFAULT = "false";
-  public static final String ACCUMULO_INSTANCE_NAME_KEY = "instance";
-  public static final String ZOOKEEPERS_KEY = "zookeepers";
   public static final String THRIFT_THREAD_POOL_SIZE_KEY = "numThreads";
   // Default number of threads from THsHaServer.Args
   public static final String THRIFT_THREAD_POOL_SIZE_DEFAULT = "5";
@@ -77,9 +74,6 @@ public class Proxy implements KeywordExecutable {
   public static final String THRIFT_SERVER_TYPE_DEFAULT = "";
   public static final ThriftServerType DEFAULT_SERVER_TYPE = ThriftServerType.getDefault();
 
-  public static final String KERBEROS_PRINCIPAL = "kerberosPrincipal";
-  public static final String KERBEROS_KEYTAB = "kerberosKeytab";
-
   public static final String THRIFT_SERVER_HOSTNAME = "thriftServerHostname";
   public static final String THRIFT_SERVER_HOSTNAME_DEFAULT = "0.0.0.0";
 
@@ -103,9 +97,12 @@ public class Proxy implements KeywordExecutable {
   }
 
   public static class Opts extends Help {
-    @Parameter(names = "-p", description = "properties file name",
+    @Parameter(names = "-p", description = "proxy.properties path",
+        converter = PropertiesConverter.class)
+    Properties proxyProps;
+    @Parameter(names = "-c", description = "accumulo-client.properties path",
         converter = PropertiesConverter.class)
-    Properties prop;
+    Properties clientProps;
   }
 
   @Override
@@ -128,41 +125,21 @@ public class Proxy implements KeywordExecutable {
     Opts opts = new Opts();
     opts.parseArgs(Proxy.class.getName(), args);
 
-    Properties props = new Properties();
-    if (opts.prop != null) {
-      props = opts.prop;
-    } else {
-      try (InputStream is = this.getClass().getClassLoader()
-          .getResourceAsStream("proxy.properties")) {
-        if (is != null) {
-          props.load(is);
-        } else {
-          System.err.println("proxy.properties needs to be specified as"
-              + " argument (using -p) or on the classpath (by putting the file in conf/)");
-          System.exit(-1);
-        }
-      }
-    }
+    Properties proxyProps = opts.proxyProps;
+    Properties clientProps = opts.clientProps;
 
     boolean useMini = Boolean
-        .parseBoolean(props.getProperty(USE_MINI_ACCUMULO_KEY, USE_MINI_ACCUMULO_DEFAULT));
+        .parseBoolean(proxyProps.getProperty(USE_MINI_ACCUMULO_KEY, USE_MINI_ACCUMULO_DEFAULT));
     boolean useMock = Boolean
-        .parseBoolean(props.getProperty(USE_MOCK_INSTANCE_KEY, USE_MOCK_INSTANCE_DEFAULT));
-    String instance = props.getProperty(ACCUMULO_INSTANCE_NAME_KEY);
-    String zookeepers = props.getProperty(ZOOKEEPERS_KEY);
-
-    if (!useMini && !useMock && instance == null) {
-      System.err.println("Properties file must contain one of : useMiniAccumulo=true,"
-          + " useMockInstance=true, or instance=<instance name>");
-      System.exit(1);
-    }
+        .parseBoolean(proxyProps.getProperty(USE_MOCK_INSTANCE_KEY, USE_MOCK_INSTANCE_DEFAULT));
 
-    if (instance != null && zookeepers == null) {
-      System.err.println("When instance is set in properties file, zookeepers must also be set.");
+    if (!useMini && !useMock && clientProps == null) {
+      System.err.println("The '-c' option must be set with an accumulo-client.properties file or"
+          + " proxy.properties must contain either useMiniAccumulo=true or useMockInstance=true");
       System.exit(1);
     }
 
-    if (!props.containsKey("port")) {
+    if (!proxyProps.containsKey("port")) {
       System.err.println("No port property");
       System.exit(1);
     }
@@ -172,8 +149,10 @@ public class Proxy implements KeywordExecutable {
       final File folder = Files.createTempDirectory(System.currentTimeMillis() + "").toFile();
       final MiniAccumuloCluster accumulo = new MiniAccumuloCluster(folder, "secret");
       accumulo.start();
-      props.setProperty("instance", accumulo.getConfig().getInstanceName());
-      props.setProperty("zookeepers", accumulo.getZooKeepers());
+      clientProps.setProperty(ClientProperty.INSTANCE_NAME.getKey(),
+          accumulo.getConfig().getInstanceName());
+      clientProps.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(),
+          accumulo.getZooKeepers());
       Runtime.getRuntime().addShutdownHook(new Thread() {
         @Override
         public void start() {
@@ -190,13 +169,16 @@ public class Proxy implements KeywordExecutable {
     }
 
     Class<? extends TProtocolFactory> protoFactoryClass = Class
-        .forName(props.getProperty("protocolFactory", TCompactProtocol.Factory.class.getName()))
+        .forName(
+            proxyProps.getProperty("protocolFactory", TCompactProtocol.Factory.class.getName()))
         .asSubclass(TProtocolFactory.class);
     TProtocolFactory protoFactory = protoFactoryClass.newInstance();
-    int port = Integer.parseInt(props.getProperty("port"));
-    String hostname = props.getProperty(THRIFT_SERVER_HOSTNAME, THRIFT_SERVER_HOSTNAME_DEFAULT);
+    int port = Integer.parseInt(proxyProps.getProperty("port"));
+    String hostname = proxyProps.getProperty(THRIFT_SERVER_HOSTNAME,
+        THRIFT_SERVER_HOSTNAME_DEFAULT);
     HostAndPort address = HostAndPort.fromParts(hostname, port);
-    ServerAddress server = createProxyServer(address, protoFactory, props);
+    proxyProps.putAll(clientProps);
+    ServerAddress server = createProxyServer(address, protoFactory, proxyProps);
     // Wait for the server to come up
     while (!server.server.isServing()) {
       Thread.sleep(100);
@@ -212,18 +194,12 @@ public class Proxy implements KeywordExecutable {
   }
 
   public static ServerAddress createProxyServer(HostAndPort address,
-      TProtocolFactory protocolFactory, Properties properties) throws Exception {
-    return createProxyServer(address, protocolFactory, properties,
-        ClientConfiguration.loadDefault());
-  }
-
-  public static ServerAddress createProxyServer(HostAndPort address,
-      TProtocolFactory protocolFactory, Properties properties, ClientConfiguration clientConf)
+                                                TProtocolFactory protocolFactory, Properties props)
       throws Exception {
     final int numThreads = Integer.parseInt(
-        properties.getProperty(THRIFT_THREAD_POOL_SIZE_KEY, THRIFT_THREAD_POOL_SIZE_DEFAULT));
+        props.getProperty(THRIFT_THREAD_POOL_SIZE_KEY, THRIFT_THREAD_POOL_SIZE_DEFAULT));
     final long maxFrameSize = ConfigurationTypeHelper.getFixedMemoryAsBytes(
-        properties.getProperty(THRIFT_MAX_FRAME_SIZE_KEY, THRIFT_MAX_FRAME_SIZE_DEFAULT));
+        props.getProperty(THRIFT_MAX_FRAME_SIZE_KEY, THRIFT_MAX_FRAME_SIZE_DEFAULT));
     final int simpleTimerThreadpoolSize = Integer
         .parseInt(Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE.getDefaultValue());
     // How frequently to try to resize the thread pool
@@ -235,7 +211,7 @@ public class Proxy implements KeywordExecutable {
     final String serverName = "Proxy", threadName = "Accumulo Thrift Proxy";
 
     // create the implementation of the proxy interface
-    ProxyServer impl = new ProxyServer(properties);
+    ProxyServer impl = new ProxyServer(props);
 
     // Wrap the implementation -- translate some exceptions
     AccumuloProxy.Iface wrappedImpl = RpcWrapper.service(impl);
@@ -244,7 +220,7 @@ public class Proxy implements KeywordExecutable {
     TProcessor processor = new AccumuloProxy.Processor<>(wrappedImpl);
 
     // Get the type of thrift server to instantiate
-    final String serverTypeStr = properties.getProperty(THRIFT_SERVER_TYPE,
+    final String serverTypeStr = props.getProperty(THRIFT_SERVER_TYPE,
         THRIFT_SERVER_TYPE_DEFAULT);
     ThriftServerType serverType = DEFAULT_SERVER_TYPE;
     if (!THRIFT_SERVER_TYPE_DEFAULT.equals(serverTypeStr)) {
@@ -255,30 +231,25 @@ public class Proxy implements KeywordExecutable {
     SaslServerConnectionParams saslParams = null;
     switch (serverType) {
       case SSL:
-        sslParams = SslConnectionParams.forClient(ClientConfConverter.toAccumuloConf(clientConf));
+        sslParams = SslConnectionParams.forClient(ClientConfConverter.toAccumuloConf(props));
         break;
       case SASL:
-        if (!clientConf.hasSasl()) {
-          // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j capability
-          log.error(
-              "FATAL: SASL thrift server was requested but it is disabled in client configuration");
-          throw new RuntimeException("SASL is not enabled in configuration");
+        if (!ClientProperty.SASL_ENABLED.getBoolean(props)) {
+          throw new IllegalStateException("SASL thrift server was requested but 'sasl.enabled' is"
+              + " not set to true in configuration");
         }
 
         // Kerberos needs to be enabled to use it
         if (!UserGroupInformation.isSecurityEnabled()) {
-          // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j capability
-          log.error("FATAL: Hadoop security is not enabled");
-          throw new RuntimeException();
+          throw new IllegalStateException("Hadoop security is not enabled");
         }
 
         // Login via principal and keytab
-        final String kerberosPrincipal = properties.getProperty(KERBEROS_PRINCIPAL, ""),
-            kerberosKeytab = properties.getProperty(KERBEROS_KEYTAB, "");
+        final String kerberosPrincipal = ClientProperty.AUTH_USERNAME.getValue(props);
+        final String kerberosKeytab = ClientProperty.AUTH_KERBEROS_KEYTAB_PATH.getValue(props);
         if (StringUtils.isBlank(kerberosPrincipal) || StringUtils.isBlank(kerberosKeytab)) {
-          // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j capability
-          log.error("FATAL: Kerberos principal and keytab must be provided");
-          throw new RuntimeException();
+          throw new IllegalStateException(String.format("Kerberos principal '%s' and keytab '%s'"
+              + " must be provided", kerberosPrincipal, kerberosKeytab));
         }
         UserGroupInformation.loginUserFromKeytab(kerberosPrincipal, kerberosKeytab);
         UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
@@ -288,12 +259,10 @@ public class Proxy implements KeywordExecutable {
         // as.
         final String shortName = ugi.getShortUserName();
         log.info("Setting server primary to {}", shortName);
-        clientConf.setProperty(ClientProperty.KERBEROS_SERVER_PRIMARY, shortName);
+        props.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(), shortName);
 
         KerberosToken token = new KerberosToken();
-        saslParams = new SaslServerConnectionParams(ClientConfConverter.toProperties(clientConf),
-            token, null);
-
+        saslParams = new SaslServerConnectionParams(props, token, null);
         processor = new UGIAssumingProcessor(processor);
 
         break;
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 0e5fc76..61f10b9 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.proxy;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -43,7 +42,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriter.Result;
 import org.apache.accumulo.core.client.ConditionalWriterConfig;
@@ -64,6 +62,7 @@ import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.ClientConfConverter;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Namespace;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
@@ -195,15 +194,7 @@ public class ProxyServer implements AccumuloProxy.Iface {
     if (useMock != null && Boolean.parseBoolean(useMock))
       instance = DeprecationUtil.makeMockInstance(this.getClass().getName());
     else {
-      ClientConfiguration clientConf;
-      if (props.containsKey("clientConfigurationFile")) {
-        String clientConfFile = props.getProperty("clientConfigurationFile");
-        clientConf = ClientConfiguration.fromFile(new File(clientConfFile));
-      } else {
-        clientConf = ClientConfiguration.loadDefault();
-      }
-      instance = new ZooKeeperInstance(clientConf.withInstance(props.getProperty("instance"))
-          .withZkHosts(props.getProperty("zookeepers")));
+      instance = new ZooKeeperInstance(ClientConfConverter.toClientConf(props));
     }
 
     try {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
index d5053ef..767f53f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosProxyIT.java
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.UGIAssumingTransport;
 import org.apache.accumulo.core.security.Authorizations;
@@ -231,7 +232,9 @@ public class KerberosProxyIT extends AccumuloITBase {
    */
   private Process startProxy(MiniAccumuloConfigImpl cfg) throws IOException {
     File proxyPropertiesFile = generateNewProxyConfiguration(cfg);
-    return mac.exec(Proxy.class, "-p", proxyPropertiesFile.getCanonicalPath());
+    File clientPropsFile = generateNewAccumuloClientConfiguration(cfg);
+    return mac.exec(Proxy.class, "-p", proxyPropertiesFile.getCanonicalPath(), "-c",
+        clientPropsFile.getCanonicalPath());
   }
 
   /**
@@ -258,11 +261,7 @@ public class KerberosProxyIT extends AccumuloITBase {
     proxyProperties.setProperty("tokenClass", KerberosToken.class.getName());
     proxyProperties.setProperty("port", Integer.toString(proxyPort));
     proxyProperties.setProperty("maxFrameSize", "16M");
-    proxyProperties.setProperty("instance", mac.getInstanceName());
-    proxyProperties.setProperty("zookeepers", mac.getZooKeepers());
     proxyProperties.setProperty("thriftServerType", "sasl");
-    proxyProperties.setProperty("kerberosPrincipal", proxyPrincipal);
-    proxyProperties.setProperty("kerberosKeytab", proxyKeytab.getCanonicalPath());
 
     // Write out the proxy.properties file
     FileWriter writer = new FileWriter(proxyPropertiesFile);
@@ -274,6 +273,31 @@ public class KerberosProxyIT extends AccumuloITBase {
     return proxyPropertiesFile;
   }
 
+  private File generateNewAccumuloClientConfiguration(MiniAccumuloConfigImpl cfg) throws IOException {
+    // Proxy configuration
+    File propsFile = new File(cfg.getConfDir(), "accumulo-client-proxy.properties");
+    if (propsFile.exists()) {
+      assertTrue("Failed to delete proxy.properties file", propsFile.delete());
+    }
+    Properties clientProps = new Properties();
+    clientProps.setProperty(ClientProperty.INSTANCE_NAME.getKey(), cfg.getInstanceName());
+    clientProps.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), cfg.getZooKeepers());
+    clientProps.setProperty(ClientProperty.AUTH_METHOD.getKey(), "kerberos");
+    clientProps.setProperty(ClientProperty.AUTH_USERNAME.getKey(), proxyPrincipal);
+    clientProps.setProperty(ClientProperty.AUTH_KERBEROS_KEYTAB_PATH.getKey(),
+        proxyKeytab.getCanonicalPath());
+    clientProps.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
+
+    // Write out the proxy.properties file
+    FileWriter writer = new FileWriter(propsFile);
+    clientProps.store(writer, "Configuration for Accumulo proxy");
+    writer.close();
+
+    log.info("Created Accumulo client configuration for proxy listening on {}", proxyPort);
+
+    return propsFile;
+  }
+
   /**
    * Restarts the thrift proxy if the previous instance is no longer running. If the proxy is still
    * running, this method does nothing.
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
index a3db52f..d9fad78 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
@@ -80,20 +80,18 @@ public class ProxyDurabilityIT extends ConfigurableMacBase {
   @Test
   public void testDurability() throws Exception {
     Connector c = getConnector();
-    Properties props = new Properties();
+    Properties proxyProps = new Properties();
     // Avoid issues with locally installed client configuration files with custom properties
     File emptyFile = Files.createTempFile(null, null).toFile();
     emptyFile.deleteOnExit();
-    props.put("instance", c.getInstance().getInstanceName());
-    props.put("zookeepers", c.getInstance().getZooKeepers());
-    props.put("tokenClass", PasswordToken.class.getName());
-    props.put("clientConfigurationFile", emptyFile.toString());
+    proxyProps.put("tokenClass", PasswordToken.class.getName());
+    proxyProps.putAll(getConnectionInfo().getProperties());
 
     TJSONProtocol.Factory protocol = new TJSONProtocol.Factory();
 
     int proxyPort = PortUtils.getRandomFreePort();
-    final TServer proxyServer = Proxy
-        .createProxyServer(HostAndPort.fromParts("localhost", proxyPort), protocol, props).server;
+    final TServer proxyServer = Proxy.createProxyServer(
+        HostAndPort.fromParts("localhost", proxyPort), protocol, proxyProps).server;
     while (!proxyServer.isServing())
       sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
     Client client = new TestProxyClient("localhost", proxyPort, protocol).proxy();
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
index be96a84..dbcbbf8 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/SimpleProxyBase.java
@@ -45,7 +45,6 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterUser;
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Namespace;
@@ -227,16 +226,10 @@ public abstract class SimpleProxyBase extends SharedMiniClusterBase {
     }
 
     props.put("tokenClass", tokenClass);
-
-    ClientConfiguration clientConfig = SharedMiniClusterBase.getCluster().getClientConfig();
-    String clientConfPath = new File(SharedMiniClusterBase.getCluster().getConfig().getConfDir(),
-        "client.conf").getAbsolutePath();
-    props.put("clientConfigurationFile", clientConfPath);
-    properties.put("clientConfigurationFile", clientConfPath);
-
+    props.putAll(SharedMiniClusterBase.getCluster().getConnectionInfo().getProperties());
     proxyPort = PortUtils.getRandomFreePort();
     proxyServer = Proxy.createProxyServer(HostAndPort.fromParts(hostname, proxyPort), factory,
-        props, clientConfig).server;
+        props).server;
     while (!proxyServer.isServing())
       sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyInstanceOperations.java b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyInstanceOperations.java
index 787c980..2b8c189 100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyInstanceOperations.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/TestProxyInstanceOperations.java
@@ -46,12 +46,12 @@ public class TestProxyInstanceOperations {
 
   @BeforeClass
   public static void setup() throws Exception {
-    Properties prop = new Properties();
-    prop.setProperty("useMockInstance", "true");
-    prop.put("tokenClass", PasswordToken.class.getName());
+    Properties proxyProps = new Properties();
+    proxyProps.setProperty("useMockInstance", "true");
+    proxyProps.put("tokenClass", PasswordToken.class.getName());
 
     proxy = Proxy.createProxyServer(HostAndPort.fromParts("localhost", port),
-        new TCompactProtocol.Factory(), prop).server;
+        new TCompactProtocol.Factory(), proxyProps).server;
     log.info("Waiting for proxy to start");
     while (!proxy.isServing()) {
       Thread.sleep(500);

-- 
To stop receiving notification emails like this one, please contact
mwalch@apache.org.