You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2015/01/26 05:51:04 UTC

[2/3] accumulo git commit: ACCUMULO-3532 Remove changes to ServerType.

ACCUMULO-3532 Remove changes to ServerType.

ServerType is in public API and can't have additions to it per semver.
Introduce a class not in the public API that lets us do what we want.


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

Branch: refs/heads/master
Commit: c1d9118ba7973eebac6b381f903893f6666cc2ef
Parents: 69ca8c9
Author: Josh Elser <el...@apache.org>
Authored: Sun Jan 25 23:49:03 2015 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Sun Jan 25 23:49:03 2015 -0500

----------------------------------------------------------------------
 .../apache/accumulo/cluster/ClusterControl.java | 18 +++---
 .../accumulo/cluster/ClusterServerType.java     | 65 ++++++++++++++++++++
 .../standalone/StandaloneAccumuloCluster.java   |  8 ++-
 .../standalone/StandaloneClusterControl.java    | 24 ++++----
 .../minicluster/MiniAccumuloCluster.java        | 12 +++-
 .../minicluster/MiniAccumuloConfig.java         |  5 +-
 .../apache/accumulo/minicluster/ServerType.java |  2 +-
 .../impl/MiniAccumuloClusterControl.java        | 21 ++++---
 .../impl/MiniAccumuloClusterImpl.java           | 25 ++++----
 .../impl/MiniAccumuloConfigImpl.java            |  8 +--
 .../impl/MiniAccumuloConfigImplTest.java        | 12 ++--
 .../apache/accumulo/test/Accumulo3010IT.java    |  6 +-
 .../apache/accumulo/test/Accumulo3047IT.java    | 10 +--
 .../org/apache/accumulo/test/CleanWalIT.java    | 10 +--
 .../accumulo/test/NoMutationRecoveryIT.java     |  6 +-
 .../test/functional/BinaryStressIT.java         | 10 +--
 .../accumulo/test/functional/BulkFileIT.java    |  4 +-
 .../functional/BulkSplitOptimizationIT.java     | 10 +--
 .../accumulo/test/functional/CleanTmpIT.java    |  6 +-
 .../accumulo/test/functional/CompactionIT.java  | 10 +--
 .../accumulo/test/functional/LargeRowIT.java    |  4 +-
 .../test/functional/MasterFailoverIT.java       |  6 +-
 .../accumulo/test/functional/ReadWriteIT.java   | 10 +--
 .../accumulo/test/functional/RestartIT.java     | 26 ++++----
 .../test/functional/RestartStressIT.java        |  6 +-
 .../functional/SimpleBalancerFairnessIT.java    |  4 +-
 .../accumulo/test/functional/SplitIT.java       | 10 +--
 .../test/functional/WriteAheadLogIT.java        |  6 +-
 28 files changed, 211 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
index e5ab27f..7d65259 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
@@ -19,8 +19,6 @@ package org.apache.accumulo.cluster;
 import java.io.IOException;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.minicluster.ServerType;
-
 /**
  * Basic functionality required to control an Accumulo cluster
  */
