Add TabletBalancer to SPI package (#1891)

Add TabletBalancer to SPI package

This change deprecates existing balancers and introduces copies of each
under the SPI package. The main balancer interface, TabletBalancer, has
been converted from an abstract class to an interface and its method
signatures use only public API types and some other data types in SPI
sub-packages. The new types used expose the same functionality as the
previous types, and the implementations generally simply wrap the old
thrift objects and delegate, but will allow for easier evolution of the
API in the future.

* Deprecated TabletId.getTableId(), which created a new Text object
  every time, in favor of new TabletId.getTable() which returns a
  TableId. All existing usages of the getTableId() method would
  immediately call toString() on the returned Text object, or construct
  a TableId from the Text object. Therefore, the new method avoids
  object creation and usages read cleaner in many cases.
* Add static factory method on KeyExtent to create from a TabletId and
  return the wrapped KeyExtent if the TabletId is a TabletIdImpl and
  otherwise constructs with the property values retrieved from the
  TabletId.
* Add functionality to PluginEnvironment.Configuration to expose the
  derived property calculation (AccumuloConfiguration.Deriver) as a
  Supplier.
* Deprecated abstract TabletBalancer class now implements the new SPI
  TabletBalancer interface and provides implementations of the SPI
  methods on that interface that convert between the new and old types
  and delegate to the old balance/getAssignments methods.
* Deprecated all balancers in org.apache.accumulo.server.master.balancer
  except for ChaoticLoadBalancer, which was moved to accumulo-test since
  its purpose is only for testing.
* Introduce types in org.apache.accumulo.core.spi.balancer.data for
  wrapping thrift types needed by the balancers.
* Add a BalancerEnvironment to the SPI to support balancer-specific
  system operations (table online status check, listing online tablets,
  etc) in an API-stable manner.
* Convert TabletBalancer from abstract class to interface with init,
  getAssignments, and balance methods. The methods take parameters
  objects which will make it easier in the future if the balancer API
  needs to evolve.
* Implement existing balancers (DefaultLoadBalancer, TableLoadBalancer,
  GroupBalancer, RegexGroupBalancer, HostRegexTableLoadBalancer) in the
  SPI package by copying existing code and converting usages of thrift
  types to new stable types, and adjusting method signatures such that
  the new TabletBalancer API is implemented. The existing class
  DefaultLoadBalancer was renamed to SimpleLoadBalancer.
* Changed default balancer and table load balancer properties to use the
  new SPI-packaged versions.
* Update Master to construct the new TabletBalancer class, consolidate
  all access to the balancer internally (getAssignments moved from 
  )

fixes #1880
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index 84e24f9..633f672 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -30,7 +30,6 @@
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
 
 /**
@@ -107,8 +106,7 @@
 
     for (Entry<TabletId,Set<SecurityErrorCode>> entry : hashMap.entrySet()) {
       TabletId tabletId = entry.getKey();
-      String tableInfo =
-          Tables.getPrintableTableInfoFromId(context, TableId.of(tabletId.getTableId().toString()));
+      String tableInfo = Tables.getPrintableTableInfoFromId(context, tabletId.getTable());
 
       if (!result.containsKey(tableInfo)) {
         result.put(tableInfo, new HashSet<>());
diff --git a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
index c73b178..55ff67f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
@@ -21,6 +21,8 @@
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.function.Function;
+import java.util.function.Supplier;
 
 import org.apache.accumulo.core.data.TableId;
 
@@ -36,7 +38,7 @@
   /**
    * @since 2.1.0
    */
-  public interface Configuration extends Iterable<Entry<String,String>> {
+  interface Configuration extends Iterable<Entry<String,String>> {
 
     /**
      * Properties with a default value will always return something when calling
@@ -122,6 +124,14 @@
      */
     @Override
     Iterator<Entry<String,String>> iterator();
+
+    /**
+     * Returns a derived value from this Configuration. The returned value supplier is thread-safe
+     * and attempts to avoid re-computation of the response. The intended use for a derived value is
+     * to ensure that configuration changes that may be made in Zookeeper, for example, are always
+     * reflected in the returned value.
+     */
+    <T> Supplier<T> getDerived(Function<Configuration,T> computeDerivedValue);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index c5b8885..ec49c81 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -551,7 +551,7 @@
         if (!e.getSecurityErrorCodes().isEmpty()) {
           var tables = new HashMap<String,Set<SecurityErrorCode>>();
           e.getSecurityErrorCodes().forEach((tabletId, secSet) -> {
-            var tableId = tabletId.getTableId().toString();
+            var tableId = tabletId.getTable().canonical();
             tables.computeIfAbsent(tableId, p -> new HashSet<>()).addAll(secSet);
           });
           log.error("Not authorized to write to tables : " + tables);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index b9fdae4..4444b28 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -552,7 +552,7 @@
         if (!e.getSecurityErrorCodes().isEmpty()) {
           var tables = new HashMap<String,Set<SecurityErrorCode>>();
           e.getSecurityErrorCodes().forEach((table, code) -> tables
-              .computeIfAbsent(table.getTableId().toString(), k -> new HashSet<>()).addAll(code));
+              .computeIfAbsent(table.getTable().canonical(), k -> new HashSet<>()).addAll(code));
           log.error("Not authorized to write to tables : " + tables);
         }
 
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 9525f34..9a2998f 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
@@ -258,7 +258,7 @@
   MANAGER_CLIENTPORT("manager.port.client", "9999", PropertyType.PORT,
       "The port used for handling client connections on the manager"),
   MANAGER_TABLET_BALANCER("manager.tablet.balancer",
-      "org.apache.accumulo.server.master.balancer.TableLoadBalancer", PropertyType.CLASSNAME,
+      "org.apache.accumulo.core.spi.balancer.TableLoadBalancer", PropertyType.CLASSNAME,
       "The balancer class that accumulo will use to make tablet assignment and "
           + "migration decisions."),
   MANAGER_BULK_RETRIES("manager.bulk.retries", "3", PropertyType.COUNT,
@@ -765,8 +765,8 @@
           + "Once this limit is reached, the buffered data is sent to the client."),
   TABLE_FILE_TYPE("table.file.type", RFile.EXTENSION, PropertyType.STRING,
       "Change the type of file a table writes"),
-  TABLE_LOAD_BALANCER("table.balancer",
-      "org.apache.accumulo.server.master.balancer.DefaultLoadBalancer", PropertyType.STRING,
+  TABLE_LOAD_BALANCER("table.balancer", "org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer",
+      PropertyType.STRING,
       "This property can be set to allow the LoadBalanceByTable load balancer"
           + " to change the called Load Balancer for this table"),
   TABLE_FILE_COMPRESSION_TYPE("table.file.compress.type", "gz", PropertyType.STRING,
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
index eb69bfc..ec17935 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/TabletId.java
@@ -26,6 +26,15 @@
  * @since 1.7.0
  */
 public interface TabletId extends Comparable<TabletId> {
+  /**
+   * @since 2.1.0
+   */
+  TableId getTable();
+
+  /**
+   * @deprecated use {@link #getTable()} and {@link TableId#canonical()} instead
+   */
+  @Deprecated(since = "2.1.0")
   Text getTableId();
 
   Text getEndRow();
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index 3a4eed0..7ed868e 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
@@ -43,6 +43,7 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -178,6 +179,20 @@
   }
 
   /**
+   * Create a KeyExtent from a {@link TabletId}.
+   *
+   * @param tabletId
+   *          the {@link TabletId} to convert to a KeyExtent
+   */
+  public static KeyExtent fromTabletId(TabletId tabletId) {
+    if (tabletId instanceof TabletIdImpl) {
+      return ((TabletIdImpl) tabletId).toKeyExtent();
+    } else {
+      return new KeyExtent(tabletId.getTable(), tabletId.getEndRow(), tabletId.getPrevEndRow());
+    }
+  }
+
+  /**
    * Return a serialized form of the table ID and end row for this extent, in a form suitable for
    * use in the row portion of metadata entries for the tablet this extent represents.
    */
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
index 5ca015d..bf970ea 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
@@ -19,12 +19,13 @@
 package org.apache.accumulo.core.dataImpl;
 
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.hadoop.io.Text;
 
 public class TabletIdImpl implements TabletId {
 
-  private KeyExtent ke;
+  private final KeyExtent ke;
 
   public TabletIdImpl(KeyExtent ke) {
     this.ke = ke;
@@ -36,6 +37,12 @@
   }
 
   @Override
+  public TableId getTable() {
+    return ke.tableId();
+  }
+
+  @Override
+  @Deprecated(since = "2.1.0")
   public Text getTableId() {
     return new Text(ke.tableId().canonical());
   }
@@ -73,4 +80,9 @@
   public Range toRange() {
     return ke.toDataRange();
   }
+
+  public KeyExtent toKeyExtent() {
+    return ke;
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
new file mode 100644
index 0000000..b490737
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/AssignmentParamsImpl.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.spi.balancer.TabletBalancer;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+
+public class AssignmentParamsImpl implements TabletBalancer.AssignmentParameters {
+  private final SortedMap<TabletServerId,TServerStatus> currentStatus;
+  private final Map<TabletId,TabletServerId> unassigned;
+  private final Map<TabletId,TabletServerId> assignmentsOut;
+  private final SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus;
+  private final Map<KeyExtent,TServerInstance> thriftUnassigned;
+  private final Map<KeyExtent,TServerInstance> thriftAssignmentsOut;
+
+  public static AssignmentParamsImpl fromThrift(
+      SortedMap<TServerInstance,TabletServerStatus> currentStatus,
+      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignmentsOut) {
+
+    SortedMap<TabletServerId,TServerStatus> currentStatusNew = new TreeMap<>();
+    currentStatus.forEach((tsi, status) -> currentStatusNew.put(new TabletServerIdImpl(tsi),
+        TServerStatusImpl.fromThrift(status)));
+    Map<TabletId,TabletServerId> unassignedNew = new HashMap<>();
+    unassigned.forEach(
+        (ke, tsi) -> unassignedNew.put(new TabletIdImpl(ke), TabletServerIdImpl.fromThrift(tsi)));
+
+    return new AssignmentParamsImpl(Collections.unmodifiableSortedMap(currentStatusNew),
+        Collections.unmodifiableMap(unassignedNew), currentStatus, unassigned, assignmentsOut);
+  }
+
+  public AssignmentParamsImpl(SortedMap<TabletServerId,TServerStatus> currentStatus,
+      Map<TabletId,TabletServerId> unassigned, Map<TabletId,TabletServerId> assignmentsOut) {
+    this.currentStatus = currentStatus;
+    this.unassigned = unassigned;
+    this.assignmentsOut = assignmentsOut;
+    this.thriftCurrentStatus = null;
+    this.thriftUnassigned = null;
+    this.thriftAssignmentsOut = null;
+  }
+
+  private AssignmentParamsImpl(SortedMap<TabletServerId,TServerStatus> currentStatus,
+      Map<TabletId,TabletServerId> unassigned,
+      SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus,
+      Map<KeyExtent,TServerInstance> thriftUnassigned,
+      Map<KeyExtent,TServerInstance> thriftAssignmentsOut) {
+    this.currentStatus = currentStatus;
+    this.unassigned = unassigned;
+    this.assignmentsOut = null;
+    this.thriftCurrentStatus = thriftCurrentStatus;
+    this.thriftUnassigned = thriftUnassigned;
+    this.thriftAssignmentsOut = thriftAssignmentsOut;
+  }
+
+  @Override
+  public SortedMap<TabletServerId,TServerStatus> currentStatus() {
+    return currentStatus;
+  }
+
+  @Override
+  public Map<TabletId,TabletServerId> unassignedTablets() {
+    return unassigned;
+  }
+
+  @Override
+  public void addAssignment(TabletId tabletId, TabletServerId tabletServerId) {
+    if (assignmentsOut != null) {
+      assignmentsOut.put(tabletId, tabletServerId);
+    }
+    if (thriftAssignmentsOut != null) {
+      thriftAssignmentsOut.put(KeyExtent.fromTabletId(tabletId),
+          TabletServerIdImpl.toThrift(tabletServerId));
+    }
+  }
+
+  public SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus() {
+    return thriftCurrentStatus;
+  }
+
+  public Map<KeyExtent,TServerInstance> thriftUnassigned() {
+    return thriftUnassigned;
+  }
+
+  public Map<KeyExtent,TServerInstance> thriftAssignmentsOut() {
+    return thriftAssignmentsOut;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
new file mode 100644
index 0000000..4053d16
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/BalanceParamsImpl.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.spi.balancer.TabletBalancer;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+
+public class BalanceParamsImpl implements TabletBalancer.BalanceParameters {
+  private final SortedMap<TabletServerId,TServerStatus> currentStatus;
+  private final Set<TabletId> currentMigrations;
+  private final List<TabletMigration> migrationsOut;
+  private final SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus;
+  private final Set<KeyExtent> thriftCurrentMigrations;
+
+  public static BalanceParamsImpl fromThrift(SortedMap<TabletServerId,TServerStatus> currentStatus,
+      SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus,
+      Set<KeyExtent> thriftCurrentMigrations) {
+    Set<TabletId> currentMigrations = thriftCurrentMigrations.stream().map(TabletIdImpl::new)
+        .collect(Collectors.toUnmodifiableSet());
+
+    return new BalanceParamsImpl(currentStatus, currentMigrations, new ArrayList<>(),
+        thriftCurrentStatus, thriftCurrentMigrations);
+  }
+
+  public BalanceParamsImpl(SortedMap<TabletServerId,TServerStatus> currentStatus,
+      Set<TabletId> currentMigrations, List<TabletMigration> migrationsOut) {
+    this.currentStatus = currentStatus;
+    this.currentMigrations = currentMigrations;
+    this.migrationsOut = migrationsOut;
+    this.thriftCurrentStatus = null;
+    this.thriftCurrentMigrations = null;
+  }
+
+  private BalanceParamsImpl(SortedMap<TabletServerId,TServerStatus> currentStatus,
+      Set<TabletId> currentMigrations, List<TabletMigration> migrationsOut,
+      SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus,
+      Set<KeyExtent> thriftCurrentMigrations) {
+    this.currentStatus = currentStatus;
+    this.currentMigrations = currentMigrations;
+    this.migrationsOut = migrationsOut;
+    this.thriftCurrentStatus = thriftCurrentStatus;
+    this.thriftCurrentMigrations = thriftCurrentMigrations;
+  }
+
+  @Override
+  public SortedMap<TabletServerId,TServerStatus> currentStatus() {
+    return currentStatus;
+  }
+
+  @Override
+  public Set<TabletId> currentMigrations() {
+    return currentMigrations;
+  }
+
+  @Override
+  public List<TabletMigration> migrationsOut() {
+    return migrationsOut;
+  }
+
+  public SortedMap<TServerInstance,TabletServerStatus> thriftCurrentStatus() {
+    return thriftCurrentStatus;
+  }
+
+  public Set<KeyExtent> thriftCurrentMigrations() {
+    return thriftCurrentMigrations;
+  }
+
+  public void addMigration(KeyExtent extent, TServerInstance oldServer, TServerInstance newServer) {
+    TabletId id = new TabletIdImpl(extent);
+    TabletServerId oldTsid = new TabletServerIdImpl(oldServer);
+    TabletServerId newTsid = new TabletServerIdImpl(newServer);
+    migrationsOut.add(new TabletMigration(id, oldTsid, newTsid));
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
new file mode 100644
index 0000000..31968a7
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TServerStatusImpl.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+
+public class TServerStatusImpl implements TServerStatus {
+  private final TabletServerStatus thriftStatus;
+  private Map<String,TableStatistics> tableInfoMap;
+
+  public static TServerStatusImpl fromThrift(TabletServerStatus tss) {
+    return (tss == null) ? null : new TServerStatusImpl(tss);
+  }
+
+  public TServerStatusImpl(TabletServerStatus thriftStatus) {
+    this.thriftStatus = requireNonNull(thriftStatus);
+    if (thriftStatus.getTableMap() == null) {
+      tableInfoMap = null;
+    } else {
+      tableInfoMap = new HashMap<>();
+      thriftStatus.getTableMap()
+          .forEach((name, info) -> tableInfoMap.put(name, TableStatisticsImpl.fromThrift(info)));
+    }
+  }
+
+  @Override
+  public Map<String,TableStatistics> getTableMap() {
+    return tableInfoMap;
+  }
+
+  public void setTableMap(Map<String,TableStatistics> tableInfoMap) {
+    this.tableInfoMap = tableInfoMap;
+  }
+
+  @Override
+  public long getLastContact() {
+    return thriftStatus.getLastContact();
+  }
+
+  @Override
+  public String getName() {
+    return thriftStatus.getName();
+  }
+
+  @Override
+  public double getOsLoad() {
+    return thriftStatus.getOsLoad();
+  }
+
+  @Override
+  public long getHoldTime() {
+    return thriftStatus.getHoldTime();
+  }
+
+  @Override
+  public long getLookups() {
+    return thriftStatus.getLookups();
+  }
+
+  @Override
+  public long getIndexCacheHits() {
+    return thriftStatus.getIndexCacheHits();
+  }
+
+  @Override
+  public long getIndexCacheRequests() {
+    return thriftStatus.getIndexCacheRequest();
+  }
+
+  @Override
+  public long getDataCacheHits() {
+    return thriftStatus.getDataCacheHits();
+  }
+
+  @Override
+  public long getDataCacheRequests() {
+    return thriftStatus.getDataCacheRequest();
+  }
+
+  @Override
+  public long getFlushes() {
+    return thriftStatus.getFlushs();
+  }
+
+  @Override
+  public long getSyncs() {
+    return thriftStatus.getSyncs();
+  }
+
+  @Override
+  public String getVersion() {
+    return thriftStatus.getVersion();
+  }
+
+  @Override
+  public long getResponseTime() {
+    return thriftStatus.getResponseTime();
+  }
+
+  @Override
+  public int compareTo(TServerStatus o) {
+    return thriftStatus.compareTo(((TServerStatusImpl) o).thriftStatus);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (o == null || getClass() != o.getClass())
+      return false;
+    TServerStatusImpl that = (TServerStatusImpl) o;
+    return thriftStatus.equals(that.thriftStatus);
+  }
+
+  @Override
+  public int hashCode() {
+    return thriftStatus.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return thriftStatus.toString();
+  }
+
+  public TabletServerStatus toThrift() {
+    return thriftStatus;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
new file mode 100644
index 0000000..93dd1eb
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TableStatisticsImpl.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+
+public class TableStatisticsImpl implements TableStatistics {
+  private final TableInfo thriftInfo;
+
+  public static TableStatisticsImpl fromThrift(TableInfo tableInfo) {
+    return tableInfo == null ? null : new TableStatisticsImpl(tableInfo);
+  }
+
+  public TableStatisticsImpl(TableInfo thriftInfo) {
+    this.thriftInfo = thriftInfo;
+  }
+
+  public TableStatisticsImpl(TableStatisticsImpl other) {
+    this.thriftInfo = new TableInfo(other.thriftInfo);
+  }
+
+  @Override
+  public long getRecords() {
+    return thriftInfo.getRecs();
+  }
+
+  @Override
+  public long getRecordsInMemory() {
+    return thriftInfo.getRecsInMemory();
+  }
+
+  @Override
+  public int getTabletCount() {
+    return thriftInfo.getTablets();
+  }
+
+  @Override
+  public int getOnlineTabletCount() {
+    return thriftInfo.getOnlineTablets();
+  }
+
+  public void setOnlineTabletCount(int onlineTabletCount) {
+    thriftInfo.setOnlineTablets(onlineTabletCount);
+  }
+
+  @Override
+  public double getIngestRate() {
+    return thriftInfo.getIngestRate();
+  }
+
+  @Override
+  public double getIngestByteRate() {
+    return thriftInfo.getIngestByteRate();
+  }
+
+  @Override
+  public double getQueryRate() {
+    return thriftInfo.getQueryRate();
+  }
+
+  @Override
+  public double getQueryByteRate() {
+    return thriftInfo.getQueryByteRate();
+  }
+
+  @Override
+  public double getScanRate() {
+    return thriftInfo.getScanRate();
+  }
+
+  @Override
+  public int compareTo(TableStatistics o) {
+    return thriftInfo.compareTo(((TableStatisticsImpl) o).thriftInfo);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (o == null || getClass() != o.getClass())
+      return false;
+    TableStatisticsImpl that = (TableStatisticsImpl) o;
+    return thriftInfo.equals(that.thriftInfo);
+  }
+
+  @Override
+  public int hashCode() {
+    return thriftInfo.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return thriftInfo.toString();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
new file mode 100644
index 0000000..8236efb
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletServerIdImpl.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.util.HostAndPort;
+
+/**
+ * @since 2.1.0
+ */
+public class TabletServerIdImpl implements TabletServerId {
+  private final TServerInstance tServerInstance;
+
+  public static TabletServerIdImpl fromThrift(TServerInstance tsi) {
+    return (tsi == null) ? null : new TabletServerIdImpl(tsi);
+  }
+
+  public TabletServerIdImpl(String host, int port, String session) {
+    requireNonNull(host);
+    this.tServerInstance = new TServerInstance(HostAndPort.fromParts(host, port), session);
+  }
+
+  public TabletServerIdImpl(TServerInstance tServerInstance) {
+    this.tServerInstance = requireNonNull(tServerInstance);
+  }
+
+  @Override
+  public String getHost() {
+    return tServerInstance.getHostAndPort().getHost();
+  }
+
+  @Override
+  public int getPort() {
+    return tServerInstance.getHostAndPort().getPort();
+  }
+
+  @Override
+  public String getSession() {
+    return tServerInstance.getSession();
+  }
+
+  @Override
+  public int compareTo(TabletServerId o) {
+    return tServerInstance.compareTo(((TabletServerIdImpl) o).tServerInstance);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (o == null || getClass() != o.getClass())
+      return false;
+    TabletServerIdImpl that = (TabletServerIdImpl) o;
+    return tServerInstance.equals(that.tServerInstance);
+  }
+
+  @Override
+  public int hashCode() {
+    return tServerInstance.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return getHost() + ':' + getPort() + '[' + getSession() + ']';
+  }
+
+  public TServerInstance toThrift() {
+    return tServerInstance;
+  }
+
+  public static TServerInstance toThrift(TabletServerId tabletServerId) {
+    if (tabletServerId instanceof TabletServerIdImpl) {
+      return ((TabletServerIdImpl) tabletServerId).toThrift();
+    } else {
+      return new TServerInstance(
+          HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()),
+          tabletServerId.getSession());
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
new file mode 100644
index 0000000..4373f84
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/manager/balancer/TabletStatisticsImpl.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.manager.balancer;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+
+public class TabletStatisticsImpl implements TabletStatistics {
+  private final TabletStats thriftStats;
+  private final TabletId tabletId;
+
+  public TabletStatisticsImpl(TabletStats thriftStats) {
+    this.thriftStats = requireNonNull(thriftStats);
+    tabletId = new TabletIdImpl(KeyExtent.fromThrift(thriftStats.getExtent()));
+  }
+
+  @Override
+  public TabletId getTabletId() {
+    return tabletId;
+  }
+
+  @Override
+  public long getNumEntries() {
+    return thriftStats.getNumEntries();
+  }
+
+  @Override
+  public long getSplitCreationTime() {
+    return thriftStats.getSplitCreationTime();
+  }
+
+  @Override
+  public double getIngestRate() {
+    return thriftStats.getIngestRate();
+  }
+
+  @Override
+  public double getQueryRate() {
+    return thriftStats.getQueryRate();
+  }
+
+  @Override
+  public int compareTo(TabletStatistics o) {
+    return thriftStats.compareTo(((TabletStatisticsImpl) o).thriftStats);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (o == null || getClass() != o.getClass())
+      return false;
+    TabletStatisticsImpl that = (TabletStatisticsImpl) o;
+    return thriftStats.equals(that.thriftStats);
+  }
+
+  @Override
+  public int hashCode() {
+    return thriftStats.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return thriftStats.toString();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/BalancerEnvironment.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/BalancerEnvironment.java
new file mode 100644
index 0000000..64d0892
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/BalancerEnvironment.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
+/**
+ * This interface is an extension of {@link ServiceEnvironment} that exposes system level
+ * information that is specific to tablet balancing.
+ *
+ * @since 2.1.0
+ */
+public interface BalancerEnvironment extends ServiceEnvironment {
+  /**
+   * Many Accumulo plugins are given table IDs as this is what Accumulo uses internally to identify
+   * tables. This provides a mapping of table names to table IDs for the purposes of translating
+   * and/or enumerating the existing tables.
+   */
+  Map<String,TableId> getTableIdMap();
+
+  /**
+   * Accumulo plugins working with a table may need to know if the table is online or not before
+   * operating on it.
+   *
+   * @param tableId
+   *          The id of the table to check.
+   * @return {@code true} if the table is online and {@code false} if not
+   */
+  boolean isTableOnline(TableId tableId);
+
+  /**
+   * Fetch the locations for each of {@code tableId}'s tablets from the metadata table. If there is
+   * no location available for a given tablet, then the returned mapping will have a {@code null}
+   * value stored for the tablet id.
+   *
+   * @param tableId
+   *          The id of the table for which to retrieve tablets.
+   * @return a mapping of {@link TabletId} to {@link TabletServerId} (or @null if no location is
+   *         available) for each tablet belonging to {@code tableId}
+   */
+  Map<TabletId,TabletServerId> listTabletLocations(TableId tableId);
+
+  /**
+   * Fetch the tablets for the given table by asking the tablet server. Useful if your balance
+   * strategy needs details at the tablet level to decide what tablets to move.
+   *
+   * @param tabletServerId
+   *          The tablet server to ask.
+   * @param tableId
+   *          The table id
+   * @return a list of tablet statistics
+   * @throws AccumuloSecurityException
+   *           tablet server disapproves of your internal System password.
+   * @throws AccumuloException
+   *           any other problem
+   */
+  List<TabletStatistics> listOnlineTabletsForTable(TabletServerId tabletServerId, TableId tableId)
+      throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Retrieve the classloader context that is configured for {@code tableId}, or {@code null} if
+   * none is configured.
+   */
+  String tableContext(TableId tableId);
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java
new file mode 100644
index 0000000..3a87f26
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/GroupBalancer.java
@@ -0,0 +1,733 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.function.Function;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.util.ComparablePair;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.commons.lang3.mutable.MutableInt;
+
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Multimap;
+
+/**
+ * A balancer that evenly spreads groups of tablets across all tablet server. This balancer
+ * accomplishes the following two goals :
+ *
+ * <ul>
+ * <li>Evenly spreads each group across all tservers.
+ * <li>Minimizes the total number of groups on each tserver.
+ * </ul>
+ *
+ * <p>
+ * To use this balancer you must extend it and implement {@link #getPartitioner()}. See
+ * {@link RegexGroupBalancer} as an example.
+ *
+ * @since 2.1.0
+ */
+public abstract class GroupBalancer implements TabletBalancer {
+
+  protected BalancerEnvironment environment;
+  private final TableId tableId;
+  private long lastRun = 0;
+
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    this.environment = balancerEnvironment;
+  }
+
+  /**
+   * @return A function that groups tablets into named groups.
+   */
+  protected abstract Function<TabletId,String> getPartitioner();
+
+  public GroupBalancer(TableId tableId) {
+    this.tableId = tableId;
+  }
+
+  protected Map<TabletId,TabletServerId> getLocationProvider() {
+    return environment.listTabletLocations(tableId);
+  }
+
+  /**
+   * The amount of time to wait between balancing.
+   */
+  protected long getWaitTime() {
+    return 60000;
+  }
+
+  /**
+   * The maximum number of migrations to perform in a single pass.
+   */
+  protected int getMaxMigrations() {
+    return 1000;
+  }
+
+  /**
+   * @return Examine current tserver and migrations and return true if balancing should occur.
+   */
+  protected boolean shouldBalance(SortedMap<TabletServerId,TServerStatus> current,
+      Set<TabletId> migrations) {
+
+    if (current.size() < 2) {
+      return false;
+    }
+
+    return migrations.stream().noneMatch(t -> t.getTable().equals(tableId));
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+
+    if (params.currentStatus().isEmpty()) {
+      return;
+    }
+
+    Function<TabletId,String> partitioner = getPartitioner();
+
+    List<ComparablePair<String,TabletId>> tabletsByGroup = new ArrayList<>();
+    for (Entry<TabletId,TabletServerId> entry : params.unassignedTablets().entrySet()) {
+      TabletServerId last = entry.getValue();
+      if (last != null) {
+        // Maintain locality
+        String fakeSessionID = " ";
+        TabletServerId simple =
+            new TabletServerIdImpl(last.getHost(), last.getPort(), fakeSessionID);
+        Iterator<TabletServerId> find = params.currentStatus().tailMap(simple).keySet().iterator();
+        if (find.hasNext()) {
+          TabletServerId tserver = find.next();
+          if (tserver.getHost().equals(last.getHost())) {
+            params.addAssignment(entry.getKey(), tserver);
+            continue;
+          }
+        }
+      }
+
+      tabletsByGroup.add(new ComparablePair<>(partitioner.apply(entry.getKey()), entry.getKey()));
+    }
+
+    Collections.sort(tabletsByGroup);
+
+    Iterator<TabletServerId> tserverIter = Iterators.cycle(params.currentStatus().keySet());
+    for (ComparablePair<String,TabletId> pair : tabletsByGroup) {
+      TabletId tabletId = pair.getSecond();
+      params.addAssignment(tabletId, tserverIter.next());
+    }
+
+  }
+
+  @Override
+  public long balance(BalanceParameters params) {
+
+    // The terminology extra and expected are used in this code. Expected tablets is the number of
+    // tablets a tserver must have for a given group and is
+    // numInGroup/numTservers. Extra tablets are any tablets more than the number expected for a
+    // given group. If numInGroup % numTservers > 0, then a tserver
+    // may have one extra tablet for a group.
+    //
+    // Assume we have 4 tservers and group A has 11 tablets.
+    // * expected tablets : group A is expected to have 2 tablets on each tservers
+    // * extra tablets : group A may have an additional tablet on each tserver. Group A has a total
+    // of 3 extra tablets.
+    //
+    // This balancer also evens out the extra tablets across all groups. The terminology
+    // extraExpected and extraExtra is used to describe these tablets.
+    // ExtraExpected is totalExtra/numTservers. ExtraExtra is totalExtra%numTservers. Each tserver
+    // should have at least expectedExtra extra tablets and at most
+    // one extraExtra tablets. All extra tablets on a tserver must be from different groups.
+    //
+    // Assume we have 6 tservers and three groups (G1, G2, G3) with 9 tablets each. Each tserver is
+    // expected to have one tablet from each group and could
+    // possibly have 2 tablets from a group. Below is an illustration of an ideal balancing of extra
+    // tablets. To understand the illustration, the first column
+    // shows tserver T1 with 2 tablets from G1, 1 tablet from G2, and two tablets from G3. EE means
+    // empty, put it there so eclipse formatting would not mess up
+    // table.
+    //
+    // T1 | T2 | T3 | T4 | T5 | T6
+    // ---+----+----+----+----+-----
+    // G3 | G2 | G3 | EE | EE | EE <-- extra extra tablets
+    // G1 | G1 | G1 | G2 | G3 | G2 <-- extra expected tablets.
+    // G1 | G1 | G1 | G1 | G1 | G1 <-- expected tablets for group 1
+    // G2 | G2 | G2 | G2 | G2 | G2 <-- expected tablets for group 2
+    // G3 | G3 | G3 | G3 | G3 | G3 <-- expected tablets for group 3
+    //
+    // Do not want to balance the extra tablets like the following. There are two problem with this.
+    // First extra tablets are not evenly spread. Since there are
+    // a total of 9 extra tablets, every tserver is expected to have at least one extra tablet.
+    // Second tserver T1 has two extra tablet for group G1. This
+    // violates the principal that a tserver can only have one extra tablet for a given group.
+    //
+    // T1 | T2 | T3 | T4 | T5 | T6
+    // ---+----+----+----+----+-----
+    // G1 | EE | EE | EE | EE | EE <--- one extra tablets from group 1
+    // G3 | G3 | G3 | EE | EE | EE <--- three extra tablets from group 3
+    // G2 | G2 | G2 | EE | EE | EE <--- three extra tablets from group 2
+    // G1 | G1 | EE | EE | EE | EE <--- two extra tablets from group 1
+    // G1 | G1 | G1 | G1 | G1 | G1 <-- expected tablets for group 1
+    // G2 | G2 | G2 | G2 | G2 | G2 <-- expected tablets for group 2
+    // G3 | G3 | G3 | G3 | G3 | G3 <-- expected tablets for group 3
+
+    if (!shouldBalance(params.currentStatus(), params.currentMigrations())) {
+      return 5000;
+    }
+
+    if (System.currentTimeMillis() - lastRun < getWaitTime()) {
+      return 5000;
+    }
+
+    MapCounter<String> groupCounts = new MapCounter<>();
+    Map<TabletServerId,TserverGroupInfo> tservers = new HashMap<>();
+
+    for (TabletServerId tsi : params.currentStatus().keySet()) {
+      tservers.put(tsi, new TserverGroupInfo(tsi));
+    }
+
+    Function<TabletId,String> partitioner = getPartitioner();
+
+    // collect stats about current state
+    for (var tablet : getLocationProvider().entrySet()) {
+      String group = partitioner.apply(tablet.getKey());
+      var loc = tablet.getValue();
+
+      if (loc == null || !tservers.containsKey(loc)) {
+        return 5000;
+      }
+
+      groupCounts.increment(group, 1);
+      TserverGroupInfo tgi = tservers.get(loc);
+      tgi.addGroup(group);
+    }
+
+    Map<String,Integer> expectedCounts = new HashMap<>();
+
+    int totalExtra = 0;
+    for (String group : groupCounts.keySet()) {
+      int groupCount = groupCounts.getInt(group);
+      totalExtra += groupCount % params.currentStatus().size();
+      expectedCounts.put(group, (groupCount / params.currentStatus().size()));
+    }
+
+    // The number of extra tablets from all groups that each tserver must have.
+    int expectedExtra = totalExtra / params.currentStatus().size();
+    int maxExtraGroups = expectedExtra + 1;
+
+    expectedCounts = Collections.unmodifiableMap(expectedCounts);
+    tservers = Collections.unmodifiableMap(tservers);
+
+    for (TserverGroupInfo tgi : tservers.values()) {
+      tgi.finishedAdding(expectedCounts);
+    }
+
+    Moves moves = new Moves();
+
+    // The order of the following steps is important, because as ordered each step should not move
+    // any tablets moved by a previous step.
+    balanceExpected(tservers, moves);
+    if (moves.size() < getMaxMigrations()) {
+      balanceExtraExpected(tservers, expectedExtra, moves);
+      if (moves.size() < getMaxMigrations()) {
+        boolean cont = balanceExtraMultiple(tservers, maxExtraGroups, moves);
+        if (cont && moves.size() < getMaxMigrations()) {
+          balanceExtraExtra(tservers, maxExtraGroups, moves);
+        }
+      }
+    }
+
+    populateMigrations(tservers.keySet(), params.migrationsOut(), moves);
+
+    lastRun = System.currentTimeMillis();
+
+    return 5000;
+  }
+
+  static class TserverGroupInfo {
+
+    private Map<String,Integer> expectedCounts;
+    private final Map<String,MutableInt> initialCounts = new HashMap<>();
+    private final Map<String,Integer> extraCounts = new HashMap<>();
+    private final Map<String,Integer> expectedDeficits = new HashMap<>();
+
+    private final TabletServerId tsi;
+    private boolean finishedAdding = false;
+
+    TserverGroupInfo(TabletServerId tsi) {
+      this.tsi = tsi;
+    }
+
+    public void addGroup(String group) {
+      checkState(!finishedAdding);
+
+      MutableInt mi = initialCounts.get(group);
+      if (mi == null) {
+        mi = new MutableInt();
+        initialCounts.put(group, mi);
+      }
+
+      mi.increment();
+    }
+
+    public void finishedAdding(Map<String,Integer> expectedCounts) {
+      checkState(!finishedAdding);
+      finishedAdding = true;
+      this.expectedCounts = expectedCounts;
+
+      for (Entry<String,Integer> entry : expectedCounts.entrySet()) {
+        String group = entry.getKey();
+        int expected = entry.getValue();
+
+        MutableInt count = initialCounts.get(group);
+        int num = count == null ? 0 : count.intValue();
+
+        if (num < expected) {
+          expectedDeficits.put(group, expected - num);
+        } else if (num > expected) {
+          extraCounts.put(group, num - expected);
+        }
+      }
+
+    }
+
+    public void moveOff(String group, int num) {
+      checkArgument(num > 0);
+      checkState(finishedAdding);
+
+      Integer extraCount = extraCounts.get(group);
+
+      // don't wrap precondition check due to https://github.com/spotbugs/spotbugs/issues/462
+      String formatString = "group=%s num=%s extraCount=%s";
+      checkArgument(extraCount != null && extraCount >= num, formatString, group, num, extraCount);
+
+      MutableInt initialCount = initialCounts.get(group);
+
+      checkArgument(initialCount.intValue() >= num);
+
+      initialCount.subtract(num);
+
+      if (extraCount - num == 0) {
+        extraCounts.remove(group);
+      } else {
+        extraCounts.put(group, extraCount - num);
+      }
+    }
+
+    public void moveTo(String group, int num) {
+      checkArgument(num > 0);
+      checkArgument(expectedCounts.containsKey(group));
+      checkState(finishedAdding);
+
+      Integer deficit = expectedDeficits.get(group);
+      if (deficit != null) {
+        if (num >= deficit) {
+          expectedDeficits.remove(group);
+          num -= deficit;
+        } else {
+          expectedDeficits.put(group, deficit - num);
+          num = 0;
+        }
+      }
+
+      if (num > 0) {
+        Integer extra = extraCounts.get(group);
+        if (extra == null) {
+          extra = 0;
+        }
+
+        extraCounts.put(group, extra + num);
+      }
+
+      // TODO could check extra constraints
+    }
+
+    public Map<String,Integer> getExpectedDeficits() {
+      checkState(finishedAdding);
+      return Collections.unmodifiableMap(expectedDeficits);
+    }
+
+    public Map<String,Integer> getExtras() {
+      checkState(finishedAdding);
+      return Collections.unmodifiableMap(extraCounts);
+    }
+
+    public TabletServerId getTabletServerId() {
+      return tsi;
+    }
+
+    @Override
+    public int hashCode() {
+      return tsi.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof TserverGroupInfo) {
+        TserverGroupInfo otgi = (TserverGroupInfo) o;
+        return tsi.equals(otgi.tsi);
+      }
+
+      return false;
+    }
+
+    @Override
+    public String toString() {
+      return tsi.toString();
+    }
+
+  }
+
+  private static class Move {
+    TserverGroupInfo dest;
+    int count;
+
+    public Move(TserverGroupInfo dest, int num) {
+      this.dest = dest;
+      this.count = num;
+    }
+  }
+
+  private static class Moves {
+
+    private final HashBasedTable<TabletServerId,String,List<Move>> moves = HashBasedTable.create();
+    private int totalMoves = 0;
+
+    public void move(String group, int num, TserverGroupInfo src, TserverGroupInfo dest) {
+      checkArgument(num > 0);
+      checkArgument(!src.equals(dest));
+
+      src.moveOff(group, num);
+      dest.moveTo(group, num);
+
+      List<Move> srcMoves = moves.get(src.getTabletServerId(), group);
+      if (srcMoves == null) {
+        srcMoves = new ArrayList<>();
+        moves.put(src.getTabletServerId(), group, srcMoves);
+      }
+
+      srcMoves.add(new Move(dest, num));
+      totalMoves += num;
+    }
+
+    public TabletServerId removeMove(TabletServerId src, String group) {
+      List<Move> srcMoves = moves.get(src, group);
+      if (srcMoves == null) {
+        return null;
+      }
+
+      Move move = srcMoves.get(srcMoves.size() - 1);
+      TabletServerId ret = move.dest.getTabletServerId();
+      totalMoves--;
+
+      move.count--;
+      if (move.count == 0) {
+        srcMoves.remove(srcMoves.size() - 1);
+        if (srcMoves.isEmpty()) {
+          moves.remove(src, group);
+        }
+      }
+
+      return ret;
+    }
+
+    public int size() {
+      return totalMoves;
+    }
+  }
+
+  private void balanceExtraExtra(Map<TabletServerId,TserverGroupInfo> tservers, int maxExtraGroups,
+      Moves moves) {
+    HashBasedTable<String,TabletServerId,TserverGroupInfo> surplusExtra = HashBasedTable.create();
+    for (TserverGroupInfo tgi : tservers.values()) {
+      Map<String,Integer> extras = tgi.getExtras();
+      if (extras.size() > maxExtraGroups) {
+        for (String group : extras.keySet()) {
+          surplusExtra.put(group, tgi.getTabletServerId(), tgi);
+        }
+      }
+    }
+
+    ArrayList<Pair<String,TabletServerId>> serversGroupsToRemove = new ArrayList<>();
+    ArrayList<TabletServerId> serversToRemove = new ArrayList<>();
+
+    for (TserverGroupInfo destTgi : tservers.values()) {
+      if (surplusExtra.isEmpty()) {
+        break;
+      }
+
+      Map<String,Integer> extras = destTgi.getExtras();
+      if (extras.size() < maxExtraGroups) {
+        serversToRemove.clear();
+        serversGroupsToRemove.clear();
+        for (String group : surplusExtra.rowKeySet()) {
+          if (!extras.containsKey(group)) {
+            TserverGroupInfo srcTgi = surplusExtra.row(group).values().iterator().next();
+
+            moves.move(group, 1, srcTgi, destTgi);
+
+            if (srcTgi.getExtras().size() <= maxExtraGroups) {
+              serversToRemove.add(srcTgi.getTabletServerId());
+            } else {
+              serversGroupsToRemove.add(new Pair<>(group, srcTgi.getTabletServerId()));
+            }
+
+            if (destTgi.getExtras().size() >= maxExtraGroups
+                || moves.size() >= getMaxMigrations()) {
+              break;
+            }
+          }
+        }
+
+        if (!serversToRemove.isEmpty()) {
+          surplusExtra.columnKeySet().removeAll(serversToRemove);
+        }
+
+        for (Pair<String,TabletServerId> pair : serversGroupsToRemove) {
+          surplusExtra.remove(pair.getFirst(), pair.getSecond());
+        }
+
+        if (moves.size() >= getMaxMigrations()) {
+          break;
+        }
+      }
+    }
+  }
+
+  private boolean balanceExtraMultiple(Map<TabletServerId,TserverGroupInfo> tservers,
+      int maxExtraGroups, Moves moves) {
+    Multimap<String,TserverGroupInfo> extraMultiple = HashMultimap.create();
+
+    for (TserverGroupInfo tgi : tservers.values()) {
+      Map<String,Integer> extras = tgi.getExtras();
+      for (Entry<String,Integer> entry : extras.entrySet()) {
+        if (entry.getValue() > 1) {
+          extraMultiple.put(entry.getKey(), tgi);
+        }
+      }
+    }
+
+    balanceExtraMultiple(tservers, maxExtraGroups, moves, extraMultiple, false);
+    if (moves.size() < getMaxMigrations() && !extraMultiple.isEmpty()) {
+      // no place to move so must exceed maxExtra temporarily... subsequent balancer calls will
+      // smooth things out
+      balanceExtraMultiple(tservers, maxExtraGroups, moves, extraMultiple, true);
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  private void balanceExtraMultiple(Map<TabletServerId,TserverGroupInfo> tservers,
+      int maxExtraGroups, Moves moves, Multimap<String,TserverGroupInfo> extraMultiple,
+      boolean alwaysAdd) {
+
+    ArrayList<Pair<String,TserverGroupInfo>> serversToRemove = new ArrayList<>();
+    for (TserverGroupInfo destTgi : tservers.values()) {
+      Map<String,Integer> extras = destTgi.getExtras();
+      if (alwaysAdd || extras.size() < maxExtraGroups) {
+        serversToRemove.clear();
+        for (String group : extraMultiple.keySet()) {
+          if (!extras.containsKey(group)) {
+            Collection<TserverGroupInfo> sources = extraMultiple.get(group);
+            Iterator<TserverGroupInfo> iter = sources.iterator();
+            TserverGroupInfo srcTgi = iter.next();
+
+            int num = srcTgi.getExtras().get(group);
+
+            moves.move(group, 1, srcTgi, destTgi);
+
+            if (num == 2) {
+              serversToRemove.add(new Pair<>(group, srcTgi));
+            }
+
+            if (destTgi.getExtras().size() >= maxExtraGroups
+                || moves.size() >= getMaxMigrations()) {
+              break;
+            }
+          }
+        }
+
+        for (Pair<String,TserverGroupInfo> pair : serversToRemove) {
+          extraMultiple.remove(pair.getFirst(), pair.getSecond());
+        }
+
+        if (extraMultiple.isEmpty() || moves.size() >= getMaxMigrations()) {
+          break;
+        }
+      }
+    }
+  }
+
+  private void balanceExtraExpected(Map<TabletServerId,TserverGroupInfo> tservers,
+      int expectedExtra, Moves moves) {
+
+    HashBasedTable<String,TabletServerId,TserverGroupInfo> extraSurplus = HashBasedTable.create();
+
+    for (TserverGroupInfo tgi : tservers.values()) {
+      Map<String,Integer> extras = tgi.getExtras();
+      if (extras.size() > expectedExtra) {
+        for (String group : extras.keySet()) {
+          extraSurplus.put(group, tgi.getTabletServerId(), tgi);
+        }
+      }
+    }
+
+    ArrayList<TabletServerId> emptyServers = new ArrayList<>();
+    ArrayList<Pair<String,TabletServerId>> emptyServerGroups = new ArrayList<>();
+    for (TserverGroupInfo destTgi : tservers.values()) {
+      if (extraSurplus.isEmpty()) {
+        break;
+      }
+
+      Map<String,Integer> extras = destTgi.getExtras();
+      if (extras.size() < expectedExtra) {
+        emptyServers.clear();
+        emptyServerGroups.clear();
+        nextGroup: for (String group : extraSurplus.rowKeySet()) {
+          if (!extras.containsKey(group)) {
+            Iterator<TserverGroupInfo> iter = extraSurplus.row(group).values().iterator();
+            TserverGroupInfo srcTgi = iter.next();
+
+            while (srcTgi.getExtras().size() <= expectedExtra) {
+              if (iter.hasNext()) {
+                srcTgi = iter.next();
+              } else {
+                continue nextGroup;
+              }
+            }
+
+            moves.move(group, 1, srcTgi, destTgi);
+
+            if (srcTgi.getExtras().size() <= expectedExtra) {
+              emptyServers.add(srcTgi.getTabletServerId());
+            } else if (srcTgi.getExtras().get(group) == null) {
+              emptyServerGroups.add(new Pair<>(group, srcTgi.getTabletServerId()));
+            }
+
+            if (destTgi.getExtras().size() >= expectedExtra || moves.size() >= getMaxMigrations()) {
+              break;
+            }
+          }
+        }
+
+        if (!emptyServers.isEmpty()) {
+          extraSurplus.columnKeySet().removeAll(emptyServers);
+        }
+
+        for (Pair<String,TabletServerId> pair : emptyServerGroups) {
+          extraSurplus.remove(pair.getFirst(), pair.getSecond());
+        }
+
+        if (moves.size() >= getMaxMigrations()) {
+          break;
+        }
+      }
+    }
+  }
+
+  private void balanceExpected(Map<TabletServerId,TserverGroupInfo> tservers, Moves moves) {
+    Multimap<String,TserverGroupInfo> groupDefecits = HashMultimap.create();
+    Multimap<String,TserverGroupInfo> groupSurplus = HashMultimap.create();
+
+    for (TserverGroupInfo tgi : tservers.values()) {
+      for (String group : tgi.getExpectedDeficits().keySet()) {
+        groupDefecits.put(group, tgi);
+      }
+
+      for (String group : tgi.getExtras().keySet()) {
+        groupSurplus.put(group, tgi);
+      }
+    }
+
+    for (String group : groupDefecits.keySet()) {
+      Collection<TserverGroupInfo> defecitServers = groupDefecits.get(group);
+      for (TserverGroupInfo defecitTsi : defecitServers) {
+        int numToMove = defecitTsi.getExpectedDeficits().get(group);
+
+        Iterator<TserverGroupInfo> surplusIter = groupSurplus.get(group).iterator();
+        while (numToMove > 0) {
+          TserverGroupInfo surplusTsi = surplusIter.next();
+
+          int available = surplusTsi.getExtras().get(group);
+
+          if (numToMove >= available) {
+            surplusIter.remove();
+          }
+
+          int transfer = Math.min(numToMove, available);
+
+          numToMove -= transfer;
+
+          moves.move(group, transfer, surplusTsi, defecitTsi);
+          if (moves.size() >= getMaxMigrations()) {
+            return;
+          }
+        }
+      }
+    }
+  }
+
+  private void populateMigrations(Set<TabletServerId> current, List<TabletMigration> migrationsOut,
+      Moves moves) {
+    if (moves.size() == 0) {
+      return;
+    }
+
+    Function<TabletId,String> partitioner = getPartitioner();
+
+    for (var tablet : getLocationProvider().entrySet()) {
+      String group = partitioner.apply(tablet.getKey());
+      var loc = tablet.getValue();
+
+      if (loc == null || !current.contains(loc)) {
+        migrationsOut.clear();
+        return;
+      }
+
+      TabletServerId dest = moves.removeMove(loc, group);
+      if (dest != null) {
+        migrationsOut.add(new TabletMigration(tablet.getKey(), loc, dest));
+        if (moves.size() == 0) {
+          break;
+        }
+      }
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
new file mode 100644
index 0000000..9038912
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
@@ -0,0 +1,569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Collection;
+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;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.PluginEnvironment;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TableStatisticsImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+
+/**
+ * This balancer creates groups of tablet servers using user-provided regular expressions over the
+ * tablet server hostnames. Then it delegates to the table balancer to balance the tablets within
+ * the resulting group of tablet servers. All tablet servers that do not match a regex are grouped
+ * into a default group.<br>
+ * Regex properties for this balancer are specified as:<br>
+ * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
+ * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets
+ * that it should not be according to the regex configuration. If this occurs then the offending
+ * tablets will be reassigned. This would cover the case where the configuration is changed and the
+ * master is restarted while the tablet servers are up. To change the out of bounds check time
+ * period, set the following property:<br>
+ * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
+ * Regex matching can be based on either the host name (default) or host ip address. To set this
+ * balancer to match the regular expressions to the tablet server IP address, then set the following
+ * property:<br>
+ * <b>table.custom.balancer.host.regex.is.ip=true</b><br>
+ * It's possible that this balancer may create a lot of migrations. To limit the number of
+ * migrations that are created during a balance call, set the following property (default 250):<br>
+ * <b>table.custom.balancer.host.regex.concurrent.migrations</b> This balancer can continue
+ * balancing even if there are outstanding migrations. To limit the number of outstanding migrations
+ * in which this balancer will continue balancing, set the following property (default 0):<br>
+ * <b>table.custom.balancer.host.regex.max.outstanding.migrations</b>
+ *
+ * @since 2.1.0
+ */
+public class HostRegexTableLoadBalancer extends TableLoadBalancer {
+
+  private static final String PROP_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey();
+
+  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
+  public static final String HOST_BALANCER_PREFIX = PROP_PREFIX + "balancer.host.regex.";
+  public static final String HOST_BALANCER_OOB_CHECK_KEY =
+      PROP_PREFIX + "balancer.host.regex.oob.period";
+  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
+  public static final String HOST_BALANCER_REGEX_USING_IPS_KEY =
+      PROP_PREFIX + "balancer.host.regex.is.ip";
+  public static final String HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY =
+      PROP_PREFIX + "balancer.host.regex.concurrent.migrations";
+  private static final int HOST_BALANCER_REGEX_MAX_MIGRATIONS_DEFAULT = 250;
+  protected static final String DEFAULT_POOL = "HostTableLoadBalancer.ALL";
+  private static final int DEFAULT_OUTSTANDING_MIGRATIONS = 0;
+  public static final String HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY =
+      PROP_PREFIX + "balancer.host.regex.max.outstanding.migrations";
+
+  private static Map<String,String> getRegexes(PluginEnvironment.Configuration conf) {
+    Map<String,String> regexes = new HashMap<>();
+    Map<String,String> customProps = conf.getWithPrefix(PROP_PREFIX);
+
+    if (customProps != null && !customProps.isEmpty()) {
+      for (Entry<String,String> customProp : customProps.entrySet()) {
+        if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
+          if (customProp.getKey().equals(HOST_BALANCER_OOB_CHECK_KEY)
+              || customProp.getKey().equals(HOST_BALANCER_REGEX_USING_IPS_KEY)
+              || customProp.getKey().equals(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY)
+              || customProp.getKey().equals(HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY)) {
+            continue;
+          }
+          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
+          String regex = customProp.getValue();
+          regexes.put(tableName, regex);
+        }
+      }
+    }
+
+    return Map.copyOf(regexes);
+  }
+
+  /**
+   * Host Regex Table Load Balance Config
+   */
+  static class HrtlbConf {
+
+    protected long oobCheckMillis =
+        ConfigurationTypeHelper.getTimeInMillis(HOST_BALANCER_OOB_DEFAULT);
+    private int maxTServerMigrations = HOST_BALANCER_REGEX_MAX_MIGRATIONS_DEFAULT;
+    private int maxOutstandingMigrations = DEFAULT_OUTSTANDING_MIGRATIONS;
+    private boolean isIpBasedRegex = false;
+    private final Map<String,String> regexes;
+    private final Map<String,Pattern> poolNameToRegexPattern;
+
+    HrtlbConf(PluginEnvironment.Configuration conf) {
+      System.out.println("building hrtlb conf");
+      String oobProperty = conf.get(HOST_BALANCER_OOB_CHECK_KEY);
+      if (oobProperty != null) {
+        oobCheckMillis = ConfigurationTypeHelper.getTimeInMillis(oobProperty);
+      }
+      String ipBased = conf.get(HOST_BALANCER_REGEX_USING_IPS_KEY);
+      if (ipBased != null) {
+        isIpBasedRegex = Boolean.parseBoolean(ipBased);
+      }
+      String migrations = conf.get(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY);
+      if (migrations != null) {
+        maxTServerMigrations = Integer.parseInt(migrations);
+      }
+      String outstanding = conf.get(HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY);
+      if (outstanding != null) {
+        maxOutstandingMigrations = Integer.parseInt(outstanding);
+      }
+
+      this.regexes = getRegexes(conf);
+
+      Map<String,Pattern> poolNameToRegexPatternBuilder = new HashMap<>();
+      regexes.forEach((k, v) -> poolNameToRegexPatternBuilder.put(k, Pattern.compile(v)));
+
+      poolNameToRegexPattern = Map.copyOf(poolNameToRegexPatternBuilder);
+    }
+  }
+
+  private static final long ONE_HOUR = 60 * 60 * 1000;
+  private static final Set<TabletId> EMPTY_MIGRATIONS = Collections.emptySet();
+  private volatile long lastOOBCheck = System.currentTimeMillis();
+  private Map<String,SortedMap<TabletServerId,TServerStatus>> pools = new HashMap<>();
+  private final Map<TabletId,TabletMigration> migrationsFromLastPass = new HashMap<>();
+  private final Map<TableId,Long> tableToTimeSinceNoMigrations = new HashMap<>();
+
+  private Supplier<HrtlbConf> hrtlbConf;
+  private LoadingCache<TableId,Supplier<Map<String,String>>> tablesRegExCache;
+
+  /**
+   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into
+   * a default pool. This could be expensive in the terms of the amount of time to recompute the
+   * groups, so HOST_BALANCER_POOL_RECHECK_KEY should be specified in the terms of minutes, not
+   * seconds or less.
+   *
+   * @param current
+   *          map of current tservers
+   * @return current servers grouped by pool name, if not a match it is put into a default pool.
+   */
+  protected synchronized Map<String,SortedMap<TabletServerId,TServerStatus>>
+      splitCurrentByRegex(SortedMap<TabletServerId,TServerStatus> current) {
+    LOG.debug("Performing pool recheck - regrouping tablet servers based on regular expressions");
+    Map<String,SortedMap<TabletServerId,TServerStatus>> newPools = new HashMap<>();
+    for (Entry<TabletServerId,TServerStatus> e : current.entrySet()) {
+      List<String> poolNames = getPoolNamesForHost(e.getKey().getHost());
+      for (String pool : poolNames) {
+        SortedMap<TabletServerId,TServerStatus> np = newPools.get(pool);
+        if (np == null) {
+          np = new TreeMap<>(current.comparator());
+          newPools.put(pool, np);
+        }
+        np.put(e.getKey(), e.getValue());
+      }
+    }
+
+    if (newPools.get(DEFAULT_POOL) == null) {
+      LOG.warn("Default pool is empty; assigning all tablet servers to the default pool");
+      SortedMap<TabletServerId,TServerStatus> dp = new TreeMap<>(current.comparator());
+      dp.putAll(current);
+      newPools.put(DEFAULT_POOL, dp);
+    }
+
+    pools = newPools;
+
+    LOG.trace("Pool to TabletServer mapping:");
+    if (LOG.isTraceEnabled()) {
+      for (Entry<String,SortedMap<TabletServerId,TServerStatus>> e : pools.entrySet()) {
+        LOG.trace("\tpool: {} -> tservers: {}", e.getKey(), e.getValue().keySet());
+      }
+    }
+    return pools;
+  }
+
+  /**
+   * Matches host against the regexes and returns the matching pool names
+   *
+   * @param host
+   *          tablet server host
+   * @return pool names, will return default pool if host matches more no regex
+   */
+  protected List<String> getPoolNamesForHost(String host) {
+    String test = host;
+    if (!hrtlbConf.get().isIpBasedRegex) {
+      try {
+        test = getNameFromIp(host);
+      } catch (UnknownHostException e1) {
+        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool",
+            e1);
+        return Collections.singletonList(DEFAULT_POOL);
+      }
+    }
+    List<String> pools = new ArrayList<>();
+    for (Entry<String,Pattern> e : hrtlbConf.get().poolNameToRegexPattern.entrySet()) {
+      if (e.getValue().matcher(test).matches()) {
+        pools.add(e.getKey());
+      }
+    }
+    if (pools.isEmpty()) {
+      pools.add(DEFAULT_POOL);
+    }
+    return pools;
+  }
+
+  protected String getNameFromIp(String hostIp) throws UnknownHostException {
+    return InetAddress.getByName(hostIp).getHostName();
+  }
+
+  private void checkTableConfig(TableId tableId) {
+    Map<String,String> tableRegexes = tablesRegExCache.getUnchecked(tableId).get();
+
+    if (!hrtlbConf.get().regexes.equals(tableRegexes)) {
+      LoggerFactory.getLogger(HostRegexTableLoadBalancer.class).warn(
+          "Table id {} has different config than system.  The per table config is ignored.",
+          tableId);
+    }
+  }
+
+  /**
+   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
+   *
+   * @param tableName
+   *          name of table
+   * @return tablet server pool name (table name or DEFAULT_POOL)
+   */
+  protected String getPoolNameForTable(String tableName) {
+    if (tableName == null) {
+      return DEFAULT_POOL;
+    }
+    return hrtlbConf.get().poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
+  }
+
+  @Override
+  public String toString() {
+    HrtlbConf myConf = hrtlbConf.get();
+    ToStringBuilder buf = new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE);
+    buf.append("\nTablet Out Of Bounds Check Interval", myConf.oobCheckMillis);
+    buf.append("\nMax Tablet Server Migrations", myConf.maxTServerMigrations);
+    buf.append("\nRegular Expressions use IPs", myConf.isIpBasedRegex);
+    buf.append("\nPools", myConf.poolNameToRegexPattern);
+    return buf.toString();
+  }
+
+  public Map<String,Pattern> getPoolNameToRegexPattern() {
+    return hrtlbConf.get().poolNameToRegexPattern;
+  }
+
+  public int getMaxMigrations() {
+    return hrtlbConf.get().maxTServerMigrations;
+  }
+
+  public int getMaxOutstandingMigrations() {
+    return hrtlbConf.get().maxOutstandingMigrations;
+  }
+
+  public long getOobCheckMillis() {
+    return hrtlbConf.get().oobCheckMillis;
+  }
+
+  public boolean isIpBasedRegex() {
+    return hrtlbConf.get().isIpBasedRegex;
+  }
+
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    super.init(balancerEnvironment);
+
+    this.hrtlbConf = balancerEnvironment.getConfiguration().getDerived(HrtlbConf::new);
+
+    tablesRegExCache =
+        CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.HOURS).build(new CacheLoader<>() {
+          @Override
+          public Supplier<Map<String,String>> load(TableId key) {
+            return balancerEnvironment.getConfiguration(key)
+                .getDerived(HostRegexTableLoadBalancer::getRegexes);
+          }
+        });
+
+    LOG.info("{}", this);
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+
+    Map<String,SortedMap<TabletServerId,TServerStatus>> pools =
+        splitCurrentByRegex(params.currentStatus());
+    // group the unassigned into tables
+    Map<TableId,Map<TabletId,TabletServerId>> groupedUnassigned = new HashMap<>();
+    params.unassignedTablets().forEach((ke, lastTserver) -> groupedUnassigned
+        .computeIfAbsent(ke.getTable(), k -> new HashMap<>()).put(ke, lastTserver));
+
+    Map<TableId,String> tableIdToTableName = environment.getTableIdMap().entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
+
+    // Send a view of the current servers to the tables tablet balancer
+    for (Entry<TableId,Map<TabletId,TabletServerId>> e : groupedUnassigned.entrySet()) {
+      Map<TabletId,TabletServerId> newAssignments = new HashMap<>();
+      String tableName = tableIdToTableName.get(e.getKey());
+      String poolName = getPoolNameForTable(tableName);
+      SortedMap<TabletServerId,TServerStatus> currentView = pools.get(poolName);
+      if (currentView == null || currentView.isEmpty()) {
+        LOG.warn("No tablet servers online for table {}, assigning within default pool", tableName);
+        currentView = pools.get(DEFAULT_POOL);
+        if (currentView == null) {
+          LOG.error(
+              "No tablet servers exist in the default pool, unable to assign tablets for table {}",
+              tableName);
+          continue;
+        }
+      }
+      LOG.debug("Sending {} tablets to balancer for table {} for assignment within tservers {}",
+          e.getValue().size(), tableName, currentView.keySet());
+      getBalancerForTable(e.getKey())
+          .getAssignments(new AssignmentParamsImpl(currentView, e.getValue(), newAssignments));
+      newAssignments.forEach(params::addAssignment);
+    }
+  }
+
+  @Override
+  public long balance(BalanceParameters params) {
+    long minBalanceTime = 20 * 1000;
+    // Iterate over the tables and balance each of them
+    Map<String,TableId> tableIdMap = environment.getTableIdMap();
+    Map<TableId,String> tableIdToTableName = tableIdMap.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey));
+    tableIdToTableName.keySet().forEach(this::checkTableConfig);
+
+    long now = System.currentTimeMillis();
+
+    HrtlbConf myConf = hrtlbConf.get();
+
+    SortedMap<TabletServerId,TServerStatus> current = params.currentStatus();
+    Set<TabletId> migrations = params.currentMigrations();
+    List<TabletMigration> migrationsOut = params.migrationsOut();
+
+    Map<String,SortedMap<TabletServerId,TServerStatus>> currentGrouped =
+        splitCurrentByRegex(params.currentStatus());
+    if ((now - this.lastOOBCheck) > myConf.oobCheckMillis) {
+      try {
+        // Check to see if a tablet is assigned outside the bounds of the pool. If so, migrate it.
+        for (String table : tableIdMap.keySet()) {
+          LOG.debug("Checking for out of bounds tablets for table {}", table);
+          String tablePoolName = getPoolNameForTable(table);
+          for (Entry<TabletServerId,TServerStatus> e : current.entrySet()) {
+            // pool names are the same as table names, except in the DEFAULT case.
+            // If this table is assigned to a pool for this host, then move on.
+            List<String> hostPools = getPoolNamesForHost(e.getKey().getHost());
+            if (hostPools.contains(tablePoolName)) {
+              continue;
+            }
+            TableId tid = tableIdMap.get(table);
+            if (tid == null) {
+              LOG.warn("Unable to check for out of bounds tablets for table {},"
+                  + " it may have been deleted or renamed.", table);
+              continue;
+            }
+            try {
+              List<TabletStatistics> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
+              if (outOfBoundsTablets == null) {
+                continue;
+              }
+              Random random = new SecureRandom();
+              for (TabletStatistics ts : outOfBoundsTablets) {
+                if (migrations.contains(ts.getTabletId())) {
+                  LOG.debug("Migration for out of bounds tablet {} has already been requested",
+                      ts.getTabletId());
+                  continue;
+                }
+                String poolName = getPoolNameForTable(table);
+                SortedMap<TabletServerId,TServerStatus> currentView = currentGrouped.get(poolName);
+                if (currentView != null) {
+                  int skip = random.nextInt(currentView.size());
+                  Iterator<TabletServerId> iter = currentView.keySet().iterator();
+                  for (int i = 0; i < skip; i++) {
+                    iter.next();
+                  }
+                  TabletServerId nextTS = iter.next();
+                  LOG.info(
+                      "Tablet {} is currently outside the bounds of the"
+                          + " regex, migrating from {} to {}",
+                      ts.getTabletId(), e.getKey(), nextTS);
+                  migrationsOut.add(new TabletMigration(ts.getTabletId(), e.getKey(), nextTS));
+                  if (migrationsOut.size() >= myConf.maxTServerMigrations) {
+                    break;
+                  }
+                } else {
+                  LOG.warn("No tablet servers online for pool {}, unable to"
+                      + " migrate out of bounds tablets", poolName);
+                }
+              }
+            } catch (AccumuloException | AccumuloSecurityException e1) {
+              LOG.error("Error in OOB check getting tablets for table {} from server {} {}", tid,
+                  e.getKey().getHost(), e);
+            }
+          }
+        }
+      } finally {
+        // this could have taken a while...get a new time
+        this.lastOOBCheck = System.currentTimeMillis();
+      }
+    }
+
+    if (!migrationsOut.isEmpty()) {
+      LOG.warn("Not balancing tables due to moving {} out of bounds tablets", migrationsOut.size());
+      LOG.info("Migrating out of bounds tablets: {}", migrationsOut);
+      return minBalanceTime;
+    }
+
+    if (migrations != null && !migrations.isEmpty()) {
+      if (migrations.size() >= myConf.maxOutstandingMigrations) {
+        LOG.warn("Not balancing tables due to {} outstanding migrations", migrations.size());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Sample up to 10 outstanding migrations: {}", Iterables.limit(migrations, 10));
+        }
+        return minBalanceTime;
+      }
+
+      LOG.debug("Current outstanding migrations of {} being applied", migrations.size());
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Sample up to 10 outstanding migrations: {}", Iterables.limit(migrations, 10));
+      }
+      migrationsFromLastPass.keySet().retainAll(migrations);
+      SortedMap<TabletServerId,TServerStatusImpl> currentCopy = new TreeMap<>();
+      current.forEach((tid, status) -> currentCopy.put(tid, (TServerStatusImpl) status));
+      Multimap<TabletServerId,String> serverTableIdCopied = HashMultimap.create();
+      for (TabletMigration migration : migrationsFromLastPass.values()) {
+        TableStatisticsImpl fromInfo = getTableInfo(currentCopy, serverTableIdCopied,
+            migration.getTablet().getTable().canonical(), migration.getOldTabletServer());
+        if (fromInfo != null) {
+          fromInfo.setOnlineTabletCount(fromInfo.getOnlineTabletCount() - 1);
+        }
+        TableStatisticsImpl toInfo = getTableInfo(currentCopy, serverTableIdCopied,
+            migration.getTablet().getTable().canonical(), migration.getNewTabletServer());
+        if (toInfo != null) {
+          toInfo.setOnlineTabletCount(toInfo.getOnlineTabletCount() + 1);
+        }
+      }
+      migrations = EMPTY_MIGRATIONS;
+    } else {
+      migrationsFromLastPass.clear();
+    }
+
+    for (TableId tableId : tableIdMap.values()) {
+      String tableName = tableIdToTableName.get(tableId);
+      String regexTableName = getPoolNameForTable(tableName);
+      SortedMap<TabletServerId,TServerStatus> currentView = currentGrouped.get(regexTableName);
+      if (currentView == null) {
+        LOG.warn("Skipping balance for table {} as no tablet servers are online.", tableName);
+        continue;
+      }
+      ArrayList<TabletMigration> newMigrations = new ArrayList<>();
+      getBalancerForTable(tableId)
+          .balance(new BalanceParamsImpl(currentView, migrations, newMigrations));
+
+      if (newMigrations.isEmpty()) {
+        tableToTimeSinceNoMigrations.remove(tableId);
+      } else if (tableToTimeSinceNoMigrations.containsKey(tableId)) {
+        if ((now - tableToTimeSinceNoMigrations.get(tableId)) > ONE_HOUR) {
+          LOG.warn("We have been consistently producing migrations for {}: {}", tableName,
+              Iterables.limit(newMigrations, 10));
+        }
+      } else {
+        tableToTimeSinceNoMigrations.put(tableId, now);
+      }
+
+      migrationsOut.addAll(newMigrations);
+      if (migrationsOut.size() >= myConf.maxTServerMigrations) {
+        break;
+      }
+    }
+
+    for (TabletMigration migration : migrationsOut) {
+      migrationsFromLastPass.put(migration.getTablet(), migration);
+    }
+
+    LOG.info("Migrating tablets for balance: {}", migrationsOut);
+    return minBalanceTime;
+  }
+
+  protected List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tabletServerId,
+      TableId tableId) throws AccumuloSecurityException, AccumuloException {
+    return environment.listOnlineTabletsForTable(tabletServerId, tableId);
+  }
+
+  /**
+   * Get a mutable table info for the specified table and server
+   */
+  private TableStatisticsImpl getTableInfo(SortedMap<TabletServerId,TServerStatusImpl> currentCopy,
+      Multimap<TabletServerId,String> serverTableIdCopied, String tableId, TabletServerId server) {
+    TableStatisticsImpl newInfo = null;
+    if (currentCopy.containsKey(server)) {
+      Map<String,TableStatistics> newTableMap = currentCopy.get(server).getTableMap();
+      if (newTableMap != null) {
+        newInfo = (TableStatisticsImpl) newTableMap.get(tableId);
+        if (newInfo != null) {
+          Collection<String> tableIdCopied = serverTableIdCopied.get(server);
+          if (tableIdCopied.isEmpty()) {
+            newTableMap = new HashMap<>(newTableMap);
+            currentCopy.get(server).setTableMap(newTableMap);
+          }
+          if (!tableIdCopied.contains(tableId)) {
+            newInfo = new TableStatisticsImpl(newInfo);
+            newTableMap.put(tableId, newInfo);
+            tableIdCopied.add(tableId);
+          }
+        }
+      }
+    }
+    return newInfo;
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/RegexGroupBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/RegexGroupBalancer.java
new file mode 100644
index 0000000..8f8ae27
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/RegexGroupBalancer.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.hadoop.io.Text;
+
+/**
+ * A {@link GroupBalancer} that groups tablets using a configurable regex. To use this balancer
+ * configure the following settings for your table then configure this balancer for your table.
+ *
+ * <ul>
+ * <li>Set {@code table.custom.balancer.group.regex.pattern} to a regular expression. This regular
+ * expression must have one group. The regex is applied to the tablet end row and whatever the regex
+ * group matches is used as the group. For example with a regex of {@code (\d\d).*} and an end row
+ * of {@code 12abc}, the group for the tablet would be {@code 12}.
+ * <li>Set {@code table.custom.balancer.group.regex.default} to a default group. This group is
+ * returned for the last tablet in the table and tablets for which the regex does not match.
+ * <li>Optionally set {@code table.custom.balancer.group.regex.wait.time} to time (can use time
+ * suffixes). This determines how long to wait between balancing. Since this balancer scans the
+ * metadata table, may want to set this higher for large tables.
+ * </ul>
+ *
+ * @since 2.1.0
+ */
+public class RegexGroupBalancer extends GroupBalancer {
+
+  public static final String REGEX_PROPERTY =
+      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.pattern";
+  public static final String DEFAUT_GROUP_PROPERTY =
+      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.default";
+  public static final String WAIT_TIME_PROPERTY =
+      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.wait.time";
+
+  private final TableId tableId;
+
+  public RegexGroupBalancer(TableId tableId) {
+    super(tableId);
+    this.tableId = tableId;
+  }
+
+  @Override
+  protected long getWaitTime() {
+    ServiceEnvironment.Configuration conf = environment.getConfiguration(tableId);
+    if (conf.isSet(WAIT_TIME_PROPERTY)) {
+      return ConfigurationTypeHelper.getTimeInMillis(conf.get(WAIT_TIME_PROPERTY));
+    }
+    return super.getWaitTime();
+  }
+
+  @Override
+  protected Function<TabletId,String> getPartitioner() {
+
+    ServiceEnvironment.Configuration conf = environment.getConfiguration(tableId);
+    String regex = conf.get(REGEX_PROPERTY);
+    final String defaultGroup = conf.get(DEFAUT_GROUP_PROPERTY);
+
+    final Pattern pattern = Pattern.compile(regex);
+
+    return input -> {
+      Text er = input.getEndRow();
+      if (er == null) {
+        return defaultGroup;
+      }
+
+      Matcher matcher = pattern.matcher(er.toString());
+      if (matcher.matches() && matcher.groupCount() == 1) {
+        return matcher.group(1);
+      }
+
+      return defaultGroup;
+    };
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
new file mode 100644
index 0000000..c19f650
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancer.java
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+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;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.spi.balancer.util.ThrottledBalancerProblemReporter;
+import org.apache.accumulo.core.spi.balancer.util.ThrottledBalancerProblemReporter.OutstandingMigrationsProblem;
+import org.apache.accumulo.core.spi.balancer.util.ThrottledBalancerProblemReporter.Problem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A simple tablet balancer that attempts to spread tablets out evenly across all available tablet
+ * servers. The goal is to achieve the same number of tablets on each tablet server.
+ *
+ * <p>
+ * Initial assignments attempt to maintain locality by assigning tablets to their previous location
+ * if possible, but otherwise assignments are made in a random fashion across all available tablet
+ * servers.
+ *
+ * @since 2.1.0
+ */
+public class SimpleLoadBalancer implements TabletBalancer {
+
+  private static final Logger log = LoggerFactory.getLogger(SimpleLoadBalancer.class);
+
+  protected BalancerEnvironment environment;
+
+  Iterator<TabletServerId> assignments;
+  // if tableToBalance is set, then only balance the given table
+  TableId tableToBalance = null;
+
+  public SimpleLoadBalancer() {}
+
+  public SimpleLoadBalancer(TableId table) {
+    tableToBalance = table;
+  }
+
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    this.environment = balancerEnvironment;
+  }
+
+  List<TabletServerId> randomize(Set<TabletServerId> locations) {
+    List<TabletServerId> result = new ArrayList<>(locations);
+    Collections.shuffle(result);
+    return result;
+  }
+
+  public TabletServerId getAssignment(SortedMap<TabletServerId,TServerStatus> locations,
+      TabletServerId last) {
+    if (locations.isEmpty())
+      return null;
+
+    if (last != null) {
+      // Maintain locality
+      String fakeSessionID = " ";
+      TabletServerId simple = new TabletServerIdImpl(last.getHost(), last.getPort(), fakeSessionID);
+      Iterator<TabletServerId> find = locations.tailMap(simple).keySet().iterator();
+      if (find.hasNext()) {
+        TabletServerId current = find.next();
+        if (current.getHost().equals(last.getHost()))
+          return current;
+      }
+    }
+
+    // The strategy here is to walk through the locations and hand them back, one at a time
+    // Grab an iterator off of the set of options; use a new iterator if it hands back something not
+    // in the current list.
+    if (assignments == null || !assignments.hasNext())
+      assignments = randomize(locations.keySet()).iterator();
+    TabletServerId result = assignments.next();
+    if (!locations.containsKey(result)) {
+      assignments = null;
+      return randomize(locations.keySet()).iterator().next();
+    }
+    return result;
+  }
+
+  static class ServerCounts implements Comparable<ServerCounts> {
+    public final TabletServerId server;
+    public int count;
+    public final TServerStatus status;
+
+    ServerCounts(int count, TabletServerId server, TServerStatus status) {
+      this.count = count;
+      this.server = server;
+      this.status = status;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(server) + count;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this || (obj instanceof ServerCounts && compareTo((ServerCounts) obj) == 0);
+    }
+
+    @Override
+    public int compareTo(ServerCounts obj) {
+      int result = count - obj.count;
+      if (result == 0)
+        return server.compareTo(obj.server);
+      return result;
+    }
+  }
+
+  public boolean getMigrations(Map<TabletServerId,TServerStatus> current,
+      List<TabletMigration> result) {
+    boolean moreBalancingNeeded = false;
+    try {
+      // no moves possible
+      if (current.size() < 2) {
+        return false;
+      }
+      final Map<TableId,Map<TabletId,TabletStatistics>> donerTabletStats = new HashMap<>();
+
+      // Sort by total number of online tablets, per server
+      int total = 0;
+      ArrayList<ServerCounts> totals = new ArrayList<>();
+      for (Entry<TabletServerId,TServerStatus> entry : current.entrySet()) {
+        int serverTotal = 0;
+        if (entry.getValue() != null && entry.getValue().getTableMap() != null) {
+          for (Entry<String,TableStatistics> e : entry.getValue().getTableMap().entrySet()) {
+            /*
+             * The check below was on entry.getKey(), but that resolves to a tabletserver not a
+             * tablename. Believe it should be e.getKey() which is a tablename
+             */
+            if (tableToBalance == null || tableToBalance.canonical().equals(e.getKey()))
+              serverTotal += e.getValue().getOnlineTabletCount();
+          }
+        }
+        totals.add(new ServerCounts(serverTotal, entry.getKey(), entry.getValue()));
+        total += serverTotal;
+      }
+
+      // order from low to high
+      totals.sort(Collections.reverseOrder());
+      int even = total / totals.size();
+      int numServersOverEven = total % totals.size();
+
+      // Move tablets from the servers with too many to the servers with
+      // the fewest but only nominate tablets to move once. This allows us
+      // to fill new servers with tablets from a mostly balanced server
+      // very quickly. However, it may take several balancing passes to move
+      // tablets from one hugely overloaded server to many slightly
+      // under-loaded servers.
+      int end = totals.size() - 1;
+      int movedAlready = 0;
+      int tooManyIndex = 0;
+      while (tooManyIndex < end) {
+        ServerCounts tooMany = totals.get(tooManyIndex);
+        int goal = even;
+        if (tooManyIndex < numServersOverEven) {
+          goal++;
+        }
+        int needToUnload = tooMany.count - goal;
+        ServerCounts tooLittle = totals.get(end);
+        int needToLoad = goal - tooLittle.count - movedAlready;
+        if (needToUnload < 1 && needToLoad < 1) {
+          break;
+        }
+        if (needToUnload >= needToLoad) {
+          result.addAll(move(tooMany, tooLittle, needToLoad, donerTabletStats));
+          end--;
+          movedAlready = 0;
+        } else {
+          result.addAll(move(tooMany, tooLittle, needToUnload, donerTabletStats));
+          movedAlready += needToUnload;
+        }
+        if (needToUnload > needToLoad) {
+          moreBalancingNeeded = true;
+        } else {
+          tooManyIndex++;
+          donerTabletStats.clear();
+        }
+      }
+
+    } finally {
+      log.trace("balance ended with {} migrations", result.size());
+    }
+    return moreBalancingNeeded;
+  }
+
+  /**
+   * Select a tablet based on differences between table loads; if the loads are even, use the
+   * busiest table
+   */
+  List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int count,
+      Map<TableId,Map<TabletId,TabletStatistics>> donerTabletStats) {
+
+    if (count == 0) {
+      return Collections.emptyList();
+    }
+
+    List<TabletMigration> result = new ArrayList<>();
+    // Copy counts so we can update them as we propose migrations
+    Map<TableId,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
+    Map<TableId,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
+
+    for (int i = 0; i < count; i++) {
+      TableId table;
+      Integer tooLittleCount;
+      if (tableToBalance == null) {
+        // find a table to migrate
+        // look for an uneven table count
+        int biggestDifference = 0;
+        TableId biggestDifferenceTable = null;
+        for (var tableEntry : tooMuchMap.entrySet()) {
+          TableId tableID = tableEntry.getKey();
+          tooLittleMap.putIfAbsent(tableID, 0);
+          int diff = tableEntry.getValue() - tooLittleMap.get(tableID);
+          if (diff > biggestDifference) {
+            biggestDifference = diff;
+            biggestDifferenceTable = tableID;
+          }
+        }
+        if (biggestDifference < 2) {
+          table = busiest(tooMuch.status.getTableMap());
+        } else {
+          table = biggestDifferenceTable;
+        }
+      } else {
+        // just balance the given table
+        table = tableToBalance;
+      }
+      Map<TabletId,TabletStatistics> onlineTabletsForTable = donerTabletStats.get(table);
+      try {
+        if (onlineTabletsForTable == null) {
+          onlineTabletsForTable = new HashMap<>();
+          List<TabletStatistics> stats = getOnlineTabletsForTable(tooMuch.server, table);
+          if (stats == null) {
+            log.warn("Unable to find tablets to move");
+            return result;
+          }
+          for (TabletStatistics stat : stats)
+            onlineTabletsForTable.put(stat.getTabletId(), stat);
+          donerTabletStats.put(table, onlineTabletsForTable);
+        }
+      } catch (Exception ex) {
+        log.error("Unable to select a tablet to move", ex);
+        return result;
+      }
+      TabletId tabletId = selectTablet(onlineTabletsForTable);
+      onlineTabletsForTable.remove(tabletId);
+      if (tabletId == null)
+        return result;
+      tooMuchMap.put(table, tooMuchMap.get(table) - 1);
+      /*
+       * If a table grows from 1 tablet then tooLittleMap.get(table) can return a null, since there
+       * is only one tabletserver that holds all of the tablets. Here we check to see if in fact
+       * that is the case and if so set the value to 0.
+       */
+      tooLittleCount = tooLittleMap.get(table);
+      if (tooLittleCount == null) {
+        tooLittleCount = 0;
+      }
+      tooLittleMap.put(table, tooLittleCount + 1);
+      tooMuch.count--;
+      tooLittle.count++;
+      result.add(new TabletMigration(tabletId, tooMuch.server, tooLittle.server));
+    }
+    return result;
+  }
+
+  protected List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tabletServerId,
+      TableId tableId) throws AccumuloSecurityException, AccumuloException {
+    return environment.listOnlineTabletsForTable(tabletServerId, tableId);
+  }
+
+  static Map<TableId,Integer> tabletCountsPerTable(TServerStatus status) {
+    Map<TableId,Integer> result = new HashMap<>();
+    if (status != null && status.getTableMap() != null) {
+      Map<String,TableStatistics> tableMap = status.getTableMap();
+      for (Entry<String,TableStatistics> entry : tableMap.entrySet()) {
+        result.put(TableId.of(entry.getKey()), entry.getValue().getOnlineTabletCount());
+      }
+    }
+    return result;
+  }
+
+  static TabletId selectTablet(Map<TabletId,TabletStatistics> extents) {
+    if (extents.isEmpty())
+      return null;
+    TabletId mostRecentlySplit = null;
+    long splitTime = 0;
+    for (Entry<TabletId,TabletStatistics> entry : extents.entrySet())
+      if (entry.getValue().getSplitCreationTime() >= splitTime) {
+        splitTime = entry.getValue().getSplitCreationTime();
+        mostRecentlySplit = entry.getKey();
+      }
+    return mostRecentlySplit;
+  }
+
+  // define what it means for a tablet to be busy
+  private static TableId busiest(Map<String,TableStatistics> tables) {
+    TableId result = null;
+    double busiest = Double.NEGATIVE_INFINITY;
+    for (Entry<String,TableStatistics> entry : tables.entrySet()) {
+      TableStatistics info = entry.getValue();
+      double busy = info.getIngestRate() + info.getQueryRate();
+      if (busy > busiest) {
+        busiest = busy;
+        result = TableId.of(entry.getKey());
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+    params.unassignedTablets().forEach((tabletId, tserverId) -> params.addAssignment(tabletId,
+        getAssignment(params.currentStatus(), tserverId)));
+  }
+
+  private final ThrottledBalancerProblemReporter problemReporter =
+      new ThrottledBalancerProblemReporter(getClass());
+  private final Problem noTserversProblem = problemReporter.createNoTabletServersProblem();
+  private final OutstandingMigrationsProblem outstandingMigrationsProblem =
+      problemReporter.createOutstandingMigrationsProblem();
+
+  @Override
+  public long balance(BalanceParameters params) {
+    // do we have any servers?
+    if (params.currentStatus().isEmpty()) {
+      problemReporter.reportProblem(noTserversProblem);
+    } else {
+      // Don't migrate if we have migrations in progress
+      if (params.currentMigrations().isEmpty()) {
+        problemReporter.clearProblemReportTimes();
+        if (getMigrations(params.currentStatus(), params.migrationsOut()))
+          return TimeUnit.SECONDS.toMillis(1);
+      } else {
+        outstandingMigrationsProblem.setMigrations(params.currentMigrations());
+        problemReporter.reportProblem(outstandingMigrationsProblem);
+      }
+    }
+    return 5 * 1000;
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancer.java
new file mode 100644
index 0000000..f046984
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancer.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.classloader.ClassLoaderUtil;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @since 2.1.0
+ */
+public class TableLoadBalancer implements TabletBalancer {
+
+  private static final Logger log = LoggerFactory.getLogger(TableLoadBalancer.class);
+
+  protected BalancerEnvironment environment;
+  Map<TableId,TabletBalancer> perTableBalancers = new HashMap<>();
+
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    this.environment = balancerEnvironment;
+  }
+
+  private TabletBalancer constructNewBalancerForTable(String clazzName, TableId tableId)
+      throws Exception {
+    String context = environment.tableContext(tableId);
+    Class<? extends TabletBalancer> clazz =
+        ClassLoaderUtil.loadClass(context, clazzName, TabletBalancer.class);
+    Constructor<? extends TabletBalancer> constructor = clazz.getConstructor(TableId.class);
+    return constructor.newInstance(tableId);
+  }
+
+  protected String getLoadBalancerClassNameForTable(TableId table) {
+    if (environment.isTableOnline(table))
+      return environment.getConfiguration(table).get(Property.TABLE_LOAD_BALANCER.getKey());
+    return null;
+  }
+
+  protected TabletBalancer getBalancerForTable(TableId tableId) {
+    TabletBalancer balancer = perTableBalancers.get(tableId);
+
+    String clazzName = getLoadBalancerClassNameForTable(tableId);
+
+    if (clazzName == null)
+      clazzName = SimpleLoadBalancer.class.getName();
+    if (balancer != null) {
+      if (!clazzName.equals(balancer.getClass().getName())) {
+        // the balancer class for this table does not match the class specified in the configuration
+        try {
+          balancer = constructNewBalancerForTable(clazzName, tableId);
+          perTableBalancers.put(tableId, balancer);
+          balancer.init(environment);
+
+          log.info("Loaded new class {} for table {}", clazzName, tableId);
+        } catch (Exception e) {
+          log.warn("Failed to load table balancer class {} for table {}", clazzName, tableId, e);
+        }
+      }
+    }
+    if (balancer == null) {
+      try {
+        balancer = constructNewBalancerForTable(clazzName, tableId);
+        log.info("Loaded class {} for table {}", clazzName, tableId);
+      } catch (Exception e) {
+        log.warn("Failed to load table balancer class {} for table {}", clazzName, tableId, e);
+      }
+
+      if (balancer == null) {
+        log.info("Using balancer {} for table {}", SimpleLoadBalancer.class.getName(), tableId);
+        balancer = new SimpleLoadBalancer(tableId);
+      }
+      perTableBalancers.put(tableId, balancer);
+      balancer.init(environment);
+    }
+    return balancer;
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+    // separate the unassigned into tables
+    Map<TableId,Map<TabletId,TabletServerId>> groupedUnassigned = new HashMap<>();
+    params.unassignedTablets()
+        .forEach((tid, lastTserver) -> groupedUnassigned
+            .computeIfAbsent(TableId.of(tid.getTable().canonical()), k -> new HashMap<>())
+            .put(tid, lastTserver));
+    for (Entry<TableId,Map<TabletId,TabletServerId>> e : groupedUnassigned.entrySet()) {
+      Map<TabletId,TabletServerId> newAssignments = new HashMap<>();
+      getBalancerForTable(e.getKey()).getAssignments(
+          new AssignmentParamsImpl(params.currentStatus(), e.getValue(), newAssignments));
+      newAssignments.forEach(params::addAssignment);
+    }
+  }
+
+  @Override
+  public long balance(BalanceParameters params) {
+    long minBalanceTime = 5 * 1000;
+    // Iterate over the tables and balance each of them
+    for (TableId tableId : environment.getTableIdMap().values()) {
+      ArrayList<TabletMigration> newMigrations = new ArrayList<>();
+      long tableBalanceTime = getBalancerForTable(tableId).balance(
+          new BalanceParamsImpl(params.currentStatus(), params.currentMigrations(), newMigrations));
+      if (tableBalanceTime < minBalanceTime)
+        minBalanceTime = tableBalanceTime;
+      params.migrationsOut().addAll(newMigrations);
+    }
+    return minBalanceTime;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java
new file mode 100644
index 0000000..0a4ef69
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+
+/**
+ * This class is responsible for managing the distribution of tablets throughout an Accumulo
+ * cluster. In most cases, users will want a balancer implementation which ensures a uniform
+ * distribution of tablets, so that no individual tablet server is handling significantly more work
+ * than any other.
+ *
+ * <p>
+ * Implementations may wish to store configuration in Accumulo's system configuration using the
+ * {@link Property#GENERAL_ARBITRARY_PROP_PREFIX}. They may also benefit from using per-table
+ * configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
+ *
+ * @since 2.1.0
+ */
+public interface TabletBalancer {
+
+  /**
+   * An interface for grouping parameters required for the balancer to assign unassigned tablets.
+   * This interface allows for evolution of the parameter set without changing the balancer's method
+   * signature.
+   *
+   * @since 2.1.0
+   */
+  interface AssignmentParameters {
+    /**
+     * @return the current status for all tablet servers (read-only)
+     */
+    SortedMap<TabletServerId,TServerStatus> currentStatus();
+
+    /**
+     * @return the tablets that need to be assigned, mapped to their previous known location
+     *         (read-only)
+     */
+    Map<TabletId,TabletServerId> unassignedTablets();
+
+    /**
+     * Assigns {@code tabletId} to {@code tabletServerId}.
+     */
+    void addAssignment(TabletId tabletId, TabletServerId tabletServerId);
+  }
+
+  /**
+   * An interface for grouping parameters required for the balancer to balance tablets. This
+   * interface allows for evolution of the parameter set without changing the balancer's method
+   * signature.
+   *
+   * @since 2.1.0
+   */
+  interface BalanceParameters {
+    /**
+     * @return the current status for all tablet servers (read-only)
+     */
+    SortedMap<TabletServerId,TServerStatus> currentStatus();
+
+    /**
+     * @return the migrations that are currently in progress (read-only)
+     */
+    Set<TabletId> currentMigrations();
+
+    /**
+     * @return a write-only map for storing new assignments made by the balancer. It is important
+     *         that any tablets found in {@link #currentMigrations()} are not included in the output
+     *         migrations.
+     */
+    List<TabletMigration> migrationsOut();
+  }
+
+  /**
+   * Initialize the TabletBalancer. This gives the balancer the opportunity to read the
+   * configuration.
+   */
+  void init(BalancerEnvironment balancerEnvironment);
+
+  /**
+   * Assign tablets to tablet servers. This method is called whenever the master finds tablets that
+   * are unassigned.
+   */
+  void getAssignments(AssignmentParameters params);
+
+  /**
+   * Ask the balancer if any migrations are necessary.
+   *
+   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires
+   * some minimum number of tservers, or a maximum number of outstanding migrations), it should
+   * issue a log message to alert operators. The message should be at WARN normally and at ERROR if
+   * the balancer knows that the problem can not self correct. It should not issue these messages
+   * more than once a minute. This method will not be called when there are unassigned tablets.
+   *
+   * @return the time, in milliseconds, to wait before re-balancing.
+   */
+  long balance(BalanceParameters params);
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TServerStatus.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TServerStatus.java
new file mode 100644
index 0000000..f14dfb7
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TServerStatus.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.data;
+
+import java.util.Map;
+
+/**
+ * @since 2.1.0
+ */
+public interface TServerStatus extends Comparable<TServerStatus> {
+  Map<String,TableStatistics> getTableMap();
+
+  long getLastContact();
+
+  String getName();
+
+  double getOsLoad();
+
+  long getHoldTime();
+
+  long getLookups();
+
+  long getIndexCacheHits();
+
+  long getIndexCacheRequests();
+
+  long getDataCacheHits();
+
+  long getDataCacheRequests();
+
+  long getFlushes();
+
+  long getSyncs();
+
+  String getVersion();
+
+  long getResponseTime();
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TableStatistics.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TableStatistics.java
new file mode 100644
index 0000000..21df202
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TableStatistics.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.data;
+
+/**
+ * @since 2.1.0
+ */
+public interface TableStatistics extends Comparable<TableStatistics> {
+  long getRecords();
+
+  long getRecordsInMemory();
+
+  int getTabletCount();
+
+  int getOnlineTabletCount();
+
+  double getIngestRate();
+
+  double getIngestByteRate();
+
+  double getQueryRate();
+
+  double getQueryByteRate();
+
+  double getScanRate();
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletMigration.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletMigration.java
new file mode 100644
index 0000000..cd00d9a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletMigration.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.data;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.TabletId;
+
+/**
+ * @since 2.1.0
+ */
+public class TabletMigration {
+  private final TabletId tabletId;
+  private final TabletServerId oldTabletServer;
+  private final TabletServerId newTabletServer;
+
+  public TabletMigration(TabletId tabletId, TabletServerId oldTabletServer,
+      TabletServerId newTabletServer) {
+    this.tabletId = requireNonNull(tabletId);
+    this.oldTabletServer = requireNonNull(oldTabletServer);
+    this.newTabletServer = requireNonNull(newTabletServer);
+  }
+
+  public TabletId getTablet() {
+    return tabletId;
+  }
+
+  public TabletServerId getOldTabletServer() {
+    return oldTabletServer;
+  }
+
+  public TabletServerId getNewTabletServer() {
+    return newTabletServer;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)
+      return true;
+    if (o == null || getClass() != o.getClass())
+      return false;
+    TabletMigration that = (TabletMigration) o;
+    return tabletId.equals(that.tabletId) && oldTabletServer.equals(that.oldTabletServer)
+        && newTabletServer.equals(that.newTabletServer);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(tabletId, oldTabletServer, newTabletServer);
+  }
+
+  @Override
+  public String toString() {
+    return tabletId + ": " + oldTabletServer + " -> " + newTabletServer;
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java
new file mode 100644
index 0000000..69d27db
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletServerId.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.data;
+
+/**
+ * @since 2.1.0
+ */
+public interface TabletServerId extends Comparable<TabletServerId> {
+  String getHost();
+
+  int getPort();
+
+  String getSession();
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
new file mode 100644
index 0000000..af38c49
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.data;
+
+import org.apache.accumulo.core.data.TabletId;
+
+/**
+ * @since 2.1.0
+ */
+public interface TabletStatistics extends Comparable<TabletStatistics> {
+  TabletId getTabletId();
+
+  long getNumEntries();
+
+  long getSplitCreationTime();
+
+  double getIngestRate();
+
+  double getQueryRate();
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/util/ThrottledBalancerProblemReporter.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/util/ThrottledBalancerProblemReporter.java
new file mode 100644
index 0000000..6a82dde
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/util/ThrottledBalancerProblemReporter.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer.util;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.WeakHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Throttles logging of balancer problems by storing the last log time for each problem and limiting
+ * reports to once per minute. The logger is generated internally from the supplied class in order
+ * to adhere to the requirements for the SPI package (which prohibits having the Logger class in the
+ * API).
+ */
+public class ThrottledBalancerProblemReporter {
+  public interface Problem {
+    void report();
+  }
+
+  public interface OutstandingMigrationsProblem extends Problem {
+    void setMigrations(Set<TabletId> migrations);
+  }
+
+  private static final long TIME_BETWEEN_WARNINGS = TimeUnit.SECONDS.toMillis(60);
+  private final WeakHashMap<Problem,Long> problemReportTimes = new WeakHashMap<>();
+  private final Logger log;
+
+  public ThrottledBalancerProblemReporter(Class<?> loggerClass) {
+    log = LoggerFactory.getLogger(loggerClass);
+  }
+
+  /**
+   * Create a new problem reporter to indicate there are no tablet servers available and balancing
+   * could not be performed. Balancers should only create a single instance of this problem, and
+   * reuse each time the problem is reported.
+   */
+  public Problem createNoTabletServersProblem() {
+    return () -> log.warn("Not balancing because we don't have any tservers.");
+  }
+
+  /**
+   * Create a new problem reporter to indicate that balancing could not be performed due to the
+   * existence of outstanding migrations. Balancers should only create a single instance of this
+   * problem and update its migrations list before each report.
+   */
+  public OutstandingMigrationsProblem createOutstandingMigrationsProblem() {
+    return new OutstandingMigrationsProblem() {
+      private Set<TabletId> migrations = Collections.emptySet();
+
+      @Override
+      public void setMigrations(Set<TabletId> migrations) {
+        this.migrations = migrations;
+      }
+
+      @Override
+      public void report() {
+        log.warn("Not balancing due to {} outstanding migrations.", migrations.size());
+        /*
+         * TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected
+         * information.
+         */
+        if (log.isDebugEnabled()) {
+          log.debug("Sample up to 10 outstanding migrations: {}",
+              migrations.stream().limit(10).map(String::valueOf).collect(Collectors.joining(", ")));
+        }
+        // Now that we've reported, clear out the migrations list so we don't hold it in memory.
+        migrations = Collections.emptySet();
+      }
+    };
+  }
+
+  /**
+   * Reports a balance problem. The {@link Problem#report()} will only be called up to once a minute
+   * for each problem that is reported repeatedly.
+   */
+  public void reportProblem(Problem problem) {
+    long reportTime = problemReportTimes.getOrDefault(problem, -1L);
+    if ((System.currentTimeMillis() - reportTime) > TIME_BETWEEN_WARNINGS) {
+      problem.report();
+      problemReportTimes.put(problem, System.currentTimeMillis());
+    }
+  }
+
+  /**
+   * Clears reported problems so that a problem report will be logged immediately the next time
+   * {@link #reportProblem(Problem)} is invoked.
+   */
+  public void clearProblemReportTimes() {
+    problemReportTimes.clear();
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java b/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java
index d0224e7..966e436 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java
@@ -37,7 +37,7 @@
   /**
    * @since 2.0.0
    */
-  public interface Configuration extends PluginEnvironment.Configuration {
+  interface Configuration extends PluginEnvironment.Configuration {
 
   }
 
@@ -46,6 +46,7 @@
    *         in zookeeper, which falls back to site configuration, which falls back to the default
    *         configuration.
    */
+  @Override
   Configuration getConfiguration();
 
   /**
@@ -55,5 +56,6 @@
    *         {@link #getConfiguration()}.
    *
    */
+  @Override
   Configuration getConfiguration(TableId tableId);
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java b/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
index 56e0b94..0009c86 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ConfigurationImpl.java
@@ -21,9 +21,12 @@
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
+import org.apache.accumulo.core.client.PluginEnvironment;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.PropertyType;
@@ -121,4 +124,13 @@
     return StreamSupport.stream(acfg.spliterator(), false)
         .filter(e -> !Property.isSensitive(e.getKey())).iterator();
   }
+
+  @Override
+  public <T> Supplier<T>
+      getDerived(Function<PluginEnvironment.Configuration,T> computeDerivedValue) {
+    Configuration outerConfiguration = this;
+    AccumuloConfiguration.Deriver<T> deriver =
+        acfg.newDeriver(entries -> computeDerivedValue.apply(outerConfiguration));
+    return deriver::derive;
+  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java
new file mode 100644
index 0000000..49219a9
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TableStatisticsImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.hadoop.io.Text;
+
+public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableLoadBalancer {
+
+  protected static class TestTable {
+    private final String tableName;
+    private final TableId id;
+
+    TestTable(String tableName, TableId id) {
+      this.tableName = tableName;
+      this.id = id;
+    }
+
+    public String getTableName() {
+      return tableName;
+    }
+
+    public TableId getId() {
+      return id;
+    }
+  }
+
+  protected static final HashMap<String,String> DEFAULT_TABLE_PROPERTIES = new HashMap<>();
+  {
+    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "7s");
+    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY,
+        "4");
+    DEFAULT_TABLE_PROPERTIES
+        .put(HostRegexTableLoadBalancer.HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY, "10");
+    DEFAULT_TABLE_PROPERTIES
+        .put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(), "r01.*");
+    DEFAULT_TABLE_PROPERTIES
+        .put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r02.*");
+    DEFAULT_TABLE_PROPERTIES.put(Property.TABLE_LOAD_BALANCER.getKey(),
+        TestSimpleBalancer.class.getName());
+  }
+
+  protected static final TestTable FOO = new TestTable("foo", TableId.of("1"));
+  protected static final TestTable BAR = new TestTable("bar", TableId.of("2"));
+  protected static final TestTable BAZ = new TestTable("baz", TableId.of("3"));
+
+  protected class TestSimpleBalancer extends SimpleLoadBalancer {
+    @Override
+    public List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver,
+        TableId tableId) {
+      String tableName = idToTableName(tableId);
+      TabletServerId initialLocation = initialTableLocation.get(tableName);
+      if (tserver.equals(initialLocation)) {
+        List<TabletStatistics> list = new ArrayList<>(5);
+        for (TabletId tabletId : tableTablets.get(tableName)) {
+          TabletStats thriftStats = new TabletStats();
+          thriftStats.setExtent(
+              new KeyExtent(tabletId.getTable(), tabletId.getEndRow(), tabletId.getPrevEndRow())
+                  .toThrift());
+          TabletStatistics stats = new TabletStatisticsImpl(thriftStats);
+          list.add(stats);
+        }
+        return list;
+      }
+      return null;
+    }
+  }
+
+  protected final Map<String,String> servers = new HashMap<>(15);
+  protected final SortedMap<TabletServerId,TServerStatus> allTabletServers = new TreeMap<>();
+  protected final Map<String,List<TabletId>> tableTablets = new HashMap<>(3);
+  protected final Map<String,TabletServerId> initialTableLocation = new HashMap<>(3);
+
+  {
+    servers.put("192.168.0.1", "r01s01");
+    servers.put("192.168.0.2", "r01s02");
+    servers.put("192.168.0.3", "r01s03");
+    servers.put("192.168.0.4", "r01s04");
+    servers.put("192.168.0.5", "r01s05");
+    servers.put("192.168.0.6", "r02s01");
+    servers.put("192.168.0.7", "r02s02");
+    servers.put("192.168.0.8", "r02s03");
+    servers.put("192.168.0.9", "r02s04");
+    servers.put("192.168.0.10", "r02s05");
+    servers.put("192.168.0.11", "r03s01");
+    servers.put("192.168.0.12", "r03s02");
+    servers.put("192.168.0.13", "r03s03");
+    servers.put("192.168.0.14", "r03s04");
+    servers.put("192.168.0.15", "r03s05");
+
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.12", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.13", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.14", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+    allTabletServers.put(new TabletServerIdImpl("192.168.0.15", 9997, Integer.toHexString(1)),
+        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+
+    initialTableLocation.put(FOO.getTableName(),
+        new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1)));
+    initialTableLocation.put(BAR.getTableName(),
+        new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1)));
+    initialTableLocation.put(BAZ.getTableName(),
+        new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1)));
+
+    tableTablets.put(FOO.getTableName(), new ArrayList<>());
+    tableTablets.get(FOO.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(FOO.getId(), new Text("1"), new Text("0"))));
+    tableTablets.get(FOO.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(FOO.getId(), new Text("2"), new Text("1"))));
+    tableTablets.get(FOO.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(FOO.getId(), new Text("3"), new Text("2"))));
+    tableTablets.get(FOO.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(FOO.getId(), new Text("4"), new Text("3"))));
+    tableTablets.get(FOO.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(FOO.getId(), new Text("5"), new Text("4"))));
+    tableTablets.put(BAR.getTableName(), new ArrayList<>());
+    tableTablets.get(BAR.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAR.getId(), new Text("11"), new Text("10"))));
+    tableTablets.get(BAR.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAR.getId(), new Text("12"), new Text("11"))));
+    tableTablets.get(BAR.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAR.getId(), new Text("13"), new Text("12"))));
+    tableTablets.get(BAR.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAR.getId(), new Text("14"), new Text("13"))));
+    tableTablets.get(BAR.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAR.getId(), new Text("15"), new Text("14"))));
+    tableTablets.put(BAZ.getTableName(), new ArrayList<>());
+    tableTablets.get(BAZ.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAZ.getId(), new Text("21"), new Text("20"))));
+    tableTablets.get(BAZ.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAZ.getId(), new Text("22"), new Text("21"))));
+    tableTablets.get(BAZ.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAZ.getId(), new Text("23"), new Text("22"))));
+    tableTablets.get(BAZ.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAZ.getId(), new Text("24"), new Text("23"))));
+    tableTablets.get(BAZ.getTableName())
+        .add(new TabletIdImpl(new KeyExtent(BAZ.getId(), new Text("25"), new Text("24"))));
+
+  }
+
+  protected boolean tabletInBounds(TabletId tabletId, TabletServerId tsi) {
+    String tid = tabletId.getTable().canonical();
+    String host = tsi.getHost();
+    if (tid.equals("1")
+        && (host.equals("192.168.0.1") || host.equals("192.168.0.2") || host.equals("192.168.0.3")
+            || host.equals("192.168.0.4") || host.equals("192.168.0.5"))) {
+      return true;
+    } else if (tid.equals("2")
+        && (host.equals("192.168.0.6") || host.equals("192.168.0.7") || host.equals("192.168.0.8")
+            || host.equals("192.168.0.9") || host.equals("192.168.0.10"))) {
+      return true;
+    } else {
+      return tid.equals("3") && (host.equals("192.168.0.11") || host.equals("192.168.0.12")
+          || host.equals("192.168.0.13") || host.equals("192.168.0.14")
+          || host.equals("192.168.0.15"));
+    }
+  }
+
+  protected String idToTableName(TableId id) {
+    if (id.equals(FOO.getId())) {
+      return FOO.getTableName();
+    } else if (id.equals(BAR.getId())) {
+      return BAR.getTableName();
+    } else if (id.equals(BAZ.getId())) {
+      return BAZ.getTableName();
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  protected TabletBalancer getBalancerForTable(TableId table) {
+    return new TestSimpleBalancer();
+  }
+
+  @Override
+  protected String getNameFromIp(String hostIp) throws UnknownHostException {
+    if (servers.containsKey(hostIp)) {
+      return servers.get(hostIp);
+    } else {
+      throw new UnknownHostException();
+    }
+  }
+
+  protected SortedMap<TabletServerId,TServerStatus> createCurrent(int numTservers) {
+    String base = "192.168.0.";
+    TreeMap<TabletServerId,TServerStatus> current = new TreeMap<>();
+    for (int i = 1; i <= numTservers; i++) {
+      TServerStatusImpl status =
+          new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus());
+      Map<String,TableStatistics> tableMap = new HashMap<>();
+      tableMap.put(FOO.getId().canonical(),
+          new TableStatisticsImpl(new org.apache.accumulo.core.master.thrift.TableInfo()));
+      tableMap.put(BAR.getId().canonical(),
+          new TableStatisticsImpl(new org.apache.accumulo.core.master.thrift.TableInfo()));
+      tableMap.put(BAZ.getId().canonical(),
+          new TableStatisticsImpl(new org.apache.accumulo.core.master.thrift.TableInfo()));
+      status.setTableMap(tableMap);
+      current.put(new TabletServerIdImpl(base + i, 9997, Integer.toHexString(1)), status);
+    }
+    // now put all of the tablets on one server
+    for (Map.Entry<String,TabletServerId> entry : initialTableLocation.entrySet()) {
+      TServerStatus status = current.get(entry.getValue());
+      if (status != null) {
+        TableId tableId = environment.getTableIdMap().get(entry.getKey());
+        ((TableStatisticsImpl) status.getTableMap().get(tableId.canonical()))
+            .setOnlineTabletCount(5);
+      }
+    }
+    return current;
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java
new file mode 100644
index 0000000..634757e
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java
@@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.function.Function;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class GroupBalancerTest {
+
+  private static final Function<TabletId,String> partitioner =
+      input -> (input == null || input.getEndRow() == null) ? null
+          : input.getEndRow().toString().substring(0, 2);
+
+  public static class TabletServers {
+    private final Set<TabletServerId> tservers = new HashSet<>();
+    private final Map<TabletId,TabletServerId> tabletLocs = new HashMap<>();
+
+    public void addTservers(String... locs) {
+      for (String loc : locs) {
+        int idx = loc.indexOf(':');
+        addTserver(loc.substring(0, idx), Integer.parseInt(loc.substring(idx + 1)));
+      }
+    }
+
+    public void addTserver(String host, int port) {
+      tservers.add(new TabletServerIdImpl(host, port, Long.toHexString(6)));
+    }
+
+    public void addTablet(String er, String host, int port) {
+      TabletServerId tsi = new TabletServerIdImpl(host, port, Long.toHexString(6));
+      tabletLocs.put(
+          new TabletIdImpl(new KeyExtent(TableId.of("b"), er == null ? null : new Text(er), null)),
+          new TabletServerIdImpl(host, port, Long.toHexString(6)));
+      tservers.add(tsi);
+    }
+
+    public void balance() {
+      balance(10000);
+    }
+
+    public void balance(final int maxMigrations) {
+      GroupBalancer balancer = new GroupBalancer(TableId.of("1")) {
+
+        @Override
+        protected Map<TabletId,TabletServerId> getLocationProvider() {
+          return tabletLocs;
+        }
+
+        @Override
+        protected Function<TabletId,String> getPartitioner() {
+          return partitioner;
+        }
+
+        @Override
+        protected long getWaitTime() {
+          return 0;
+        }
+
+        @Override
+        protected int getMaxMigrations() {
+          return maxMigrations;
+        }
+      };
+
+      balance(balancer, maxMigrations);
+    }
+
+    public void balance(TabletBalancer balancer, int maxMigrations) {
+
+      while (true) {
+        Set<TabletId> migrations = new HashSet<>();
+        List<TabletMigration> migrationsOut = new ArrayList<>();
+        SortedMap<TabletServerId,TServerStatus> current = new TreeMap<>();
+
+        for (TabletServerId tsi : tservers) {
+          current.put(tsi, new TServerStatusImpl(
+              new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        }
+
+        balancer.balance(new BalanceParamsImpl(current, migrations, migrationsOut));
+
+        assertTrue("Max Migration exceeded " + maxMigrations + " " + migrationsOut.size(),
+            migrationsOut.size() <= (maxMigrations + 5));
+
+        for (TabletMigration tabletMigration : migrationsOut) {
+          assertEquals(tabletLocs.get(tabletMigration.getTablet()),
+              tabletMigration.getOldTabletServer());
+          assertTrue(tservers.contains(tabletMigration.getNewTabletServer()));
+
+          tabletLocs.put(tabletMigration.getTablet(), tabletMigration.getNewTabletServer());
+        }
+
+        if (migrationsOut.isEmpty()) {
+          break;
+        }
+      }
+
+      checkBalance();
+    }
+
+    void checkBalance() {
+      MapCounter<String> groupCounts = new MapCounter<>();
+      Map<TabletServerId,MapCounter<String>> tserverGroupCounts = new HashMap<>();
+
+      for (Entry<TabletId,TabletServerId> entry : tabletLocs.entrySet()) {
+        String group = partitioner.apply(entry.getKey());
+        TabletServerId loc = entry.getValue();
+
+        groupCounts.increment(group, 1);
+        MapCounter<String> tgc = tserverGroupCounts.get(loc);
+        if (tgc == null) {
+          tgc = new MapCounter<>();
+          tserverGroupCounts.put(loc, tgc);
+        }
+
+        tgc.increment(group, 1);
+      }
+
+      Map<String,Integer> expectedCounts = new HashMap<>();
+
+      int totalExtra = 0;
+      for (String group : groupCounts.keySet()) {
+        long groupCount = groupCounts.get(group);
+        totalExtra += groupCount % tservers.size();
+        expectedCounts.put(group, (int) (groupCount / tservers.size()));
+      }
+
+      // The number of extra tablets from all groups that each tserver must have.
+      int expectedExtra = totalExtra / tservers.size();
+      int maxExtraGroups = expectedExtra + ((totalExtra % tservers.size() > 0) ? 1 : 0);
+
+      for (Entry<TabletServerId,MapCounter<String>> entry : tserverGroupCounts.entrySet()) {
+        MapCounter<String> tgc = entry.getValue();
+        int tserverExtra = 0;
+        for (String group : groupCounts.keySet()) {
+          assertTrue(tgc.get(group) >= expectedCounts.get(group));
+          assertTrue(
+              "Group counts not as expected group:" + group + " actual:" + tgc.get(group)
+                  + " expected:" + (expectedCounts.get(group) + 1) + " tserver:" + entry.getKey(),
+              tgc.get(group) <= expectedCounts.get(group) + 1);
+          tserverExtra += tgc.get(group) - expectedCounts.get(group);
+        }
+
+        assertTrue(tserverExtra >= expectedExtra);
+        assertTrue(tserverExtra <= maxExtraGroups);
+      }
+    }
+  }
+
+  @Test
+  public void testSingleGroup() {
+
+    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
+        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
+
+    for (String[] suffixes : tests) {
+      for (int maxTS = 1; maxTS <= 4; maxTS++) {
+        TabletServers tservers = new TabletServers();
+        int ts = 0;
+        for (String s : suffixes) {
+          tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+        }
+
+        tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
+        tservers.balance();
+        tservers.balance();
+      }
+    }
+  }
+
+  @Test
+  public void testTwoGroups() {
+    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
+        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
+
+    for (String[] suffixes1 : tests) {
+      for (String[] suffixes2 : tests) {
+        for (int maxTS = 1; maxTS <= 4; maxTS++) {
+          TabletServers tservers = new TabletServers();
+          int ts = 0;
+          for (String s : suffixes1) {
+            tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+          }
+
+          for (String s : suffixes2) {
+            tservers.addTablet("02" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+          }
+
+          tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
+          tservers.balance();
+          tservers.balance();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testThreeGroups() {
+    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
+        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
+        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
+
+    for (String[] suffixes1 : tests) {
+      for (String[] suffixes2 : tests) {
+        for (String[] suffixes3 : tests) {
+          for (int maxTS = 1; maxTS <= 4; maxTS++) {
+            TabletServers tservers = new TabletServers();
+            int ts = 0;
+            for (String s : suffixes1) {
+              tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+            }
+
+            for (String s : suffixes2) {
+              tservers.addTablet("02" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+            }
+
+            for (String s : suffixes3) {
+              tservers.addTablet("03" + s, "192.168.1." + ((ts++ % maxTS) + 1), 9997);
+            }
+
+            tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
+            tservers.balance();
+            tservers.balance();
+          }
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testManySingleTabletGroups() {
+
+    for (int numGroups = 1; numGroups <= 13; numGroups++) {
+      for (int maxTS = 1; maxTS <= 4; maxTS++) {
+        TabletServers tservers = new TabletServers();
+        int ts = 0;
+
+        for (int group = 1; group <= numGroups; group++) {
+          tservers.addTablet(String.format("%02d:p", group), "192.168.1." + ((ts++ % maxTS) + 1),
+              9997);
+        }
+
+        tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
+
+        tservers.balance();
+        tservers.balance();
+      }
+    }
+  }
+
+  @Test
+  public void testMaxMigrations() {
+
+    for (int max : new int[] {1, 2, 3, 7, 10, 30}) {
+      TabletServers tservers = new TabletServers();
+
+      for (int i = 1; i <= 9; i++) {
+        tservers.addTablet("01" + i, "192.168.1.1", 9997);
+      }
+
+      for (int i = 1; i <= 4; i++) {
+        tservers.addTablet("02" + i, "192.168.1.2", 9997);
+      }
+
+      for (int i = 1; i <= 5; i++) {
+        tservers.addTablet("03" + i, "192.168.1.3", 9997);
+      }
+
+      tservers.addTservers("192.168.1.4:9997", "192.168.1.5:9997");
+
+      tservers.balance(max);
+    }
+  }
+
+  @Test
+  public void bigTest() {
+    TabletServers tservers = new TabletServers();
+    Random rand = new SecureRandom();
+
+    for (int g = 1; g <= 60; g++) {
+      for (int t = 1; t <= 241; t++) {
+        tservers.addTablet(String.format("%02d:%d", g, t), "192.168.1." + (rand.nextInt(249) + 1),
+            9997);
+      }
+    }
+
+    for (int i = 1; i <= 250; i++) {
+      tservers.addTserver("192.168.1." + i, 9997);
+    }
+
+    tservers.balance(1000);
+  }
+
+  @Test
+  public void bigTest2() {
+    TabletServers tservers = new TabletServers();
+    Random rand = new SecureRandom();
+
+    for (int g = 1; g <= 60; g++) {
+      for (int t = 1; t <= rand.nextInt(1000); t++) {
+        tservers.addTablet(String.format("%02d:%d", g, t), "192.168.1." + (rand.nextInt(249) + 1),
+            9997);
+      }
+    }
+
+    for (int i = 1; i <= 250; i++) {
+      tservers.addTserver("192.168.1." + i, 9997);
+    }
+
+    tservers.balance(1000);
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
new file mode 100644
index 0000000..6440fa3
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.junit.Test;
+
+public class HostRegexTableLoadBalancerReconfigurationTest
+    extends BaseHostRegexTableLoadBalancerTest {
+
+  private final Map<TabletId,TabletServerId> assignments = new HashMap<>();
+
+  @Test
+  public void testConfigurationChanges() {
+    HashMap<String,TableId> tables = new HashMap<>();
+    tables.put(FOO.getTableName(), FOO.getId());
+    tables.put(BAR.getTableName(), BAR.getId());
+    tables.put(BAZ.getTableName(), BAZ.getId());
+
+    ConfigurationCopy config = new ConfigurationCopy(SiteConfiguration.auto());
+    DEFAULT_TABLE_PROPERTIES.forEach(config::set);
+    ConfigurationImpl configImpl = new ConfigurationImpl(config);
+    BalancerEnvironment environment = createMock(BalancerEnvironment.class);
+    expect(environment.getConfiguration()).andReturn(configImpl).anyTimes();
+    expect(environment.getTableIdMap()).andReturn(tables).anyTimes();
+    expect(environment.getConfiguration(anyObject(TableId.class))).andReturn(configImpl).anyTimes();
+    replay(environment);
+    init(environment);
+
+    Map<TabletId,TabletServerId> unassigned = new HashMap<>();
+    for (List<TabletId> tablets : tableTablets.values()) {
+      for (TabletId tablet : tablets) {
+        unassigned.put(tablet, null);
+      }
+    }
+    this.getAssignments(
+        new AssignmentParamsImpl(Collections.unmodifiableSortedMap(allTabletServers),
+            Collections.unmodifiableMap(unassigned), assignments));
+    assertEquals(15, assignments.size());
+    // Ensure unique tservers
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      for (Entry<TabletId,TabletServerId> e2 : assignments.entrySet()) {
+        if (e.getKey().equals(e2.getKey())) {
+          continue;
+        }
+        if (e.getValue().equals(e2.getValue())) {
+          fail("Assignment failure. " + e.getKey() + " and " + e2.getKey()
+              + " are assigned to the same host: " + e.getValue());
+        }
+      }
+    }
+    // Ensure assignments are correct
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      if (!tabletInBounds(e.getKey(), e.getValue())) {
+        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
+      }
+    }
+    Set<TabletId> migrations = new HashSet<>();
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    // Wait to trigger the out of bounds check which will call our version of
+    // getOnlineTabletsForTable
+    UtilWaitThread.sleep(3000);
+    this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(allTabletServers),
+        migrations, migrationsOut));
+    assertEquals(0, migrationsOut.size());
+    // Change property, simulate call by TableConfWatcher
+
+    config.set(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r01.*");
+
+    // Wait to trigger the out of bounds check and the repool check
+    UtilWaitThread.sleep(10000);
+    this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(allTabletServers),
+        migrations, migrationsOut));
+    assertEquals(5, migrationsOut.size());
+    for (TabletMigration migration : migrationsOut) {
+      assertTrue(migration.getNewTabletServer().getHost().startsWith("192.168.0.1")
+          || migration.getNewTabletServer().getHost().startsWith("192.168.0.2")
+          || migration.getNewTabletServer().getHost().startsWith("192.168.0.3")
+          || migration.getNewTabletServer().getHost().startsWith("192.168.0.4")
+          || migration.getNewTabletServer().getHost().startsWith("192.168.0.5"));
+    }
+  }
+
+  @Override
+  public List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver, TableId tableId) {
+    List<TabletStatistics> tablets = new ArrayList<>();
+    // Report assignment information
+    for (Entry<TabletId,TabletServerId> e : this.assignments.entrySet()) {
+      if (e.getValue().equals(tserver) && e.getKey().getTable().equals(tableId)) {
+        TabletStats ts = new TabletStats();
+        TabletId tid = e.getKey();
+        ts.setExtent(
+            new KeyExtent(tid.getTable(), tid.getEndRow(), tid.getPrevEndRow()).toThrift());
+        tablets.add(new TabletStatisticsImpl(ts));
+      }
+    }
+    return tablets;
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java
new file mode 100644
index 0000000..33fb741
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancerTest.java
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.regex.Pattern;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.junit.Test;
+
+public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalancerTest {
+
+  public void init(Map<String,String> tableProperties) {
+    HashMap<String,TableId> tables = new HashMap<>();
+    tables.put(FOO.getTableName(), FOO.getId());
+    tables.put(BAR.getTableName(), BAR.getId());
+    tables.put(BAZ.getTableName(), BAZ.getId());
+
+    ConfigurationCopy config = new ConfigurationCopy(SiteConfiguration.auto());
+    tableProperties.forEach(config::set);
+    ConfigurationImpl configImpl = new ConfigurationImpl(config);
+    BalancerEnvironment environment = createMock(BalancerEnvironment.class);
+    expect(environment.getConfiguration()).andReturn(configImpl).anyTimes();
+    expect(environment.getTableIdMap()).andReturn(tables).anyTimes();
+    expect(environment.getConfiguration(anyObject(TableId.class))).andReturn(configImpl).anyTimes();
+    replay(environment);
+    init(environment);
+  }
+
+  @Test
+  public void testInit() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    assertEquals("OOB check interval value is incorrect", 7000, this.getOobCheckMillis());
+    assertEquals("Max migrations is incorrect", 4, this.getMaxMigrations());
+    assertEquals("Max outstanding migrations is incorrect", 10, this.getMaxOutstandingMigrations());
+    assertFalse(isIpBasedRegex());
+    Map<String,Pattern> patterns = this.getPoolNameToRegexPattern();
+    assertEquals(2, patterns.size());
+    assertTrue(patterns.containsKey(FOO.getTableName()));
+    assertEquals(Pattern.compile("r01.*").pattern(), patterns.get(FOO.getTableName()).pattern());
+    assertTrue(patterns.containsKey(BAR.getTableName()));
+    assertEquals(Pattern.compile("r02.*").pattern(), patterns.get(BAR.getTableName()).pattern());
+  }
+
+  @Test
+  public void testBalance() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Set<TabletId> migrations = new HashSet<>();
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    long wait =
+        this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(createCurrent(15)),
+            migrations, migrationsOut));
+    assertEquals(20000, wait);
+    // should balance four tablets in one of the tables before reaching max
+    assertEquals(4, migrationsOut.size());
+
+    // now balance again passing in the new migrations
+    for (TabletMigration m : migrationsOut) {
+      migrations.add(m.getTablet());
+    }
+    migrationsOut.clear();
+    wait = this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(createCurrent(15)),
+        migrations, migrationsOut));
+    assertEquals(20000, wait);
+    // should balance four tablets in one of the other tables before reaching max
+    assertEquals(4, migrationsOut.size());
+
+    // now balance again passing in the new migrations
+    for (TabletMigration m : migrationsOut) {
+      migrations.add(m.getTablet());
+    }
+    migrationsOut.clear();
+    wait = this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(createCurrent(15)),
+        migrations, migrationsOut));
+    assertEquals(20000, wait);
+    // should balance four tablets in one of the other tables before reaching max
+    assertEquals(4, migrationsOut.size());
+
+    // now balance again passing in the new migrations
+    for (TabletMigration m : migrationsOut) {
+      migrations.add(m.getTablet());
+    }
+    migrationsOut.clear();
+    wait = this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(createCurrent(15)),
+        migrations, migrationsOut));
+    assertEquals(20000, wait);
+    // no more balancing to do
+    assertEquals(0, migrationsOut.size());
+  }
+
+  @Test
+  public void testBalanceWithTooManyOutstandingMigrations() {
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    init(DEFAULT_TABLE_PROPERTIES);
+    // lets say we already have migrations ongoing for the FOO and BAR table extends (should be 5 of
+    // each of them) for a total of 10
+    Set<TabletId> migrations = new HashSet<>();
+    migrations.addAll(tableTablets.get(FOO.getTableName()));
+    migrations.addAll(tableTablets.get(BAR.getTableName()));
+    long wait =
+        this.balance(new BalanceParamsImpl(Collections.unmodifiableSortedMap(createCurrent(15)),
+            migrations, migrationsOut));
+    assertEquals(20000, wait);
+    // no migrations should have occurred as 10 is the maxOutstandingMigrations
+    assertEquals(0, migrationsOut.size());
+  }
+
+  @Test
+  public void testSplitCurrentByRegexUsingHostname() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<String,SortedMap<TabletServerId,TServerStatus>> groups =
+        this.splitCurrentByRegex(createCurrent(15));
+    assertEquals(3, groups.size());
+    assertTrue(groups.containsKey(FOO.getTableName()));
+    SortedMap<TabletServerId,TServerStatus> fooHosts = groups.get(FOO.getTableName());
+    assertEquals(5, fooHosts.size());
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1))));
+    assertTrue(groups.containsKey(BAR.getTableName()));
+    SortedMap<TabletServerId,TServerStatus> barHosts = groups.get(BAR.getTableName());
+    assertEquals(5, barHosts.size());
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1))));
+    assertTrue(groups.containsKey(DEFAULT_POOL));
+    SortedMap<TabletServerId,TServerStatus> defHosts = groups.get(DEFAULT_POOL);
+    assertEquals(5, defHosts.size());
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.12", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.13", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.14", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.15", 9997, Integer.toHexString(1))));
+  }
+
+  @Test
+  public void testSplitCurrentByRegexUsingOverlappingPools() {
+    HashMap<String,String> props = new HashMap<>(DEFAULT_TABLE_PROPERTIES);
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(), "r.*");
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r01.*|r02.*");
+    init(props);
+    Map<String,SortedMap<TabletServerId,TServerStatus>> groups =
+        this.splitCurrentByRegex(createCurrent(15));
+
+    // Groups foo, bar, and the default pool which contains all known hosts
+    assertEquals(3, groups.size());
+    assertTrue(groups.containsKey(FOO.getTableName()));
+    assertTrue(groups.containsKey(DEFAULT_POOL));
+    for (String pool : new String[] {FOO.getTableName(), DEFAULT_POOL}) {
+      SortedMap<TabletServerId,TServerStatus> fooHosts = groups.get(pool);
+      assertEquals(15, fooHosts.size());
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.12", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.13", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.14", 9997, Integer.toHexString(1))));
+      assertTrue(fooHosts
+          .containsKey(new TabletServerIdImpl("192.168.0.15", 9997, Integer.toHexString(1))));
+    }
+
+    assertTrue(groups.containsKey(BAR.getTableName()));
+    SortedMap<TabletServerId,TServerStatus> barHosts = groups.get(BAR.getTableName());
+    assertEquals(10, barHosts.size());
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1))));
+  }
+
+  @Test
+  public void testSplitCurrentByRegexUsingIP() {
+    HashMap<String,String> props = new HashMap<>();
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "30s");
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_REGEX_USING_IPS_KEY, "true");
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(),
+        "192\\.168\\.0\\.[1-5]");
+    props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(),
+        "192\\.168\\.0\\.[6-9]|192\\.168\\.0\\.10");
+    init(props);
+
+    assertTrue(isIpBasedRegex());
+    Map<String,SortedMap<TabletServerId,TServerStatus>> groups =
+        this.splitCurrentByRegex(createCurrent(15));
+    assertEquals(3, groups.size());
+    assertTrue(groups.containsKey(FOO.getTableName()));
+    SortedMap<TabletServerId,TServerStatus> fooHosts = groups.get(FOO.getTableName());
+    assertEquals(5, fooHosts.size());
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1))));
+    assertTrue(
+        fooHosts.containsKey(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1))));
+    assertTrue(groups.containsKey(BAR.getTableName()));
+    SortedMap<TabletServerId,TServerStatus> barHosts = groups.get(BAR.getTableName());
+    assertEquals(5, barHosts.size());
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1))));
+    assertTrue(
+        barHosts.containsKey(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1))));
+    assertTrue(groups.containsKey(DEFAULT_POOL));
+    SortedMap<TabletServerId,TServerStatus> defHosts = groups.get(DEFAULT_POOL);
+    assertEquals(5, defHosts.size());
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.12", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.13", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.14", 9997, Integer.toHexString(1))));
+    assertTrue(
+        defHosts.containsKey(new TabletServerIdImpl("192.168.0.15", 9997, Integer.toHexString(1))));
+  }
+
+  @Test
+  public void testAllUnassigned() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    Map<TabletId,TabletServerId> unassigned = new HashMap<>();
+    for (List<TabletId> extents : tableTablets.values()) {
+      for (TabletId tabletId : extents) {
+        unassigned.put(tabletId, null);
+      }
+    }
+    this.getAssignments(
+        new AssignmentParamsImpl(Collections.unmodifiableSortedMap(allTabletServers),
+            Collections.unmodifiableMap(unassigned), assignments));
+    assertEquals(15, assignments.size());
+    // Ensure unique tservers
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      for (Entry<TabletId,TabletServerId> e2 : assignments.entrySet()) {
+        if (e.getKey().equals(e2.getKey())) {
+          continue;
+        }
+        if (e.getValue().equals(e2.getValue())) {
+          fail("Assignment failure");
+        }
+      }
+    }
+    // Ensure assignments are correct
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      if (!tabletInBounds(e.getKey(), e.getValue())) {
+        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
+      }
+    }
+  }
+
+  @Test
+  public void testAllAssigned() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    this.getAssignments(new AssignmentParamsImpl(
+        Collections.unmodifiableSortedMap(allTabletServers), Map.of(), assignments));
+    assertEquals(0, assignments.size());
+  }
+
+  @Test
+  public void testPartiallyAssigned() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    Map<TabletId,TabletServerId> unassigned = new HashMap<>();
+    int i = 0;
+    for (List<TabletId> tablets : tableTablets.values()) {
+      for (TabletId tabletId : tablets) {
+        if ((i % 2) == 0) {
+          unassigned.put(tabletId, null);
+        }
+        i++;
+      }
+    }
+    this.getAssignments(
+        new AssignmentParamsImpl(Collections.unmodifiableSortedMap(allTabletServers),
+            Collections.unmodifiableMap(unassigned), assignments));
+    assertEquals(unassigned.size(), assignments.size());
+    // Ensure unique tservers
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      for (Entry<TabletId,TabletServerId> e2 : assignments.entrySet()) {
+        if (e.getKey().equals(e2.getKey())) {
+          continue;
+        }
+        if (e.getValue().equals(e2.getValue())) {
+          fail("Assignment failure");
+        }
+      }
+    }
+    // Ensure assignments are correct
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      if (!tabletInBounds(e.getKey(), e.getValue())) {
+        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
+      }
+    }
+  }
+
+  @Test
+  public void testUnassignedWithNoTServers() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    Map<TabletId,TabletServerId> unassigned = new HashMap<>();
+    for (TabletId tabletId : tableTablets.get(BAR.getTableName())) {
+      unassigned.put(tabletId, null);
+    }
+    SortedMap<TabletServerId,TServerStatus> current = createCurrent(15);
+    // Remove the BAR tablet servers from current
+    List<TabletServerId> removals = new ArrayList<>();
+    for (Entry<TabletServerId,TServerStatus> e : current.entrySet()) {
+      if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7")
+          || e.getKey().getHost().equals("192.168.0.8")
+          || e.getKey().getHost().equals("192.168.0.9")
+          || e.getKey().getHost().equals("192.168.0.10")) {
+        removals.add(e.getKey());
+      }
+    }
+    for (TabletServerId r : removals) {
+      current.remove(r);
+    }
+    this.getAssignments(new AssignmentParamsImpl(Collections.unmodifiableSortedMap(current),
+        Collections.unmodifiableMap(unassigned), assignments));
+    assertEquals(unassigned.size(), assignments.size());
+    // Ensure assignments are correct
+    // Ensure tablets are assigned in default pool
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      if (tabletInBounds(e.getKey(), e.getValue())) {
+        fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
+      }
+    }
+  }
+
+  @Test
+  public void testUnassignedWithNoDefaultPool() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    Map<TabletId,TabletServerId> unassigned = new HashMap<>();
+    for (TabletId tabletId : tableTablets.get(BAR.getTableName())) {
+      unassigned.put(tabletId, null);
+    }
+
+    SortedMap<TabletServerId,TServerStatus> current = createCurrent(15);
+    // Remove the BAR tablet servers and default pool from current
+    List<TabletServerId> removals = new ArrayList<>();
+    for (Entry<TabletServerId,TServerStatus> e : current.entrySet()) {
+      if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7")
+          || e.getKey().getHost().equals("192.168.0.8")
+          || e.getKey().getHost().equals("192.168.0.9")
+          || e.getKey().getHost().equals("192.168.0.10")
+          || e.getKey().getHost().equals("192.168.0.11")
+          || e.getKey().getHost().equals("192.168.0.12")
+          || e.getKey().getHost().equals("192.168.0.13")
+          || e.getKey().getHost().equals("192.168.0.14")
+          || e.getKey().getHost().equals("192.168.0.15")) {
+        removals.add(e.getKey());
+      }
+    }
+
+    for (TabletServerId r : removals) {
+      current.remove(r);
+    }
+
+    this.getAssignments(new AssignmentParamsImpl(Collections.unmodifiableSortedMap(current),
+        Collections.unmodifiableMap(unassigned), assignments));
+    assertEquals(unassigned.size(), assignments.size());
+
+    // Ensure tablets are assigned in default pool
+    for (Entry<TabletId,TabletServerId> e : assignments.entrySet()) {
+      if (tabletInBounds(e.getKey(), e.getValue())) {
+        fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
+      }
+    }
+  }
+
+  @Test
+  public void testOutOfBoundsTablets() {
+    init(DEFAULT_TABLE_PROPERTIES);
+    // Wait to trigger the out of bounds check which will call our version of
+    // getOnlineTabletsForTable
+    UtilWaitThread.sleep(11000);
+    Set<TabletId> migrations = new HashSet<>();
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    this.balance(new BalanceParamsImpl(createCurrent(15), migrations, migrationsOut));
+    assertEquals(2, migrationsOut.size());
+  }
+
+  @Override
+  public List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver, TableId tableId) {
+    // Report incorrect information so that balance will create an assignment
+    List<TabletStatistics> tablets = new ArrayList<>();
+    if (tableId.equals(BAR.getId()) && tserver.getHost().equals("192.168.0.1")) {
+      // Report that we have a bar tablet on this server
+      TKeyExtent tke = new TKeyExtent();
+      tke.setTable(BAR.getId().canonical().getBytes(UTF_8));
+      tke.setEndRow("11".getBytes());
+      tke.setPrevEndRow("10".getBytes());
+      TabletStats tstats = new TabletStats();
+      tstats.setExtent(tke);
+      TabletStatistics ts = new TabletStatisticsImpl(tstats);
+      tablets.add(ts);
+    } else if (tableId.equals(FOO.getId()) && tserver.getHost().equals("192.168.0.6")) {
+      // Report that we have a foo tablet on this server
+      TKeyExtent tke = new TKeyExtent();
+      tke.setTable(FOO.getId().canonical().getBytes(UTF_8));
+      tke.setEndRow("1".getBytes());
+      tke.setPrevEndRow("0".getBytes());
+      TabletStats tstats = new TabletStats();
+      tstats.setExtent(tke);
+      TabletStatistics ts = new TabletStatisticsImpl(tstats);
+      tablets.add(ts);
+    }
+    return tablets;
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java
new file mode 100644
index 0000000..663d315
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SimpleLoadBalancerTest {
+
+  static class FakeTServer {
+    List<TabletId> tablets = new ArrayList<>();
+
+    TServerStatus getStatus() {
+      org.apache.accumulo.core.master.thrift.TabletServerStatus result =
+          new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+      result.tableMap = new HashMap<>();
+      for (TabletId tabletId : tablets) {
+        TableInfo info = result.tableMap.get(tabletId.getTable().canonical());
+        if (info == null)
+          result.tableMap.put(tabletId.getTable().canonical(), info = new TableInfo());
+        info.onlineTablets++;
+        info.recs = info.onlineTablets;
+        info.ingestRate = 123.;
+        info.queryRate = 456.;
+      }
+      return new TServerStatusImpl(result);
+    }
+  }
+
+  Map<TabletServerId,FakeTServer> servers = new HashMap<>();
+  Map<TabletId,TabletServerId> last = new HashMap<>();
+
+  class TestSimpleLoadBalancer extends SimpleLoadBalancer {
+
+    @Override
+    protected List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver,
+        TableId tableId) {
+      List<TabletStatistics> result = new ArrayList<>();
+      for (TabletId tabletId : servers.get(tserver).tablets) {
+        if (tabletId.getTable().equals(tableId)) {
+          KeyExtent extent = new KeyExtent(tableId, tabletId.getEndRow(), tabletId.getPrevEndRow());
+          TabletStats stats =
+              new TabletStats(new TabletStats(extent.toThrift(), null, null, null, 0L, 0., 0., 0));
+          result.add(new TabletStatisticsImpl(stats));
+        }
+      }
+      return result;
+    }
+  }
+
+  @Before
+  public void setUp() {
+    last.clear();
+    servers.clear();
+  }
+
+  @Test
+  public void testAssignMigrations() {
+    servers.put(new TabletServerIdImpl("127.0.0.1", 1234, "a"), new FakeTServer());
+    servers.put(new TabletServerIdImpl("127.0.0.2", 1234, "b"), new FakeTServer());
+    servers.put(new TabletServerIdImpl("127.0.0.3", 1234, "c"), new FakeTServer());
+    List<TabletId> metadataTable = new ArrayList<>();
+    String table = "t1";
+    metadataTable.add(makeTablet(table, null, null));
+    table = "t2";
+    metadataTable.add(makeTablet(table, "a", null));
+    metadataTable.add(makeTablet(table, null, "a"));
+    table = "t3";
+    metadataTable.add(makeTablet(table, "a", null));
+    metadataTable.add(makeTablet(table, "b", "a"));
+    metadataTable.add(makeTablet(table, "c", "b"));
+    metadataTable.add(makeTablet(table, "d", "c"));
+    metadataTable.add(makeTablet(table, "e", "d"));
+    metadataTable.add(makeTablet(table, null, "e"));
+    Collections.sort(metadataTable);
+
+    TestSimpleLoadBalancer balancer = new TestSimpleLoadBalancer();
+
+    SortedMap<TabletServerId,TServerStatus> current = new TreeMap<>();
+    for (Entry<TabletServerId,FakeTServer> entry : servers.entrySet()) {
+      current.put(entry.getKey(), entry.getValue().getStatus());
+    }
+    assignTablets(metadataTable, servers, current, balancer);
+
+    // Verify that the counts on the tables are correct
+    Map<String,Integer> expectedCounts = new HashMap<>();
+    expectedCounts.put("t1", 1);
+    expectedCounts.put("t2", 1);
+    expectedCounts.put("t3", 2);
+    checkBalance(metadataTable, servers, expectedCounts);
+
+    // Rebalance once
+    for (Entry<TabletServerId,FakeTServer> entry : servers.entrySet()) {
+      current.put(entry.getKey(), entry.getValue().getStatus());
+    }
+
+    // Nothing should happen, we are balanced
+    ArrayList<TabletMigration> out = new ArrayList<>();
+    balancer.getMigrations(current, out);
+    assertEquals(out.size(), 0);
+
+    // Take down a tabletServer
+    TabletServerId first = current.keySet().iterator().next();
+    current.remove(first);
+    FakeTServer remove = servers.remove(first);
+
+    // reassign offline extents
+    assignTablets(remove.tablets, servers, current, balancer);
+    checkBalance(metadataTable, servers, null);
+  }
+
+  private void assignTablets(List<TabletId> metadataTable, Map<TabletServerId,FakeTServer> servers,
+      SortedMap<TabletServerId,TServerStatus> status, TestSimpleLoadBalancer balancer) {
+    // Assign tablets
+    for (TabletId tabletId : metadataTable) {
+      TabletServerId assignment = balancer.getAssignment(status, last.get(tabletId));
+      assertNotNull(assignment);
+      assertFalse(servers.get(assignment).tablets.contains(tabletId));
+      servers.get(assignment).tablets.add(tabletId);
+      last.put(tabletId, assignment);
+    }
+  }
+
+  SortedMap<TabletServerId,TServerStatus> getAssignments(Map<TabletServerId,FakeTServer> servers) {
+    SortedMap<TabletServerId,TServerStatus> result = new TreeMap<>();
+    for (Entry<TabletServerId,FakeTServer> entry : servers.entrySet()) {
+      result.put(entry.getKey(), entry.getValue().getStatus());
+    }
+    return result;
+  }
+
+  @Test
+  public void testUnevenAssignment() {
+    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
+      String cString = Character.toString(c);
+      TabletServerId tsid = new TabletServerIdImpl("127.0.0.1", c, cString);
+      FakeTServer fakeTServer = new FakeTServer();
+      servers.put(tsid, fakeTServer);
+      fakeTServer.tablets.add(makeTablet(cString, null, null));
+    }
+    // Put more tablets on one server, but not more than the number of servers
+    Entry<TabletServerId,FakeTServer> first = servers.entrySet().iterator().next();
+    first.getValue().tablets.add(makeTablet("newTable", "a", null));
+    first.getValue().tablets.add(makeTablet("newTable", "b", "a"));
+    first.getValue().tablets.add(makeTablet("newTable", "c", "b"));
+    first.getValue().tablets.add(makeTablet("newTable", "d", "c"));
+    first.getValue().tablets.add(makeTablet("newTable", "e", "d"));
+    first.getValue().tablets.add(makeTablet("newTable", "f", "e"));
+    first.getValue().tablets.add(makeTablet("newTable", "g", "f"));
+    first.getValue().tablets.add(makeTablet("newTable", "h", "g"));
+    first.getValue().tablets.add(makeTablet("newTable", "i", null));
+    TestSimpleLoadBalancer balancer = new TestSimpleLoadBalancer();
+    Set<TabletId> migrations = Collections.emptySet();
+    int moved = 0;
+    // balance until we can't balance no more!
+    while (true) {
+      List<TabletMigration> migrationsOut = new ArrayList<>();
+      balancer.balance(new BalanceParamsImpl(getAssignments(servers), migrations, migrationsOut));
+      if (migrationsOut.isEmpty())
+        break;
+      for (TabletMigration migration : migrationsOut) {
+        if (servers.get(migration.getOldTabletServer()).tablets.remove(migration.getTablet()))
+          moved++;
+        servers.get(migration.getNewTabletServer()).tablets.add(migration.getTablet());
+      }
+    }
+    assertEquals(8, moved);
+  }
+
+  @Test
+  public void testUnevenAssignment2() {
+    // make 26 servers
+    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
+      TabletServerId tsid = new TabletServerIdImpl("127.0.0.1", c, Character.toString(c));
+      FakeTServer fakeTServer = new FakeTServer();
+      servers.put(tsid, fakeTServer);
+    }
+    // put 60 tablets on 25 of them
+    List<Entry<TabletServerId,FakeTServer>> shortList = new ArrayList<>(servers.entrySet());
+    Entry<TabletServerId,FakeTServer> shortServer = shortList.remove(0);
+    int c = 0;
+    for (int i = 0; i < 60; i++) {
+      for (Entry<TabletServerId,FakeTServer> entry : shortList) {
+        entry.getValue().tablets.add(makeTablet("t" + c, null, null));
+      }
+    }
+    // put 10 on the that short server:
+    for (int i = 0; i < 10; i++) {
+      shortServer.getValue().tablets.add(makeTablet("s" + i, null, null));
+    }
+
+    TestSimpleLoadBalancer balancer = new TestSimpleLoadBalancer();
+    Set<TabletId> migrations = Collections.emptySet();
+    int moved = 0;
+    // balance until we can't balance no more!
+    while (true) {
+      List<TabletMigration> migrationsOut = new ArrayList<>();
+      balancer.balance(new BalanceParamsImpl(getAssignments(servers), migrations, migrationsOut));
+      if (migrationsOut.isEmpty())
+        break;
+      for (TabletMigration migration : migrationsOut) {
+        if (servers.get(migration.getOldTabletServer()).tablets.remove(migration.getTablet()))
+          moved++;
+        last.remove(migration.getTablet());
+        servers.get(migration.getNewTabletServer()).tablets.add(migration.getTablet());
+        last.put(migration.getTablet(), migration.getNewTabletServer());
+      }
+    }
+    // average is 58, with 2 at 59: we need 48 more moved to the short server
+    assertEquals(48, moved);
+  }
+
+  private void checkBalance(List<TabletId> metadataTable, Map<TabletServerId,FakeTServer> servers,
+      Map<String,Integer> expectedCounts) {
+    // Verify they are spread evenly over the cluster
+    int average = metadataTable.size() / servers.size();
+    for (FakeTServer server : servers.values()) {
+      int diff = server.tablets.size() - average;
+      if (diff < 0)
+        fail("average number of tablets is " + average + " but a server has "
+            + server.tablets.size());
+      if (diff > 1)
+        fail("average number of tablets is " + average + " but a server has "
+            + server.tablets.size());
+    }
+
+    if (expectedCounts != null) {
+      for (FakeTServer server : servers.values()) {
+        Map<String,Integer> counts = new HashMap<>();
+        server.tablets.forEach(tabletId -> {
+          String t = tabletId.getTable().canonical();
+          counts.putIfAbsent(t, 0);
+          counts.put(t, counts.get(t) + 1);
+        });
+        counts.forEach((k, v) -> assertEquals(expectedCounts.get(k), v));
+      }
+    }
+  }
+
+  private static TabletId makeTablet(String table, String end, String prev) {
+    return new TabletIdImpl(new KeyExtent(TableId.of(table), toText(end), toText(prev)));
+  }
+
+  private static Text toText(String value) {
+    if (value != null)
+      return new Text(value);
+    return null;
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java
new file mode 100644
index 0000000..47786cc
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.balancer;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class TableLoadBalancerTest {
+
+  private static final Map<String,String> TABLE_ID_MAP =
+      Map.of("t1", "a1", "t2", "b12", "t3", "c4");
+
+  private static TabletServerId mkts(String host, int port, String session) {
+    return new TabletServerIdImpl(host, port, session);
+  }
+
+  private static TServerStatus status(Object... config) {
+    org.apache.accumulo.core.master.thrift.TabletServerStatus thriftStatus =
+        new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+    thriftStatus.tableMap = new HashMap<>();
+    String tablename = null;
+    for (Object c : config) {
+      if (c instanceof String) {
+        tablename = (String) c;
+      } else {
+        TableInfo info = new TableInfo();
+        int count = (Integer) c;
+        info.onlineTablets = count;
+        info.tablets = count;
+        thriftStatus.tableMap.put(tablename, info);
+      }
+    }
+    return new TServerStatusImpl(thriftStatus);
+  }
+
+  static SortedMap<TabletServerId,TServerStatus> state;
+
+  static List<TabletStatistics> generateFakeTablets(TabletServerId tserver, TableId tableId) {
+    List<TabletStatistics> result = new ArrayList<>();
+    TServerStatus tableInfo = state.get(tserver);
+    // generate some fake tablets
+    for (int i = 0; i < tableInfo.getTableMap().get(tableId.canonical()).getOnlineTabletCount();
+        i++) {
+      TabletStats stats = new TabletStats();
+      stats.extent =
+          new KeyExtent(tableId, new Text(tserver.getHost() + String.format("%03d", i + 1)),
+              new Text(tserver.getHost() + String.format("%03d", i))).toThrift();
+      result.add(new TabletStatisticsImpl(stats));
+    }
+    return result;
+  }
+
+  public static class TestSimpleLoadBalancer extends SimpleLoadBalancer {
+
+    public TestSimpleLoadBalancer(TableId table) {
+      super(table);
+    }
+
+    @Override
+    public void init(BalancerEnvironment balancerEnvironment) {}
+
+    @Override
+    public List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver,
+        TableId tableId) {
+      return generateFakeTablets(tserver, tableId);
+    }
+  }
+
+  @Test
+  public void test() {
+    BalancerEnvironment environment = createMock(BalancerEnvironment.class);
+    ConfigurationCopy cc = new ConfigurationCopy(
+        Map.of(Property.TABLE_LOAD_BALANCER.getKey(), TestSimpleLoadBalancer.class.getName()));
+    ConfigurationImpl tableConfig = new ConfigurationImpl(cc);
+
+    Map<String,TableId> tableIdMap = TABLE_ID_MAP.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> TableId.of(e.getValue())));
+    expect(environment.getTableIdMap()).andReturn(tableIdMap).anyTimes();
+    expect(environment.isTableOnline(anyObject(TableId.class))).andReturn(true).anyTimes();
+    expect(environment.getConfiguration(anyObject(TableId.class))).andReturn(tableConfig)
+        .anyTimes();
+    expect(environment.tableContext(anyObject(TableId.class))).andReturn(null).anyTimes();
+
+    replay(environment);
+
+    String t1Id = TABLE_ID_MAP.get("t1"), t2Id = TABLE_ID_MAP.get("t2"),
+        t3Id = TABLE_ID_MAP.get("t3");
+    state = new TreeMap<>();
+    TabletServerId svr = mkts("10.0.0.1", 1234, "0x01020304");
+    state.put(svr, status(t1Id, 10, t2Id, 10, t3Id, 10));
+
+    Set<TabletId> migrations = Collections.emptySet();
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    TableLoadBalancer tls = new TableLoadBalancer();
+    tls.init(environment);
+    tls.balance(new BalanceParamsImpl(state, migrations, migrationsOut));
+    assertEquals(0, migrationsOut.size());
+
+    state.put(mkts("10.0.0.2", 2345, "0x02030405"), status());
+    tls = new TableLoadBalancer();
+    tls.init(environment);
+    tls.balance(new BalanceParamsImpl(state, migrations, migrationsOut));
+    int count = 0;
+    Map<TableId,Integer> movedByTable = new HashMap<>();
+    movedByTable.put(TableId.of(t1Id), 0);
+    movedByTable.put(TableId.of(t2Id), 0);
+    movedByTable.put(TableId.of(t3Id), 0);
+    for (TabletMigration migration : migrationsOut) {
+      if (migration.getOldTabletServer().equals(svr)) {
+        count++;
+      }
+      TableId key = migration.getTablet().getTable();
+      movedByTable.put(key, movedByTable.get(key) + 1);
+    }
+    assertEquals(15, count);
+    for (Integer moved : movedByTable.values()) {
+      assertEquals(5, moved.intValue());
+    }
+  }
+
+}
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java
index 4a2567a..c23930e 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordWriter.java
@@ -187,7 +187,7 @@
       if (!e.getSecurityErrorCodes().isEmpty()) {
         var tables = new HashMap<String,Set<SecurityErrorCode>>();
         e.getSecurityErrorCodes().forEach((tabletId, codes) -> tables
-            .computeIfAbsent(tabletId.getTableId().toString(), k -> new HashSet<>()).addAll(codes));
+            .computeIfAbsent(tabletId.getTable().canonical(), k -> new HashSet<>()).addAll(codes));
         log.error("Not authorized to write to tables : " + tables);
       }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java
