You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2014/01/13 19:50:21 UTC

[2/3] ACCUMULO-2151 separated MAC API from implementation

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
new file mode 100644
index 0000000..2931aca
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -0,0 +1,443 @@
+/*
+ * 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.minicluster.impl;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.StringUtil;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.server.util.PortUtils;
+
+/**
+ * Holds configuration for {@link MiniAccumuloClusterImpl}. Required configurations must be passed to constructor(s) and all other configurations are optional.
+ * 
+ * @since 1.5.0
+ */
+public class MiniAccumuloConfigImpl {
+
+  private static final String DEFAULT_INSTANCE_SECRET = "DONTTELL";
+
+  private File dir = null;
+  private String rootPassword = null;
+  private Map<String,String> siteConfig = new HashMap<String,String>();
+  private int numTservers = 2;
+  private Map<ServerType,Long> memoryConfig = new HashMap<ServerType,Long>();
+  private boolean jdwpEnabled = false;
+  private Map<String,String> systemProperties = new HashMap<String,String>();
+
+  private String instanceName = "miniInstance";
+
+  private File libDir;
+  private File confDir;
+  private File zooKeeperDir;
+  private File accumuloDir;
+  private File logDir;
+  private File walogDir;
+
+  private Integer zooKeeperPort;
+  private long defaultMemorySize = 128 * 1024 * 1024;
+
+  private boolean initialized = false;
+
+  private boolean useMiniDFS = false;
+
+  private String[] classpathItems = null;
+
+  private String[] nativePathItems = null;
+
+  /**
+   * @param dir
+   *          An empty or nonexistant directory that Accumulo and Zookeeper can store data in. Creating the directory is left to the user. Java 7, Guava, and
+   *          Junit provide methods for creating temporary directories.
+   * @param rootPassword
+   *          The initial password for the Accumulo root user
+   */
+  public MiniAccumuloConfigImpl(File dir, String rootPassword) {
+    this.dir = dir;
+    this.rootPassword = rootPassword;
+  }
+
+  /**
+   * Set directories and fully populate site config
+   */
+  MiniAccumuloConfigImpl initialize() {
+
+    // Sanity checks
+    if (this.getDir().exists() && !this.getDir().isDirectory())
+      throw new IllegalArgumentException("Must pass in directory, " + this.getDir() + " is a file");
+
+    if (this.getDir().exists() && this.getDir().list().length != 0)
+      throw new IllegalArgumentException("Directory " + this.getDir() + " is not empty");
+
+    if (!initialized) {
+      libDir = new File(dir, "lib");
+      confDir = new File(dir, "conf");
+      accumuloDir = new File(dir, "accumulo");
+      zooKeeperDir = new File(dir, "zookeeper");
+      logDir = new File(dir, "logs");
+      walogDir = new File(dir, "walogs");
+
+      String[] paths = {"$ACCUMULO_HOME/lib/.*.jar", "$ZOOKEEPER_HOME/zookeeper[^.].*.jar", "$HADOOP_PREFIX/[^.].*.jar", "$HADOOP_PREFIX/lib/[^.].*.jar",
+          "$HADOOP_PREFIX/share/hadoop/common/.*.jar", "$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar", "$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar",
+          "$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar"};
+
+      String classpath = StringUtil.join(Arrays.asList(paths), ",");
+
+      mergeProp(Property.INSTANCE_DFS_URI.getKey(), "file:///");
+      mergeProp(Property.INSTANCE_DFS_DIR.getKey(), accumuloDir.getAbsolutePath());
+      mergeProp(Property.INSTANCE_SECRET.getKey(), DEFAULT_INSTANCE_SECRET);
+      mergeProp(Property.TSERV_PORTSEARCH.getKey(), "true");
+      mergeProp(Property.LOGGER_DIR.getKey(), walogDir.getAbsolutePath());
+      mergeProp(Property.TSERV_DATACACHE_SIZE.getKey(), "10M");
+      mergeProp(Property.TSERV_INDEXCACHE_SIZE.getKey(), "10M");
+      mergeProp(Property.TSERV_MAXMEM.getKey(), "50M");
+      mergeProp(Property.TSERV_WALOG_MAX_SIZE.getKey(), "100M");
+      mergeProp(Property.TSERV_NATIVEMAP_ENABLED.getKey(), "false");
+      mergeProp(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey() + ".password", getRootPassword());
+      // since there is a small amount of memory, check more frequently for majc... setting may not be needed in 1.5
+      mergeProp(Property.TSERV_MAJC_DELAY.getKey(), "3");
+      mergeProp(Property.GENERAL_CLASSPATHS.getKey(), classpath);
+      mergeProp(Property.GENERAL_DYNAMIC_CLASSPATHS.getKey(), libDir.getAbsolutePath() + "/[^.].*[.]jar");
+      mergeProp(Property.GC_CYCLE_DELAY.getKey(), "4s");
+      mergeProp(Property.GC_CYCLE_START.getKey(), "0s");
+      mergePropWithRandomPort(Property.MASTER_CLIENTPORT.getKey());
+      mergePropWithRandomPort(Property.TRACE_PORT.getKey());
+      mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
+      mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
+      mergePropWithRandomPort(Property.GC_PORT.getKey());
+
+      // zookeeper port should be set explicitly in this class, not just on the site config
+      if (zooKeeperPort == null)
+        zooKeeperPort = PortUtils.getRandomFreePort();
+      siteConfig.put(Property.INSTANCE_ZK_HOST.getKey(), "localhost:" + zooKeeperPort);
+      initialized = true;
+    }
+    return this;
+  }
+
+  /**
+   * Set a given key/value on the site config if it doesn't already exist
+   */
+  private void mergeProp(String key, String value) {
+    if (!siteConfig.containsKey(key)) {
+      siteConfig.put(key, value);
+    }
+  }
+
+  /**
+   * Sets a given key with a random port for the value on the site config if it doesn't already exist.
+   */
+  private void mergePropWithRandomPort(String key) {
+    if (!siteConfig.containsKey(key)) {
+      siteConfig.put(key, "0");
+    }
+  }
+
+  /**
+   * Calling this method is optional. If not set, it defaults to two.
+   * 
+   * @param numTservers
+   *          the number of tablet servers that mini accumulo cluster should start
+   */
+  public MiniAccumuloConfigImpl setNumTservers(int numTservers) {
+    if (numTservers < 1)
+      throw new IllegalArgumentException("Must have at least one tablet server");
+    this.numTservers = numTservers;
+    return this;
+  }
+
+  /**
+   * Calling this method is optional. If not set, defaults to 'miniInstance'
+   * 
+   * @since 1.6.0
+   */
+  public MiniAccumuloConfigImpl setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+    return this;
+  }
+
+  /**
+   * Calling this method is optional. If not set, it defaults to an empty map.
+   * 
+   * @param siteConfig
+   *          key/values that you normally put in accumulo-site.xml can be put here.
+   */
+  public MiniAccumuloConfigImpl setSiteConfig(Map<String,String> siteConfig) {
+    this.siteConfig = new HashMap<String,String>(siteConfig);
+    return this;
+  }
+
+  /**
+   * Calling this method is optional. A random port is generated by default
+   * 
+   * @param zooKeeperPort
+   *          A valid (and unused) port to use for the zookeeper
+   * 
+   * @since 1.6.0
+   */
+  public MiniAccumuloConfigImpl setZooKeeperPort(int zooKeeperPort) {
+    this.zooKeeperPort = zooKeeperPort;
+    return this;
+  }
+
+  /**
+   * Sets the amount of memory to use in the master process. Calling this method is optional. Default memory is 128M
+   * 
+   * @param serverType
+   *          the type of server to apply the memory settings
+   * @param memory
+   *          amount of memory to set
+   * 
+   * @param memoryUnit
+   *          the units for which to apply with the memory size
+   * 
+   * @since 1.6.0
+   */
+  public MiniAccumuloConfigImpl setMemory(ServerType serverType, long memory, MemoryUnit memoryUnit) {
+    this.memoryConfig.put(serverType, memoryUnit.toBytes(memory));
+    return this;
+  }
+
+  /**
+   * Sets the default memory size to use. This value is also used when a ServerType has not been configured explicitly. Calling this method is optional. Default
+   * memory is 128M
+   * 
+   * @param memory
+   *          amount of memory to set
+   * 
+   * @param memoryUnit
+   *          the units for which to apply with the memory size
+   * 
+   * @since 1.6.0
+   */
+  public MiniAccumuloConfigImpl setDefaultMemory(long memory, MemoryUnit memoryUnit) {
+    this.defaultMemorySize = memoryUnit.toBytes(memory);
+    return this;
+  }
+
+  /**
+   * @return a copy of the site config
+   */
+  public Map<String,String> getSiteConfig() {
+    return new HashMap<String,String>(siteConfig);
+  }
+
+  /**
+   * @return name of configured instance
+   * 
+   * @since 1.6.0
+   */
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  /**
+   * @return The configured zookeeper port
+   * 
+   * @since 1.6.0
+   */
+  public int getZooKeeperPort() {
+    return zooKeeperPort;
+  }
+
+  File getLibDir() {
+    return libDir;
+  }
+
+  File getConfDir() {
+    return confDir;
+  }
+
+  File getZooKeeperDir() {
+    return zooKeeperDir;
+  }
+
+  public File getAccumuloDir() {
+    return accumuloDir;
+  }
+
+  public File getLogDir() {
+    return logDir;
+  }
+
+  File getWalogDir() {
+    return walogDir;
+  }
+
+  /**
+   * @param serverType
+   *          get configuration for this server type
+   * 
+   * @return memory configured in bytes, returns default if this server type is not configured
+   * 
+   * @since 1.6.0
+   */
+  public long getMemory(ServerType serverType) {
+    return memoryConfig.containsKey(serverType) ? memoryConfig.get(serverType) : defaultMemorySize;
+  }
+
+  /**
+   * @return memory configured in bytes
+   * 
+   * @since 1.6.0
+   */
+  public long getDefaultMemory() {
+    return defaultMemorySize;
+  }
+
+  /**
+   * @return zookeeper connection string
+   * 
+   * @since 1.6.0
+   */
+  public String getZooKeepers() {
+    return siteConfig.get(Property.INSTANCE_ZK_HOST.getKey());
+  }
+
+  /**
+   * @return the base directory of the cluster configuration
+   */
+  public File getDir() {
+    return dir;
+  }
+
+  /**
+   * @return the root password of this cluster configuration
+   */
+  public String getRootPassword() {
+    return rootPassword;
+  }
+
+  /**
+   * @return the number of tservers configured for this cluster
+   */
+  public int getNumTservers() {
+    return numTservers;
+  }
+
+  /**
+   * @return is the current configuration in jdwpEnabled mode?
+   * 
+   * @since 1.6.0
+   */
+  public boolean isJDWPEnabled() {
+    return jdwpEnabled;
+  }
+
+  /**
+   * @param jdwpEnabled
+   *          should the processes run remote jdwpEnabled servers?
+   * @return the current instance
+   * 
+   * @since 1.6.0
+   */
+  public MiniAccumuloConfigImpl setJDWPEnabled(boolean jdwpEnabled) {
+    this.jdwpEnabled = jdwpEnabled;
+    return this;
+  }
+
+  public boolean useMiniDFS() {
+    return useMiniDFS;
+  }
+
+  public void useMiniDFS(boolean useMiniDFS) {
+    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");
+  }
+  
+  /**
+   * sets system properties set for service processes
+   * 
+   * @since 1.6.0
+   */
+  public void setSystemProperties(Map<String,String> systemProperties) {
+    this.systemProperties = new HashMap<String,String>(systemProperties);
+  }
+
+  /**
+   * @return a copy of the system properties for service processes
+   * 
+   * @since 1.6.0
+   */
+  public Map<String,String> getSystemProperties() {
+    return new HashMap<String,String>(systemProperties);
+  }
+
+  /**
+   * Gets the classpath elements to use when spawning processes.
+   * 
+   * @return the classpathItems, if set
+   * 
+   * @since 1.6.0
+   */
+  public String[] getClasspathItems() {
+    return classpathItems;
+  }
+
+  /**
+   * Sets the classpath elements to use when spawning processes.
+   * 
+   * @param classpathItems
+   *          the classpathItems to set
+   * @since 1.6.0
+   */
+  public void setClasspathItems(String... classpathItems) {
+    this.classpathItems = classpathItems;
+  }
+
+  /**
+   * @return the paths to use for loading native libraries
+   * 
+   * @since 1.6.0
+   */
+  public String[] getNativeLibPaths() {
+    return this.nativePathItems == null ? new String[0] : this.nativePathItems;
+  }
+
+  /**
+   * Sets the path for processes to use for loading native libraries
+   * 
+   * @param nativePathItems
+   *          the nativePathItems to set
+   * @since 1.6.0
+   */
+  public void setNativeLibPaths(String... nativePathItems) {
+    this.nativePathItems = nativePathItems;
+  }
+
+  /**
+   * Sets arbitrary configuration properties.
+   * 
+   * @since 1.6.0
+   */
+  public void setProperty(Property p, String value) {
+    this.siteConfig.put(p.getKey(), value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessNotFoundException.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessNotFoundException.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessNotFoundException.java
new file mode 100644
index 0000000..522f511
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessNotFoundException.java
@@ -0,0 +1,23 @@
+/*
+ * 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.minicluster.impl;
+
+public class ProcessNotFoundException extends Exception {
+
+  private static final long serialVersionUID = 1L;
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessReference.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessReference.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessReference.java
new file mode 100644
index 0000000..9aa2449
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/ProcessReference.java
@@ -0,0 +1,47 @@
+/*
+ * 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.minicluster.impl;
+
+/**
+ * Opaque handle to a process.
+ */
+public class ProcessReference {
+  private Process process;
+
+  ProcessReference(Process process) {
+    this.process = process;
+  }
+
+  @Override
+  public String toString() {
+    return process.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    return process.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof Process) {
+      return process == obj;
+    }
+    return this == obj;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterGCTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterGCTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterGCTest.java
deleted file mode 100644
index 8a01f99..0000000
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterGCTest.java
+++ /dev/null
@@ -1,153 +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
- *
- *     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.minicluster;
-
-import java.io.File;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.filefilter.SuffixFileFilter;
-import org.apache.commons.io.filefilter.TrueFileFilter;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableMap;
-
-/**
- * 
- */
-public class MiniAccumuloClusterGCTest {
-  private static final Logger log = Logger.getLogger(MiniAccumuloClusterGCTest.class);
-  
-  private static File testDir = new File(System.getProperty("user.dir") + "/target/" + MiniAccumuloClusterGCTest.class.getName());
-  private static MiniAccumuloConfig macConfig;
-  private static MiniAccumuloCluster accumulo;
-  private static final String passwd = "password";
-  
-  @BeforeClass
-  public static void setupMiniCluster() throws Exception {
-    FileUtils.deleteQuietly(testDir);
-    testDir.mkdir();
-    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
-
-    macConfig = new MiniAccumuloConfig(testDir, passwd);
-    macConfig.setNumTservers(1);
-
-    // And tweak the settings to make it run often
-    Map<String,String> config = ImmutableMap.of(Property.GC_CYCLE_DELAY.getKey(), "1s", Property.GC_CYCLE_START.getKey(), "0s");
-    macConfig.setSiteConfig(config);
-
-    accumulo = new MiniAccumuloCluster(macConfig);
-    accumulo.start();
-  }
-  
-  @AfterClass
-  public static void tearDownMiniCluster() throws Exception {
-    accumulo.stop();
-  }
-  
-  // This test seems to be a little too unstable for a unit test
-  @Ignore
-  public void test() throws Exception {
-    ZooKeeperInstance inst = new ZooKeeperInstance(accumulo.getClientConfig());
-    Connector c = inst.getConnector("root", new PasswordToken(passwd));
-
-    final String table = "foobar";
-    c.tableOperations().create(table);
-    
-    final String tableId = c.tableOperations().tableIdMap().get(table);
-
-    BatchWriter bw = null;
-
-    // Add some data
-    try {
-      bw = c.createBatchWriter(table, new BatchWriterConfig().setMaxMemory(100000l).setMaxLatency(100, TimeUnit.MILLISECONDS).setMaxWriteThreads(1));
-      Mutation m = new Mutation("a");
-      for (int i = 0; i < 500; i++) {
-        m.put("colf", Integer.toString(i), "");
-      }
-
-      bw.addMutation(m);
-    } finally {
-      if (null != bw) {
-        bw.close();
-      }
-    }
-
-    File accumuloDir = new File(testDir, "accumulo");
-    File tables = new File(accumuloDir.getAbsolutePath(), "tables");
-    File myTable = new File(tables, tableId);
-    
-    log.trace("Files before compaction: " + FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE));
-
-    final boolean flush = true, wait = true;
-
-    // Compact the tables to get some rfiles which we can gc
-    c.tableOperations().compact(table, null, null, flush, wait);
-
-    Collection<File> filesAfterCompaction = FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE);
-    int fileCountAfterCompaction = filesAfterCompaction.size();
-    
-    log.trace("Files after compaction: " + filesAfterCompaction);
-
-    // Sleep for 10s to let the GC do its thing
-    for (int i = 1; i < 10; i++) {
-      Thread.sleep(1000);
-      filesAfterCompaction = FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE);
-      
-      log.trace("Files in loop: " + filesAfterCompaction);
-      
-      int fileCountAfterGCWait = filesAfterCompaction.size();
-
-      if (fileCountAfterGCWait < fileCountAfterCompaction) {
-        return;
-      }
-    }
-
-    Assert.fail("Expected to find less files after compaction and pause for GC");
-  }
-
-  @Test(timeout = 10000)
-  public void testAccurateProcessListReturned() throws Exception {
-    Map<ServerType,Collection<ProcessReference>> procs = accumulo.getProcesses();
-
-    for (ServerType t : new ServerType[] {ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.ZOOKEEPER, ServerType.GARBAGE_COLLECTOR}) {
-      Assert.assertTrue(procs.containsKey(t));
-      Collection<ProcessReference> procRefs = procs.get(t);
-      Assert.assertTrue(1 <= procRefs.size());
-
-      for (ProcessReference procRef : procRefs) {
-        Assert.assertNotNull(procRef);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
index fe6f5ed..4d774e4 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -19,9 +19,7 @@ package org.apache.accumulo.minicluster;
 import static org.junit.Assert.assertEquals;
 
 import java.io.File;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.UUID;
@@ -210,23 +208,6 @@ public class MiniAccumuloClusterTest {
     }
   }
 
-  @Test(timeout = 10000)
-  public void testAccurateProcessListReturned() throws Exception {
-    Map<ServerType,Collection<ProcessReference>> procs = accumulo.getProcesses();
-
-    Assert.assertTrue(procs.containsKey(ServerType.GARBAGE_COLLECTOR));
-
-    for (ServerType t : new ServerType[] {ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.ZOOKEEPER}) {
-      Assert.assertTrue(procs.containsKey(t));
-      Collection<ProcessReference> procRefs = procs.get(t);
-      Assert.assertTrue(1 <= procRefs.size());
-
-      for (ProcessReference procRef : procRefs) {
-        Assert.assertNotNull(procRef);
-      }
-    }
-  }
-
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloConfigTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloConfigTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloConfigTest.java
deleted file mode 100644
index 8a764d2..0000000
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloConfigTest.java
+++ /dev/null
@@ -1,82 +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
- *
- *     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.minicluster;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.Property;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-public class MiniAccumuloConfigTest {
-
-  static TemporaryFolder tempFolder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  @BeforeClass
-  public static void setUp() throws IOException {
-    tempFolder.create();
-  }
-
-  @Test
-  public void testZookeeperPort() {
-
-    // set specific zookeeper port
-    MiniAccumuloConfig config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").setZooKeeperPort(5000).initialize();
-    assertEquals(5000, config.getZooKeeperPort());
-
-    // generate zookeeper port
-    config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").initialize();
-    assertTrue(config.getZooKeeperPort() > 0);
-  }
-
-  @Test
-  public void testSiteConfig() {
-
-    // constructor site config overrides default props
-    Map<String,String> siteConfig = new HashMap<String,String>();
-    siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), "hdfs://");
-    MiniAccumuloConfig config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").setSiteConfig(siteConfig).initialize();
-    assertEquals("hdfs://", config.getSiteConfig().get(Property.INSTANCE_DFS_URI.getKey()));
-  }
-
-  @Test
-  public void testMemoryConfig() {
-
-    MiniAccumuloConfig config = new MiniAccumuloConfig(tempFolder.getRoot(), "password").initialize();
-    config.setDefaultMemory(96, MemoryUnit.MEGABYTE);
-    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.MASTER));
-    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.TABLET_SERVER));
-    assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
-    config.setMemory(ServerType.MASTER, 256, MemoryUnit.MEGABYTE);
-    assertEquals(256 * 1024 * 1024l, config.getMemory(ServerType.MASTER));
-    assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
-    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.TABLET_SERVER));
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    tempFolder.delete();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterGCTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterGCTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterGCTest.java
new file mode 100644
index 0000000..4793014
--- /dev/null
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterGCTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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.minicluster.impl;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.SuffixFileFilter;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * 
+ */
+public class MiniAccumuloClusterGCTest {
+  private static final Logger log = Logger.getLogger(MiniAccumuloClusterGCTest.class);
+  private static File testDir = new File(System.getProperty("user.dir") + "/target/" + MiniAccumuloClusterGCTest.class.getName());
+  private static MiniAccumuloConfigImpl macConfig;
+  private static MiniAccumuloClusterImpl accumulo;
+  private static final String passwd = "password";
+  
+  @BeforeClass
+  public static void setupMiniCluster() throws Exception {
+    FileUtils.deleteQuietly(testDir);
+    testDir.mkdir();
+    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
+
+    macConfig = new MiniAccumuloConfigImpl(testDir, passwd);
+    macConfig.setNumTservers(1);
+
+    // And tweak the settings to make it run often
+    Map<String,String> config = ImmutableMap.of(Property.GC_CYCLE_DELAY.getKey(), "1s", Property.GC_CYCLE_START.getKey(), "0s");
+    macConfig.setSiteConfig(config);
+
+    accumulo = new MiniAccumuloClusterImpl(macConfig);
+    accumulo.start();
+  }
+  
+  @AfterClass
+  public static void tearDownMiniCluster() throws Exception {
+    accumulo.stop();
+  }
+  
+  // This test seems to be a little too unstable for a unit test
+  @Ignore
+  public void test() throws Exception {
+    ZooKeeperInstance inst = new ZooKeeperInstance(accumulo.getClientConfig());
+    Connector c = inst.getConnector("root", new PasswordToken(passwd));
+
+    final String table = "foobar";
+    c.tableOperations().create(table);
+    
+    final String tableId = c.tableOperations().tableIdMap().get(table);
+
+    BatchWriter bw = null;
+
+    // Add some data
+    try {
+      bw = c.createBatchWriter(table, new BatchWriterConfig().setMaxMemory(100000l).setMaxLatency(100, TimeUnit.MILLISECONDS).setMaxWriteThreads(1));
+      Mutation m = new Mutation("a");
+      for (int i = 0; i < 500; i++) {
+        m.put("colf", Integer.toString(i), "");
+      }
+
+      bw.addMutation(m);
+    } finally {
+      if (null != bw) {
+        bw.close();
+      }
+    }
+
+    File accumuloDir = new File(testDir, "accumulo");
+    File tables = new File(accumuloDir.getAbsolutePath(), "tables");
+    File myTable = new File(tables, tableId);
+    
+    log.trace("Files before compaction: " + FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE));
+
+    final boolean flush = true, wait = true;
+
+    // Compact the tables to get some rfiles which we can gc
+    c.tableOperations().compact(table, null, null, flush, wait);
+
+    Collection<File> filesAfterCompaction = FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE);
+    int fileCountAfterCompaction = filesAfterCompaction.size();
+    
+    log.trace("Files after compaction: " + filesAfterCompaction);
+
+    // Sleep for 10s to let the GC do its thing
+    for (int i = 1; i < 10; i++) {
+      Thread.sleep(1000);
+      filesAfterCompaction = FileUtils.listFiles(myTable, new SuffixFileFilter(".rf"), TrueFileFilter.TRUE);
+      
+      log.trace("Files in loop: " + filesAfterCompaction);
+      
+      int fileCountAfterGCWait = filesAfterCompaction.size();
+
+      if (fileCountAfterGCWait < fileCountAfterCompaction) {
+        return;
+      }
+    }
+
+    Assert.fail("Expected to find less files after compaction and pause for GC");
+  }
+
+  @Test(timeout = 10000)
+  public void testAccurateProcessListReturned() throws Exception {
+    Map<ServerType,Collection<ProcessReference>> procs = accumulo.getProcesses();
+
+    for (ServerType t : new ServerType[] {ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.ZOOKEEPER, ServerType.GARBAGE_COLLECTOR}) {
+      Assert.assertTrue(procs.containsKey(t));
+      Collection<ProcessReference> procRefs = procs.get(t);
+      Assert.assertTrue(1 <= procRefs.size());
+
+      for (ProcessReference procRef : procRefs) {
+        Assert.assertNotNull(procRef);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
new file mode 100644
index 0000000..99f8d7d
--- /dev/null
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.minicluster.impl;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class MiniAccumuloClusterImplTest {
+
+  public static File testDir;
+
+  private static MiniAccumuloClusterImpl accumulo;
+
+  @BeforeClass
+  public static void setupMiniCluster() throws Exception {
+    Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
+
+    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
+    baseDir.mkdirs();
+    testDir = new File(baseDir, MiniAccumuloClusterImplTest.class.getName());
+    FileUtils.deleteQuietly(testDir);
+    testDir.mkdir();
+
+    MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(testDir, "superSecret").setJDWPEnabled(true);
+    accumulo = new MiniAccumuloClusterImpl(config);
+    accumulo.start();
+  }
+
+  @Test(timeout = 10000)
+  public void testAccurateProcessListReturned() throws Exception {
+    Map<ServerType,Collection<ProcessReference>> procs = accumulo.getProcesses();
+
+    Assert.assertTrue(procs.containsKey(ServerType.GARBAGE_COLLECTOR));
+
+    for (ServerType t : new ServerType[] {ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.ZOOKEEPER}) {
+      Assert.assertTrue(procs.containsKey(t));
+      Collection<ProcessReference> procRefs = procs.get(t);
+      Assert.assertTrue(1 <= procRefs.size());
+
+      for (ProcessReference procRef : procRefs) {
+        Assert.assertNotNull(procRef);
+      }
+    }
+  }
+
+  @AfterClass
+  public static void tearDownMiniCluster() throws Exception {
+    accumulo.stop();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java
new file mode 100644
index 0000000..c0f7053
--- /dev/null
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.minicluster.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class MiniAccumuloConfigImplTest {
+
+  static TemporaryFolder tempFolder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    tempFolder.create();
+  }
+
+  @Test
+  public void testZookeeperPort() {
+
+    // set specific zookeeper port
+    MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(tempFolder.getRoot(), "password").setZooKeeperPort(5000).initialize();
+    assertEquals(5000, config.getZooKeeperPort());
+
+    // generate zookeeper port
+    config = new MiniAccumuloConfigImpl(tempFolder.getRoot(), "password").initialize();
+    assertTrue(config.getZooKeeperPort() > 0);
+  }
+
+  @Test
+  public void testSiteConfig() {
+
+    // constructor site config overrides default props
+    Map<String,String> siteConfig = new HashMap<String,String>();
+    siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), "hdfs://");
+    MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(tempFolder.getRoot(), "password").setSiteConfig(siteConfig).initialize();
+    assertEquals("hdfs://", config.getSiteConfig().get(Property.INSTANCE_DFS_URI.getKey()));
+  }
+
+  @Test
+  public void testMemoryConfig() {
+
+    MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(tempFolder.getRoot(), "password").initialize();
+    config.setDefaultMemory(96, MemoryUnit.MEGABYTE);
+    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.MASTER));
+    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.TABLET_SERVER));
+    assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
+    config.setMemory(ServerType.MASTER, 256, MemoryUnit.MEGABYTE);
+    assertEquals(256 * 1024 * 1024l, config.getMemory(ServerType.MASTER));
+    assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
+    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.TABLET_SERVER));
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    tempFolder.delete();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
----------------------------------------------------------------------
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 7597fc7..3e404b1 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
@@ -98,7 +98,7 @@ public class Proxy {
       final MiniAccumuloCluster accumulo = new MiniAccumuloCluster(folder, "secret");
       accumulo.start();
       opts.prop.setProperty("instance", accumulo.getConfig().getInstanceName());
-      opts.prop.setProperty("zookeepers", accumulo.getConfig().getZooKeepers());
+      opts.prop.setProperty("zookeepers", accumulo.getZooKeepers());
       Runtime.getRuntime().addShutdownHook(new Thread() {
         public void start() {
           try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java b/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
index 91f04b2..91ae830 100644
--- a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
+++ b/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
@@ -168,7 +168,7 @@ public class SimpleProxyIT {
 
     Properties props = new Properties();
     props.put("instance", accumulo.getConfig().getInstanceName());
-    props.put("zookeepers", accumulo.getConfig().getZooKeepers());
+    props.put("zookeepers", accumulo.getZooKeepers());
     props.put("tokenClass", PasswordToken.class.getName());
 
     protocolClass = getRandomProtocol();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java b/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
index 31d59e9..e902818 100644
--- a/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
+++ b/test/src/test/java/org/apache/accumulo/fate/zookeeper/ZooLockTest.java
@@ -102,15 +102,15 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testDeleteParent() throws Exception {
-    accumulo.getConfig().getZooKeepers();
+    accumulo.getZooKeepers();
 
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
 
     // intentionally created parent after lock
     zk.mkdirs(parent);
@@ -135,11 +135,11 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testNoParent() throws Exception {
-    accumulo.getConfig().getZooKeepers();
+    accumulo.getZooKeepers();
 
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
@@ -157,14 +157,14 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testDeleteLock() throws Exception {
-    accumulo.getConfig().getZooKeepers();
+    accumulo.getZooKeepers();
 
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
@@ -190,14 +190,14 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testDeleteWaiting() throws Exception {
-    accumulo.getConfig().getZooKeepers();
+    accumulo.getZooKeepers();
 
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes());
+    ZooReaderWriter zk = ZooReaderWriter.getInstance(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes());
     zk.mkdirs(parent);
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
@@ -212,7 +212,7 @@ public class ZooLockTest {
     Assert.assertNull(lw.exception);
     Assert.assertNull(lw.reason);
 
-    ZooLock zl2 = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl2 = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     TestALW lw2 = new TestALW();
 
@@ -221,7 +221,7 @@ public class ZooLockTest {
     Assert.assertFalse(lw2.locked);
     Assert.assertFalse(zl2.isLocked());
 
-    ZooLock zl3 = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl3 = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     TestALW lw3 = new TestALW();
 
@@ -258,16 +258,16 @@ public class ZooLockTest {
 
   @Test(timeout = 10000)
   public void testUnexpectedEvent() throws Exception {
-    accumulo.getConfig().getZooKeepers();
+    accumulo.getZooKeepers();
 
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooKeeper zk = new ZooKeeper(accumulo.getConfig().getZooKeepers(), 30000, null);
+    ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 30000, null);
     zk.addAuthInfo("digest", "secret".getBytes());
 
     zk.create(parent, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 30000, "digest", "secret".getBytes(), parent);
 
     Assert.assertFalse(zl.isLocked());
 
@@ -301,9 +301,9 @@ public class ZooLockTest {
   public void testTryLock() throws Exception {
     String parent = "/zltest-" + this.hashCode() + "-l" + pdCount++;
 
-    ZooLock zl = new ZooLock(accumulo.getConfig().getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
+    ZooLock zl = new ZooLock(accumulo.getZooKeepers(), 1000, "digest", "secret".getBytes(), parent);
 
-    ZooKeeper zk = new ZooKeeper(accumulo.getConfig().getZooKeepers(), 1000, null);
+    ZooKeeper zk = new ZooKeeper(accumulo.getZooKeepers(), 1000, null);
     zk.addAuthInfo("digest", "secret".getBytes());
 
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
index b735e08..dc44854 100644
--- a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -45,7 +45,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.commons.io.FileUtils;
@@ -93,7 +93,7 @@ public class AuditMessageIT extends ConfigurableMacIT {
    */
   private ArrayList<String> getAuditMessages(String stepName) throws IOException {
 
-    for (MiniAccumuloCluster.LogWriter lw : getCluster().getLogWriters()) {
+    for (MiniAccumuloClusterImpl.LogWriter lw : getCluster().getLogWriters()) {
       lw.flush();
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
index 3c07547..ed04f7e 100644
--- a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
@@ -34,16 +34,16 @@ import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
-import org.apache.accumulo.minicluster.ProcessReference;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.junit.Test;
 
 public class CleanWalIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "3s");
     cfg.setNumTservers(1);
     cfg.useMiniDFS(true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/ConfigurableMajorCompactionIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConfigurableMajorCompactionIT.java b/test/src/test/java/org/apache/accumulo/test/ConfigurableMajorCompactionIT.java
index 334fa85..ff3aa3d 100644
--- a/test/src/test/java/org/apache/accumulo/test/ConfigurableMajorCompactionIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ConfigurableMajorCompactionIT.java
@@ -36,7 +36,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.accumulo.tserver.compaction.CompactionPlan;
 import org.apache.accumulo.tserver.compaction.CompactionStrategy;
@@ -47,7 +47,7 @@ import org.junit.Test;
 public class ConfigurableMajorCompactionIT extends ConfigurableMacIT {
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1s");
     cfg.setSiteConfig(siteConfig);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
index 39df878..b98d452 100644
--- a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
@@ -26,7 +26,7 @@ import java.util.Collections;
 
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.accumulo.test.functional.FunctionalTestUtils;
@@ -40,7 +40,7 @@ public class DumpConfigIT extends ConfigurableMacIT {
   public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TABLE_FILE_BLOCK_SIZE.getKey(), "1234567"));
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
index 8234079..f5b1ddb 100644
--- a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
@@ -40,7 +40,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -67,7 +67,7 @@ public class VolumeIT extends ConfigurableMacIT {
   }
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     // Run MAC on two locations in the local file system
     cfg.setProperty(Property.INSTANCE_VOLUMES, v1.toString() + "," + v2.toString());
     super.configure(cfg);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
index f74b205..ecfcf59 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
@@ -27,8 +27,8 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.util.CertUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Logger;
@@ -45,7 +45,7 @@ public abstract class AbstractMacIT {
   @Rule
   public TestName testName = new TestName();
 
-  protected static void cleanUp(MiniAccumuloCluster cluster) {
+  protected static void cleanUp(MiniAccumuloClusterImpl cluster) {
     if (cluster != null)
       try {
         cluster.stop();
@@ -85,13 +85,13 @@ public abstract class AbstractMacIT {
     return names;
   }
 
-  protected static void configureForEnvironment(MiniAccumuloConfig cfg, Class<?> testClass, File folder) {
+  protected static void configureForEnvironment(MiniAccumuloConfigImpl cfg, Class<?> testClass, File folder) {
     if ("true".equals(System.getProperty("org.apache.accumulo.test.functional.useSslForIT"))) {
       configureForSsl(cfg, folder);
     }
   }
 
-  protected static void configureForSsl(MiniAccumuloConfig cfg, File folder) {
+  protected static void configureForSsl(MiniAccumuloConfigImpl cfg, File folder) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
     if ("true".equals(siteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
       // already enabled; don't mess with it

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
index bf2f6b6..08242ff 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
@@ -35,7 +35,7 @@ import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.fate.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
@@ -43,7 +43,7 @@ import org.junit.Test;
 public class BalanceAfterCommsFailureIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.GENERAL_RPC_TIMEOUT.getKey(), "2s"));
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
index 0297ffc..0afd651 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
@@ -34,14 +34,14 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 public class BatchScanSplitIT extends ConfigurableMacIT {
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "0"));
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
index 383e07b..16130bf 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
@@ -29,14 +29,14 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.junit.Test;
 
 public class BigRootTabletIT extends ConfigurableMacIT {
   // ACCUMULO-542: A large root tablet will fail to load if it does't fit in the tserver scan buffers
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.TABLE_SCAN_MAXMEM.getKey(), "1024");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "60m");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
index ebe2ca6..4a10a34 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BinaryStressIT.java
@@ -24,7 +24,7 @@ import java.util.Map;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -33,7 +33,7 @@ import org.junit.Test;
 public class BinaryStressIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "50K");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index d615cb8..8824167 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -40,14 +40,14 @@ import org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor;
 import org.apache.accumulo.core.file.keyfunctor.RowFunctor;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 public class BloomFilterIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setDefaultMemory(1, MemoryUnit.GIGABYTE);
     cfg.setNumTservers(1);
     Map<String,String> siteConfig = new HashMap<String, String>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 337d0f7..204fd77 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.fs.FileSystem;
 import org.junit.Rule;
@@ -46,7 +46,7 @@ public class BulkSplitOptimizationIT extends ConfigurableMacIT {
   public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
index 08d2536..9a4a904 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
@@ -23,7 +23,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.master.balancer.ChaoticLoadBalancer;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -33,7 +33,7 @@ import org.junit.Test;
 public class ChaoticBalancerIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> siteConfig = new HashMap<String, String>();
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/CleanTmpIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CleanTmpIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CleanTmpIT.java
index d455fff..df1b51b 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CleanTmpIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CleanTmpIT.java
@@ -31,9 +31,9 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
-import org.apache.accumulo.minicluster.ProcessReference;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
@@ -41,7 +41,7 @@ import org.junit.Test;
 public class CleanTmpIT extends ConfigurableMacIT {
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> props = new HashMap<String,String>();
     props.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "3s");
     cfg.setSiteConfig(props);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java
index cba24fd..3f8fcb3 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -38,7 +38,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,7 +52,7 @@ public class CompactionIT extends ConfigurableMacIT {
   public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> map = new HashMap<String,String>();
     map.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "4");
     map.put(Property.TSERV_MAJC_DELAY.getKey(), "1");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
index 87f6bd7..305d300 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
@@ -37,7 +37,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -67,7 +67,7 @@ public class ConcurrencyIT extends ConfigurableMacIT {
   }
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1"));
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
index 7d2c795..cc43cc9 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
@@ -36,7 +36,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.tserver.compaction.CompactionPlan;
 import org.apache.accumulo.tserver.compaction.CompactionStrategy;
 import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
@@ -46,7 +46,7 @@ import org.junit.Test;
 public class ConfigurableCompactionIT extends ConfigurableMacIT {
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
index a7abcb1..8dafe53 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ConfigurableMacIT.java
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.MonitorUtil;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -35,18 +35,18 @@ import org.junit.Before;
 public class ConfigurableMacIT extends AbstractMacIT {
   public static final Logger log = Logger.getLogger(ConfigurableMacIT.class);
 
-  public MiniAccumuloCluster cluster;
+  public MiniAccumuloClusterImpl cluster;
 
-  public void configure(MiniAccumuloConfig cfg) {}
+  public void configure(MiniAccumuloConfigImpl cfg) {}
 
   @Before
   public void setUp() throws Exception {
-    MiniAccumuloConfig cfg = new MiniAccumuloConfig(createTestDir(this.getClass().getName()), ROOT_PASSWORD);
+    MiniAccumuloConfigImpl cfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName()), ROOT_PASSWORD);
     cfg.setNativeLibPaths(NativeMapIT.nativeMapLocation().getAbsolutePath());
     configure(cfg);
     cfg.setProperty(Property.TSERV_NATIVEMAP_ENABLED, Boolean.TRUE.toString());
     configureForEnvironment(cfg, getClass(), createSharedTestDir(this.getClass().getName() + "-ssl"));
-    cluster = new MiniAccumuloCluster(cfg);
+    cluster = new MiniAccumuloClusterImpl(cfg);
     cluster.start();
   }
 
@@ -55,7 +55,7 @@ public class ConfigurableMacIT extends AbstractMacIT {
     cleanUp(cluster);
   }
 
-  public MiniAccumuloCluster getCluster() {
+  public MiniAccumuloClusterImpl getCluster() {
     return cluster;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
index 1f62f3c..88f936f 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
@@ -30,14 +30,14 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 public class DeleteEverythingIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
index 28d7d81..fe51039 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DeleteIT.java
@@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals;
 
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestRandomDeletes;
@@ -37,7 +37,7 @@ public class DeleteIT extends ConfigurableMacIT {
     assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
   }
 
-  public static void deleteTest(Connector c, MiniAccumuloCluster cluster) throws Exception {
+  public static void deleteTest(Connector c, MiniAccumuloClusterImpl cluster) throws Exception {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     TestIngest.Opts opts = new TestIngest.Opts();
     vopts.rows = opts.rows = 1000;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
index 4ec59b0..e83baee 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/DynamicThreadPoolsIT.java
@@ -30,7 +30,7 @@ import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.junit.Test;
@@ -38,7 +38,7 @@ import org.junit.Test;
 public class DynamicThreadPoolsIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setNumTservers(1);
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "100ms"));
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
index dcf72c8..559ae67 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
@@ -71,8 +71,8 @@ import org.apache.accumulo.examples.simple.shard.Index;
 import org.apache.accumulo.examples.simple.shard.Query;
 import org.apache.accumulo.examples.simple.shard.Reverse;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster.LogWriter;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl.LogWriter;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.tracer.TraceServer;
@@ -86,7 +86,7 @@ public class ExamplesIT extends ConfigurableMacIT {
   BatchWriterOpts bwOpts = new BatchWriterOpts();
 
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setDefaultMemory(cfg.getDefaultMemory() * 2, MemoryUnit.BYTE);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
index 08a6f51..c863735 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
@@ -42,8 +42,8 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster.LogWriter;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl.LogWriter;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -148,7 +148,7 @@ public class FunctionalTestUtils {
     return result.toString();
   }
 
-  static String readAll(MiniAccumuloCluster c, Class<?> klass, Process p) throws Exception {
+  static String readAll(MiniAccumuloClusterImpl c, Class<?> klass, Process p) throws Exception {
     for (LogWriter writer : c.getLogWriters())
       writer.flush();
     return readAll(new FileInputStream(c.getConfig().getLogDir() + "/" + klass.getSimpleName() + "_" + p.hashCode() + ".out"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 44c57c1..36ff31d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -49,10 +49,10 @@ import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
-import org.apache.accumulo.minicluster.ProcessNotFoundException;
-import org.apache.accumulo.minicluster.ProcessReference;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessNotFoundException;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -69,7 +69,7 @@ public class GarbageCollectorIT extends ConfigurableMacIT {
   private static final String OUR_SECRET = "itsreallysecret";
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     Map<String,String> settings = new HashMap<String,String>();
     settings.put(Property.INSTANCE_SECRET.getKey(), OUR_SECRET);
     settings.put(Property.GC_CYCLE_START.getKey(), "1");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9d1e171e/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index dfd416f..d8f38c8 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -33,8 +33,8 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.minicluster.MiniAccumuloConfig;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.start.Main;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -44,7 +44,7 @@ import org.junit.Test;
 public class HalfDeadTServerIT extends ConfigurableMacIT {
   
   @Override
-  public void configure(MiniAccumuloConfig cfg) {
+  public void configure(MiniAccumuloConfigImpl cfg) {
     cfg.setNumTservers(1);
     Map<String,String> siteConfig = new HashMap<String,String>();
     siteConfig.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "15s");