@@ -44,40 +42,40 @@ public interface ClusterControl {
   /**
    * Starts all occurrences of the given server
    */
-  void startAllServers(ServerType server) throws IOException;
+  void startAllServers(ClusterServerType server) throws IOException;
 
   /**
    * Start the given process on the host
    */
-  void start(ServerType server, String hostname) throws IOException;
+  void start(ClusterServerType server, String hostname) throws IOException;
 
   /**
    * Stops all occurrences of the given server
    */
-  void stopAllServers(ServerType server) throws IOException;
+  void stopAllServers(ClusterServerType server) throws IOException;
 
   /**
    * Stop the given process on the host
    */
-  void stop(ServerType server, String hostname) throws IOException;
+  void stop(ClusterServerType server, String hostname) throws IOException;
 
   /**
    * Send the provided signal to the process on the host
    */
-  void signal(ServerType server, String hostname, String signal) throws IOException;
+  void signal(ClusterServerType server, String hostname, String signal) throws IOException;
 
   /**
    * Send SIGSTOP to the given process on the host
    */
-  void suspend(ServerType server, String hostname) throws IOException;
+  void suspend(ClusterServerType server, String hostname) throws IOException;
 
   /**
    * Send SIGCONT to the given process on the host
    */
-  void resume(ServerType server, String hostname) throws IOException;
+  void resume(ClusterServerType server, String hostname) throws IOException;
 
   /**
    * Send SIGKILL to the given process on the host
    */
-  void kill(ServerType server, String hostname) throws IOException;
+  void kill(ClusterServerType server, String hostname) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterServerType.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterServerType.java b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterServerType.java
new file mode 100644
index 0000000..9a30796
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterServerType.java
@@ -0,0 +1,65 @@
+/*
+ * 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.cluster;
+
+import org.apache.accumulo.minicluster.ServerType;
+
+/**
+ * {@link ServerType} is in the public API. This lets us work around that annoyance.
+ */
+public enum ClusterServerType {
+  MASTER("Master"), ZOOKEEPER("ZooKeeper"), TABLET_SERVER("TServer"), GARBAGE_COLLECTOR("GC"), TRACER("Tracer"), MONITOR("Monitor");
+
+  private final String prettyPrint;
+
+  public String prettyPrint() {
+    return prettyPrint;
+  }
+
+  ClusterServerType(String prettyPrint) {
+    this.prettyPrint = prettyPrint;
+  }
+
+  public static ClusterServerType get(ServerType type) {
+    switch (type) {
+      case MASTER:
+        return MASTER;
+      case ZOOKEEPER:
+        return ZOOKEEPER;
+      case TABLET_SERVER:
+        return TABLET_SERVER;
+      case GARBAGE_COLLECTOR:
+        return GARBAGE_COLLECTOR;
+    }
+    throw new IllegalArgumentException("Unknown server type");
+  }
+
+  public ServerType toServerType() {
+    switch (this) {
+      case MASTER:
+        return ServerType.MASTER;
+      case ZOOKEEPER:
+        return ServerType.ZOOKEEPER;
+      case TABLET_SERVER:
+        return ServerType.TABLET_SERVER;
+      case GARBAGE_COLLECTOR:
+        return ServerType.GARBAGE_COLLECTOR;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index 58536ed..33a29dd 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
@@ -29,7 +30,6 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.master.state.SetGoalState;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -112,7 +112,8 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
 
     control.exec(SetGoalState.class, new String[] {"NORMAL"});
 
-    for (ServerType type : Arrays.asList(ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.TRACER, ServerType.GARBAGE_COLLECTOR, ServerType.MONITOR)) {
+    for (ClusterServerType type : Arrays.asList(ClusterServerType.MASTER, ClusterServerType.TABLET_SERVER, ClusterServerType.TRACER,
+        ClusterServerType.GARBAGE_COLLECTOR, ClusterServerType.MONITOR)) {
       control.startAllServers(type);
     }
   }
@@ -123,7 +124,8 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
 
     // TODO We can check the hosts files, but that requires us to be on a host with the installation. Limitation at the moment.
 
-    for (ServerType type : Arrays.asList(ServerType.MASTER, ServerType.TABLET_SERVER, ServerType.TRACER, ServerType.GARBAGE_COLLECTOR, ServerType.MONITOR)) {
+    for (ClusterServerType type : Arrays.asList(ClusterServerType.MASTER, ClusterServerType.TABLET_SERVER, ClusterServerType.TRACER,
+        ClusterServerType.GARBAGE_COLLECTOR, ClusterServerType.MONITOR)) {
       control.stopAllServers(type);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
index 9b467bc..e377daf 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
@@ -28,9 +28,9 @@ import java.util.List;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.cluster.RemoteShell;
 import org.apache.accumulo.cluster.RemoteShellOptions;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.util.Shell.ExitCodeException;
@@ -137,7 +137,7 @@ public class StandaloneClusterControl implements ClusterControl {
   }
 
   @Override
-  public void startAllServers(ServerType server) throws IOException {
+  public void startAllServers(ClusterServerType server) throws IOException {
     File confDir = getConfDir();
 
     switch (server) {
@@ -181,7 +181,7 @@ public class StandaloneClusterControl implements ClusterControl {
   }
 
   @Override
-  public void start(ServerType server, String hostname) throws IOException {
+  public void start(ClusterServerType server, String hostname) throws IOException {
     String[] cmd = new String[] {startServerPath, hostname, getProcessString(server)};
     Entry<Integer,String> pair = exec(hostname, cmd);
     if (0 != pair.getKey()) {
@@ -190,7 +190,7 @@ public class StandaloneClusterControl implements ClusterControl {
   }
 
   @Override
-  public void stopAllServers(ServerType server) throws IOException {
+  public void stopAllServers(ClusterServerType server) throws IOException {
     File confDir = getConfDir();
 
     switch (server) {
@@ -226,14 +226,14 @@ public class StandaloneClusterControl implements ClusterControl {
   }
 
   @Override
-  public void stop(ServerType server, String hostname) throws IOException {
+  public void stop(ClusterServerType server, String hostname) throws IOException {
     // TODO Use `accumulo admin stop` for tservers, instrument clean stop for GC, monitor, tracer instead kill
 
     kill(server, hostname);
   }
 
   @Override
-  public void signal(ServerType server, String hostname, String signal) throws IOException {
+  public void signal(ClusterServerType server, String hostname, String signal) throws IOException {
     String pid = getPid(server, accumuloHome, hostname);
 
     if (pid.trim().isEmpty()) {
@@ -261,21 +261,21 @@ public class StandaloneClusterControl implements ClusterControl {
   }
 
   @Override
-  public void suspend(ServerType server, String hostname) throws IOException {
+  public void suspend(ClusterServerType server, String hostname) throws IOException {
     signal(server, hostname, "SIGSTOP");
   }
 
   @Override
-  public void resume(ServerType server, String hostname) throws IOException {
+  public void resume(ClusterServerType server, String hostname) throws IOException {
     signal(server, hostname, "SIGCONT");
   }
 
   @Override
-  public void kill(ServerType server, String hostname) throws IOException {
+  public void kill(ClusterServerType server, String hostname) throws IOException {
     signal(server, hostname, "SIGKILL");
   }
 
-  protected String getPid(ServerType server, String accumuloHome, String hostname) throws IOException {
+  protected String getPid(ClusterServerType server, String accumuloHome, String hostname) throws IOException {
     String[] getPidCommand = getPidCommand(server, accumuloHome);
     Entry<Integer,String> ret = exec(hostname, getPidCommand);
     if (0 != ret.getKey()) {
@@ -285,13 +285,13 @@ public class StandaloneClusterControl implements ClusterControl {
     return ret.getValue();
   }
 
-  protected String[] getPidCommand(ServerType server, String accumuloHome) {
+  protected String[] getPidCommand(ClusterServerType server, String accumuloHome) {
     // Lifted from stop-server.sh to get the PID
     return new String[] {"ps", "aux", "|", "fgrep", accumuloHome, "|", "fgrep", getProcessString(server), "|", "fgrep", "-v", "grep", "|", "fgrep", "-v",
         "ssh", "|", "awk", "'{print \\$2}'", "|", "head", "-1", "|", "tr", "-d", "'\\n'"};
   }
 
-  protected String getProcessString(ServerType server) {
+  protected String getProcessString(ClusterServerType server) {
     switch (server) {
       case TABLET_SERVER:
         return "tserver";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 87dfff8..913da2b 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -18,8 +18,10 @@ package org.apache.accumulo.minicluster;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
@@ -75,7 +77,15 @@ public class MiniAccumuloCluster {
    * @since 1.6.0
    */
   public Set<Pair<ServerType,Integer>> getDebugPorts() {
-    return impl.getDebugPorts();
+    Set<Pair<ClusterServerType,Integer>> implPorts = impl.getDebugPorts();
+    Set<Pair<ServerType,Integer>> returnPorts = new HashSet<Pair<ServerType,Integer>>();
+    for (Pair<ClusterServerType,Integer> pair : implPorts) {
+      ServerType st = pair.getFirst().toServerType();
+      if (null != st) {
+        returnPorts.add(new Pair<ServerType,Integer>(st, pair.getSecond()));
+      }
+    }
+    return returnPorts;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index b71b771..19a5fb0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.minicluster;
 import java.io.File;
 import java.util.Map;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 
 /**
@@ -121,7 +122,7 @@ public class MiniAccumuloConfig {
    * @since 1.6.0
    */
   public MiniAccumuloConfig setMemory(ServerType serverType, long memory, MemoryUnit memoryUnit) {
-    impl.setMemory(serverType, memory, memoryUnit);
+    impl.setMemory(ClusterServerType.get(serverType), memory, memoryUnit);
     return this;
   }
 
@@ -176,7 +177,7 @@ public class MiniAccumuloConfig {
    * @since 1.6.0
    */
   public long getMemory(ServerType serverType) {
-    return impl.getMemory(serverType);
+    return impl.getMemory(ClusterServerType.get(serverType));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
index f2dae8f..3590a20 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
@@ -21,7 +21,7 @@ package org.apache.accumulo.minicluster;
  */
 
 public enum ServerType {
-  MASTER("Master"), ZOOKEEPER("ZooKeeper"), TABLET_SERVER("TServer"), GARBAGE_COLLECTOR("GC"), TRACER("Tracer"), MONITOR("Monitor");
+  MASTER("Master"), ZOOKEEPER("ZooKeeper"), TABLET_SERVER("TServer"), GARBAGE_COLLECTOR("GC");
 
   private final String prettyPrint;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterControl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterControl.java
index 839285a..c2faf81 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterControl.java
@@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.minicluster.ServerType;
@@ -62,7 +63,7 @@ public class MiniAccumuloClusterControl implements ClusterControl {
     this.cluster = cluster;
   }
 
-  public void start(ServerType server) throws IOException {
+  public void start(ClusterServerType server) throws IOException {
     start(server, null);
   }
 
@@ -125,12 +126,12 @@ public class MiniAccumuloClusterControl implements ClusterControl {
   }
 
   @Override
-  public synchronized void startAllServers(ServerType server) throws IOException {
+  public synchronized void startAllServers(ClusterServerType server) throws IOException {
     start(server, null);
   }
 
   @Override
-  public synchronized void start(ServerType server, String hostname) throws IOException {
+  public synchronized void start(ClusterServerType server, String hostname) throws IOException {
     switch (server) {
       case TABLET_SERVER:
         synchronized (tabletServerProcesses) {
@@ -170,16 +171,16 @@ public class MiniAccumuloClusterControl implements ClusterControl {
   }
 
   @Override
-  public synchronized void stopAllServers(ServerType server) throws IOException {
+  public synchronized void stopAllServers(ClusterServerType server) throws IOException {
     stop(server);
   }
 
-  public void stop(ServerType server) throws IOException {
+  public void stop(ClusterServerType server) throws IOException {
     stop(server, null);
   }
 
   @Override
-  public synchronized void stop(ServerType server, String hostname) throws IOException {
+  public synchronized void stop(ClusterServerType server, String hostname) throws IOException {
     switch (server) {
       case MASTER:
         if (null != masterProcess) {
@@ -284,17 +285,17 @@ public class MiniAccumuloClusterControl implements ClusterControl {
   }
 
   @Override
-  public void signal(ServerType server, String hostname, String signal) throws IOException {
+  public void signal(ClusterServerType server, String hostname, String signal) throws IOException {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void suspend(ServerType server, String hostname) throws IOException {
+  public void suspend(ClusterServerType server, String hostname) throws IOException {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void resume(ServerType server, String hostname) throws IOException {
+  public void resume(ClusterServerType server, String hostname) throws IOException {
     throw new UnsupportedOperationException();
   }
 
@@ -368,7 +369,7 @@ public class MiniAccumuloClusterControl implements ClusterControl {
   }
 
   @Override
-  public void kill(ServerType server, String hostname) throws IOException {
+  public void kill(ClusterServerType server, String hostname) throws IOException {
     stop(server, hostname);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 8188be7..6e67c9f 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -52,6 +52,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -164,7 +165,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
   private boolean initialized = false;
 
-  private Set<Pair<ServerType,Integer>> debugPorts = new HashSet<Pair<ServerType,Integer>>();
+  private Set<Pair<ClusterServerType,Integer>> debugPorts = new HashSet<Pair<ClusterServerType,Integer>>();
 
   private File zooCfgFile;
   private String dfsUri;
@@ -324,7 +325,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     return process;
   }
 
-  Process _exec(Class<?> clazz, ServerType serverType, String... args) throws IOException {
+  Process _exec(Class<?> clazz, ClusterServerType serverType, String... args) throws IOException {
 
     List<String> jvmOpts = new ArrayList<String>();
     jvmOpts.add("-Xmx" + config.getMemory(serverType));
@@ -332,7 +333,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     if (config.isJDWPEnabled()) {
       Integer port = PortUtils.getRandomFreePort();
       jvmOpts.addAll(buildRemoteDebugParams(port));
-      debugPorts.add(new Pair<ServerType,Integer>(serverType, port));
+      debugPorts.add(new Pair<ClusterServerType,Integer>(serverType, port));
     }
     return _exec(clazz, jvmOpts, args);
   }
@@ -528,7 +529,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         });
       }
 
-      control.start(ServerType.ZOOKEEPER);
+      control.start(ClusterServerType.ZOOKEEPER);
 
       if (!initialized) {
         // sleep a little bit to let zookeeper come up before calling init, seems to work better
@@ -567,7 +568,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     log.info("Starting MAC against instance {} and zookeeper(s) {}.", config.getInstanceName(), config.getZooKeepers());
 
-    control.start(ServerType.TABLET_SERVER);
+    control.start(ClusterServerType.TABLET_SERVER);
 
     int ret = 0;
     for (int i = 0; i < 5; i++) {
@@ -580,8 +581,8 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       throw new RuntimeException("Could not set master goal state, process returned " + ret + ". Check the logs in " + config.getLogDir() + " for errors.");
     }
 
-    control.start(ServerType.MASTER);
-    control.start(ServerType.GARBAGE_COLLECTOR);
+    control.start(ClusterServerType.MASTER);
+    control.start(ClusterServerType.GARBAGE_COLLECTOR);
 
     if (null == executor) {
       executor = Executors.newSingleThreadExecutor();
@@ -596,7 +597,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
    * @return generated remote debug ports if in debug mode.
    * @since 1.6.0
    */
-  public Set<Pair<ServerType,Integer>> getDebugPorts() {
+  public Set<Pair<ClusterServerType,Integer>> getDebugPorts() {
     return debugPorts;
   }
 
@@ -657,10 +658,10 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     MiniAccumuloClusterControl control = getClusterControl();
 
-    control.stop(ServerType.GARBAGE_COLLECTOR, null);
-    control.stop(ServerType.MASTER, null);
-    control.stop(ServerType.TABLET_SERVER, null);
-    control.stop(ServerType.ZOOKEEPER, null);
+    control.stop(ClusterServerType.GARBAGE_COLLECTOR, null);
+    control.stop(ClusterServerType.MASTER, null);
+    control.stop(ClusterServerType.TABLET_SERVER, null);
+    control.stop(ClusterServerType.ZOOKEEPER, null);
 
     // ACCUMULO-2985 stop the ExecutorService after we finished using it to stop accumulo procs
     if (null != executor) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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
index b8651ef..10d5a9c 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -24,11 +24,11 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
@@ -47,7 +47,7 @@ public class MiniAccumuloConfigImpl {
   private Map<String,String> siteConfig = new HashMap<String,String>();
   private Map<String,String> configuredSiteConig = new HashMap<String,String>();
   private int numTservers = 2;
-  private Map<ServerType,Long> memoryConfig = new HashMap<ServerType,Long>();
+  private Map<ClusterServerType,Long> memoryConfig = new HashMap<ClusterServerType,Long>();
   private boolean jdwpEnabled = false;
   private Map<String,String> systemProperties = new HashMap<String,String>();
 
@@ -321,7 +321,7 @@ public class MiniAccumuloConfigImpl {
    *
    * @since 1.6.0
    */
-  public MiniAccumuloConfigImpl setMemory(ServerType serverType, long memory, MemoryUnit memoryUnit) {
+  public MiniAccumuloConfigImpl setMemory(ClusterServerType serverType, long memory, MemoryUnit memoryUnit) {
     this.memoryConfig.put(serverType, memoryUnit.toBytes(memory));
     return this;
   }
@@ -416,7 +416,7 @@ public class MiniAccumuloConfigImpl {
    *
    * @since 1.6.0
    */
-  public long getMemory(ServerType serverType) {
+  public long getMemory(ClusterServerType serverType) {
     return memoryConfig.containsKey(serverType) ? memoryConfig.get(serverType) : defaultMemorySize;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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
index dc616df..5fd6986 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImplTest.java
@@ -24,9 +24,9 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 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;
@@ -77,13 +77,13 @@ public class MiniAccumuloConfigImplTest {
 
     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.getMemory(ClusterServerType.MASTER));
+    assertEquals(96 * 1024 * 1024l, config.getMemory(ClusterServerType.TABLET_SERVER));
     assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
-    config.setMemory(ServerType.MASTER, 256, MemoryUnit.MEGABYTE);
-    assertEquals(256 * 1024 * 1024l, config.getMemory(ServerType.MASTER));
+    config.setMemory(ClusterServerType.MASTER, 256, MemoryUnit.MEGABYTE);
+    assertEquals(256 * 1024 * 1024l, config.getMemory(ClusterServerType.MASTER));
     assertEquals(96 * 1024 * 1024l, config.getDefaultMemory());
-    assertEquals(96 * 1024 * 1024l, config.getMemory(ServerType.TABLET_SERVER));
+    assertEquals(96 * 1024 * 1024l, config.getMemory(ClusterServerType.TABLET_SERVER));
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/Accumulo3010IT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/Accumulo3010IT.java b/test/src/test/java/org/apache/accumulo/test/Accumulo3010IT.java
index 512e666..eff7251 100644
--- a/test/src/test/java/org/apache/accumulo/test/Accumulo3010IT.java
+++ b/test/src/test/java/org/apache/accumulo/test/Accumulo3010IT.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.test;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -31,7 +32,6 @@ 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.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.RawLocalFileSystem;
@@ -77,10 +77,10 @@ public class Accumulo3010IT extends AccumuloClusterIT {
     ClusterControl control = cluster.getClusterControl();
 
     // kill the tablet servers
-    control.stopAllServers(ServerType.TABLET_SERVER);
+    control.stopAllServers(ClusterServerType.TABLET_SERVER);
 
     // recover
-    control.startAllServers(ServerType.TABLET_SERVER);
+    control.startAllServers(ClusterServerType.TABLET_SERVER);
 
     // ensure the table is readable
     for (@SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/Accumulo3047IT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/Accumulo3047IT.java b/test/src/test/java/org/apache/accumulo/test/Accumulo3047IT.java
index cc12668..ff8dae0 100644
--- a/test/src/test/java/org/apache/accumulo/test/Accumulo3047IT.java
+++ b/test/src/test/java/org/apache/accumulo/test/Accumulo3047IT.java
@@ -21,6 +21,7 @@ import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -37,7 +38,6 @@ import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -87,7 +87,7 @@ public class Accumulo3047IT extends AccumuloClusterIT {
     iops.setProperty(Property.GC_CYCLE_START.getKey(), "0s");
     log.info("Restarting garbage collector");
 
-    getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
+    getCluster().getClusterControl().stopAllServers(ClusterServerType.GARBAGE_COLLECTOR);
 
     Instance instance = getConnector().getInstance();
     ZooCache zcache = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
@@ -104,7 +104,7 @@ public class Accumulo3047IT extends AccumuloClusterIT {
 
     log.info("GC lock was lost");
 
-    getCluster().getClusterControl().startAllServers(ServerType.GARBAGE_COLLECTOR);
+    getCluster().getClusterControl().startAllServers(ClusterServerType.GARBAGE_COLLECTOR);
     log.info("Garbage collector was restarted");
 
     gcLockData = null;
@@ -129,8 +129,8 @@ public class Accumulo3047IT extends AccumuloClusterIT {
       iops.setProperty(Property.GC_CYCLE_START.getKey(), gcCycleStart);
     }
     log.info("Restarting garbage collector");
-    getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
-    getCluster().getClusterControl().startAllServers(ServerType.GARBAGE_COLLECTOR);
+    getCluster().getClusterControl().stopAllServers(ClusterServerType.GARBAGE_COLLECTOR);
+    getCluster().getClusterControl().startAllServers(ClusterServerType.GARBAGE_COLLECTOR);
     log.info("Garbage collector was restarted");
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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 1fcd5a4..b2298f7 100644
--- a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
@@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.Map.Entry;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -35,7 +36,6 @@ 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.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.FunctionalTestUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -88,10 +88,10 @@ public class CleanWalIT extends AccumuloClusterIT {
     m.put("cf", "cq", "value");
     bw.addMutation(m);
     bw.close();
-    getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
     // all 3 tables should do recovery, but the bug doesn't really remove the log file references
 
-    getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
 
     for (String table : new String[] {MetadataTable.NAME, RootTable.NAME})
       conn.tableOperations().flush(table, null, null, true);
@@ -112,10 +112,10 @@ public class CleanWalIT extends AccumuloClusterIT {
     conn.tableOperations().flush(MetadataTable.NAME, null, null, true);
     conn.tableOperations().flush(RootTable.NAME, null, null, true);
     try {
-      getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
       UtilWaitThread.sleep(3 * 1000);
     } finally {
-      getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
     assertEquals(0, count(tableName, conn));
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
index 10b8810..00faf77 100644
--- a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -38,7 +39,6 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.FunctionalTestUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -83,8 +83,8 @@ public class NoMutationRecoveryIT extends AccumuloClusterIT {
     conn.tableOperations().flush(RootTable.NAME, null, null, true);
 
     ClusterControl control = cluster.getClusterControl();
-    control.stopAllServers(ServerType.TABLET_SERVER);
-    control.startAllServers(ServerType.TABLET_SERVER);
+    control.stopAllServers(ClusterServerType.TABLET_SERVER);
+    control.startAllServers(ClusterServerType.TABLET_SERVER);
 
     Scanner s = conn.createScanner(table, Authorizations.EMPTY);
     int count = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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 5a41d32..fbe504e 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,6 +24,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
@@ -34,7 +35,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -73,8 +73,8 @@ public class BinaryStressIT extends AccumuloClusterIT {
     iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), "0");
     iops.setProperty(Property.TSERV_MAXMEM.getKey(), "50K");
 
-    getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-    getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+    getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
   }
 
   @After
@@ -84,8 +84,8 @@ public class BinaryStressIT extends AccumuloClusterIT {
       iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
       iops.setProperty(Property.TSERV_MAXMEM.getKey(), maxMem);
 
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
index 67de2e6..27a6cce 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BulkFileIT.java
@@ -23,6 +23,7 @@ import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -34,7 +35,6 @@ import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.hadoop.conf.Configuration;
@@ -47,7 +47,7 @@ public class BulkFileIT extends AccumuloClusterIT {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration conf) {
-    cfg.setMemory(ServerType.TABLET_SERVER, 128 * 4, MemoryUnit.MEGABYTE);
+    cfg.setMemory(ClusterServerType.TABLET_SERVER, 128 * 4, MemoryUnit.MEGABYTE);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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 27b5147..442507f 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
@@ -20,6 +20,7 @@ import static com.google.common.base.Charsets.UTF_8;
 
 import java.util.Collections;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
@@ -27,7 +28,6 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.conf.Configuration;
@@ -63,8 +63,8 @@ public class BulkSplitOptimizationIT extends AccumuloClusterIT {
     majcDelay = conn.instanceOperations().getSystemConfiguration().get(Property.TSERV_MAJC_DELAY.getKey());
     if (!"1s".equals(majcDelay)) {
       conn.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1s");
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
   }
 
@@ -73,8 +73,8 @@ public class BulkSplitOptimizationIT extends AccumuloClusterIT {
     if (null != majcDelay) {
       Connector conn = getConnector();
       conn.instanceOperations().setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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 1387983..65422c9 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
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -38,7 +39,6 @@ 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.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -104,8 +104,8 @@ public class CleanTmpIT extends AccumuloClusterIT {
     // Make the file
     fs.create(tmp).close();
     log.info("Created tmp file {}", tmp.toString());
-    getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-    getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+    getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
 
     Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
     assertEquals(2, FunctionalTestUtils.count(scanner));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/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 dc877dc..907d17d 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
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -34,7 +35,6 @@ 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.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.hadoop.conf.Configuration;
@@ -78,8 +78,8 @@ public class CompactionIT extends AccumuloClusterIT {
       iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), "1");
       iops.setProperty(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), "1");
 
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
   }
 
@@ -93,8 +93,8 @@ public class CompactionIT extends AccumuloClusterIT {
       iops.setProperty(Property.TSERV_MAJC_DELAY.getKey(), majcDelay);
       iops.setProperty(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), majcMaxConcurrent);
 
-      getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/LargeRowIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/LargeRowIT.java b/test/src/test/java/org/apache/accumulo/test/functional/LargeRowIT.java
index a417213..3448523 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/LargeRowIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/LargeRowIT.java
@@ -23,6 +23,7 @@ import java.util.Map.Entry;
 import java.util.Random;
 import java.util.TreeSet;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -36,7 +37,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.hadoop.conf.Configuration;
@@ -52,7 +52,7 @@ public class LargeRowIT extends AccumuloClusterIT {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setMemory(ServerType.TABLET_SERVER, cfg.getMemory(ServerType.TABLET_SERVER) * 2, MemoryUnit.BYTE);
+    cfg.setMemory(ClusterServerType.TABLET_SERVER, cfg.getMemory(ClusterServerType.TABLET_SERVER) * 2, MemoryUnit.BYTE);
     cfg.setSiteConfig(Collections.singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "10ms"));
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MasterFailoverIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
index 0bd303a..7634f10 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MasterFailoverIT.java
@@ -19,12 +19,12 @@ package org.apache.accumulo.test.functional;
 import java.util.Collections;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -53,9 +53,9 @@ public class MasterFailoverIT extends AccumuloClusterIT {
     TestIngest.ingest(c, opts, new BatchWriterOpts());
 
     ClusterControl control = cluster.getClusterControl();
-    control.stopAllServers(ServerType.MASTER);
+    control.stopAllServers(ClusterServerType.MASTER);
     // start up a new one
-    control.startAllServers(ServerType.MASTER);
+    control.startAllServers(ClusterServerType.MASTER);
     // talk to it
     c.tableOperations().rename(names[0], names[1]);
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index a9f8887..17d6584 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
@@ -63,7 +64,6 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestMultiTableIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -97,7 +97,7 @@ public class ReadWriteIT extends AccumuloClusterIT {
     // Start accumulo, create a table, insert some data, verify we can read it out.
     // Shutdown cleanly.
     log.debug("Starting Monitor");
-    cluster.getClusterControl().startAllServers(ServerType.MONITOR);
+    cluster.getClusterControl().startAllServers(ClusterServerType.MONITOR);
     Connector connector = getConnector();
     String tableName = getUniqueNames(1)[0];
     ingest(connector, ROWS, COLS, 50, 0, tableName);
@@ -128,9 +128,9 @@ public class ReadWriteIT extends AccumuloClusterIT {
       }
     } while (null != masterLockData);
 
-    control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
-    control.stopAllServers(ServerType.MONITOR);
-    control.stopAllServers(ServerType.TRACER);
+    control.stopAllServers(ClusterServerType.GARBAGE_COLLECTOR);
+    control.stopAllServers(ClusterServerType.MONITOR);
+    control.stopAllServers(ClusterServerType.TRACER);
     log.debug("success!");
     // Restarting everything
     cluster.start();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
index 8d5a1dd..4e55ab4 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -29,6 +29,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
@@ -42,7 +43,6 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -128,8 +128,8 @@ public class RestartIT extends AccumuloClusterIT {
       }
     });
 
-    control.stopAllServers(ServerType.MASTER);
-    control.startAllServers(ServerType.MASTER);
+    control.stopAllServers(ClusterServerType.MASTER);
+    control.startAllServers(ClusterServerType.MASTER);
     assertEquals(0, ret.get().intValue());
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
   }
@@ -146,11 +146,11 @@ public class RestartIT extends AccumuloClusterIT {
 
     // TODO implement a kill all too?
     // cluster.stop() would also stop ZooKeeper
-    control.stopAllServers(ServerType.MASTER);
-    control.stopAllServers(ServerType.TRACER);
-    control.stopAllServers(ServerType.TABLET_SERVER);
-    control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
-    control.stopAllServers(ServerType.MONITOR);
+    control.stopAllServers(ClusterServerType.MASTER);
+    control.stopAllServers(ClusterServerType.TRACER);
+    control.stopAllServers(ClusterServerType.TABLET_SERVER);
+    control.stopAllServers(ClusterServerType.GARBAGE_COLLECTOR);
+    control.stopAllServers(ClusterServerType.MONITOR);
 
     ZooReader zreader = new ZooReader(c.getInstance().getZooKeepers(), c.getInstance().getZooKeepersSessionTimeOut());
     ZooCache zcache = new ZooCache(zreader, null);
@@ -165,7 +165,7 @@ public class RestartIT extends AccumuloClusterIT {
 
     cluster.start();
     UtilWaitThread.sleep(5);
-    control.stopAllServers(ServerType.MASTER);
+    control.stopAllServers(ClusterServerType.MASTER);
 
     masterLockData = new byte[0];
     do {
@@ -202,7 +202,7 @@ public class RestartIT extends AccumuloClusterIT {
       }
     });
 
-    control.stopAllServers(ServerType.MASTER);
+    control.stopAllServers(ClusterServerType.MASTER);
 
     ZooReader zreader = new ZooReader(c.getInstance().getZooKeepers(), c.getInstance().getZooKeepersSessionTimeOut());
     ZooCache zcache = new ZooCache(zreader, null);
@@ -229,7 +229,7 @@ public class RestartIT extends AccumuloClusterIT {
     VOPTS.tableName = tableName;
     TestIngest.ingest(c, OPTS, BWOPTS);
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
-    cluster.getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+    cluster.getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
     cluster.start();
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
   }
@@ -243,7 +243,7 @@ public class RestartIT extends AccumuloClusterIT {
     c.tableOperations().create(tableName);
     // Original test started and then stopped a GC. Not sure why it did this. The GC was
     // already running by default, and it would have nothing to do after only creating a table
-    control.stopAllServers(ServerType.TABLET_SERVER);
+    control.stopAllServers(ClusterServerType.TABLET_SERVER);
 
     cluster.start();
     c.tableOperations().create(names[1]);
@@ -257,7 +257,7 @@ public class RestartIT extends AccumuloClusterIT {
     OPTS.tableName = tableName;
     TestIngest.ingest(c, OPTS, BWOPTS);
     try {
-      getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
       getCluster().getClusterControl().adminStopAll();
     } finally {
       getCluster().start();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
index 1febcd7..b965420 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -27,13 +27,13 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -121,8 +121,8 @@ public class RestartStressIT extends AccumuloClusterIT {
 
     for (int i = 0; i < 2; i++) {
       UtilWaitThread.sleep(10 * 1000);
-      control.stopAllServers(ServerType.TABLET_SERVER);
-      control.startAllServers(ServerType.TABLET_SERVER);
+      control.stopAllServers(ClusterServerType.TABLET_SERVER);
+      control.startAllServers(ClusterServerType.TABLET_SERVER);
     }
     assertEquals(0, retCode.get().intValue());
     VOPTS.tableName = tableName;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index a2b3811..5242117 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.impl.MasterClient;
@@ -37,7 +38,6 @@ 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.MemoryUnit;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.trace.instrument.Tracer;
@@ -53,7 +53,7 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
     siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
     cfg.setSiteConfig(siteConfig);
-    cfg.setMemory(ServerType.TABLET_SERVER, cfg.getMemory(ServerType.TABLET_SERVER) * 3, MemoryUnit.BYTE);
+    cfg.setMemory(ClusterServerType.TABLET_SERVER, cfg.getMemory(ClusterServerType.TABLET_SERVER) * 3, MemoryUnit.BYTE);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
index 66d80fa..2f010c0 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
@@ -39,7 +40,6 @@ 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.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.CheckForMetadataProblems;
 import org.apache.accumulo.test.TestIngest;
@@ -91,8 +91,8 @@ public class SplitIT extends AccumuloClusterIT {
     boolean restarted = false;
     if (!tservMaxMem.equals("5K")) {
       iops.setProperty(Property.TSERV_MAXMEM.getKey(), "5K");
-      getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getCluster().getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
       restarted = true;
     }
 
@@ -111,8 +111,8 @@ public class SplitIT extends AccumuloClusterIT {
       log.info("Resetting {}={}", Property.TSERV_MAXMEM.getKey(), tservMaxMem);
       getConnector().instanceOperations().setProperty(Property.TSERV_MAXMEM.getKey(), tservMaxMem);
       tservMaxMem = null;
-      getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-      getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+      getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+      getCluster().getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     }
     if (null != tservMajcDelay) {
       log.info("Resetting {}={}", Property.TSERV_MAJC_DELAY.getKey(), tservMajcDelay);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c1d9118b/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index 9a3a9c5..bfca75b 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -19,12 +19,12 @@ package org.apache.accumulo.test.functional;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.accumulo.cluster.ClusterServerType;
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
@@ -63,8 +63,8 @@ public class WriteAheadLogIT extends AccumuloClusterIT {
     VerifyIngest.Opts vopts = new VerifyIngest.Opts();
     vopts.tableName = tableName;
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
-    getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-    getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().stopAllServers(ClusterServerType.TABLET_SERVER);
+    getCluster().getClusterControl().startAllServers(ClusterServerType.TABLET_SERVER);
     VerifyIngest.verifyIngest(c, vopts, new ScannerOpts());
   }