You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2023/06/23 19:05:44 UTC

[accumulo] branch elasticity updated: Modified MiniAccumuloCluster configuration to include cluster config (#3533)

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

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


The following commit(s) were added to refs/heads/elasticity by this push:
     new d58312fcff Modified MiniAccumuloCluster configuration to include cluster config (#3533)
d58312fcff is described below

commit d58312fcffd36e0beaccd81668dddf31b2cceca8
Author: Dave Marion <dl...@apache.org>
AuthorDate: Fri Jun 23 15:05:37 2023 -0400

    Modified MiniAccumuloCluster configuration to include cluster config (#3533)
    
    Created an object used by MiniAccumuloConfigImpl that contains the
    number of Compactions, ScanServers, and TabletServers for the default
    resource group. This object allows test writers to change the defaults
    and add other resource groups for their test. When MAC starts, it starts
    all of the Compactors, ScanServers, and TabletServers for all of the groups.
    Resource groups can be added to the configuration in the middle of a test
    and then calling start(ServerType) will start up the new processes.
---
 .../org/apache/accumulo/core/conf/Property.java    |   2 +-
 .../hadoop/its/mapred/AccumuloOutputFormatIT.java  |   2 +-
 .../its/mapreduce/AccumuloInputFormatIT.java       |   2 +-
 .../apache/accumulo/cluster/ClusterControl.java    |  11 -
 .../standalone/StandaloneClusterControl.java       |   6 -
 .../accumulo/minicluster/MiniAccumuloConfig.java   |  10 +-
 .../ClusterServerConfiguration.java                | 104 +++++++++
 .../MiniAccumuloClusterControl.java                | 240 +++++++++++++--------
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |  48 +++--
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  63 +-----
 .../minicluster/MiniAccumuloClusterTest.java       |   2 +-
 .../MiniAccumuloClusterImplTest.java               |   2 +-
 .../org/apache/accumulo/test/AuditMessageIT.java   |   2 +-
 .../accumulo/test/BadDeleteMarkersCreatedIT.java   |   2 +-
 .../accumulo/test/BulkImportSequentialRowsIT.java  |   2 +-
 .../java/org/apache/accumulo/test/CleanWalIT.java  |   2 +-
 .../accumulo/test/CountNameNodeOpsBulkIT.java      |   2 +-
 .../apache/accumulo/test/GarbageCollectWALIT.java  |   2 +-
 .../accumulo/test/InterruptibleScannersIT.java     |   2 +-
 .../org/apache/accumulo/test/IteratorEnvIT.java    |   2 +-
 .../org/apache/accumulo/test/LargeSplitRowIT.java  |   2 +-
 .../apache/accumulo/test/MetaGetsReadersIT.java    |   2 +-
 .../test/MissingWalHeaderCompletesRecoveryIT.java  |   2 +-
 .../test/RecoveryCompactionsAreFlushesIT.java      |   2 +-
 .../test/ScanServerConcurrentTabletScanIT.java     |   4 +-
 .../org/apache/accumulo/test/ScanServerIT.java     |   2 +-
 .../accumulo/test/ScanServerIT_NoServers.java      |   2 +-
 .../accumulo/test/ScanServerMetadataEntriesIT.java |   2 +-
 .../accumulo/test/ScanServerMultipleScansIT.java   |   2 +-
 .../accumulo/test/TabletServerGivesUpIT.java       |   2 +-
 .../accumulo/test/TabletServerHdfsRestartIT.java   |   2 +-
 .../org/apache/accumulo/test/TotalQueuedIT.java    |   2 +-
 .../java/org/apache/accumulo/test/UnusedWALIT.java |   2 +-
 .../accumulo/test/VerifySerialRecoveryIT.java      |   2 +-
 .../org/apache/accumulo/test/VolumeChooserIT.java  |   2 +-
 .../org/apache/accumulo/test/YieldScannersIT.java  |   2 +-
 .../compaction/ExternalCompactionMetricsIT.java    |  10 +-
 .../compaction/ExternalCompactionProgressIT.java   |   4 +-
 .../compaction/ExternalCompactionTestUtils.java    |   1 +
 .../test/compaction/ExternalCompaction_1_IT.java   |  32 +--
 .../test/compaction/ExternalCompaction_2_IT.java   |  23 +-
 .../test/compaction/ExternalCompaction_3_IT.java   |  10 +-
 .../BalanceInPresenceOfOfflineTableIT.java         |   6 +-
 .../accumulo/test/functional/BloomFilterIT.java    |   2 +-
 .../apache/accumulo/test/functional/BulkNewIT.java |   3 -
 .../accumulo/test/functional/CleanTmpIT.java       |   2 +-
 .../test/functional/ConfigurableMacBase.java       |   5 +
 .../accumulo/test/functional/DurabilityIT.java     |   2 +-
 .../test/functional/HalfDeadTServerIT.java         |   2 +-
 .../functional/IteratorMincClassCastBugIT.java     |   2 +-
 .../accumulo/test/functional/KerberosIT.java       |   2 +-
 .../test/functional/KerberosRenewalIT.java         |   2 +-
 .../test/functional/ManagerAssignmentIT.java       |   2 +-
 .../test/functional/ManyWriteAheadLogsIT.java      |   2 +-
 .../test/functional/MemoryStarvedMajCIT.java       |   2 +-
 .../test/functional/MemoryStarvedMinCIT.java       |   2 +-
 .../test/functional/MemoryStarvedScanIT.java       |   2 +-
 .../accumulo/test/functional/MetadataIT.java       |   2 +-
 .../test/functional/OnDemandTabletUnloadingIT.java |   4 +-
 .../test/functional/RegexGroupBalanceIT.java       |   2 +-
 .../test/functional/SessionBlockVerifyIT.java      |   2 +-
 .../test/functional/SessionDurabilityIT.java       |   2 +-
 .../accumulo/test/functional/TabletMetadataIT.java |   2 +-
 .../accumulo/test/functional/WALSunnyDayIT.java    |   2 +-
 .../test/functional/WatchTheWatchCountIT.java      |   2 +-
 .../accumulo/test/manager/SuspendedTabletsIT.java  |   4 +-
 .../apache/accumulo/test/metrics/MetricsIT.java    |   1 -
 .../accumulo/test/shell/ShellCreateTableIT.java    |   2 +-
 .../apache/accumulo/test/shell/ShellServerIT.java  |   2 +-
 69 files changed, 415 insertions(+), 272 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index d163c5bf59..06bc8383c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -1075,7 +1075,7 @@ public enum Property {
   COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
       "Properties in this category affect the behavior of the accumulo compactor server.", "2.1.0"),
   @Experimental
-  COMPACTOR_PORTSEARCH("compactor.port.search", "false", PropertyType.BOOLEAN,
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
       "If the compactor.port.client is in use, search higher ports until one is available",
       "2.1.0"),
   @Experimental
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
index 39e450e154..de1ea9660f 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloOutputFormatIT.java
@@ -64,7 +64,7 @@ public class AccumuloOutputFormatIT extends ConfigurableMacBase {
   @Override
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "1");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   // Prevent regression of ACCUMULO-3709.
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
index f209fc0a7c..c5b3d82ac4 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
@@ -89,7 +89,7 @@ public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   @BeforeEach
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 be8d358214..a1e14cfc39 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/ClusterControl.java
@@ -21,7 +21,6 @@ package org.apache.accumulo.cluster;
 import java.io.IOException;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.minicluster.ServerType;
 
 /**
@@ -47,16 +46,6 @@ public interface ClusterControl {
    */
   void adminStopAll() throws IOException;
 
-  /**
-   * Start instances of Compactors
-   *
-   * @param compactor compactor class
-   * @param limit number of compactors to start
-   * @param queueName name of queue
-   */
-  void startCompactors(Class<? extends Compactor> compactor, int limit, String queueName)
-      throws IOException;
-
   /**
    * Starts all occurrences of the given server
    */
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 e503e779b0..4443f7f31c 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
@@ -34,7 +34,6 @@ import java.util.Map.Entry;
 import org.apache.accumulo.cluster.ClusterControl;
 import org.apache.accumulo.cluster.RemoteShell;
 import org.apache.accumulo.cluster.RemoteShellOptions;
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.manager.state.SetGoalState;
 import org.apache.accumulo.minicluster.ServerType;
@@ -368,9 +367,4 @@ public class StandaloneClusterControl implements ClusterControl {
     }
   }
 
-  @Override
-  public void startCompactors(Class<? extends Compactor> compactor, int limit, String queueName)
-      throws IOException {
-    throw new UnsupportedOperationException("Not yet implemented.");
-  }
 }
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 2945eb7650..6446dfd44c 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -56,8 +56,9 @@ public class MiniAccumuloConfig {
    *
    * @param numTservers the number of tablet servers that mini accumulo cluster should start
    */
+  // ELASTICITY_TODO: Deprecate in 3.0.0 and remove in elasticity on the merge
   public MiniAccumuloConfig setNumTservers(int numTservers) {
-    impl.setNumTservers(numTservers);
+    // impl.setNumTservers(numTservers);
     return this;
   }
 
@@ -67,8 +68,9 @@ public class MiniAccumuloConfig {
    * @param numScanServers the number of scan servers that mini accumulo cluster should start
    * @since 2.1.0
    */
+  // ELASTICITY_TODO: Deprecate in 3.0.0 and remove in elasticity on the merge
   public MiniAccumuloConfig setNumScanServers(int numScanServers) {
-    impl.setNumScanServers(numScanServers);
+    // impl.setNumScanServers(numScanServers);
     return this;
   }
 
@@ -224,8 +226,10 @@ public class MiniAccumuloConfig {
   /**
    * @return the number of tservers configured for this cluster
    */
+  // ELASTICITY_TODO: Deprecate in 3.0.0 and remove in elasticity on the merge
   public int getNumTservers() {
-    return impl.getNumTservers();
+    return impl.getClusterServerConfiguration().getTabletServerConfiguration().values().stream()
+        .reduce(0, Integer::sum);
   }
 
   /**
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/ClusterServerConfiguration.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/ClusterServerConfiguration.java
new file mode 100644
index 0000000000..05e37b5584
--- /dev/null
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/ClusterServerConfiguration.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.miniclusterImpl;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.accumulo.core.lock.ServiceLockData;
+
+public class ClusterServerConfiguration {
+
+  private final Map<String,Integer> compactors;
+  private final Map<String,Integer> sservers;
+  private final Map<String,Integer> tservers;
+
+  /**
+   * Creates the default configuration with 1 each of Compactor and ScanServer and 2 TabletServers
+   * in the default resource group
+   */
+  public ClusterServerConfiguration() {
+    this(1, 1, 2);
+  }
+
+  /**
+   * Creates the default Configuration using the parameters
+   *
+   * @param numCompactors number of compactors in the default resource group
+   * @param numSServers number of scan servers in the default resource group
+   * @param numTServers number of tablet servers in the default resource group
+   */
+  public ClusterServerConfiguration(int numCompactors, int numSServers, int numTServers) {
+    compactors = new HashMap<>();
+    compactors.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numCompactors);
+    sservers = new HashMap<>();
+    sservers.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numSServers);
+    tservers = new HashMap<>();
+    tservers.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numTServers);
+  }
+
+  public void setNumDefaultCompactors(int numCompactors) {
+    compactors.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numCompactors);
+  }
+
+  public void setNumDefaultScanServers(int numSServers) {
+    sservers.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numSServers);
+  }
+
+  public void setNumDefaultTabletServers(int numTServers) {
+    tservers.put(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME, numTServers);
+  }
+
+  public void addCompactorResourceGroup(String resourceGroupName, int numCompactors) {
+    compactors.put(resourceGroupName, numCompactors);
+  }
+
+  public void addScanServerResourceGroup(String resourceGroupName, int numScanServers) {
+    sservers.put(resourceGroupName, numScanServers);
+  }
+
+  public void addTabletServerResourceGroup(String resourceGroupName, int numTabletServers) {
+    tservers.put(resourceGroupName, numTabletServers);
+  }
+
+  public Map<String,Integer> getCompactorConfiguration() {
+    return Collections.unmodifiableMap(compactors);
+  }
+
+  public Map<String,Integer> getScanServerConfiguration() {
+    return Collections.unmodifiableMap(sservers);
+  }
+
+  public Map<String,Integer> getTabletServerConfiguration() {
+    return Collections.unmodifiableMap(tservers);
+  }
+
+  public void clearCompactorResourceGroups() {
+    Iterator<String> iter = compactors.keySet().iterator();
+    while (iter.hasNext()) {
+      String resourceGroup = iter.next();
+      if (!resourceGroup.equals(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME)) {
+        compactors.remove(resourceGroup);
+      }
+    }
+  }
+
+}
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
index 71f8b55101..cfac24841c 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
@@ -23,6 +23,8 @@ import static java.util.Objects.requireNonNull;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -58,9 +60,9 @@ public class MiniAccumuloClusterControl implements ClusterControl {
   Process managerProcess = null;
   Process gcProcess = null;
   Process monitor = null;
-  final List<Process> tabletServerProcesses = new ArrayList<>();
-  final List<Process> scanServerProcesses = new ArrayList<>();
-  final List<Process> compactorProcesses = new ArrayList<>();
+  final Map<String,List<Process>> tabletServerProcesses = new HashMap<>();
+  final Map<String,List<Process>> scanServerProcesses = new HashMap<>();
+  final Map<String,List<Process>> compactorProcesses = new HashMap<>();
 
   public MiniAccumuloClusterControl(MiniAccumuloClusterImpl cluster) {
     requireNonNull(cluster);
@@ -116,20 +118,6 @@ public class MiniAccumuloClusterControl implements ClusterControl {
     }
   }
 
-  @Override
-  public synchronized void startCompactors(Class<? extends Compactor> compactor, int limit,
-      String queueName) throws IOException {
-    synchronized (compactorProcesses) {
-      int count =
-          Math.min(limit, cluster.getConfig().getNumCompactors() - compactorProcesses.size());
-      for (int i = 0; i < count; i++) {
-        compactorProcesses.add(
-            cluster.exec(compactor, "-o", Property.COMPACTOR_QUEUE_NAME.getKey() + "=" + queueName)
-                .getProcess());
-      }
-    }
-  }
-
   @Override
   public synchronized void startAllServers(ServerType server) throws IOException {
     start(server, null);
@@ -137,61 +125,117 @@ public class MiniAccumuloClusterControl implements ClusterControl {
 
   @Override
   public synchronized void start(ServerType server, String hostname) throws IOException {
-    start(server, Collections.emptyMap(), Integer.MAX_VALUE);
+    start(server, Collections.emptyMap(), Integer.MAX_VALUE, null);
   }
 
   public synchronized void start(ServerType server, Map<String,String> configOverrides, int limit)
       throws IOException {
+    start(server, configOverrides, limit, null);
+  }
+
+  public synchronized void start(ServerType server, Map<String,String> configOverrides, int limit,
+      Class<?> classOverride) throws IOException {
     if (limit <= 0) {
       return;
     }
 
+    Class<?> classToUse;
+    if (classOverride != null) {
+      classToUse = classOverride;
+    } else {
+      switch (server) {
+        case COMPACTOR:
+          classToUse = Compactor.class;
+          break;
+        case SCAN_SERVER:
+          classToUse = ScanServer.class;
+          break;
+        case TABLET_SERVER:
+          classToUse = TabletServer.class;
+          break;
+        case GARBAGE_COLLECTOR:
+          classToUse = SimpleGarbageCollector.class;
+          break;
+        case MANAGER:
+          classToUse = Manager.class;
+          break;
+        case MONITOR:
+          classToUse = Monitor.class;
+          break;
+        case ZOOKEEPER:
+          classToUse = ZooKeeperServerMain.class;
+          break;
+        default:
+          throw new IllegalArgumentException("Unhandled server type: " + server);
+      }
+    }
+
     switch (server) {
       case TABLET_SERVER:
         synchronized (tabletServerProcesses) {
-          int count = 0;
-          for (int i = tabletServerProcesses.size();
-              count < limit && i < cluster.getConfig().getNumTservers(); i++, ++count) {
-            tabletServerProcesses
-                .add(cluster._exec(TabletServer.class, server, configOverrides).getProcess());
+          Map<String,Integer> tserverGroups =
+              cluster.getConfig().getClusterServerConfiguration().getTabletServerConfiguration();
+          for (Entry<String,Integer> e : tserverGroups.entrySet()) {
+            List<Process> processes =
+                tabletServerProcesses.computeIfAbsent(e.getKey(), k -> new ArrayList<>());
+            int count = 0;
+            for (int i = processes.size(); count < limit && i < e.getValue(); i++, ++count) {
+              processes.add(cluster._exec(classToUse, server, configOverrides).getProcess());
+            }
           }
         }
         break;
       case MANAGER:
         if (managerProcess == null) {
-          managerProcess = cluster._exec(Manager.class, server, configOverrides).getProcess();
+          managerProcess = cluster._exec(classToUse, server, configOverrides).getProcess();
         }
         break;
       case ZOOKEEPER:
         if (zooKeeperProcess == null) {
-          zooKeeperProcess = cluster._exec(ZooKeeperServerMain.class, server, configOverrides,
-              cluster.getZooCfgFile().getAbsolutePath()).getProcess();
+          zooKeeperProcess = cluster
+              ._exec(classToUse, server, configOverrides, cluster.getZooCfgFile().getAbsolutePath())
+              .getProcess();
         }
         break;
       case GARBAGE_COLLECTOR:
         if (gcProcess == null) {
-          gcProcess =
-              cluster._exec(SimpleGarbageCollector.class, server, configOverrides).getProcess();
+          gcProcess = cluster._exec(classToUse, server, configOverrides).getProcess();
         }
         break;
       case MONITOR:
         if (monitor == null) {
-          monitor = cluster._exec(Monitor.class, server, configOverrides).getProcess();
+          monitor = cluster._exec(classToUse, server, configOverrides).getProcess();
         }
         break;
       case SCAN_SERVER:
         synchronized (scanServerProcesses) {
-          int count = 0;
-          for (int i = scanServerProcesses.size();
-              count < limit && i < cluster.getConfig().getNumScanServers(); i++, ++count) {
-            scanServerProcesses
-                .add(cluster._exec(ScanServer.class, server, configOverrides).getProcess());
+          Map<String,Integer> sserverGroups =
+              cluster.getConfig().getClusterServerConfiguration().getScanServerConfiguration();
+          for (Entry<String,Integer> e : sserverGroups.entrySet()) {
+            List<Process> processes =
+                scanServerProcesses.computeIfAbsent(e.getKey(), k -> new ArrayList<>());
+            int count = 0;
+            for (int i = processes.size(); count < limit && i < e.getValue(); i++, ++count) {
+              processes.add(cluster._exec(classToUse, server, configOverrides).getProcess());
+            }
           }
         }
         break;
       case COMPACTOR:
-        startCompactors(Compactor.class, cluster.getConfig().getNumCompactors(),
-            configOverrides.get("QUEUE_NAME"));
+        synchronized (compactorProcesses) {
+          Map<String,Integer> compactorGroups =
+              cluster.getConfig().getClusterServerConfiguration().getCompactorConfiguration();
+          for (Entry<String,Integer> e : compactorGroups.entrySet()) {
+            List<Process> processes =
+                compactorProcesses.computeIfAbsent(e.getKey(), k -> new ArrayList<>());
+            int count = 0;
+            for (int i = processes.size(); count < limit && i < e.getValue(); i++, ++count) {
+              processes.add(cluster
+                  .exec(classToUse, "-o", Property.COMPACTOR_QUEUE_NAME.getKey() + "=" + e.getKey())
+                  .getProcess());
+            }
+          }
+        }
         break;
       default:
         throw new UnsupportedOperationException("Cannot start process for " + server);
@@ -252,15 +296,17 @@ public class MiniAccumuloClusterControl implements ClusterControl {
       case TABLET_SERVER:
         synchronized (tabletServerProcesses) {
           try {
-            for (Process tserver : tabletServerProcesses) {
-              try {
-                cluster.stopProcessWithTimeout(tserver, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("TabletServer did not fully stop after 30 seconds", e);
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-              }
-            }
+            tabletServerProcesses.values().forEach(list -> {
+              list.forEach(process -> {
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                } catch (InterruptedException e) {
+                  Thread.currentThread().interrupt();
+                }
+              });
+            });
           } finally {
             tabletServerProcesses.clear();
           }
@@ -282,15 +328,17 @@ public class MiniAccumuloClusterControl implements ClusterControl {
       case SCAN_SERVER:
         synchronized (scanServerProcesses) {
           try {
-            for (Process sserver : scanServerProcesses) {
-              try {
-                cluster.stopProcessWithTimeout(sserver, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("ScanServer did not fully stop after 30 seconds", e);
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-              }
-            }
+            scanServerProcesses.values().forEach(list -> {
+              list.forEach(process -> {
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                } catch (InterruptedException e) {
+                  Thread.currentThread().interrupt();
+                }
+              });
+            });
           } finally {
             scanServerProcesses.clear();
           }
@@ -299,15 +347,17 @@ public class MiniAccumuloClusterControl implements ClusterControl {
       case COMPACTOR:
         synchronized (compactorProcesses) {
           try {
-            for (Process compactor : compactorProcesses) {
-              try {
-                cluster.stopProcessWithTimeout(compactor, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("Compactor did not fully stop after 30 seconds", e);
-              } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-              }
-            }
+            compactorProcesses.values().forEach(list -> {
+              list.forEach(process -> {
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                } catch (InterruptedException e) {
+                  Thread.currentThread().interrupt();
+                }
+              });
+            });
           } finally {
             compactorProcesses.clear();
           }
@@ -351,16 +401,20 @@ public class MiniAccumuloClusterControl implements ClusterControl {
         break;
       case TABLET_SERVER:
         synchronized (tabletServerProcesses) {
-          for (Process tserver : tabletServerProcesses) {
-            if (procRef.getProcess().equals(tserver)) {
-              tabletServerProcesses.remove(tserver);
-              try {
-                cluster.stopProcessWithTimeout(tserver, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("TabletServer did not fully stop after 30 seconds", e);
+          for (List<Process> plist : tabletServerProcesses.values()) {
+            Iterator<Process> iter = plist.iterator();
+            while (!found && iter.hasNext()) {
+              Process process = iter.next();
+              if (procRef.getProcess().equals(process)) {
+                iter.remove();
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                }
+                found = true;
+                break;
               }
-              found = true;
-              break;
             }
           }
         }
@@ -389,29 +443,39 @@ public class MiniAccumuloClusterControl implements ClusterControl {
         break;
       case SCAN_SERVER:
         synchronized (scanServerProcesses) {
-          for (Process sserver : scanServerProcesses) {
-            if (procRef.getProcess().equals(sserver)) {
-              scanServerProcesses.remove(sserver);
-              try {
-                cluster.stopProcessWithTimeout(sserver, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("ScanServer did not fully stop after 30 seconds", e);
+          for (List<Process> plist : scanServerProcesses.values()) {
+            Iterator<Process> iter = plist.iterator();
+            while (!found && iter.hasNext()) {
+              Process process = iter.next();
+              if (procRef.getProcess().equals(process)) {
+                iter.remove();
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                }
+                found = true;
+                break;
               }
-              found = true;
-              break;
             }
           }
         }
         break;
       case COMPACTOR:
         synchronized (compactorProcesses) {
-          for (Process compactor : compactorProcesses) {
-            if (procRef.getProcess().equals(compactor)) {
-              compactorProcesses.remove(compactor);
-              try {
-                cluster.stopProcessWithTimeout(compactor, 30, TimeUnit.SECONDS);
-              } catch (ExecutionException | TimeoutException e) {
-                log.warn("Compactor did not fully stop after 30 seconds", e);
+          for (List<Process> plist : compactorProcesses.values()) {
+            Iterator<Process> iter = plist.iterator();
+            while (!found && iter.hasNext()) {
+              Process process = iter.next();
+              if (procRef.getProcess().equals(process)) {
+                iter.remove();
+                try {
+                  cluster.stopProcessWithTimeout(process, 30, TimeUnit.SECONDS);
+                } catch (ExecutionException | TimeoutException e) {
+                  log.warn("TabletServer did not fully stop after 30 seconds", e);
+                }
+                found = true;
+                break;
               }
             }
           }
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 25edd7cd8b..6c5e0d82d0 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -62,7 +62,6 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.client.Accumulo;
@@ -602,6 +601,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         config.getZooKeepers());
 
     control.start(ServerType.TABLET_SERVER);
+    control.start(ServerType.SCAN_SERVER);
 
     int ret = 0;
     for (int i = 0; i < 5; i++) {
@@ -631,11 +631,12 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         throw new IllegalStateException("No Compactor queues configured.");
       }
       for (String name : queues) {
-        control.startCompactors(Compactor.class, getConfig().getNumCompactors(), name);
+        config.getClusterServerConfiguration().addCompactorResourceGroup(name, 1);
       }
     } catch (ClassNotFoundException e) {
       throw new IllegalArgumentException("Unable to find declared CompactionPlanner class", e);
     }
+    control.start(ServerType.COMPACTOR);
 
     verifyUp();
 
@@ -708,17 +709,30 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     waitForProcessStart(getClusterControl().gcProcess, "GC");
 
     int tsExpectedCount = 0;
-    for (Process tsp : getClusterControl().tabletServerProcesses) {
-      tsExpectedCount++;
-      requireNonNull(tsp, "Error starting TabletServer " + tsExpectedCount + " - no process");
-      waitForProcessStart(tsp, "TabletServer" + tsExpectedCount);
+    for (List<Process> tabletServerProcesses : getClusterControl().tabletServerProcesses.values()) {
+      for (Process tsp : tabletServerProcesses) {
+        tsExpectedCount++;
+        requireNonNull(tsp, "Error starting TabletServer " + tsExpectedCount + " - no process");
+        waitForProcessStart(tsp, "TabletServer" + tsExpectedCount);
+      }
+    }
+
+    int ssExpectedCount = 0;
+    for (List<Process> scanServerProcesses : getClusterControl().scanServerProcesses.values()) {
+      for (Process tsp : scanServerProcesses) {
+        ssExpectedCount++;
+        requireNonNull(tsp, "Error starting ScanServer " + ssExpectedCount + " - no process");
+        waitForProcessStart(tsp, "ScanServer" + ssExpectedCount);
+      }
     }
 
     int ecExpectedCount = 0;
-    for (Process ecp : getClusterControl().compactorProcesses) {
-      ecExpectedCount++;
-      requireNonNull(ecp, "Error starting compactor " + ecExpectedCount + " - no process");
-      waitForProcessStart(ecp, "Compactor" + ecExpectedCount);
+    for (List<Process> compactorProcesses : getClusterControl().compactorProcesses.values()) {
+      for (Process ecp : compactorProcesses) {
+        ecExpectedCount++;
+        requireNonNull(ecp, "Error starting compactor " + ecExpectedCount + " - no process");
+        waitForProcessStart(ecp, "Compactor" + ecExpectedCount);
+      }
     }
 
     try (ZooKeeper zk = new ZooKeeper(getZooKeepers(), 60000, event -> log.warn("{}", event))) {
@@ -785,7 +799,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       String rootPath = Constants.ZROOT + "/" + instanceId;
       int tsActualCount = 0;
       try {
-        while (tsActualCount < tsExpectedCount) {
+        while (tsActualCount < ssExpectedCount) {
           tsActualCount = 0;
           for (String child : zk.getChildren(rootPath + Constants.ZTSERVERS, null)) {
             if (zk.getChildren(rootPath + Constants.ZTSERVERS + "/" + child, null).isEmpty()) {
@@ -861,13 +875,13 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     Map<ServerType,Collection<ProcessReference>> result = new HashMap<>();
     MiniAccumuloClusterControl control = getClusterControl();
     result.put(ServerType.MANAGER, references(control.managerProcess));
-    result.put(ServerType.TABLET_SERVER,
-        references(control.tabletServerProcesses.toArray(new Process[0])));
-    result.put(ServerType.COMPACTOR,
-        references(control.compactorProcesses.toArray(new Process[0])));
+    result.put(ServerType.TABLET_SERVER, references(control.tabletServerProcesses.values().stream()
+        .flatMap(List::stream).collect(Collectors.toList()).toArray(new Process[0])));
+    result.put(ServerType.COMPACTOR, references(control.compactorProcesses.values().stream()
+        .flatMap(List::stream).collect(Collectors.toList()).toArray(new Process[0])));
     if (control.scanServerProcesses != null) {
-      result.put(ServerType.SCAN_SERVER,
-          references(control.scanServerProcesses.toArray(new Process[0])));
+      result.put(ServerType.SCAN_SERVER, references(control.scanServerProcesses.values().stream()
+          .flatMap(List::stream).collect(Collectors.toList()).toArray(new Process[0])));
     }
     if (control.zooKeeperProcess != null) {
       result.put(ServerType.ZOOKEEPER, references(control.zooKeeperProcess));
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index ee5a0c4724..a287ec7a6b 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -58,9 +58,6 @@ public class MiniAccumuloConfigImpl {
   private Map<String,String> siteConfig = new HashMap<>();
   private Map<String,String> configuredSiteConig = new HashMap<>();
   private Map<String,String> clientProps = new HashMap<>();
-  private int numTservers = 2;
-  private int numScanServers = 0;
-  private int numCompactors = 1;
   private Map<ServerType,Long> memoryConfig = new HashMap<>();
   private boolean jdwpEnabled = false;
   private Map<String,String> systemProperties = new HashMap<>();
@@ -102,6 +99,8 @@ public class MiniAccumuloConfigImpl {
 
   private Consumer<MiniAccumuloConfigImpl> preStartConfigProcessor;
 
+  private final ClusterServerConfiguration serverConfiguration;
+
   /**
    * @param dir An empty or nonexistent 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
@@ -111,6 +110,7 @@ public class MiniAccumuloConfigImpl {
   public MiniAccumuloConfigImpl(File dir, String rootPassword) {
     this.dir = dir;
     this.rootPassword = rootPassword;
+    this.serverConfiguration = new ClusterServerConfiguration();
   }
 
   /**
@@ -250,32 +250,6 @@ public class MiniAccumuloConfigImpl {
     }
   }
 
-  /**
-   * 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, it defaults to two.
-   *
-   * @param numScanServers the number of tablet servers that mini accumulo cluster should start
-   */
-  public MiniAccumuloConfigImpl setNumScanServers(int numScanServers) {
-    if (numScanServers < 0) {
-      throw new IllegalArgumentException("Must have zero or more scan servers");
-    }
-    this.numScanServers = numScanServers;
-    return this;
-  }
-
   /**
    * Calling this method is optional. If not set, defaults to 'miniInstance'
    *
@@ -524,17 +498,10 @@ public class MiniAccumuloConfigImpl {
   }
 
   /**
-   * @return the number of tservers configured for this cluster
+   * @return ClusterServerConfiguration
    */
-  public int getNumTservers() {
-    return numTservers;
-  }
-
-  /**
-   * @return the number of scan servers configured for this cluster
-   */
-  public int getNumScanServers() {
-    return numScanServers;
+  public ClusterServerConfiguration getClusterServerConfiguration() {
+    return serverConfiguration;
   }
 
   /**
@@ -783,24 +750,6 @@ public class MiniAccumuloConfigImpl {
     this.rootUserName = rootUserName;
   }
 
-  /**
-   * @return number of Compactors
-   * @since 2.1.0
-   */
-  public int getNumCompactors() {
-    return numCompactors;
-  }
-
-  /**
-   * Set number of Compactors
-   *
-   * @param numCompactors number of compactors
-   * @since 2.1.0
-   */
-  public void setNumCompactors(int numCompactors) {
-    this.numCompactors = numCompactors;
-  }
-
   /**
    * Set the object that will be used to modify the site configuration right before it's written out
    * a file. This would be useful in the case where the configuration needs to be updated based on a
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 91f5b884c5..e1229c418f 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -187,7 +187,7 @@ public class MiniAccumuloClusterTest extends WithTestNames {
   public void testDebugPorts() {
 
     Set<Pair<ServerType,Integer>> debugPorts = accumulo.getDebugPorts();
-    assertEquals(5, debugPorts.size());
+    assertEquals(6, debugPorts.size());
     for (Pair<ServerType,Integer> debugPort : debugPorts) {
       assertTrue(debugPort.getSecond() > 0);
     }
diff --git a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
index 5ec9e6850f..dcca5af1ae 100644
--- a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
@@ -71,7 +71,7 @@ public class MiniAccumuloClusterImplTest {
     MiniAccumuloConfigImpl config =
         new MiniAccumuloConfigImpl(testDir, "superSecret").setJDWPEnabled(true);
     // expressly set number of tservers since we assert it later, in case the default changes
-    config.setNumTservers(NUM_TSERVERS);
+    config.getClusterServerConfiguration().setNumDefaultTabletServers(NUM_TSERVERS);
     accumulo = new MiniAccumuloClusterImpl(config);
     accumulo.start();
     // create a table to ensure there are some entries in the !0 table
diff --git a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
index 2c1afb7667..cb88399c98 100644
--- a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -91,7 +91,7 @@ public class AuditMessageIT extends ConfigurableMacBase {
 
   @Override
   public void beforeClusterStart(MiniAccumuloConfigImpl cfg) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   // Must be static to survive Junit re-initialising the class every time.
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index f5f02e839b..d067e47fdc 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -68,7 +68,7 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
     cfg.setProperty(Property.GC_CYCLE_START, "0s");
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java b/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
index 0a70abf10e..2db11f8d60 100644
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BulkImportSequentialRowsIT.java
@@ -57,7 +57,7 @@ public class BulkImportSequentialRowsIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // Need more than one tserver
-    cfg.setNumTservers(2);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(2);
 
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
index 2a20c5bbc7..d1b328178d 100644
--- a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
@@ -62,7 +62,7 @@ public class CleanWalIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
index 4efdc52d22..ea4213270f 100644
--- a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
@@ -68,7 +68,7 @@ public class CountNameNodeOpsBulkIT extends ConfigurableMacBase {
 
   @Override
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.useMiniDFS(true);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
index 2fe0862aef..45bd1a5f7a 100644
--- a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
@@ -53,7 +53,7 @@ public class GarbageCollectWALIT extends ConfigurableMacBase {
     cfg.setProperty(Property.INSTANCE_ZK_HOST, "5s");
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/InterruptibleScannersIT.java b/test/src/main/java/org/apache/accumulo/test/InterruptibleScannersIT.java
index c2f2fc09e1..f93014febf 100644
--- a/test/src/main/java/org/apache/accumulo/test/InterruptibleScannersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/InterruptibleScannersIT.java
@@ -48,7 +48,7 @@ public class InterruptibleScannersIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java b/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
index ea7432ba67..00b5224ddc 100644
--- a/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
@@ -62,7 +62,7 @@ public class IteratorEnvIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   private AccumuloClient client;
diff --git a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
index ea97f8263e..2681f8f2d8 100644
--- a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java
@@ -66,7 +66,7 @@ public class LargeSplitRowIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
 
     Map<String,String> siteConfig = Map.of(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
     cfg.setSiteConfig(siteConfig);
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
index 34ff974bf5..6e4642e766 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
@@ -51,7 +51,7 @@ public class MetaGetsReadersIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.TSERV_SCAN_MAX_OPENFILES, "2");
     cfg.setProperty(Property.TABLE_BLOCKCACHE_ENABLED, "false");
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 87e931b637..50c8bab80a 100644
--- a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -72,7 +72,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration conf) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
     // Make sure the GC doesn't delete the file before the metadata reference is added
     cfg.setProperty(Property.GC_CYCLE_START, "999999s");
diff --git a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
index 93375d8e51..7657b0c260 100644
--- a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
@@ -57,7 +57,7 @@ public class RecoveryCompactionsAreFlushesIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     // file system supports recovery
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java
index beea10ba26..0a9009a8df 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java
@@ -63,7 +63,7 @@ public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
         org.apache.hadoop.conf.Configuration coreSite) {
-      cfg.setNumScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(1);
       cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
       cfg.setProperty(Property.SSERV_MINTHREADS, "4");
     }
@@ -94,7 +94,7 @@ public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
     overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
         cacheEnabled ? "300m" : "0m");
     SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
-        1);
+        1, null);
     while (zrw.getChildren(scanServerRoot).size() == 0) {
       Thread.sleep(500);
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
index b68f311871..7eef3d59dd 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java
@@ -77,7 +77,7 @@ public class ScanServerIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
         org.apache.hadoop.conf.Configuration coreSite) {
-      cfg.setNumScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(1);
 
       // Timeout scan sessions after being idle for 3 seconds
       cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
index 3e3bd7a183..c6aa4d64a2 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java
@@ -63,7 +63,7 @@ public class ScanServerIT_NoServers extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
         org.apache.hadoop.conf.Configuration coreSite) {
-      cfg.setNumScanServers(0);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(0);
 
       // Timeout scan sessions after being idle for 3 seconds
       cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
index 4e1f3dafb4..14567b9e96 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
@@ -79,7 +79,7 @@ public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
         org.apache.hadoop.conf.Configuration coreSite) {
-      cfg.setNumScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(1);
       cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
       cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java
index 6283037c80..22dd9c52a7 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java
@@ -75,7 +75,7 @@ public class ScanServerMultipleScansIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
         org.apache.hadoop.conf.Configuration coreSite) {
-      cfg.setNumScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(1);
       cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
index a6bb86ed04..2c023859e7 100644
--- a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
@@ -44,7 +44,7 @@ public class TabletServerGivesUpIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.useMiniDFS(true);
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "10s");
     cfg.setProperty(Property.TSERV_WAL_TOLERATED_CREATION_FAILURES, "10");
     cfg.setProperty(Property.TSERV_WAL_TOLERATED_WAIT_INCREMENT, "0s");
diff --git a/test/src/main/java/org/apache/accumulo/test/TabletServerHdfsRestartIT.java b/test/src/main/java/org/apache/accumulo/test/TabletServerHdfsRestartIT.java
index d274bf0f51..782a9f8bec 100644
--- a/test/src/main/java/org/apache/accumulo/test/TabletServerHdfsRestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TabletServerHdfsRestartIT.java
@@ -47,7 +47,7 @@ public class TabletServerHdfsRestartIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.useMiniDFS(true);
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
index 3bf605403f..b3de6ce48a 100644
--- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
@@ -50,7 +50,7 @@ public class TotalQueuedIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.useMiniDFS();
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
index 4e99e59612..7c90c74505 100644
--- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -63,7 +63,7 @@ public class UnusedWALIT extends ConfigurableMacBase {
     final long logSize = 1024 * 1024 * 10;
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, Long.toString(logSize));
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     hadoopCoreSite.set("fs.namenode.fs-limits.min-block-size", Long.toString(logSize));
diff --git a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
index a8e84ba9f4..e66f778cf9 100644
--- a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
@@ -73,7 +73,7 @@ public class VerifySerialRecoveryIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setProperty(Property.TSERV_ASSIGNMENT_MAXCONCURRENT, "20");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
index a0cba2784c..e134524065 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -96,7 +96,7 @@ public class VolumeChooserIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // Get 2 tablet servers
-    cfg.setNumTservers(2);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(2);
     namespace1 = "ns_" + getUniqueNames(2)[0];
     namespace2 = "ns_" + getUniqueNames(2)[1];
 
diff --git a/test/src/main/java/org/apache/accumulo/test/YieldScannersIT.java b/test/src/main/java/org/apache/accumulo/test/YieldScannersIT.java
index 313c0853c9..2fd8f1859e 100644
--- a/test/src/main/java/org/apache/accumulo/test/YieldScannersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/YieldScannersIT.java
@@ -63,7 +63,7 @@ public class YieldScannersIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
index b7b9162a93..2dd5f16f50 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
@@ -32,7 +32,6 @@ import java.util.Map;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
@@ -62,9 +61,9 @@ public class ExternalCompactionMetricsIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
       ExternalCompactionTestUtils.configureMiniCluster(cfg, coreSite);
-      cfg.setNumCompactors(2);
+      cfg.getClusterServerConfiguration().setNumDefaultCompactors(2);
       // use one tserver so that queue metrics are not spread across tservers
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
 
       // Tell the server processes to use a StatsDMeterRegistry that will be configured
       // to push all metrics to the sink we started.
@@ -142,8 +141,9 @@ public class ExternalCompactionMetricsIT extends SharedMiniClusterBase {
         sawDCQ2_10 |= match(qm, "DCQ2", "10");
       }
 
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE1);
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE2);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE2, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
       boolean sawDCQ1_0 = false;
       boolean sawDCQ2_0 = false;
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
index 3cde93d22c..1946541251 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
@@ -33,7 +33,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -73,6 +72,7 @@ public class ExternalCompactionProgressIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
     ExternalCompactionTestUtils.configureMiniCluster(cfg, coreSite);
+    cfg.getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
   }
 
   @Test
@@ -83,8 +83,6 @@ public class ExternalCompactionProgressIT extends AccumuloClusterHarness {
       createTable(client, table1, "cs1");
       writeData(client, table1, ROWS);
 
-      cluster.getClusterControl().startCompactors(Compactor.class, 1, QUEUE1);
-
       Thread checkerThread = startChecker();
       checkerThread.start();
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
index a401b497b3..c0563c9187 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
@@ -197,6 +197,7 @@ public class ExternalCompactionTestUtils {
     clProps.put(ClientProperty.BATCH_WRITER_LATENCY_MAX.getKey(), "2s");
     cfg.setClientProps(clProps);
 
+    // configure the compaction services to use the queues
     cfg.setProperty("tserver.compaction.major.service.cs1.planner",
         DefaultCompactionPlanner.class.getName());
     cfg.setProperty("tserver.compaction.major.service.cs1.planner.opts.executors",
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
index 081d15ab1f..7bbee23f0c 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
@@ -52,7 +52,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.compactor.ExtCEnv.CompactorIterEnv;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -106,7 +105,6 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
       ExternalCompactionTestUtils.configureMiniCluster(cfg, coreSite);
-      cfg.setNumCompactors(2);
     }
   }
 
@@ -121,6 +119,7 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
   public void tearDown() throws Exception {
     // The ExternalDoNothingCompactor needs to be restarted between tests
     getCluster().getClusterControl().stop(ServerType.COMPACTOR);
+    getCluster().getConfig().getClusterServerConfiguration().clearCompactorResourceGroups();
   }
 
   public static class TestFilter extends Filter {
@@ -180,8 +179,9 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
       writeData(client, table1);
       writeData(client, table2);
 
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE1);
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE2);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE2, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
       compact(client, table1, 2, QUEUE1, true);
       verify(client, table1, 2);
@@ -210,12 +210,16 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
         }
       });
       // Start our TServer that will not commit the compaction
-      ProcessInfo tserverProcess = getCluster().exec(ExternalCompactionTServer.class);
+      getCluster().getClusterControl().start(TABLET_SERVER, null, 1,
+          ExternalCompactionTServer.class);
+      getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
 
       createTable(client, table1, "cs3", 2);
       writeData(client, table1);
 
-      getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE3);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE3, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+          ExternalDoNothingCompactor.class);
 
       compact(client, table1, 2, QUEUE3, false);
       TableId tid = getCluster().getServerContext().getTableId(table1);
@@ -239,7 +243,6 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
       // compaction above in the test. Even though the external compaction was cancelled
       // because we split the table, FaTE will continue to queue up a compaction
       client.tableOperations().cancelCompaction(table1);
-      getCluster().stopProcessWithTimeout(tserverProcess.getProcess(), 30, TimeUnit.SECONDS);
       getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
     } finally {
       // We stopped the TServer and started our own, restart the original TabletServers
@@ -258,7 +261,8 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
 
       writeData(client, table1);
 
-      getCluster().getClusterControl().startCompactors(Compactor.class, 2, QUEUE4);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE4, 2);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
       compact(client, table1, 3, QUEUE4, true);
 
@@ -270,7 +274,8 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
   public void testConfigurer() throws Exception {
     String tableName = this.getUniqueNames(1)[0];
 
-    getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE5);
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE5, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
     try (AccumuloClient client =
         Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
@@ -345,7 +350,8 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
         Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
       createTable(client, table1, "cs6");
       writeData(client, table1);
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE6);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE6, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
       compact(client, table1, 2, QUEUE6, true);
       verify(client, table1, 2);
 
@@ -383,7 +389,8 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
         Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
       createTable(client, table3, "cs7");
       writeData(client, table3);
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE7);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE7, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
       compact(client, table3, 2, QUEUE7, false);
 
       // ExternalCompactionTServer will not commit the compaction. Wait for the
@@ -463,7 +470,8 @@ public class ExternalCompaction_1_IT extends SharedMiniClusterBase {
     try (final AccumuloClient client =
         Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
 
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE8);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE8, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
       createTable(client, tableName, "cs8");
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
index c3ff5b3a2c..f946f1d901 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
@@ -20,7 +20,6 @@ package org.apache.accumulo.test.compaction;
 
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.MAX_DATA;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE1;
-import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE2;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE3;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE4;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE5;
@@ -44,7 +43,6 @@ import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
@@ -87,12 +85,15 @@ public class ExternalCompaction_2_IT extends SharedMiniClusterBase {
   public void tearDown() throws Exception {
     // The ExternalDoNothingCompactor needs to be restarted between tests
     getCluster().getClusterControl().stop(ServerType.COMPACTOR);
+    getCluster().getConfig().getClusterServerConfiguration().clearCompactorResourceGroups();
   }
 
   @Test
   public void testSplitCancelsExternalCompaction() throws Exception {
 
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE1);
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
@@ -184,7 +185,8 @@ public class ExternalCompaction_2_IT extends SharedMiniClusterBase {
       t.start();
 
       // Start the compactor
-      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE2);
+      getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
+      getCluster().getClusterControl().start(ServerType.COMPACTOR);
 
       // Wait for the compaction to start by waiting for 1 external compaction column
       Set<ExternalCompactionId> ecids = ExternalCompactionTestUtils
@@ -230,7 +232,9 @@ public class ExternalCompaction_2_IT extends SharedMiniClusterBase {
   @Test
   public void testUserCompactionCancellation() throws Exception {
 
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE3);
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE3, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
@@ -265,7 +269,9 @@ public class ExternalCompaction_2_IT extends SharedMiniClusterBase {
   @Test
   public void testDeleteTableCancelsUserExternalCompaction() throws Exception {
 
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE4);
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE4, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
@@ -294,7 +300,10 @@ public class ExternalCompaction_2_IT extends SharedMiniClusterBase {
 
   @Test
   public void testDeleteTableCancelsExternalCompaction() throws Exception {
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE5);
+
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE5, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java
index 9de9a0d57a..871a1ee95a 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java
@@ -76,12 +76,15 @@ public class ExternalCompaction_3_IT extends SharedMiniClusterBase {
   public void tearDown() throws Exception {
     // The ExternalDoNothingCompactor needs to be restarted between tests
     getCluster().getClusterControl().stop(ServerType.COMPACTOR);
+    getCluster().getConfig().getClusterServerConfiguration().clearCompactorResourceGroups();
   }
 
   @Test
   public void testMergeCancelsExternalCompaction() throws Exception {
 
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE1);
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE1, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
@@ -139,7 +142,10 @@ public class ExternalCompaction_3_IT extends SharedMiniClusterBase {
 
   @Test
   public void testCoordinatorRestartsDuringCompaction() throws Exception {
-    getCluster().getClusterControl().startCompactors(ExternalDoNothingCompactor.class, 1, QUEUE2);
+
+    getCluster().getConfig().getClusterServerConfiguration().addCompactorResourceGroup(QUEUE2, 1);
+    getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
+        ExternalDoNothingCompactor.class);
 
     String table1 = this.getUniqueNames(1)[0];
     try (AccumuloClient client =
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index e20955e71d..bec21401e6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
@@ -71,8 +72,9 @@ public class BalanceInPresenceOfOfflineTableIT extends AccumuloClusterHarness {
     siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "50ms");
     cfg.setSiteConfig(siteConfig);
     // ensure we have two tservers
-    if (cfg.getNumTservers() < 2) {
-      cfg.setNumTservers(2);
+    if (cfg.getClusterServerConfiguration().getTabletServerConfiguration()
+        .get(ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME) < 2) {
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(2);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 0ccf7b311d..647d17e890 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -63,7 +63,7 @@ public class BloomFilterIT extends AccumuloClusterHarness {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setDefaultMemory(1, MemoryUnit.GIGABYTE);
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     Map<String,String> siteConfig = cfg.getSiteConfig();
     siteConfig.put(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "10M");
     cfg.setSiteConfig(siteConfig);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
index 044a20d399..2480cebf1d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
@@ -45,7 +45,6 @@ import java.util.TreeSet;
 import java.util.concurrent.ExecutionException;
 import java.util.stream.Collectors;
 
-import org.apache.accumulo.compactor.Compactor;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -509,8 +508,6 @@ public class BulkNewIT extends SharedMiniClusterBase {
   @Test
   public void testManyFiles() throws Exception {
 
-    getCluster().getClusterControl().startCompactors(Compactor.class, 1, "user-small");
-
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       String dir = getDir("/testBulkFile-");
       FileSystem fs = getCluster().getFileSystem();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
index 7c7a231269..b2fff12631 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
@@ -60,7 +60,7 @@ public class CleanTmpIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
index 13a51aa627..c4b1f173f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableMacBase.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.harness.AccumuloITBase;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.accumulo.miniclusterImpl.ClusterServerConfiguration;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.miniclusterImpl.ZooKeeperBindException;
@@ -147,6 +148,10 @@ public class ConfigurableMacBase extends AccumuloITBase {
         lastException);
   }
 
+  public ClusterServerConfiguration getMiniClusterDescription() {
+    return new ClusterServerConfiguration();
+  }
+
   @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
   private void createMiniAccumulo() throws Exception {
     // createTestDir will give us a empty directory, we don't need to clean it up ourselves
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 906fe2b6a2..a2683c34f1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -55,7 +55,7 @@ public class DurabilityIT extends ConfigurableMacBase {
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   static final long N = 100000;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index de85287b1d..ad06b85d37 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -74,7 +74,7 @@ public class HalfDeadTServerIT extends ConfigurableMacBase {
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // configure only one tserver from mini; mini won't less us configure 0, so instead, we will
     // start only 1, and kill it to start our own in the desired simulation environment
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setProperty(Property.GENERAL_RPC_TIMEOUT, "5s");
     cfg.setProperty(Property.TSERV_NATIVEMAP_ENABLED, Boolean.FALSE.toString());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/IteratorMincClassCastBugIT.java b/test/src/main/java/org/apache/accumulo/test/functional/IteratorMincClassCastBugIT.java
index a7d5a322f4..906ded1eef 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/IteratorMincClassCastBugIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/IteratorMincClassCastBugIT.java
@@ -62,7 +62,7 @@ public class IteratorMincClassCastBugIT extends AccumuloClusterHarness {
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // this bug only shows up when not using native maps
     cfg.setProperty(Property.TSERV_NATIVEMAP_ENABLED, "false");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 010c2b6ac3..faec8673fc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -145,7 +145,7 @@ public class KerberosIT extends AccumuloITBase {
 
         });
 
-    mac.getConfig().setNumTservers(1);
+    mac.getConfig().getClusterServerConfiguration().setNumDefaultTabletServers(1);
     mac.start();
     // Enabled kerberos auth
     Configuration conf = new Configuration(false);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
index 804ece8b49..6c413426cc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
@@ -120,7 +120,7 @@ public class KerberosRenewalIT extends AccumuloITBase {
       cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     });
 
-    mac.getConfig().setNumTservers(1);
+    mac.getConfig().getClusterServerConfiguration().setNumDefaultTabletServers(1);
     mac.start();
     // Enabled kerberos auth
     Configuration conf = new Configuration(false);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
index 93626de958..8857f41659 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
@@ -87,7 +87,7 @@ public class ManagerAssignmentIT extends SharedMiniClusterBase {
   @BeforeAll
   public static void beforeAll() throws Exception {
     SharedMiniClusterBase.startMiniClusterWithConfig((cfg, core) -> {
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       cfg.setProperty(Property.TSERV_ASSIGNMENT_MAXCONCURRENT, "10");
       cfg.setProperty(Property.GENERAL_THREADPOOL_SIZE, "10");
       cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL, "5s");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
index fe4b792170..c37d77e784 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
@@ -68,7 +68,7 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
     // lots of closed WALs for all write patterns. This test ensures code that directly handles many
     // tablets referencing many different WALs is working.
     cfg.setProperty(Property.TABLE_MINC_COMPACT_IDLETIME, "1h");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java
index e4ded303e6..551a2d09c6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java
@@ -54,7 +54,7 @@ public class MemoryStarvedMajCIT extends SharedMiniClusterBase {
 
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
       // Configure the LowMemoryDetector in the TabletServer
       // check on 1s intervals and set low mem condition if more than 80% of
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java
index 6418d98ef4..6fc27529f7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java
@@ -53,7 +53,7 @@ public class MemoryStarvedMinCIT extends SharedMiniClusterBase {
 
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
       // Configure the LowMemoryDetector in the TabletServer
       // check on 1s intervals and set low mem condition if more than 80% of
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java
index c0036d358d..66d74312cc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java
@@ -61,7 +61,7 @@ public class MemoryStarvedScanIT extends SharedMiniClusterBase {
 
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
       // Configure the LowMemoryDetector in the TabletServer
       // check on 1s intervals and set low mem condition if more than 80% of
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
index 0571dbce7c..186f181e4a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
@@ -68,7 +68,7 @@ public class MetadataIT extends AccumuloClusterHarness {
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
index 1d591289c8..6578927ea2 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java
@@ -94,8 +94,8 @@ public class OnDemandTabletUnloadingIT extends SharedMiniClusterBase {
     metricConsumer.start();
 
     SharedMiniClusterBase.startMiniClusterWithConfig((cfg, core) -> {
-      cfg.setNumTservers(1);
-      cfg.setNumScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultScanServers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       cfg.setProperty(Property.MANAGER_TABLET_GROUP_WATCHER_INTERVAL,
           Integer.toString(managerTabletGroupWatcherInterval));
       cfg.setProperty(Property.TSERV_ONDEMAND_UNLOADER_INTERVAL,
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
index b7194b86bd..bf5aaab74e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
@@ -59,7 +59,7 @@ public class RegexGroupBalanceIT extends ConfigurableMacBase {
 
   @Override
   public void beforeClusterStart(MiniAccumuloConfigImpl cfg) {
-    cfg.setNumTservers(4);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(4);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
index 8cd6197242..efd3962b8c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SessionBlockVerifyIT.java
@@ -71,7 +71,7 @@ public class SessionBlockVerifyIT extends ScanSessionTimeOutIT {
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> siteConfig = cfg.getSiteConfig();
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     siteConfig.put(Property.TSERV_SESSION_MAXIDLE.getKey(), getMaxIdleTimeString());
     siteConfig.put(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(), "11");
     cfg.setSiteConfig(siteConfig);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
index d3fc490886..572b1d0b53 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java
@@ -56,7 +56,7 @@ public class SessionDurabilityIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
index 7b715936ec..7f6d07abbb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
@@ -50,7 +50,7 @@ public class TabletMetadataIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration conf) {
-    cfg.setNumTservers(NUM_TSERVERS);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(NUM_TSERVERS);
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index c245e340bb..11da5126f9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -83,7 +83,7 @@ public class WALSunnyDayIT extends ConfigurableMacBase {
     cfg.setProperty(TSERV_WAL_MAX_SIZE, "1M");
     cfg.setProperty(TSERV_WAL_REPLICATION, "1");
     cfg.setProperty(INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index 47348aa054..df446a6b19 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@ -48,7 +48,7 @@ public class WatchTheWatchCountIT extends ConfigurableMacBase {
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(3);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(3);
   }
 
   @SuppressFBWarnings(value = "UNENCRYPTED_SOCKET",
diff --git a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
index 125af5b23c..ca8b32cfa1 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
@@ -100,7 +100,7 @@ public class SuspendedTabletsIT extends ConfigurableMacBase {
     cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "5s");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     // Start with 1 tserver, we'll increase that later
-    cfg.setNumTservers(1);
+    cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
     // config custom balancer to keep all metadata on one server
     cfg.setProperty(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "1ms");
     cfg.setProperty(Property.MANAGER_TABLET_BALANCER.getKey(),
@@ -146,7 +146,7 @@ public class SuspendedTabletsIT extends ConfigurableMacBase {
     metadataTserverProcess = procs.iterator().next();
 
     // Update the number of tservers and start the new tservers.
-    getCluster().getConfig().setNumTservers(TSERVERS);
+    getCluster().getConfig().getClusterServerConfiguration().setNumDefaultTabletServers(TSERVERS);
     getCluster().start();
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
index 95494218b7..f941fcfc08 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
@@ -74,7 +74,6 @@ public class MetricsIT extends ConfigurableMacBase implements MetricsProducer {
 
   @Override
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(2);
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
     cfg.setProperty(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL, "1s");
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
index ed4a1bd18a..dad3a2ae10 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
@@ -74,7 +74,7 @@ public class ShellCreateTableIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
       // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       // Set the min span to 0 so we will definitely get all the traces back. See ACCUMULO-4365
       Map<String,String> siteConf = cfg.getSiteConfig();
       cfg.setSiteConfig(siteConf);
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
index 6ccd6b7f15..2b45ac8687 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
@@ -123,7 +123,7 @@ public class ShellServerIT extends SharedMiniClusterBase {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
       // Only one tserver to avoid race conditions on ZK propagation (auths and configuration)
-      cfg.setNumTservers(1);
+      cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
       // Set the min span to 0 so we will definitely get all the traces back. See ACCUMULO-4365
       Map<String,String> siteConf = cfg.getSiteConfig();
       cfg.setSiteConfig(siteConf);