index fbde6e4..64aaed3 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordWriter.java
@@ -188,7 +188,7 @@
       if (!e.getSecurityErrorCodes().isEmpty()) {
         HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<>();
         e.getSecurityErrorCodes().forEach((tabletId, codes) -> tables
-            .computeIfAbsent(tabletId.getTableId().toString(), k -> new HashSet<>()).addAll(codes));
+            .computeIfAbsent(tabletId.getTable().canonical(), k -> new HashSet<>()).addAll(codes));
         log.error("Not authorized to write to tables : " + tables);
       }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
index b0845f3..08b7609 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
@@ -65,4 +65,8 @@
     String ctx = ClassLoaderUtil.tableContext(srvCtx.getTableConfiguration(tableId));
     return ConfigurationTypeHelper.getClassInstance(ctx, className, base);
   }
+
+  public ServerContext getContext() {
+    return srvCtx;
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
new file mode 100644
index 0000000..a3fd1b2
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.manager.balancer;
+
+import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
+import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.classloader.ClassLoaderUtil;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServiceEnvironmentImpl;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BalancerEnvironmentImpl extends ServiceEnvironmentImpl implements BalancerEnvironment {
+  private static final Logger log = LoggerFactory.getLogger(BalancerEnvironmentImpl.class);
+
+  public BalancerEnvironmentImpl(ServerContext ctx) {
+    super(ctx);
+  }
+
+  @Override
+  public Map<String,TableId> getTableIdMap() {
+    return Tables.getNameToIdMap(getContext());
+  }
+
+  @Override
+  public boolean isTableOnline(TableId tableId) {
+    return TableState.ONLINE.equals(Tables.getTableState(getContext(), tableId));
+  }
+
+  @Override
+  public Map<TabletId,TabletServerId> listTabletLocations(TableId tableId) {
+    Map<TabletId,TabletServerId> tablets = new LinkedHashMap<>();
+    for (var tm : TabletsMetadata.builder().forTable(tableId).fetch(LOCATION, PREV_ROW)
+        .build(getContext())) {
+      tablets.put(new TabletIdImpl(tm.getExtent()),
+          TabletServerIdImpl.fromThrift(tm.getLocation()));
+    }
+    return tablets;
+  }
+
+  @Override
+  public List<TabletStatistics> listOnlineTabletsForTable(TabletServerId tabletServerId,
+      TableId tableId) throws AccumuloException, AccumuloSecurityException {
+    log.debug("Scanning tablet server {} for table {}", tabletServerId, tableId);
+    try {
+      TabletClientService.Client client = ThriftUtil.getClient(
+          new TabletClientService.Client.Factory(),
+          HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()), getContext());
+      try {
+        return client
+            .getTabletStats(TraceUtil.traceInfo(), getContext().rpcCreds(), tableId.canonical())
+            .stream().map(TabletStatisticsImpl::new).collect(Collectors.toList());
+      } catch (TTransportException e) {
+        log.error("Unable to connect to {}: ", tabletServerId, e);
+      } finally {
+        ThriftUtil.returnClient(client);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e);
+    } catch (TException e) {
+      throw new AccumuloException(e);
+    }
+    return null;
+  }
+
+  @Override
+  public String tableContext(TableId tableId) {
+    return ClassLoaderUtil.tableContext(getContext().getTableConfiguration(tableId));
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
deleted file mode 100644
index d20a397..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.server.master.balancer;
-
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A chaotic load balancer used for testing. It constantly shuffles tablets, preventing them from
- * resting in a single location for very long. This is not designed for performance, do not use on
- * production systems. I'm calling it the LokiLoadBalancer.
- *
- * <p>
- * Will balance randomly, maintaining distribution
- */
-public class ChaoticLoadBalancer extends TabletBalancer {
-  private static final Logger log = LoggerFactory.getLogger(ChaoticLoadBalancer.class);
-
-  public ChaoticLoadBalancer() {}
-
-  // Required constructor
-  public ChaoticLoadBalancer(String tableName) {}
-
-  Random r = new SecureRandom();
-
-  @Override
-  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
-    long total = assignments.size() + unassigned.size();
-    long avg = (long) Math.ceil(((double) total) / current.size());
-    Map<TServerInstance,Long> toAssign = new HashMap<>();
-    List<TServerInstance> tServerArray = new ArrayList<>();
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      long numTablets = 0;
-      for (TableInfo ti : e.getValue().getTableMap().values()) {
-        numTablets += ti.tablets;
-      }
-      if (numTablets <= avg) {
-        tServerArray.add(e.getKey());
-        toAssign.put(e.getKey(), avg - numTablets);
-      }
-    }
-
-    if (tServerArray.isEmpty()) {
-      // No tservers to assign to
-      return;
-    }
-
-    for (KeyExtent ke : unassigned.keySet()) {
-      int index = r.nextInt(tServerArray.size());
-      TServerInstance dest = tServerArray.get(index);
-      assignments.put(ke, dest);
-      long remaining = toAssign.get(dest) - 1;
-      if (remaining == 0) {
-        tServerArray.remove(index);
-        toAssign.remove(dest);
-      } else {
-        toAssign.put(dest, remaining);
-      }
-    }
-  }
-
-  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
-
-  @Override
-  public long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-    Map<TServerInstance,Long> numTablets = new HashMap<>();
-    List<TServerInstance> underCapacityTServer = new ArrayList<>();
-
-    if (!migrations.isEmpty()) {
-      outstandingMigrations.migrations = migrations;
-      constraintNotMet(outstandingMigrations);
-      return 100;
-    }
-    resetBalancerErrors();
-
-    boolean moveMetadata = r.nextInt(4) == 0;
-    long totalTablets = 0;
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      long tabletCount = 0;
-      for (TableInfo ti : e.getValue().getTableMap().values()) {
-        tabletCount += ti.tablets;
-      }
-      numTablets.put(e.getKey(), tabletCount);
-      underCapacityTServer.add(e.getKey());
-      totalTablets += tabletCount;
-    }
-    // totalTablets is fuzzy due to asynchronicity of the stats
-    // *1.2 to handle fuzziness, and prevent locking for 'perfect' balancing scenarios
-    long avg = (long) Math.ceil(((double) totalTablets) / current.size() * 1.2);
-
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      for (String tableId : e.getValue().getTableMap().keySet()) {
-        TableId id = TableId.of(tableId);
-        if (!moveMetadata && MetadataTable.ID.equals(id))
-          continue;
-        try {
-          for (TabletStats ts : getOnlineTabletsForTable(e.getKey(), id)) {
-            KeyExtent ke = KeyExtent.fromThrift(ts.extent);
-            int index = r.nextInt(underCapacityTServer.size());
-            TServerInstance dest = underCapacityTServer.get(index);
-            if (dest.equals(e.getKey()))
-              continue;
-            migrationsOut.add(new TabletMigration(ke, e.getKey(), dest));
-            if (numTablets.put(dest, numTablets.get(dest) + 1) > avg)
-              underCapacityTServer.remove(index);
-            if (numTablets.put(e.getKey(), numTablets.get(e.getKey()) - 1) <= avg
-                && !underCapacityTServer.contains(e.getKey()))
-              underCapacityTServer.add(e.getKey());
-
-            // We can get some craziness with only 1 tserver, so lets make sure there's always an
-            // option!
-            if (underCapacityTServer.isEmpty())
-              underCapacityTServer.addAll(numTablets.keySet());
-          }
-        } catch (ThriftSecurityException e1) {
-          // Shouldn't happen, but carry on if it does
-          log.debug(
-              "Encountered ThriftSecurityException.  This should not happen.  Carrying on anyway.",
-              e1);
-        } catch (TException e1) {
-          // Shouldn't happen, but carry on if it does
-          log.debug("Encountered TException.  This should not happen.  Carrying on anyway.", e1);
-        }
-      }
-    }
-
-    return 100;
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index c12dcfb..5a0f859 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@ -39,6 +39,10 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.TabletBalancer} instead
+ */
+@Deprecated(since = "2.1.0")
 public class DefaultLoadBalancer extends TabletBalancer {
 
   private static final Logger log = LoggerFactory.getLogger(DefaultLoadBalancer.class);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
index 3ce20d3..d6ec3f4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
@@ -64,7 +64,10 @@
  * <p>
  * To use this balancer you must extend it and implement {@link #getPartitioner()}. See
  * {@link RegexGroupBalancer} as an example.
+ *
+ * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.GroupBalancer} instead.
  */
+@Deprecated(since = "2.1.0")
 public abstract class GroupBalancer extends TabletBalancer {
 
   private final TableId tableId;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
index d877af0..2083e25 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
@@ -86,7 +86,10 @@
  * in which this balancer will continue balancing, set the following property (default 0):<br>
  * <b>table.custom.balancer.host.regex.max.outstanding.migrations</b>
  *
+ * @deprecated since 2.1.0. Use
+ *             {@link org.apache.accumulo.core.spi.balancer.HostRegexTableLoadBalancer} instead
  */
+@Deprecated(since = "2.1.0")
 public class HostRegexTableLoadBalancer extends TableLoadBalancer {
 
   private static final String PROP_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
index d116112..35c6370 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
@@ -44,7 +44,11 @@
  * suffixes). This determines how long to wait between balancing. Since this balancer scans the
  * metadata table, may want to set this higher for large tables.
  * </ul>
+ *
+ * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.RegexGroupBalancer}
+ *             instead.
  */
+@Deprecated(since = "2.1.0")
 public class RegexGroupBalancer extends GroupBalancer {
 
   public static final String REGEX_PROPERTY =
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
index b13670a..89db503 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
@@ -39,6 +39,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.TableLoadBalancer}
+ *             instead.
+ */
+@Deprecated(since = "2.1.0")
 public class TableLoadBalancer extends TabletBalancer {
 
   private static final Logger log = LoggerFactory.getLogger(TableLoadBalancer.class);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 9584f72..a1a7a7c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -29,15 +29,19 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.accumulo.server.manager.balancer.BalancerEnvironmentImpl;
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
@@ -46,6 +50,8 @@
 
 import com.google.common.collect.Iterables;
 
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
 /**
  * This class is responsible for managing the distribution of tablets throughout an Accumulo
  * cluster. In most cases, users will want a balancer implementation which ensures a uniform
@@ -56,13 +62,41 @@
  * Implementations may wish to store configuration in Accumulo's system configuration using the
  * {@link Property#GENERAL_ARBITRARY_PROP_PREFIX}. They may also benefit from using per-table
  * configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
+ *
+ * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.TabletBalancer}
+ *             instead.
  */
-public abstract class TabletBalancer {
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
+    justification = "Class is deprecated and will be removed.")
+public abstract class TabletBalancer
+    implements org.apache.accumulo.core.spi.balancer.TabletBalancer {
 
   private static final Logger log = LoggerFactory.getLogger(TabletBalancer.class);
 
   protected ServerContext context;
 
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    var bei = (BalancerEnvironmentImpl) balancerEnvironment;
+    init(bei.getContext());
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+    AssignmentParamsImpl api = (AssignmentParamsImpl) params;
+    getAssignments(api.thriftCurrentStatus(), api.thriftUnassigned(), api.thriftAssignmentsOut());
+  }
+
+  @Override
+  public long balance(BalanceParameters params) {
+    BalanceParamsImpl bpi = (BalanceParamsImpl) params;
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    long result = balance(bpi.thriftCurrentStatus(), bpi.thriftCurrentMigrations(), migrationsOut);
+    migrationsOut.forEach(mo -> bpi.addMigration(mo.tablet, mo.oldServer, mo.newServer));
+    return result;
+  }
+
   /**
    * Initialize the TabletBalancer. This gives the balancer the opportunity to read the
    * configuration.
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java
index d937ae3..ed2156e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java
@@ -21,6 +21,10 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.TServerInstance;
 
+/**
+ * @deprecated since 2.1.0. Use balancers in org.apache.accumulo.core.spi.balancer instead.
+ */
+@Deprecated(since = "2.1.0")
 public class TabletMigration {
   public KeyExtent tablet;
   public TServerInstance oldServer;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index 272b653..8627f1f 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -55,6 +55,7 @@
 import org.apache.hadoop.io.Text;
 import org.easymock.EasyMock;
 
+@Deprecated(since = "2.1.0")
 public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableLoadBalancer {
 
   protected static class TestTable {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java
deleted file mode 100644
index a4df8ec..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.server.master.balancer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.hadoop.io.Text;
-import org.junit.Test;
-
-public class ChaoticLoadBalancerTest {
-
-  class FakeTServer {
-    List<KeyExtent> extents = new ArrayList<>();
-
-    TabletServerStatus getStatus() {
-      TabletServerStatus result = new TabletServerStatus();
-      result.tableMap = new HashMap<>();
-      for (KeyExtent extent : extents) {
-        TableId table = extent.tableId();
-        TableInfo info = result.tableMap.get(table.canonical());
-        if (info == null)
-          result.tableMap.put(table.canonical(), info = new TableInfo());
-        info.onlineTablets++;
-        info.recs = info.onlineTablets;
-        info.ingestRate = 123.;
-        info.queryRate = 456.;
-      }
-      return result;
-    }
-  }
-
-  Map<TServerInstance,FakeTServer> servers = new HashMap<>();
-
-  class TestChaoticLoadBalancer extends ChaoticLoadBalancer {
-
-    @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId table) {
-      List<TabletStats> result = new ArrayList<>();
-      for (KeyExtent extent : servers.get(tserver).extents) {
-        if (extent.tableId().equals(table)) {
-          result.add(new TabletStats(extent.toThrift(), null, null, null, 0L, 0., 0., 0));
-        }
-      }
-      return result;
-    }
-  }
-
-  @Test
-  public void testAssignMigrations() {
-    servers.clear();
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), "a"),
-        new FakeTServer());
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1235), "b"),
-        new FakeTServer());
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1236), "c"),
-        new FakeTServer());
-    Map<KeyExtent,TServerInstance> metadataTable = new TreeMap<>();
-    String table = "t1";
-    metadataTable.put(makeExtent(table, null, null), null);
-    table = "t2";
-    metadataTable.put(makeExtent(table, "a", null), null);
-    metadataTable.put(makeExtent(table, null, "a"), null);
-    table = "t3";
-    metadataTable.put(makeExtent(table, "a", null), null);
-    metadataTable.put(makeExtent(table, "b", "a"), null);
-    metadataTable.put(makeExtent(table, "c", "b"), null);
-    metadataTable.put(makeExtent(table, "d", "c"), null);
-    metadataTable.put(makeExtent(table, "e", "d"), null);
-    metadataTable.put(makeExtent(table, null, "e"), null);
-
-    TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer();
-
-    SortedMap<TServerInstance,TabletServerStatus> current = new TreeMap<>();
-    for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) {
-      current.put(entry.getKey(), entry.getValue().getStatus());
-    }
-
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    balancer.getAssignments(getAssignments(servers), metadataTable, assignments);
-
-    assertEquals(assignments.size(), metadataTable.size());
-  }
-
-  SortedMap<TServerInstance,TabletServerStatus>
-      getAssignments(Map<TServerInstance,FakeTServer> servers) {
-    SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<>();
-    for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) {
-      result.put(entry.getKey(), entry.getValue().getStatus());
-    }
-    return result;
-  }
-
-  @Test
-  public void testUnevenAssignment() {
-    servers.clear();
-    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
-      String cString = Character.toString(c);
-      HostAndPort fakeAddress = HostAndPort.fromParts("127.0.0.1", c);
-      TServerInstance tsi = new TServerInstance(fakeAddress, cString);
-      FakeTServer fakeTServer = new FakeTServer();
-      servers.put(tsi, fakeTServer);
-      fakeTServer.extents.add(makeExtent(cString, null, null));
-    }
-    // Put more tablets on one server, but not more than the number of servers
-    Entry<TServerInstance,FakeTServer> first = servers.entrySet().iterator().next();
-    first.getValue().extents.add(makeExtent("newTable", "a", null));
-    first.getValue().extents.add(makeExtent("newTable", "b", "a"));
-    first.getValue().extents.add(makeExtent("newTable", "c", "b"));
-    first.getValue().extents.add(makeExtent("newTable", "d", "c"));
-    first.getValue().extents.add(makeExtent("newTable", "e", "d"));
-    first.getValue().extents.add(makeExtent("newTable", "f", "e"));
-    first.getValue().extents.add(makeExtent("newTable", "g", "f"));
-    first.getValue().extents.add(makeExtent("newTable", "h", "g"));
-    first.getValue().extents.add(makeExtent("newTable", "i", null));
-    TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer();
-    Set<KeyExtent> migrations = Collections.emptySet();
-
-    // Just want to make sure it gets some migrations, randomness prevents guarantee of a defined
-    // amount, or even expected amount
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    while (!migrationsOut.isEmpty()) {
-      balancer.balance(getAssignments(servers), migrations, migrationsOut);
-    }
-  }
-
-  private static KeyExtent makeExtent(String table, String end, String prev) {
-    return new KeyExtent(TableId.of(table), toText(end), toText(prev));
-  }
-
-  private static Text toText(String value) {
-    if (value != null)
-      return new Text(value);
-    return null;
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
index 5cb0ebe..28490ae 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
@@ -45,6 +45,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+@Deprecated(since = "2.1.0")
 public class DefaultLoadBalancerTest {
 
   class FakeTServer {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
index f09a68f..c6becce 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
@@ -43,6 +43,7 @@
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
+@Deprecated(since = "2.1.0")
 public class GroupBalancerTest {
 
   private static Function<KeyExtent,String> partitioner = new Function<>() {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
index 66e4f05..c6d4145 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
@@ -43,6 +43,7 @@
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.junit.Test;
 
+@Deprecated(since = "2.1.0")
 public class HostRegexTableLoadBalancerReconfigurationTest
     extends BaseHostRegexTableLoadBalancerTest {
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
index ec74f36..2b7c025 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
@@ -51,6 +51,7 @@
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.junit.Test;
 
+@Deprecated(since = "2.1.0")
 public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalancerTest {
 
   public void init() {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index b2b1c48..cd88059 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -50,6 +50,7 @@
 import org.easymock.EasyMock;
 import org.junit.Test;
 
+@Deprecated(since = "2.1.0")
 public class TableLoadBalancerTest {
 
   private static Map<String,String> TABLE_ID_MAP = Map.of("t1", "a1", "t2", "b12", "t3", "c4");
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/Master.java b/server/manager/src/main/java/org/apache/accumulo/master/Master.java
index 116698c..4d9d256 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/Master.java
@@ -44,6 +44,7 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -60,6 +61,10 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Iface;
@@ -78,6 +83,12 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
+import org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer;
+import org.apache.accumulo.core.spi.balancer.TabletBalancer;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
 import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
@@ -105,15 +116,13 @@
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.manager.balancer.BalancerEnvironmentImpl;
 import org.apache.accumulo.server.master.LiveTServerSet;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
-import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
-import org.apache.accumulo.server.master.balancer.TabletBalancer;
 import org.apache.accumulo.server.master.state.CurrentState;
 import org.apache.accumulo.server.master.state.DeadServerList;
 import org.apache.accumulo.server.master.state.MergeInfo;
 import org.apache.accumulo.server.master.state.MergeState;
-import org.apache.accumulo.server.master.state.TabletMigration;
 import org.apache.accumulo.server.master.state.TabletServerState;
 import org.apache.accumulo.server.master.state.TabletStateStore;
 import org.apache.accumulo.server.replication.ZooKeeperInitialization;
@@ -194,7 +203,8 @@
 
   ZooLock masterLock = null;
   private TServer clientService = null;
-  TabletBalancer tabletBalancer;
+  private volatile TabletBalancer tabletBalancer;
+  private final BalancerEnvironment balancerEnvironment;
 
   private MasterState state = MasterState.INITIAL;
 
@@ -202,6 +212,8 @@
 
   volatile SortedMap<TServerInstance,TabletServerStatus> tserverStatus =
       Collections.unmodifiableSortedMap(new TreeMap<>());
+  volatile SortedMap<TabletServerId,TServerStatus> tserverStatusForBalancer =
+      Collections.unmodifiableSortedMap(new TreeMap<>());
   final ServerBulkImportStatus bulkImportStatus = new ServerBulkImportStatus();
 
   private final AtomicBoolean masterInitialized = new AtomicBoolean(false);
@@ -364,6 +376,7 @@
   Master(ServerOpts opts, String[] args) throws IOException {
     super("master", opts, args);
     ServerContext context = super.getContext();
+    balancerEnvironment = new BalancerEnvironmentImpl(context);
 
     AccumuloConfiguration aconf = context.getConfiguration();
 
@@ -373,9 +386,7 @@
     ThriftTransportPool.getInstance()
         .setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
     tserverSet = new LiveTServerSet(context, this);
-    this.tabletBalancer = Property.createInstanceFromPropertyName(aconf,
-        Property.MANAGER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
-    this.tabletBalancer.init(context);
+    initializeBalancer();
 
     this.security = AuditedSecurityOperation.getInstance(context);
 
@@ -800,7 +811,9 @@
 
     private long updateStatus() {
       Set<TServerInstance> currentServers = tserverSet.getCurrentServers();
-      tserverStatus = gatherTableInformation(currentServers);
+      TreeMap<TabletServerId,TServerStatus> temp = new TreeMap<>();
+      tserverStatus = gatherTableInformation(currentServers, temp);
+      tserverStatusForBalancer = Collections.unmodifiableSortedMap(temp);
       checkForHeldServer(tserverStatus);
 
       if (!badServers.isEmpty()) {
@@ -853,34 +866,57 @@
     }
 
     private long balanceTablets() {
-      List<TabletMigration> migrationsOut = new ArrayList<>();
-      long wait = tabletBalancer.balance(Collections.unmodifiableSortedMap(tserverStatus),
-          migrationsSnapshot(), migrationsOut);
+      BalanceParamsImpl params = BalanceParamsImpl.fromThrift(tserverStatusForBalancer,
+          tserverStatus, migrationsSnapshot());
+      long wait = tabletBalancer.balance(params);
 
-      for (TabletMigration m : TabletBalancer.checkMigrationSanity(tserverStatus.keySet(),
-          migrationsOut)) {
-        if (migrations.containsKey(m.tablet)) {
+      for (TabletMigration m : checkMigrationSanity(tserverStatusForBalancer.keySet(),
+          params.migrationsOut())) {
+        KeyExtent ke = KeyExtent.fromTabletId(m.getTablet());
+        if (migrations.containsKey(ke)) {
           log.warn("balancer requested migration more than once, skipping {}", m);
           continue;
         }
-        migrations.put(m.tablet, m.newServer);
+        TServerInstance tserverInstance = TabletServerIdImpl.toThrift(m.getNewTabletServer());
+        migrations.put(ke, tserverInstance);
         log.debug("migration {}", m);
       }
-      if (migrationsOut.isEmpty()) {
+      if (params.migrationsOut().isEmpty()) {
         synchronized (balancedNotifier) {
           balancedNotifier.notifyAll();
         }
       } else {
-        nextEvent.event("Migrating %d more tablets, %d total", migrationsOut.size(),
+        nextEvent.event("Migrating %d more tablets, %d total", params.migrationsOut().size(),
             migrations.size());
       }
       return wait;
     }
 
+    private List<TabletMigration> checkMigrationSanity(Set<TabletServerId> current,
+        List<TabletMigration> migrations) {
+      return migrations.stream().filter(m -> {
+        boolean includeMigration = false;
+        if (m.getTablet() == null) {
+          log.error("Balancer gave back a null tablet {}", m);
+        } else if (m.getNewTabletServer() == null) {
+          log.error("Balancer did not set the destination {}", m);
+        } else if (m.getOldTabletServer() == null) {
+          log.error("Balancer did not set the source {}", m);
+        } else if (!current.contains(m.getOldTabletServer())) {
+          log.warn("Balancer wants to move a tablet from a server that is not current: {}", m);
+        } else if (!current.contains(m.getNewTabletServer())) {
+          log.warn("Balancer wants to move a tablet to a server that is not current: {}", m);
+        } else {
+          includeMigration = true;
+        }
+        return includeMigration;
+      }).collect(Collectors.toList());
+    }
+
   }
 
-  private SortedMap<TServerInstance,TabletServerStatus>
-      gatherTableInformation(Set<TServerInstance> currentServers) {
+  private SortedMap<TServerInstance,TabletServerStatus> gatherTableInformation(
+      Set<TServerInstance> currentServers, SortedMap<TabletServerId,TServerStatus> balancerMap) {
     final long rpcTimeout = getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT);
     int threads = getConfiguration().getCount(Property.MANAGER_STATUS_THREAD_POOL_SIZE);
     ExecutorService tp = ThreadPools.createExecutorService(getConfiguration(),
@@ -958,6 +994,8 @@
 
     // Threads may still modify map after shutdownNow is called, so create an immutable snapshot.
     SortedMap<TServerInstance,TabletServerStatus> info = ImmutableSortedMap.copyOf(result);
+    tserverStatus.forEach((tsi, status) -> balancerMap.put(new TabletServerIdImpl(tsi),
+        TServerStatusImpl.fromThrift(status)));
 
     synchronized (badServers) {
       badServers.keySet().retainAll(currentServers);
@@ -1652,9 +1690,9 @@
 
   @Override
   public Set<KeyExtent> migrationsSnapshot() {
-    Set<KeyExtent> migrationKeys = new HashSet<>();
+    Set<KeyExtent> migrationKeys;
     synchronized (migrations) {
-      migrationKeys.addAll(migrations.keySet());
+      migrationKeys = new HashSet<>(migrations.keySet());
     }
     return Collections.unmodifiableSet(migrationKeys);
   }
@@ -1688,4 +1726,21 @@
     return masterInitialized.get();
   }
 
+  void initializeBalancer() {
+    var localTabletBalancer = Property.createInstanceFromPropertyName(getConfiguration(),
+        Property.MANAGER_TABLET_BALANCER, TabletBalancer.class, new SimpleLoadBalancer());
+    localTabletBalancer.init(balancerEnvironment);
+    tabletBalancer = localTabletBalancer;
+  }
+
+  Class<?> getBalancerClass() {
+    return tabletBalancer.getClass();
+  }
+
+  void getAssignments(SortedMap<TServerInstance,TabletServerStatus> currentStatus,
+      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignedOut) {
+    AssignmentParamsImpl params =
+        AssignmentParamsImpl.fromThrift(currentStatus, unassigned, assignedOut);
+    tabletBalancer.getAssignments(params);
+  }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index d9bab5b..3b7bafc 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -48,7 +48,6 @@
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -85,8 +84,6 @@
 import org.apache.accumulo.master.tserverOps.ShutdownTServer;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
-import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer;
-import org.apache.accumulo.server.master.balancer.TabletBalancer;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretManager;
@@ -364,7 +361,7 @@
 
   @Override
   public void setSystemProperty(TInfo info, TCredentials c, String property, String value)
-      throws ThriftSecurityException, TException {
+      throws TException {
     if (!master.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -453,12 +450,8 @@
     // resolve without warning; any warnings should have already occurred
     String resolved = DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {});
     if (resolved.equals(Property.MANAGER_TABLET_BALANCER.getKey())) {
-      AccumuloConfiguration conf = master.getConfiguration();
-      TabletBalancer balancer = Property.createInstanceFromPropertyName(conf,
-          Property.MANAGER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
-      balancer.init(master.getContext());
-      master.tabletBalancer = balancer;
-      log.info("tablet balancer changed to {}", master.tabletBalancer.getClass().getName());
+      master.initializeBalancer();
+      log.info("tablet balancer changed to {}", master.getBalancerClass().getName());
     }
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/manager/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index 450b623..5167e92 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -879,7 +879,7 @@
       Map<KeyExtent,TServerInstance> unassigned) {
     if (!tLists.currentTServers.isEmpty()) {
       Map<KeyExtent,TServerInstance> assignedOut = new HashMap<>();
-      master.tabletBalancer.getAssignments(tLists.currentTServers, unassigned, assignedOut);
+      master.getAssignments(tLists.currentTServers, unassigned, assignedOut);
       for (Entry<KeyExtent,TServerInstance> assignment : assignedOut.entrySet()) {
         if (unassigned.containsKey(assignment.getKey())) {
           if (assignment.getValue() != null) {
diff --git a/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java b/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java
new file mode 100644
index 0000000..c2dd1e2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
+import org.apache.accumulo.core.spi.balancer.TabletBalancer;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.spi.balancer.util.ThrottledBalancerProblemReporter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A chaotic load balancer used for testing. It constantly shuffles tablets, preventing them from
+ * resting in a single location for very long. This is not designed for performance, do not use on
+ * production systems. I'm calling it the LokiLoadBalancer.
+ *
+ * <p>
+ * Will balance randomly, maintaining distribution
+ */
+public class ChaoticLoadBalancer implements TabletBalancer {
+  private static final Logger log = LoggerFactory.getLogger(ChaoticLoadBalancer.class);
+
+  protected BalancerEnvironment environment;
+  Random r = new SecureRandom();
+
+  public ChaoticLoadBalancer() {}
+
+  @Override
+  public void init(BalancerEnvironment balancerEnvironment) {
+    this.environment = balancerEnvironment;
+  }
+
+  @Override
+  public void getAssignments(AssignmentParameters params) {
+    long total = params.unassignedTablets().size();
+    long avg = (long) Math.ceil(((double) total) / params.currentStatus().size());
+    Map<TabletServerId,Long> toAssign = new HashMap<>();
+    List<TabletServerId> tServerArray = new ArrayList<>();
+    for (Entry<TabletServerId,TServerStatus> e : params.currentStatus().entrySet()) {
+      long numTablets = 0;
+      for (TableStatistics ti : e.getValue().getTableMap().values()) {
+        numTablets += ti.getTabletCount();
+      }
+      if (numTablets <= avg) {
+        tServerArray.add(e.getKey());
+        toAssign.put(e.getKey(), avg - numTablets);
+      }
+    }
+
+    if (tServerArray.isEmpty()) {
+      // No tservers to assign to
+      return;
+    }
+
+    for (TabletId tabletId : params.unassignedTablets().keySet()) {
+      int index = r.nextInt(tServerArray.size());
+      TabletServerId dest = tServerArray.get(index);
+      params.addAssignment(tabletId, dest);
+      long remaining = toAssign.get(dest) - 1;
+      if (remaining == 0) {
+        tServerArray.remove(index);
+        toAssign.remove(dest);
+      } else {
+        toAssign.put(dest, remaining);
+      }
+    }
+  }
+
+  private final ThrottledBalancerProblemReporter problemReporter =
+      new ThrottledBalancerProblemReporter(getClass());
+  private final ThrottledBalancerProblemReporter.OutstandingMigrationsProblem outstandingMigrationsProblem =
+      problemReporter.createOutstandingMigrationsProblem();
+
+  @Override
+  public long balance(BalanceParameters params) {
+    Map<TabletServerId,Long> numTablets = new HashMap<>();
+    List<TabletServerId> underCapacityTServer = new ArrayList<>();
+
+    if (!params.currentMigrations().isEmpty()) {
+      outstandingMigrationsProblem.setMigrations(params.currentMigrations());
+      problemReporter.reportProblem(outstandingMigrationsProblem);
+      return 100;
+    }
+    problemReporter.clearProblemReportTimes();
+
+    boolean moveMetadata = r.nextInt(4) == 0;
+    long totalTablets = 0;
+    for (Entry<TabletServerId,TServerStatus> e : params.currentStatus().entrySet()) {
+      long tabletCount = 0;
+      for (TableStatistics ti : e.getValue().getTableMap().values()) {
+        tabletCount += ti.getTabletCount();
+      }
+      numTablets.put(e.getKey(), tabletCount);
+      underCapacityTServer.add(e.getKey());
+      totalTablets += tabletCount;
+    }
+    // totalTablets is fuzzy due to asynchronicity of the stats
+    // *1.2 to handle fuzziness, and prevent locking for 'perfect' balancing scenarios
+    long avg = (long) Math.ceil(((double) totalTablets) / params.currentStatus().size() * 1.2);
+
+    for (Entry<TabletServerId,TServerStatus> e : params.currentStatus().entrySet()) {
+      for (String tableId : e.getValue().getTableMap().keySet()) {
+        TableId id = TableId.of(tableId);
+        if (!moveMetadata && MetadataTable.ID.equals(id))
+          continue;
+        try {
+          for (TabletStatistics ts : getOnlineTabletsForTable(e.getKey(), id)) {
+            int index = r.nextInt(underCapacityTServer.size());
+            TabletServerId dest = underCapacityTServer.get(index);
+            if (dest.equals(e.getKey()))
+              continue;
+            params.migrationsOut().add(new TabletMigration(ts.getTabletId(), e.getKey(), dest));
+            if (numTablets.put(dest, numTablets.get(dest) + 1) > avg)
+              underCapacityTServer.remove(index);
+            if (numTablets.put(e.getKey(), numTablets.get(e.getKey()) - 1) <= avg
+                && !underCapacityTServer.contains(e.getKey()))
+              underCapacityTServer.add(e.getKey());
+
+            // We can get some craziness with only 1 tserver, so lets make sure there's always an
+            // option!
+            if (underCapacityTServer.isEmpty())
+              underCapacityTServer.addAll(numTablets.keySet());
+          }
+        } catch (AccumuloSecurityException e1) {
+          // Shouldn't happen, but carry on if it does
+          log.debug(
+              "Encountered AccumuloSecurityException.  This should not happen.  Carrying on anyway.",
+              e1);
+        } catch (AccumuloException e1) {
+          // Shouldn't happen, but carry on if it does
+          log.debug("Encountered AccumuloException.  This should not happen.  Carrying on anyway.",
+              e1);
+        }
+      }
+    }
+
+    return 100;
+  }
+
+  protected List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tabletServerId,
+      TableId tableId) throws AccumuloException, AccumuloSecurityException {
+    return environment.listOnlineTabletsForTable(tabletServerId, tableId);
+  }
+
+}
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 c19924b..7c8f8e1 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
@@ -43,6 +43,7 @@
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
@@ -94,7 +95,12 @@
   public void setupTables() throws AccumuloException, AccumuloSecurityException,
       TableExistsException, TableNotFoundException {
     accumuloClient = Accumulo.newClient().from(getClientProps()).build();
-    // Need at least two tservers
+    // Need at least two tservers -- wait for them to start before failing
+    for (int retries = 0; retries < 5; ++retries) {
+      if (accumuloClient.instanceOperations().getTabletServers().size() >= 2)
+        break;
+      UtilWaitThread.sleep(TimeUnit.SECONDS.toMillis(2));
+    }
     Assume.assumeTrue("Not enough tservers to run test",
         accumuloClient.instanceOperations().getTabletServers().size() >= 2);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
index 204eedb..216404b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ChaoticBalancerIT.java
@@ -29,7 +29,7 @@
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.master.balancer.ChaoticLoadBalancer;
+import org.apache.accumulo.test.ChaoticLoadBalancer;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
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 508290d..a128ae1 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
@@ -40,8 +40,8 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.spi.balancer.RegexGroupBalancer;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.master.balancer.RegexGroupBalancer;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
diff --git a/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java
new file mode 100644
index 0000000..073b408
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
+import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
+import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
+import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
+import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
+import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
+import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
+import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
+import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class ChaoticLoadBalancerTest {
+
+  static class FakeTServer {
+    List<TabletId> tablets = new ArrayList<>();
+
+    TServerStatus getStatus() {
+      org.apache.accumulo.core.master.thrift.TabletServerStatus thriftStatus =
+          new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+      thriftStatus.tableMap = new HashMap<>();
+      for (TabletId extent : tablets) {
+        TableId table = extent.getTable();
+        TableInfo info = thriftStatus.tableMap.get(table.canonical());
+        if (info == null)
+          thriftStatus.tableMap.put(table.canonical(), info = new TableInfo());
+        info.onlineTablets++;
+        info.recs = info.onlineTablets;
+        info.ingestRate = 123.;
+        info.queryRate = 456.;
+      }
+
+      return new TServerStatusImpl(thriftStatus);
+    }
+  }
+
+  Map<TabletServerId,FakeTServer> servers = new HashMap<>();
+
+  class TestChaoticLoadBalancer extends ChaoticLoadBalancer {
+
+    @Override
+    public List<TabletStatistics> getOnlineTabletsForTable(TabletServerId tserver, TableId table) {
+      List<TabletStatistics> result = new ArrayList<>();
+      for (TabletId tabletId : servers.get(tserver).tablets) {
+        if (tabletId.getTable().equals(table)) {
+          KeyExtent extent =
+              new KeyExtent(tabletId.getTable(), tabletId.getEndRow(), tabletId.getPrevEndRow());
+          TabletStats tstats = new TabletStats(extent.toThrift(), null, null, null, 0L, 0., 0., 0);
+          result.add(new TabletStatisticsImpl(tstats));
+        }
+      }
+      return result;
+    }
+  }
+
+  @Test
+  public void testAssignMigrations() {
+    servers.clear();
+    servers.put(new TabletServerIdImpl("127.0.0.1", 1234, "a"), new FakeTServer());
+    servers.put(new TabletServerIdImpl("127.0.0.1", 1235, "b"), new FakeTServer());
+    servers.put(new TabletServerIdImpl("127.0.0.1", 1236, "c"), new FakeTServer());
+    Map<TabletId,TabletServerId> metadataTable = new TreeMap<>();
+    String table = "t1";
+    metadataTable.put(makeTablet(table, null, null), null);
+    table = "t2";
+    metadataTable.put(makeTablet(table, "a", null), null);
+    metadataTable.put(makeTablet(table, null, "a"), null);
+    table = "t3";
+    metadataTable.put(makeTablet(table, "a", null), null);
+    metadataTable.put(makeTablet(table, "b", "a"), null);
+    metadataTable.put(makeTablet(table, "c", "b"), null);
+    metadataTable.put(makeTablet(table, "d", "c"), null);
+    metadataTable.put(makeTablet(table, "e", "d"), null);
+    metadataTable.put(makeTablet(table, null, "e"), null);
+
+    TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer();
+
+    Map<TabletId,TabletServerId> assignments = new HashMap<>();
+    balancer.getAssignments(
+        new AssignmentParamsImpl(getAssignments(servers), metadataTable, assignments));
+
+    assertEquals(assignments.size(), metadataTable.size());
+  }
+
+  SortedMap<TabletServerId,TServerStatus> getAssignments(Map<TabletServerId,FakeTServer> servers) {
+    SortedMap<TabletServerId,TServerStatus> result = new TreeMap<>();
+    for (Entry<TabletServerId,FakeTServer> entry : servers.entrySet()) {
+      result.put(entry.getKey(), entry.getValue().getStatus());
+    }
+    return result;
+  }
+
+  @Test
+  public void testUnevenAssignment() {
+    servers.clear();
+    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
+      String cString = Character.toString(c);
+      TabletServerId tsi = new TabletServerIdImpl("127.0.0.1", c, cString);
+      FakeTServer fakeTServer = new FakeTServer();
+      servers.put(tsi, fakeTServer);
+      fakeTServer.tablets.add(makeTablet(cString, null, null));
+    }
+    // Put more tablets on one server, but not more than the number of servers
+    Entry<TabletServerId,FakeTServer> first = servers.entrySet().iterator().next();
+    first.getValue().tablets.add(makeTablet("newTable", "a", null));
+    first.getValue().tablets.add(makeTablet("newTable", "b", "a"));
+    first.getValue().tablets.add(makeTablet("newTable", "c", "b"));
+    first.getValue().tablets.add(makeTablet("newTable", "d", "c"));
+    first.getValue().tablets.add(makeTablet("newTable", "e", "d"));
+    first.getValue().tablets.add(makeTablet("newTable", "f", "e"));
+    first.getValue().tablets.add(makeTablet("newTable", "g", "f"));
+    first.getValue().tablets.add(makeTablet("newTable", "h", "g"));
+    first.getValue().tablets.add(makeTablet("newTable", "i", null));
+    TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer();
+    Set<TabletId> migrations = Collections.emptySet();
+
+    // Just want to make sure it gets some migrations, randomness prevents guarantee of a defined
+    // amount, or even expected amount
+    List<TabletMigration> migrationsOut = new ArrayList<>();
+    while (!migrationsOut.isEmpty()) {
+      balancer.balance(new BalanceParamsImpl(getAssignments(servers), migrations, migrationsOut));
+    }
+  }
+
+  private static TabletId makeTablet(String table, String end, String prev) {
+    return new TabletIdImpl(new KeyExtent(TableId.of(table), toText(end), toText(prev)));
+  }
+
+  private static Text toText(String value) {
+    if (value != null)
+      return new Text(value);
+    return null;
+  }
+
+}