You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/11/05 06:35:08 UTC

[hbase] 14/14: HBASE-22932 Add rs group management methods in Admin and AsyncAdmin (#657)

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

zhangduo pushed a commit to branch HBASE-22514
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 4a2251fa938390e934f6fcc40bac30744d3159e8
Author: linkaline <li...@gmail.com>
AuthorDate: Tue Nov 5 14:15:51 2019 +0800

    HBASE-22932 Add rs group management methods in Admin and AsyncAdmin (#657)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |   78 ++
 .../hadoop/hbase/client/AdminOverAsyncAdmin.java   |   52 +
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |   80 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |   51 +
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  209 +++-
 .../hbase/shaded/protobuf/RequestConverter.java    |   37 +-
 .../hbase/rsgroup/IntegrationTestRSGroup.java      |    4 +-
 .../src/main/protobuf/Master.proto                 |   19 +
 .../org/apache/hadoop/hbase/master/HMaster.java    |    2 +-
 .../hadoop/hbase/master/MasterCoprocessorHost.java |    6 +-
 .../hadoop/hbase/master/MasterRpcServices.java     |  171 +++-
 .../apache/hadoop/hbase/master/MasterServices.java |    9 +
 .../hbase/rsgroup/DisabledRSGroupInfoManager.java  |   13 +-
 .../apache/hadoop/hbase/rsgroup/RSGroupAdmin.java  |    4 +
 .../hadoop/hbase/rsgroup/RSGroupAdminClient.java   | 1041 ++++++++++++++++++--
 .../hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java |   25 +-
 .../hadoop/hbase/rsgroup/RSGroupAdminServer.java   |  495 ----------
 .../hbase/rsgroup/RSGroupAdminServiceImpl.java     |   83 +-
 .../hadoop/hbase/rsgroup/RSGroupInfoManager.java   |   20 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java      |  444 ++++++++-
 .../hbase/rsgroup/RSGroupMajorCompactionTTL.java   |    7 +-
 .../apache/hadoop/hbase/rsgroup/RSGroupUtil.java   |    8 +-
 .../hbase/security/access/AccessController.java    |   81 ++
 .../hbase/master/MockNoopMasterServices.java       |    5 +
 .../hbase/rsgroup/TestMigrateRSGroupInfo.java      |   28 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java   |  131 ++-
 .../hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java   |  130 +--
 .../hadoop/hbase/rsgroup/TestRSGroupsBalance.java  |   15 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java     |   97 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBasics.java   |   43 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsKillRS.java   |   26 +-
 .../hbase/rsgroup/TestRSGroupsOfflineMode.java     |    6 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsWithACL.java  |   47 +-
 .../hbase/rsgroup/VerifyingRSGroupAdminClient.java |   29 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   52 +
 35 files changed, 2577 insertions(+), 971 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 5419c16..2b4eaf8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
@@ -53,6 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2262,4 +2264,80 @@ public interface Admin extends Abortable, Closeable {
    */
   boolean isSnapshotCleanupEnabled() throws IOException;
 
+  /**
+   * Creates a new RegionServer group with the given name
+   * @param groupName the name of the group
+   * @throws IOException if a remote or network exception occurs
+   */
+  void addRSGroup(String groupName) throws IOException;
+
+  /**
+   * Get group info for the given group name
+   * @param groupName the group name
+   * @return group info
+   * @throws IOException if a remote or network exception occurs
+   */
+  RSGroupInfo getRSGroup(String groupName) throws IOException;
+
+  /**
+   * Get group info for the given hostPort
+   * @param hostPort HostPort to get RSGroupInfo for
+   * @throws IOException if a remote or network exception occurs
+   */
+  RSGroupInfo getRSGroup(Address hostPort) throws IOException;
+
+  /**
+   * Get group info for the given table
+   * @param tableName table name to get RSGroupInfo for
+   * @throws IOException if a remote or network exception occurs
+   */
+  RSGroupInfo getRSGroup(TableName tableName) throws IOException;
+
+  /**
+   * Lists current set of RegionServer groups
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<RSGroupInfo> listRSGroups() throws IOException;
+
+  /**
+   * Remove RegionServer group associated with the given name
+   * @param groupName the group name
+   * @throws IOException if a remote or network exception occurs
+   */
+  void removeRSGroup(String groupName) throws IOException;
+
+  /**
+   * Remove decommissioned servers from group
+   *  1. Sometimes we may find the server aborted due to some hardware failure and we must offline
+   *     the server for repairing. Or we need to move some servers to join other clusters.
+   *     So we need to remove these servers from the group.
+   *  2. Dead/recovering/live servers will be disallowed.
+   * @param servers set of servers to remove
+   * @throws IOException if a remote or network exception occurs
+   */
+  void removeRSGroup(Set<Address> servers) throws IOException;
+
+  /**
+   * Move given set of servers to the specified target RegionServer group
+   * @param servers set of servers to move
+   * @param targetGroup the group to move servers to
+   * @throws IOException if a remote or network exception occurs
+   */
+  void moveToRSGroup(Set<Address> servers, String targetGroup) throws IOException;
+
+  /**
+   * Set the RegionServer group for tables
+   * @param tables tables to set group for
+   * @param groupName group name for tables
+   * @throws IOException if a remote or network exception occurs
+   */
+  void setRSGroup(Set<TableName> tables, String groupName) throws IOException;
+
+  /**
+   * Balance regions in the given RegionServer group
+   * @param groupName the group name
+   * @return boolean Whether balance ran or not
+   * @throws IOException if a remote or network exception occurs
+   */
+  boolean balanceRSGroup(String groupName) throws IOException;
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
index f1f5b2a..060170c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -960,4 +962,54 @@ class AdminOverAsyncAdmin implements Admin {
     return get(admin.isSnapshotCleanupEnabled());
   }
 
+  @Override
+  public RSGroupInfo getRSGroup(String groupName) throws IOException {
+    return get(admin.getRSGroup(groupName));
+  }
+
+  @Override
+  public void moveToRSGroup(Set<Address> servers, String groupName) throws IOException {
+    get(admin.moveToRSGroup(servers, groupName));
+  }
+
+  @Override
+  public void addRSGroup(String groupName) throws IOException {
+    get(admin.addRSGroup(groupName));
+  }
+
+  @Override
+  public void removeRSGroup(String groupName) throws IOException {
+    get(admin.removeRSGroup(groupName));
+  }
+
+  @Override
+  public boolean balanceRSGroup(String groupName) throws IOException {
+    return get(admin.balanceRSGroup(groupName));
+  }
+
+  @Override
+  public List<RSGroupInfo> listRSGroups() throws IOException {
+    return get(admin.listRSGroups());
+  }
+
+  @Override
+  public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
+    return get(admin.getRSGroup(hostPort));
+  }
+
+  @Override
+  public void removeRSGroup(Set<Address> servers) throws IOException {
+    get(admin.removeRSGroup(servers));
+  }
+
+  @Override
+  public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
+    return get(admin.getRSGroup(tableName));
+  }
+
+  @Override
+  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
+    get(admin.setRSGroup(tables, groupName));
+  }
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index a53646d..12926cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -40,12 +40,14 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -1381,7 +1383,7 @@ public interface AsyncAdmin {
    * @param newTableName name of the new table where the table will be created
    * @param preserveSplits True if the splits should be preserved
    */
-  CompletableFuture<Void>  cloneTableSchema(final TableName tableName,
+  CompletableFuture<Void> cloneTableSchema(final TableName tableName,
       final TableName newTableName, final boolean preserveSplits);
 
   /**
@@ -1507,4 +1509,80 @@ public interface AsyncAdmin {
    */
   CompletableFuture<Boolean> isSnapshotCleanupEnabled();
 
+  /**
+   * Creates a new RegionServer group with the given name
+   * @param groupName the name of the group
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Void> addRSGroup(String groupName);
+
+  /**
+   * Get group info for the given group name
+   * @param groupName the group name
+   * @return group info
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<RSGroupInfo> getRSGroup(String groupName);
+
+  /**
+   * Get group info for the given hostPort
+   * @param hostPort HostPort to get RSGroupInfo for
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<RSGroupInfo> getRSGroup(Address hostPort);
+
+  /**
+   * Get group info for the given table
+   * @param tableName table name to get RSGroupInfo for
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<RSGroupInfo> getRSGroup(TableName tableName);
+
+  /**
+   * Lists current set of RegionServer groups
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<List<RSGroupInfo>> listRSGroups();
+
+  /**
+   * Remove RegionServer group associated with the given name
+   * @param groupName the group name
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Void> removeRSGroup(String groupName);
+
+  /**
+   * Remove decommissioned servers from group
+   *  1. Sometimes we may find the server aborted due to some hardware failure and we must offline
+   *     the server for repairing. Or we need to move some servers to join other clusters.
+   *     So we need to remove these servers from the group.
+   *  2. Dead/recovering/live servers will be disallowed.
+   * @param servers set of servers to remove
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Void> removeRSGroup(Set<Address> servers);
+
+  /**
+   * Move given set of servers to the specified target RegionServer group
+   * @param servers set of servers to move
+   * @param groupName the group to move servers to
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Void> moveToRSGroup(Set<Address> servers, String groupName);
+
+  /**
+   * Set the RegionServer group for tables
+   * @param tables tables to set group for
+   * @param groupName group name for tables
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Void> setRSGroup(Set<TableName> tables, String groupName);
+
+  /**
+   * Balance regions in the given RegionServer group
+   * @param groupName the group name
+   * @return boolean Whether balance ran or not
+   * @throws IOException if a remote or network exception occurs
+   */
+  CompletableFuture<Boolean> balanceRSGroup(String groupName);
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index ffb484c..459c225 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -36,12 +36,14 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -838,4 +840,53 @@ class AsyncHBaseAdmin implements AsyncAdmin {
     return wrap(rawAdmin.isSnapshotCleanupEnabled());
   }
 
+  public CompletableFuture<RSGroupInfo> getRSGroup(String groupName) {
+    return wrap(rawAdmin.getRSGroup(groupName));
+  }
+
+  @Override
+  public CompletableFuture<Void> moveToRSGroup(Set<Address> servers, String groupName) {
+    return wrap(rawAdmin.moveToRSGroup(servers, groupName));
+  }
+
+  @Override
+  public CompletableFuture<Void> addRSGroup(String groupName) {
+    return wrap(rawAdmin.addRSGroup(groupName));
+  }
+
+  @Override
+  public CompletableFuture<Void> removeRSGroup(String groupName) {
+    return wrap(rawAdmin.removeRSGroup(groupName));
+  }
+
+  @Override
+  public CompletableFuture<Boolean> balanceRSGroup(String groupName) {
+    return wrap(rawAdmin.balanceRSGroup(groupName));
+  }
+
+  @Override
+  public CompletableFuture<List<RSGroupInfo>> listRSGroups() {
+    return wrap(rawAdmin.listRSGroups());
+  }
+
+  @Override
+  public CompletableFuture<RSGroupInfo> getRSGroup(Address hostPort) {
+    return wrap(rawAdmin.getRSGroup(hostPort));
+  }
+
+  @Override
+  public CompletableFuture<Void> removeRSGroup(Set<Address> servers) {
+    return wrap(rawAdmin.removeRSGroup(servers));
+  }
+
+  @Override
+  public CompletableFuture<RSGroupInfo> getRSGroup(TableName tableName) {
+    return wrap(rawAdmin.getRSGroup(tableName));
+  }
+
+  @Override
+  public CompletableFuture<Void> setRSGroup(Set<TableName> tables, String groupName) {
+    return wrap(rawAdmin.setRSGroup(tables, groupName));
+  }
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 8c6ec0d..399ae28 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
@@ -206,8 +208,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedur
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos
-    .IsSnapshotCleanupEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
@@ -258,8 +259,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormali
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos
-    .SetSnapshotCleanupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
@@ -285,6 +285,18 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuo
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -3867,24 +3879,189 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Boolean> snapshotCleanupSwitch(final boolean on,
       final boolean sync) {
-    return this.<Boolean>newMasterCaller()
-        .action((controller, stub) -> this
-            .call(controller, stub,
-                RequestConverter.buildSetSnapshotCleanupRequest(on, sync),
-                MasterService.Interface::switchSnapshotCleanup,
-                SetSnapshotCleanupResponse::getPrevSnapshotCleanup))
-        .call();
+    return this.<Boolean>newMasterCaller().action((controller, stub) -> this
+        .call(controller, stub, RequestConverter.buildSetSnapshotCleanupRequest(on, sync),
+            MasterService.Interface::switchSnapshotCleanup,
+            SetSnapshotCleanupResponse::getPrevSnapshotCleanup)).call();
   }
 
   @Override
   public CompletableFuture<Boolean> isSnapshotCleanupEnabled() {
-    return this.<Boolean>newMasterCaller()
+    return this.<Boolean>newMasterCaller().action((controller, stub) -> this
+        .call(controller, stub, RequestConverter.buildIsSnapshotCleanupEnabledRequest(),
+            MasterService.Interface::isSnapshotCleanupEnabled,
+            IsSnapshotCleanupEnabledResponse::getEnabled)).call();
+  }
+
+  @Override
+  public CompletableFuture<Void> moveToRSGroup(Set<Address> servers, String groupName) {
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this.
+            <MoveServersRequest, MoveServersResponse, Void> call(controller, stub,
+                RequestConverter.buildMoveServersRequest(servers, groupName),
+              (s, c, req, done) -> s.moveServers(c, req, done), resp -> null))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<Void> addRSGroup(String groupName) {
+    return this.<Void> newMasterCaller()
+        .action(((controller, stub) -> this.
+            <AddRSGroupRequest, AddRSGroupResponse, Void> call(controller, stub,
+                AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
+              (s, c, req, done) -> s.addRSGroup(c, req, done), resp -> null)))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<Void> removeRSGroup(String groupName) {
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this.
+            <RemoveRSGroupRequest, RemoveRSGroupResponse, Void> call(controller, stub,
+                RemoveRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
+              (s, c, req, done) -> s.removeRSGroup(c, req, done), resp -> null))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<Boolean> balanceRSGroup(String groupName) {
+    return this.<Boolean> newMasterCaller()
+        .action((controller, stub) -> this.
+            <BalanceRSGroupRequest, BalanceRSGroupResponse, Boolean> call(controller, stub,
+                BalanceRSGroupRequest.newBuilder().setRSGroupName(groupName).build(),
+              (s, c, req, done) -> s.balanceRSGroup(c, req, done), resp -> resp.getBalanceRan()))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<List<RSGroupInfo>> listRSGroups() {
+    return this.<List<RSGroupInfo>> newMasterCaller()
         .action((controller, stub) -> this
-            .call(controller, stub,
-                RequestConverter.buildIsSnapshotCleanupEnabledRequest(),
-                MasterService.Interface::isSnapshotCleanupEnabled,
-                IsSnapshotCleanupEnabledResponse::getEnabled))
+            .<ListRSGroupInfosRequest, ListRSGroupInfosResponse, List<RSGroupInfo>> call(
+                controller, stub, ListRSGroupInfosRequest.getDefaultInstance(),
+              (s, c, req, done) -> s.listRSGroupInfos(c, req, done),
+              resp -> resp.getRSGroupInfoList().stream()
+                  .map(r -> ProtobufUtil.toGroupInfo(r))
+                  .collect(Collectors.toList())))
         .call();
   }
 
+  @Override
+  public CompletableFuture<RSGroupInfo> getRSGroup(Address hostPort) {
+    CompletableFuture<RSGroupInfo> future = new CompletableFuture<>();
+    addListener(listRSGroups(), (groups, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      for (RSGroupInfo rsGroupInfo : groups) {
+        if (rsGroupInfo.getServers().contains(hostPort)){
+          future.complete(rsGroupInfo);
+          return;
+        }
+      }
+      future.complete(null);
+    });
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<Void> removeRSGroup(Set<Address> servers) {
+    return this.<Void> newMasterCaller()
+        .action((controller, stub) -> this.
+            <RemoveServersRequest, RemoveServersResponse, Void> call(controller, stub,
+                RequestConverter.buildRemoveServersRequest(servers),
+              (s, c, req, done) -> s.removeServers(c, req, done), resp -> null))
+        .call();
+  }
+
+  @Override
+  public CompletableFuture<Void> setRSGroup(Set<TableName> tables, String groupName) {
+    CompletableFuture<Void> future = new CompletableFuture<>();
+    for (TableName tableName : tables) {
+      addListener(tableExists(tableName), (exist, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+          return;
+        }
+        if (!exist) {
+          future.completeExceptionally(new TableNotFoundException(tableName));
+          return;
+        }
+      });
+    }
+    addListener(listTableDescriptors(new ArrayList<>(tables)), ((tableDescriptions, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      if (tableDescriptions == null || tableDescriptions.isEmpty()) {
+        future.complete(null);
+        return;
+      }
+      List<TableDescriptor> newTableDescriptors = new ArrayList<>();
+      for (TableDescriptor td : tableDescriptions) {
+        newTableDescriptors
+            .add(TableDescriptorBuilder.newBuilder(td).setRegionServerGroup(groupName).build());
+      }
+      addListener(CompletableFuture.allOf(
+        newTableDescriptors.stream().map(this::modifyTable).toArray(CompletableFuture[]::new)),
+        (v, e) -> {
+          if (e != null) {
+            future.completeExceptionally(e);
+          } else {
+            future.complete(v);
+          }
+        });
+    }));
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<RSGroupInfo> getRSGroup(TableName table) {
+    CompletableFuture<RSGroupInfo> future = new CompletableFuture<>();
+    addListener(getDescriptor(table), (td, err) -> {
+      if (err != null) {
+        // return null instead of err to keep compatible with old semantics
+        // todo: need to change both this and UTs
+        future.complete(null);
+        return;
+      }
+      addListener(listRSGroups(), (groups, err2) -> {
+        if (err2 != null) {
+          future.completeExceptionally(err2);
+          return;
+        }
+        for (RSGroupInfo rsGroupInfo : groups) {
+          if (rsGroupInfo.getTables().contains(table)) {
+            future.complete(rsGroupInfo);
+            return;
+          }
+        }
+        future.complete(null);
+      });
+    });
+    return future;
+  }
+
+  @Override
+  public CompletableFuture<RSGroupInfo> getRSGroup(String groupName) {
+    CompletableFuture<RSGroupInfo> future = new CompletableFuture<>();
+    addListener(listRSGroups(), (groups, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      for (RSGroupInfo rsGroupInfo : groups) {
+        if (rsGroupInfo.getName().equals(groupName)){
+          future.complete(rsGroupInfo);
+          return;
+        }
+      }
+      future.complete(null);
+    });
+    return future;
+  }
+
+
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index d45423c..55040cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -26,7 +26,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
@@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -64,7 +64,9 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -121,8 +123,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJ
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos
-    .IsSnapshotCleanupEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotCleanupEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
@@ -137,8 +138,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleaner
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos
-    .SetSnapshotCleanupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSnapshotCleanupRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetTableStateInMetaRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest;
@@ -147,6 +147,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
@@ -1934,4 +1936,29 @@ public final class RequestConverter {
     return IsSnapshotCleanupEnabledRequest.newBuilder().build();
   }
 
+  public static MoveServersRequest buildMoveServersRequest(Set<Address> servers,
+      String targetGroup) {
+    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
+    for (Address el : servers) {
+      hostPorts.add(
+          HBaseProtos.ServerName.newBuilder().setHostName(el.getHostname()).setPort(el.getPort())
+              .build());
+    }
+    return MoveServersRequest.newBuilder().setTargetGroup(targetGroup).addAllServers(hostPorts)
+            .build();
+  }
+
+  public static RemoveServersRequest buildRemoveServersRequest(Set<Address> servers) {
+    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
+    for(Address el: servers) {
+      hostPorts.add(HBaseProtos.ServerName.newBuilder()
+          .setHostName(el.getHostname())
+          .setPort(el.getPort())
+          .build());
+    }
+    return RemoveServersRequest.newBuilder()
+        .addAllServers(hostPorts)
+        .build();
+  }
+
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
index 4f8b3ad..ef3a93a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
@@ -82,7 +82,7 @@ public class IntegrationTestRSGroup extends TestRSGroupsBase {
         LOG.info("Waiting for cleanup to finish "+ rsGroupAdmin.listRSGroups());
         //Might be greater since moving servers back to default
         //is after starting a server
-        return rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size()
+        return rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().size()
             >= NUM_SLAVES_BASE;
       }
     });
@@ -93,7 +93,7 @@ public class IntegrationTestRSGroup extends TestRSGroupsBase {
         LOG.info("Waiting for regionservers to be registered "+ rsGroupAdmin.listRSGroups());
         //Might be greater since moving servers back to default
         //is after starting a server
-        return rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size()
+        return rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().size()
             == getNumServers();
       }
     });
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index fee9ab8..37878f4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -37,6 +37,7 @@ import "Quota.proto";
 import "Replication.proto";
 import "Snapshot.proto";
 import "AccessControl.proto";
+import "RSGroupAdmin.proto";
 
 /* Column-level protobufs */
 
@@ -1081,6 +1082,24 @@ service MasterService {
   /** returns a list of namespace names */
   rpc ListNamespaces(ListNamespacesRequest)
     returns(ListNamespacesResponse);
+
+  rpc MoveServers(MoveServersRequest)
+    returns (MoveServersResponse);
+
+  rpc AddRSGroup(AddRSGroupRequest)
+    returns (AddRSGroupResponse);
+
+  rpc RemoveRSGroup(RemoveRSGroupRequest)
+    returns (RemoveRSGroupResponse);
+
+  rpc BalanceRSGroup(BalanceRSGroupRequest)
+    returns (BalanceRSGroupResponse);
+
+  rpc ListRSGroupInfos(ListRSGroupInfosRequest)
+    returns (ListRSGroupInfosResponse);
+
+  rpc RemoveServers(RemoveServersRequest)
+    returns (RemoveServersResponse);
 }
 
 // HBCK Service definitions.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 581b010..ade1215 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -3836,7 +3836,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   public static void decorateMasterConfiguration(Configuration conf) {
     String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
     String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
-    if (!plugins.contains(cleanerClass)) {
+    if (plugins == null || !plugins.contains(cleanerClass)) {
       conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
     }
     if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 47ef3d0..abd297c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -1772,7 +1772,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void preDecommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException {
+  public void preDecommissionRegionServers(List<ServerName> servers, boolean offload)
+      throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
@@ -1781,7 +1782,8 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postDecommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException {
+  public void postDecommissionRegionServers(List<ServerName> servers, boolean offload)
+      throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
       @Override
       public void call(MasterObserver observer) throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 12292cf..cf26cd2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -94,6 +94,7 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
@@ -118,6 +119,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
@@ -303,6 +305,18 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaSta
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.AddRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.BalanceRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.MoveServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
@@ -346,9 +360,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.Snapshot
  */
 @InterfaceAudience.Private
 @SuppressWarnings("deprecation")
-public class MasterRpcServices extends RSRpcServices
-      implements MasterService.BlockingInterface, RegionServerStatusService.BlockingInterface,
-        LockService.BlockingInterface, HbckService.BlockingInterface {
+public class MasterRpcServices extends RSRpcServices implements MasterService.BlockingInterface,
+    RegionServerStatusService.BlockingInterface,
+    LockService.BlockingInterface, HbckService.BlockingInterface {
   private static final Logger LOG = LoggerFactory.getLogger(MasterRpcServices.class.getName());
   private static final Logger AUDITLOG =
       LoggerFactory.getLogger("SecurityLogger."+MasterRpcServices.class.getName());
@@ -2844,4 +2858,155 @@ public class MasterRpcServices extends RSRpcServices
     }
     return true;
   }
+
+  @Override
+  public MoveServersResponse moveServers(RpcController controller, MoveServersRequest request)
+      throws ServiceException {
+    Set<Address> hostPorts = Sets.newHashSet();
+    MoveServersResponse.Builder builder = MoveServersResponse.newBuilder();
+    for (HBaseProtos.ServerName el : request.getServersList()) {
+      hostPorts.add(Address.fromParts(el.getHostName(), el.getPort()));
+    }
+    LOG.info(master.getClientIdAuditPrefix() + " move servers " + hostPorts + " to rsgroup " +
+        request.getTargetGroup());
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preMoveServers(hostPorts, request.getTargetGroup());
+      }
+      master.getRSGroupInfoManager().moveServers(hostPorts, request.getTargetGroup());
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postMoveServers(hostPorts, request.getTargetGroup());
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public AddRSGroupResponse addRSGroup(RpcController controller, AddRSGroupRequest request)
+      throws ServiceException {
+    AddRSGroupResponse.Builder builder = AddRSGroupResponse.newBuilder();
+    LOG.info(master.getClientIdAuditPrefix() + " add rsgroup " + request.getRSGroupName());
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preAddRSGroup(request.getRSGroupName());
+      }
+      master.getRSGroupInfoManager().addRSGroup(new RSGroupInfo(request.getRSGroupName()));
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postAddRSGroup(request.getRSGroupName());
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public RemoveRSGroupResponse removeRSGroup(RpcController controller, RemoveRSGroupRequest request)
+      throws ServiceException {
+    RemoveRSGroupResponse.Builder builder = RemoveRSGroupResponse.newBuilder();
+    LOG.info(master.getClientIdAuditPrefix() + " remove rsgroup " + request.getRSGroupName());
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preRemoveRSGroup(request.getRSGroupName());
+      }
+      master.getRSGroupInfoManager().removeRSGroup(request.getRSGroupName());
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postRemoveRSGroup(request.getRSGroupName());
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public BalanceRSGroupResponse balanceRSGroup(RpcController controller,
+      BalanceRSGroupRequest request) throws ServiceException {
+    BalanceRSGroupResponse.Builder builder = BalanceRSGroupResponse.newBuilder();
+    LOG.info(
+        master.getClientIdAuditPrefix() + " balance rsgroup, group=" + request.getRSGroupName());
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preBalanceRSGroup(request.getRSGroupName());
+      }
+      boolean balancerRan =
+          master.getRSGroupInfoManager().balanceRSGroup(request.getRSGroupName());
+      builder.setBalanceRan(balancerRan);
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postBalanceRSGroup(request.getRSGroupName(), balancerRan);
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public ListRSGroupInfosResponse listRSGroupInfos(RpcController controller,
+      ListRSGroupInfosRequest request) throws ServiceException {
+    ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
+    LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preListRSGroups();
+      }
+      List<RSGroupInfo> rsGroupInfos = master.getRSGroupInfoManager().listRSGroups().stream()
+          .map(RSGroupInfo::new).collect(Collectors.toList());
+      Map<String, RSGroupInfo> name2Info = new HashMap<>();
+      List<TableDescriptor> needToFill =
+          new ArrayList<>(master.getTableDescriptors().getAll().values());
+      for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
+        name2Info.put(rsGroupInfo.getName(), rsGroupInfo);
+        for (TableDescriptor td : master.getTableDescriptors().getAll().values()) {
+          if (rsGroupInfo.containsTable(td.getTableName())){
+            needToFill.remove(td);
+          }
+        }
+      }
+      for (TableDescriptor td : needToFill) {
+        String groupName = td.getRegionServerGroup().orElse(RSGroupInfo.DEFAULT_GROUP);
+        RSGroupInfo rsGroupInfo = name2Info.get(groupName);
+        if (rsGroupInfo != null) {
+          rsGroupInfo.addTable(td.getTableName());
+        }
+      }
+      for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
+        // TODO: this can be done at once outside this loop, do not need to scan all every time.
+        builder.addRSGroupInfo(ProtobufUtil.toProtoGroupInfo(rsGroupInfo));
+      }
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postListRSGroups();
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public RemoveServersResponse removeServers(RpcController controller,
+      RemoveServersRequest request) throws ServiceException {
+    RemoveServersResponse.Builder builder = RemoveServersResponse.newBuilder();
+    Set<Address> servers = Sets.newHashSet();
+    for (HBaseProtos.ServerName el : request.getServersList()) {
+      servers.add(Address.fromParts(el.getHostName(), el.getPort()));
+    }
+    LOG.info(master.getClientIdAuditPrefix() + " remove decommissioned servers from rsgroup: " +
+        servers);
+    try {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preRemoveServers(servers);
+      }
+      master.getRSGroupInfoManager().removeServers(servers);
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postRemoveServers(servers);
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return builder.build();
+  }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index ae55352..68e5c76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
+import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -540,4 +541,12 @@ public interface MasterServices extends Server {
    * @return the {@link RSGroupInfoManager}
    */
   RSGroupInfoManager getRSGroupInfoManager();
+
+  /**
+   * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
+   * false is returned.
+   *
+   * @return The state of the load balancer, or false if the load balancer isn't defined.
+   */
+  boolean isBalancerOn();
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/DisabledRSGroupInfoManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/DisabledRSGroupInfoManager.java
index c7c521c..30dd4d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/DisabledRSGroupInfoManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/DisabledRSGroupInfoManager.java
@@ -57,8 +57,7 @@ class DisabledRSGroupInfoManager implements RSGroupInfoManager {
   }
 
   @Override
-  public Set<Address> moveServers(Set<Address> servers, String srcGroup, String dstGroup)
-    throws IOException {
+  public void moveServers(Set<Address> servers, String targetGroupName) throws IOException {
     throw new DoNotRetryIOException("RSGroup is disabled");
   }
 
@@ -107,4 +106,14 @@ class DisabledRSGroupInfoManager implements RSGroupInfoManager {
   public RSGroupInfo getRSGroupForTable(TableName tableName) throws IOException {
     return null;
   }
+
+  @Override
+  public boolean balanceRSGroup(String groupName) throws IOException {
+    throw new DoNotRetryIOException("RSGroup is disabled");
+  }
+
+  @Override
+  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
+    throw new DoNotRetryIOException("RSGroup is disabled");
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
index 344d0b3..3de6965 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdmin.java
@@ -20,12 +20,16 @@ package org.apache.hadoop.hbase.rsgroup;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Group user API interface used between client and server.
+ *
+ * @deprecated Keep it here only for tests, using {@link Admin} instead.
  */
+@Deprecated
 @InterfaceAudience.Private
 public interface RSGroupAdmin {
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
index 07f0efd..7cf4ed1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminClient.java
@@ -20,12 +20,33 @@ package org.apache.hadoop.hbase.rsgroup;
 import com.google.protobuf.ServiceException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CacheEvictionStats;
+import org.apache.hadoop.hbase.ClusterMetrics;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.CompactType;
+import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.SnapshotDescription;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.replication.TableCFs;
+import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -38,13 +59,26 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.GetRSGroupI
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.GetRSGroupInfoResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.ListRSGroupInfosRequest;
-import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersAndTablesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveTablesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdminService;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
+import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
+import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -52,27 +86,866 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /**
  * Client used for managing region server group information.
+ *
+ * @deprecated Keep it here only for tests, using {@link Admin} instead.
  */
+@Deprecated
 @InterfaceAudience.Private
-public class RSGroupAdminClient implements RSGroupAdmin {
+public class RSGroupAdminClient implements RSGroupAdmin, Admin {
   private RSGroupAdminService.BlockingInterface stub;
   private Admin admin;
 
-  public RSGroupAdminClient(Connection conn) throws IOException {
-    admin = conn.getAdmin();
-    stub = RSGroupAdminService.newBlockingStub(admin.coprocessorService());
+  public RSGroupAdminClient(Connection conn) throws IOException {
+    admin = conn.getAdmin();
+    stub = RSGroupAdminService.newBlockingStub(admin.coprocessorService());
+  }
+
+  // for writing UTs
+  @VisibleForTesting
+  protected RSGroupAdminClient() {
+  }
+
+  @Override
+  public int getOperationTimeout() {
+    return 0;
+  }
+
+  @Override
+  public int getSyncWaitTimeout() {
+    return 0;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+
+  }
+
+  @Override
+  public boolean isAborted() {
+    return false;
+  }
+
+  @Override
+  public Connection getConnection() {
+    return null;
+  }
+
+  @Override
+  public boolean tableExists(TableName tableName) throws IOException {
+    return false;
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors() throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(boolean includeSysTables) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public TableName[] listTableNames() throws IOException {
+    return new TableName[0];
+  }
+
+  @Override
+  public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
+    return new TableName[0];
+  }
+
+  @Override
+  public TableDescriptor getDescriptor(TableName tableName)
+      throws TableNotFoundException, IOException {
+    return null;
+  }
+
+  @Override
+  public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
+      throws IOException {
+
+  }
+
+  @Override
+  public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> deleteTableAsync(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> enableTableAsync(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> disableTableAsync(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean isTableEnabled(TableName tableName) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isTableDisabled(TableName tableName) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isTableAvailable(TableName tableName) throws IOException {
+    return false;
+  }
+
+  @Override
+  public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> modifyColumnFamilyAsync(TableName tableName,
+      ColumnFamilyDescriptor columnFamily) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void flush(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void flushRegion(byte[] regionName) throws IOException {
+
+  }
+
+  @Override
+  public void flushRegionServer(ServerName serverName) throws IOException {
+
+  }
+
+  @Override
+  public void compact(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void compactRegion(byte[] regionName) throws IOException {
+
+  }
+
+  @Override
+  public void compact(TableName tableName, byte[] columnFamily) throws IOException {
+
+  }
+
+  @Override
+  public void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
+
+  }
+
+  @Override
+  public void compact(TableName tableName, CompactType compactType)
+      throws IOException, InterruptedException {
+
+  }
+
+  @Override
+  public void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
+      throws IOException, InterruptedException {
+
+  }
+
+  @Override
+  public void majorCompact(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void majorCompactRegion(byte[] regionName) throws IOException {
+
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException {
+
+  }
+
+  @Override
+  public void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
+
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, CompactType compactType)
+      throws IOException, InterruptedException {
+
+  }
+
+  @Override
+  public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
+      throws IOException, InterruptedException {
+
+  }
+
+  @Override
+  public Map<ServerName, Boolean> compactionSwitch(boolean switchState,
+      List<String> serverNamesList) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void compactRegionServer(ServerName serverName) throws IOException {
+
+  }
+
+  @Override
+  public void majorCompactRegionServer(ServerName serverName) throws IOException {
+
+  }
+
+  @Override
+  public void move(byte[] encodedRegionName) throws IOException {
+
+  }
+
+  @Override
+  public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
+
+  }
+
+  @Override
+  public void assign(byte[] regionName) throws IOException {
+
+  }
+
+  @Override
+  public void unassign(byte[] regionName, boolean force) throws IOException {
+
+  }
+
+  @Override
+  public void offline(byte[] regionName) throws IOException {
+
+  }
+
+  @Override
+  public boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean balance() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean balance(boolean force) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isBalancerEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public CacheEvictionStats clearBlockCache(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean normalize() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isNormalizerEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean normalizerSwitch(boolean on) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean catalogJanitorSwitch(boolean onOrOff) throws IOException {
+    return false;
+  }
+
+  @Override
+  public int runCatalogJanitor() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public boolean isCatalogJanitorEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean cleanerChoreSwitch(boolean onOrOff) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean runCleanerChore() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isCleanerChoreEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public void split(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void split(TableName tableName, byte[] splitPoint) throws IOException {
+
+  }
+
+  @Override
+  public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+
+  }
+
+  @Override
+  public void stopMaster() throws IOException {
+
+  }
+
+  @Override
+  public boolean isMasterInMaintenanceMode() throws IOException {
+    return false;
+  }
+
+  @Override
+  public void stopRegionServer(String hostnamePort) throws IOException {
+
+  }
+
+  @Override
+  public ClusterMetrics getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return null;
+  }
+
+  @Override
+  public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> deleteNamespaceAsync(String name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name)
+      throws NamespaceNotFoundException, IOException {
+    return null;
+  }
+
+  @Override
+  public String[] listNamespaces() throws IOException {
+    return new String[0];
+  }
+
+  @Override
+  public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
+    return new NamespaceDescriptor[0];
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
+    return null;
+  }
+
+  @Override
+  public TableName[] listTableNamesByNamespace(String name) throws IOException {
+    return new TableName[0];
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void close() {
+
+  }
+
+  @Override
+  public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public String getProcedures() throws IOException {
+    return null;
+  }
+
+  @Override
+  public String getLocks() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException {
+
+  }
+
+  @Override
+  public CompactionState getCompactionState(TableName tableName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public CompactionState getCompactionState(TableName tableName, CompactType compactType)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void snapshot(SnapshotDescription snapshot)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
+
+  }
+
+  @Override
+  public Future<Void> snapshotAsync(SnapshotDescription snapshot)
+      throws IOException, SnapshotCreationException {
+    return null;
+  }
+
+  @Override
+  public boolean isSnapshotFinished(SnapshotDescription snapshot)
+      throws IOException, HBaseSnapshotException, UnknownSnapshotException {
+    return false;
+  }
+
+  @Override
+  public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException {
+
+  }
+
+  @Override
+  public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
+      throws IOException, RestoreSnapshotException {
+
+  }
+
+  @Override
+  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
+      boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
+    return null;
+  }
+
+  @Override
+  public void execProcedure(String signature, String instance, Map<String, String> props)
+      throws IOException {
+
+  }
+
+  @Override
+  public byte[] execProcedureWithReturn(String signature, String instance,
+      Map<String, String> props) throws IOException {
+    return new byte[0];
+  }
+
+  @Override
+  public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
+      throws IOException {
+    return false;
+  }
+
+  @Override
+  public List<SnapshotDescription> listSnapshots() throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
+      Pattern snapshotNamePattern) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void deleteSnapshot(String snapshotName) throws IOException {
+
   }
 
-  // for writing UTs
-  @VisibleForTesting
-  protected RSGroupAdminClient() {
+  @Override
+  public void deleteSnapshots(Pattern pattern) throws IOException {
+
+  }
+
+  @Override
+  public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
+      throws IOException {
+
+  }
+
+  @Override
+  public void setQuota(QuotaSettings quota) throws IOException {
+
+  }
+
+  @Override
+  public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
+    return null;
+  }
+
+  @Override
+  public CoprocessorRpcChannel coprocessorService() {
+    return null;
+  }
+
+  @Override
+  public CoprocessorRpcChannel coprocessorService(ServerName serverName) {
+    return null;
+  }
+
+  @Override
+  public void updateConfiguration(ServerName server) throws IOException {
+
+  }
+
+  @Override
+  public void updateConfiguration() throws IOException {
+
+  }
+
+  @Override
+  public List<SecurityCapability> getSecurityCapabilities() throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isSplitEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isMergeEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> enableReplicationPeerAsync(String peerId) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> disableReplicationPeerAsync(String peerId) throws IOException {
+    return null;
+  }
+
+  @Override
+  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> updateReplicationPeerConfigAsync(String peerId,
+      ReplicationPeerConfig peerConfig) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
+    return null;
+  }
+
+  @Override
+  public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void decommissionRegionServers(List<ServerName> servers, boolean offload)
+      throws IOException {
+
+  }
+
+  @Override
+  public List<ServerName> listDecommissionedRegionServers() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
+      throws IOException {
+
+  }
+
+  @Override
+  public List<TableCFs> listReplicatedTableCFs() throws IOException {
+    return null;
+  }
+
+  @Override
+  public void enableTableReplication(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void disableTableReplication(TableName tableName) throws IOException {
+
+  }
+
+  @Override
+  public void clearCompactionQueues(ServerName serverName, Set<String> queues)
+      throws IOException, InterruptedException {
+
+  }
+
+  @Override
+  public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
+      throws IOException {
+
+  }
+
+  @Override
+  public boolean switchRpcThrottle(boolean enable) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isRpcThrottleEnabled() throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException {
+    return false;
+  }
+
+  @Override
+  public Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException {
+    return null;
+  }
+
+  @Override
+  public Map<TableName, ? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots(
+      ServerName serverName) throws IOException {
+    return null;
+  }
+
+  @Override
+  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException {
+    return null;
+  }
+
+  @Override
+  public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
+      throws IOException {
+
+  }
+
+  @Override
+  public void revoke(UserPermission userPermission) throws IOException {
+
+  }
+
+  @Override
+  public List<UserPermission> getUserPermissions(
+      GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
+    return null;
+  }
+
+  @Override
+  public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public boolean snapshotCleanupSwitch(boolean on, boolean synchronous) throws IOException {
+    return false;
+  }
+
+  @Override
+  public boolean isSnapshotCleanupEnabled() throws IOException {
+    return false;
   }
 
   @Override
   public RSGroupInfo getRSGroupInfo(String groupName) throws IOException {
+    return getRSGroup(groupName);
+  }
+
+  @Override
+  public void moveServers(Set<Address> servers, String targetGroup) throws IOException {
+    moveToRSGroup(servers, targetGroup);
+  }
+
+  @Override
+  public void addRSGroup(String groupName) throws IOException {
+    AddRSGroupRequest request = AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build();
+    try {
+      stub.addRSGroup(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  @Override
+  public RSGroupInfo getRSGroup(String groupName) throws IOException {
     try {
       GetRSGroupInfoResponse resp = stub.getRSGroupInfo(null,
-        GetRSGroupInfoRequest.newBuilder().setRSGroupName(groupName).build());
+          GetRSGroupInfoRequest.newBuilder().setRSGroupName(groupName).build());
       if (resp.hasRSGroupInfo()) {
         return ProtobufUtil.toGroupInfo(resp.getRSGroupInfo());
       }
@@ -82,11 +955,13 @@ public class RSGroupAdminClient implements RSGroupAdmin {
     }
   }
 
-  public RSGroupInfo getRSGroupInfoOfTable(TableName tableName) throws IOException {
-    GetRSGroupInfoOfTableRequest request = GetRSGroupInfoOfTableRequest.newBuilder().setTableName(
-        ProtobufUtil.toProtoTableName(tableName)).build();
+  @Override
+  public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
+    GetRSGroupInfoOfServerRequest request = GetRSGroupInfoOfServerRequest.newBuilder().setServer(
+        HBaseProtos.ServerName.newBuilder().setHostName(hostPort.getHostname())
+            .setPort(hostPort.getPort()).build()).build();
     try {
-      GetRSGroupInfoOfTableResponse resp = stub.getRSGroupInfoOfTable(null, request);
+      GetRSGroupInfoOfServerResponse resp = stub.getRSGroupInfoOfServer(null, request);
       if (resp.hasRSGroupInfo()) {
         return ProtobufUtil.toGroupInfo(resp.getRSGroupInfo());
       }
@@ -97,55 +972,76 @@ public class RSGroupAdminClient implements RSGroupAdmin {
   }
 
   @Override
-  public void moveServers(Set<Address> servers, String targetGroup) throws IOException {
-    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
-    for(Address el: servers) {
-      hostPorts.add(HBaseProtos.ServerName.newBuilder()
-        .setHostName(el.getHostname())
-        .setPort(el.getPort())
-        .build());
+  public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
+    GetRSGroupInfoOfTableRequest request = GetRSGroupInfoOfTableRequest.newBuilder()
+        .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
+    try {
+      GetRSGroupInfoOfTableResponse resp = stub.getRSGroupInfoOfTable(null, request);
+      if (resp.hasRSGroupInfo()) {
+        return ProtobufUtil.toGroupInfo(resp.getRSGroupInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
-    MoveServersRequest request = MoveServersRequest.newBuilder()
-            .setTargetGroup(targetGroup)
-            .addAllServers(hostPorts)
-            .build();
+  }
+
+  @Override
+  public void removeRSGroup(String name) throws IOException {
+    RemoveRSGroupRequest request = RemoveRSGroupRequest.newBuilder().setRSGroupName(name).build();
     try {
-      stub.moveServers(null, request);
+      stub.removeRSGroup(null, request);
     } catch (ServiceException e) {
       throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
-  public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
-    MoveTablesRequest.Builder builder = MoveTablesRequest.newBuilder().setTargetGroup(targetGroup);
-    for(TableName tableName: tables) {
-      builder.addTableName(ProtobufUtil.toProtoTableName(tableName));
-      if (!admin.tableExists(tableName)) {
-        throw new TableNotFoundException(tableName);
-      }
+  @Override
+  public void removeRSGroup(Set<Address> servers) throws IOException {
+    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
+    for (Address el : servers) {
+      hostPorts.add(
+          HBaseProtos.ServerName.newBuilder().setHostName(el.getHostname()).setPort(el.getPort())
+              .build());
     }
+    RemoveServersRequest request =
+        RemoveServersRequest.newBuilder().addAllServers(hostPorts).build();
     try {
-      stub.moveTables(null, builder.build());
+      stub.removeServers(null, request);
     } catch (ServiceException e) {
       throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
   @Override
-  public void addRSGroup(String groupName) throws IOException {
-    AddRSGroupRequest request = AddRSGroupRequest.newBuilder().setRSGroupName(groupName).build();
+  public void moveToRSGroup(Set<Address> servers, String targetGroup) throws IOException {
+    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
+    for (Address el : servers) {
+      hostPorts.add(
+          HBaseProtos.ServerName.newBuilder().setHostName(el.getHostname()).setPort(el.getPort())
+              .build());
+    }
+    MoveServersRequest request =
+        MoveServersRequest.newBuilder().setTargetGroup(targetGroup).addAllServers(hostPorts)
+            .build();
     try {
-      stub.addRSGroup(null, request);
+      stub.moveServers(null, request);
     } catch (ServiceException e) {
       throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
   @Override
-  public void removeRSGroup(String name) throws IOException {
-    RemoveRSGroupRequest request = RemoveRSGroupRequest.newBuilder().setRSGroupName(name).build();
+  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
+    MoveTablesRequest.Builder builder = MoveTablesRequest.newBuilder().setTargetGroup(groupName);
+    for (TableName tableName : tables) {
+      builder.addTableName(ProtobufUtil.toProtoTableName(tableName));
+      if (!admin.tableExists(tableName)) {
+        throw new TableNotFoundException(tableName);
+      }
+    }
     try {
-      stub.removeRSGroup(null, request);
+      stub.moveTables(null, builder.build());
     } catch (ServiceException e) {
       throw ProtobufUtil.handleRemoteException(e);
     }
@@ -153,8 +1049,8 @@ public class RSGroupAdminClient implements RSGroupAdmin {
 
   @Override
   public boolean balanceRSGroup(String groupName) throws IOException {
-    BalanceRSGroupRequest request = BalanceRSGroupRequest.newBuilder()
-        .setRSGroupName(groupName).build();
+    BalanceRSGroupRequest request =
+        BalanceRSGroupRequest.newBuilder().setRSGroupName(groupName).build();
     try {
       return stub.balanceRSGroup(null, request).getBalanceRan();
     } catch (ServiceException e) {
@@ -165,10 +1061,11 @@ public class RSGroupAdminClient implements RSGroupAdmin {
   @Override
   public List<RSGroupInfo> listRSGroups() throws IOException {
     try {
-      List<RSGroupProtos.RSGroupInfo> resp = stub.listRSGroupInfos(null,
-          ListRSGroupInfosRequest.getDefaultInstance()).getRSGroupInfoList();
+      List<RSGroupProtos.RSGroupInfo> resp =
+          stub.listRSGroupInfos(null, ListRSGroupInfosRequest.getDefaultInstance())
+              .getRSGroupInfoList();
       List<RSGroupInfo> result = new ArrayList<>(resp.size());
-      for(RSGroupProtos.RSGroupInfo entry : resp) {
+      for (RSGroupProtos.RSGroupInfo entry : resp) {
         result.add(ProtobufUtil.toGroupInfo(entry));
       }
       return result;
@@ -179,62 +1076,12 @@ public class RSGroupAdminClient implements RSGroupAdmin {
 
   @Override
   public RSGroupInfo getRSGroupOfServer(Address hostPort) throws IOException {
-    GetRSGroupInfoOfServerRequest request = GetRSGroupInfoOfServerRequest.newBuilder()
-            .setServer(HBaseProtos.ServerName.newBuilder()
-                .setHostName(hostPort.getHostname())
-                .setPort(hostPort.getPort())
-                .build())
-            .build();
-    try {
-      GetRSGroupInfoOfServerResponse resp = stub.getRSGroupInfoOfServer(null, request);
-      if (resp.hasRSGroupInfo()) {
-        return ProtobufUtil.toGroupInfo(resp.getRSGroupInfo());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  public void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String targetGroup)
-      throws IOException {
-    MoveServersAndTablesRequest.Builder builder =
-            MoveServersAndTablesRequest.newBuilder().setTargetGroup(targetGroup);
-    for(Address el: servers) {
-      builder.addServers(HBaseProtos.ServerName.newBuilder()
-              .setHostName(el.getHostname())
-              .setPort(el.getPort())
-              .build());
-    }
-    for(TableName tableName: tables) {
-      builder.addTableName(ProtobufUtil.toProtoTableName(tableName));
-      if (!admin.tableExists(tableName)) {
-        throw new TableNotFoundException(tableName);
-      }
-    }
-    try {
-      stub.moveServersAndTables(null, builder.build());
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
+    return getRSGroup(hostPort);
   }
 
   @Override
   public void removeServers(Set<Address> servers) throws IOException {
-    Set<HBaseProtos.ServerName> hostPorts = Sets.newHashSet();
-    for(Address el: servers) {
-      hostPorts.add(HBaseProtos.ServerName.newBuilder()
-          .setHostName(el.getHostname())
-          .setPort(el.getPort())
-          .build());
-    }
-    RemoveServersRequest request = RemoveServersRequest.newBuilder()
-        .addAllServers(hostPorts)
-        .build();
-    try {
-      stub.removeServers(null, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
+    removeRSGroup(servers);
   }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 353b4d2..621e822 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -25,12 +25,8 @@ import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 /**
  * @deprecated Keep it here only for compatibility with old client, all the logics have been moved
  *             into core of HBase.
@@ -42,8 +38,6 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor {
   // Only instance of RSGroupInfoManager. RSGroup aware load balancers ask for this instance on
   // their setup.
   private MasterServices master;
-  private RSGroupInfoManager groupInfoManager;
-  private RSGroupAdminServer groupAdminServer;
   private RSGroupAdminServiceImpl groupAdminService = new RSGroupAdminServiceImpl();
 
   @Override
@@ -51,19 +45,8 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor {
     if (!(env instanceof HasMasterServices)) {
       throw new IOException("Does not implement HMasterServices");
     }
-
     master = ((HasMasterServices) env).getMasterServices();
-    groupInfoManager = master.getRSGroupInfoManager();
-    groupAdminServer = new RSGroupAdminServer(master, groupInfoManager);
-    AccessChecker accessChecker = ((HasMasterServices) env).getMasterServices().getAccessChecker();
-
-    // set the user-provider.
-    UserProvider userProvider = UserProvider.instantiate(env.getConfiguration());
-    groupAdminService.initialize(master, groupAdminServer, accessChecker, userProvider);
-  }
-
-  @Override
-  public void stop(CoprocessorEnvironment env) {
+    groupAdminService.initialize(master);
   }
 
   @Override
@@ -72,11 +55,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor {
   }
 
   RSGroupInfoManager getGroupInfoManager() {
-    return groupInfoManager;
+    return master.getRSGroupInfoManager();
   }
 
-  @VisibleForTesting
-  RSGroupAdminServiceImpl getGroupAdminService() {
-    return groupAdminService;
-  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
deleted file mode 100644
index 1e324e5..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ /dev/null
@@ -1,495 +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.hadoop.hbase.rsgroup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Function;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
-import org.apache.hadoop.hbase.net.Address;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-
-/**
- * Service to support Region Server Grouping (HBase-6721).
- */
-@InterfaceAudience.Private
-public class RSGroupAdminServer implements RSGroupAdmin {
-  private static final Logger LOG = LoggerFactory.getLogger(RSGroupAdminServer.class);
-  static final String KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE = "should keep at least " +
-          "one server in 'default' RSGroup.";
-
-  private MasterServices master;
-  final RSGroupInfoManager rsGroupInfoManager;
-
-  /** Define the config key of retries threshold when movements failed */
-  //made package private for testing
-  static final String FAILED_MOVE_MAX_RETRY = "hbase.rsgroup.move.max.retry";
-
-  /** Define the default number of retries */
-  //made package private for testing
-  static final int DEFAULT_MAX_RETRY_VALUE = 50;
-
-  private int moveMaxRetry;
-
-  public RSGroupAdminServer(MasterServices master, RSGroupInfoManager rsGroupInfoManager) {
-    this.master = master;
-    this.rsGroupInfoManager = rsGroupInfoManager;
-    this.moveMaxRetry = master.getConfiguration().getInt(FAILED_MOVE_MAX_RETRY,
-        DEFAULT_MAX_RETRY_VALUE);
-  }
-
-  @Override
-  public RSGroupInfo getRSGroupInfo(String groupName) throws IOException {
-    return rsGroupInfoManager.getRSGroup(groupName);
-  }
-
-  private void checkOnlineServersOnly(Set<Address> servers) throws ConstraintException {
-    // This uglyness is because we only have Address, not ServerName.
-    // Online servers are keyed by ServerName.
-    Set<Address> onlineServers = new HashSet<>();
-    for(ServerName server: master.getServerManager().getOnlineServers().keySet()) {
-      onlineServers.add(server.getAddress());
-    }
-    for (Address address: servers) {
-      if (!onlineServers.contains(address)) {
-        throw new ConstraintException(
-            "Server " + address + " is not an online server in 'default' RSGroup.");
-      }
-    }
-  }
-
-  /**
-   * Check passed name. Fail if nulls or if corresponding RSGroupInfo not found.
-   * @return The RSGroupInfo named <code>name</code>
-   */
-  private RSGroupInfo getAndCheckRSGroupInfo(String name) throws IOException {
-    if (StringUtils.isEmpty(name)) {
-      throw new ConstraintException("RSGroup cannot be null.");
-    }
-    RSGroupInfo rsGroupInfo = getRSGroupInfo(name);
-    if (rsGroupInfo == null) {
-      throw new ConstraintException("RSGroup does not exist: " + name);
-    }
-    return rsGroupInfo;
-  }
-
-  /**
-   * @return List of Regions associated with this <code>server</code>.
-   */
-  private List<RegionInfo> getRegions(final Address server) {
-    LinkedList<RegionInfo> regions = new LinkedList<>();
-    for (Map.Entry<RegionInfo, ServerName> el :
-        master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
-      if (el.getValue() == null) {
-        continue;
-      }
-
-      if (el.getValue().getAddress().equals(server)) {
-        addRegion(regions, el.getKey());
-      }
-    }
-    for (RegionStateNode state : master.getAssignmentManager().getRegionsInTransition()) {
-      if (state.getRegionLocation() != null &&
-          state.getRegionLocation().getAddress().equals(server)) {
-        addRegion(regions, state.getRegionInfo());
-      }
-    }
-    return regions;
-  }
-
-  private void addRegion(final LinkedList<RegionInfo> regions, RegionInfo hri) {
-    // If meta, move it last otherwise other unassigns fail because meta is not
-    // online for them to update state in. This is dodgy. Needs to be made more
-    // robust. See TODO below.
-    if (hri.isMetaRegion()) {
-      regions.addLast(hri);
-    } else {
-      regions.addFirst(hri);
-    }
-  }
-
-  /**
-   * Move every region from servers which are currently located on these servers, but should not be
-   * located there.
-   * @param servers the servers that will move to new group
-   * @param targetGroupName the target group name
-   * @throws IOException if moving the server and tables fail
-   */
-  private void moveServerRegionsFromGroup(Set<Address> servers, String targetGroupName)
-    throws IOException {
-    moveRegionsBetweenGroups(servers, targetGroupName, rs -> getRegions(rs), info -> {
-      try {
-        String groupName = RSGroupUtil.getRSGroupInfo(master, rsGroupInfoManager, info.getTable())
-          .map(RSGroupInfo::getName).orElse(RSGroupInfo.DEFAULT_GROUP);
-        return groupName.equals(targetGroupName);
-      } catch (IOException e) {
-        LOG.warn("Failed to test group for region {} and target group {}", info, targetGroupName);
-        return false;
-      }
-    }, rs -> rs.getHostname());
-  }
-
-  private <T> void moveRegionsBetweenGroups(Set<T> regionsOwners, String targetGroupName,
-      Function<T, List<RegionInfo>> getRegionsInfo, Function<RegionInfo, Boolean> validation,
-      Function<T, String> getOwnerName) throws IOException {
-    boolean hasRegionsToMove;
-    int retry = 0;
-    Set<T> allOwners = new HashSet<>(regionsOwners);
-    Set<String> failedRegions = new HashSet<>();
-    IOException toThrow = null;
-    do {
-      hasRegionsToMove = false;
-      for (Iterator<T> iter = allOwners.iterator(); iter.hasNext(); ) {
-        T owner = iter.next();
-        // Get regions that are associated with this server and filter regions by group tables.
-        for (RegionInfo region : getRegionsInfo.apply(owner)) {
-          if (!validation.apply(region)) {
-            LOG.info("Moving region {}, which do not belong to RSGroup {}",
-                region.getShortNameToLog(), targetGroupName);
-            try {
-              this.master.getAssignmentManager().move(region);
-              failedRegions.remove(region.getRegionNameAsString());
-            } catch (IOException ioe) {
-              LOG.debug("Move region {} from group failed, will retry, current retry time is {}",
-                  region.getShortNameToLog(), retry, ioe);
-              toThrow = ioe;
-              failedRegions.add(region.getRegionNameAsString());
-            }
-            if (master.getAssignmentManager().getRegionStates().
-                getRegionState(region).isFailedOpen()) {
-              continue;
-            }
-            hasRegionsToMove = true;
-          }
-        }
-
-        if (!hasRegionsToMove) {
-          LOG.info("No more regions to move from {} to RSGroup", getOwnerName.apply(owner));
-          iter.remove();
-        }
-      }
-
-      retry++;
-      try {
-        rsGroupInfoManager.wait(1000);
-      } catch (InterruptedException e) {
-        LOG.warn("Sleep interrupted", e);
-        Thread.currentThread().interrupt();
-      }
-    } while (hasRegionsToMove && retry <= moveMaxRetry);
-
-    //has up to max retry time or there are no more regions to move
-    if (hasRegionsToMove) {
-      // print failed moved regions, for later process conveniently
-      String msg = String
-          .format("move regions for group %s failed, failed regions: %s", targetGroupName,
-              failedRegions);
-      LOG.error(msg);
-      throw new DoNotRetryIOException(
-          msg + ", just record the last failed region's cause, more details in server log",
-          toThrow);
-    }
-  }
-
-  @Override
-  public void moveServers(Set<Address> servers, String targetGroupName) throws IOException {
-    if (servers == null) {
-      throw new ConstraintException("The list of servers to move cannot be null.");
-    }
-    if (servers.isEmpty()) {
-      // For some reason this difference between null servers and isEmpty is important distinction.
-      // TODO. Why? Stuff breaks if I equate them.
-      return;
-    }
-    //check target group
-    getAndCheckRSGroupInfo(targetGroupName);
-
-    // Hold a lock on the manager instance while moving servers to prevent
-    // another writer changing our state while we are working.
-    synchronized (rsGroupInfoManager) {
-      // Presume first server's source group. Later ensure all servers are from this group.
-      Address firstServer = servers.iterator().next();
-      RSGroupInfo srcGrp = rsGroupInfoManager.getRSGroupOfServer(firstServer);
-      if (srcGrp == null) {
-        // Be careful. This exception message is tested for in TestRSGroupsBase...
-        throw new ConstraintException("Source RSGroup for server " + firstServer
-            + " does not exist.");
-      }
-      // Only move online servers (when moving from 'default') or servers from other
-      // groups. This prevents bogus servers from entering groups
-      if (RSGroupInfo.DEFAULT_GROUP.equals(srcGrp.getName())) {
-        if (srcGrp.getServers().size() <= servers.size()) {
-          throw new ConstraintException(KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE);
-        }
-        checkOnlineServersOnly(servers);
-      }
-      // Ensure all servers are of same rsgroup.
-      for (Address server: servers) {
-        String tmpGroup = rsGroupInfoManager.getRSGroupOfServer(server).getName();
-        if (!tmpGroup.equals(srcGrp.getName())) {
-          throw new ConstraintException("Move server request should only come from one source " +
-              "RSGroup. Expecting only " + srcGrp.getName() + " but contains " + tmpGroup);
-        }
-      }
-      if (srcGrp.getServers().size() <= servers.size()) {
-        // check if there are still tables reference this group
-        for (TableDescriptor td : master.getTableDescriptors().getAll().values()) {
-          Optional<String> optGroupName = td.getRegionServerGroup();
-          if (optGroupName.isPresent() && optGroupName.get().equals(srcGrp.getName())) {
-            throw new ConstraintException(
-                "Cannot leave a RSGroup " + srcGrp.getName() + " that contains tables('" +
-                    td.getTableName() + "' at least) without servers to host them.");
-          }
-        }
-      }
-
-      // MovedServers may be < passed in 'servers'.
-      Set<Address> movedServers = rsGroupInfoManager.moveServers(servers, srcGrp.getName(),
-          targetGroupName);
-      moveServerRegionsFromGroup(movedServers, targetGroupName);
-      LOG.info("Move servers done: {} => {}", srcGrp.getName(), targetGroupName);
-    }
-  }
-
-  @Override
-  public void addRSGroup(String name) throws IOException {
-    rsGroupInfoManager.addRSGroup(new RSGroupInfo(name));
-  }
-
-  @Override
-  public void removeRSGroup(String name) throws IOException {
-    // Hold a lock on the manager instance while moving servers to prevent
-    // another writer changing our state while we are working.
-    synchronized (rsGroupInfoManager) {
-      RSGroupInfo rsGroupInfo = rsGroupInfoManager.getRSGroup(name);
-      if (rsGroupInfo == null) {
-        throw new ConstraintException("RSGroup " + name + " does not exist");
-      }
-      int serverCount = rsGroupInfo.getServers().size();
-      if (serverCount > 0) {
-        throw new ConstraintException("RSGroup " + name + " has " + serverCount +
-          " servers; you must remove these servers from the RSGroup before" +
-          " the RSGroup can be removed.");
-      }
-      for (TableDescriptor td : master.getTableDescriptors().getAll().values()) {
-        if (td.getRegionServerGroup().map(name::equals).orElse(false)) {
-          throw new ConstraintException("RSGroup " + name + " is already referenced by " +
-            td.getTableName() + "; you must remove all the tables from the rsgroup before " +
-            "the rsgroup can be removed.");
-        }
-      }
-      for (NamespaceDescriptor ns : master.getClusterSchema().getNamespaces()) {
-        String nsGroup = ns.getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP);
-        if (nsGroup != null && nsGroup.equals(name)) {
-          throw new ConstraintException(
-            "RSGroup " + name + " is referenced by namespace: " + ns.getName());
-        }
-      }
-      rsGroupInfoManager.removeRSGroup(name);
-    }
-  }
-
-  @Override
-  public boolean balanceRSGroup(String groupName) throws IOException {
-    ServerManager serverManager = master.getServerManager();
-    LoadBalancer balancer = master.getLoadBalancer();
-
-    synchronized (balancer) {
-      // If balance not true, don't run balancer.
-      if (!((HMaster) master).isBalancerOn()) {
-        return false;
-      }
-
-      if (getRSGroupInfo(groupName) == null) {
-        throw new ConstraintException("RSGroup does not exist: " + groupName);
-      }
-      // Only allow one balance run at at time.
-      Map<String, RegionState> groupRIT = rsGroupGetRegionsInTransition(groupName);
-      if (groupRIT.size() > 0) {
-        LOG.debug("Not running balancer because {} region(s) in transition: {}", groupRIT.size(),
-          StringUtils.abbreviate(
-            master.getAssignmentManager().getRegionStates().getRegionsInTransition().toString(),
-            256));
-        return false;
-      }
-      if (serverManager.areDeadServersInProgress()) {
-        LOG.debug("Not running balancer because processing dead regionserver(s): {}",
-          serverManager.getDeadServers());
-        return false;
-      }
-
-      // We balance per group instead of per table
-      List<RegionPlan> plans = new ArrayList<>();
-      Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
-        getRSGroupAssignmentsByTable(groupName);
-      for (Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> tableMap : assignmentsByTable
-        .entrySet()) {
-        LOG.info("Creating partial plan for table {} : {}", tableMap.getKey(), tableMap.getValue());
-        List<RegionPlan> partialPlans = balancer.balanceCluster(tableMap.getValue());
-        LOG.info("Partial plan for table {} : {}", tableMap.getKey(), partialPlans);
-        if (partialPlans != null) {
-          plans.addAll(partialPlans);
-        }
-      }
-      boolean balancerRan = !plans.isEmpty();
-      if (balancerRan) {
-        LOG.info("RSGroup balance {} starting with plan count: {}", groupName, plans.size());
-        master.executeRegionPlansWithThrottling(plans);
-        LOG.info("RSGroup balance " + groupName + " completed");
-      }
-      return balancerRan;
-    }
-  }
-
-  @Override
-  public List<RSGroupInfo> listRSGroups() throws IOException {
-    return rsGroupInfoManager.listRSGroups();
-  }
-
-  @Override
-  public RSGroupInfo getRSGroupOfServer(Address hostPort) throws IOException {
-    return rsGroupInfoManager.getRSGroupOfServer(hostPort);
-  }
-
-  @Override
-  public void removeServers(Set<Address> servers) throws IOException {
-    if (servers == null || servers.isEmpty()) {
-      throw new ConstraintException("The set of servers to remove cannot be null or empty.");
-    }
-    // Hold a lock on the manager instance while moving servers to prevent
-    // another writer changing our state while we are working.
-    synchronized (rsGroupInfoManager) {
-      // check the set of servers
-      checkForDeadOrOnlineServers(servers);
-      rsGroupInfoManager.removeServers(servers);
-      LOG.info("Remove decommissioned servers {} from RSGroup done", servers);
-    }
-  }
-
-  private boolean isTableInGroup(TableName tableName, String groupName,
-    Set<TableName> tablesInGroupCache) throws IOException {
-    if (tablesInGroupCache.contains(tableName)) {
-      return true;
-    }
-    if (RSGroupUtil.getRSGroupInfo(master, rsGroupInfoManager, tableName).map(RSGroupInfo::getName)
-      .orElse(RSGroupInfo.DEFAULT_GROUP).equals(groupName)) {
-      tablesInGroupCache.add(tableName);
-      return true;
-    }
-    return false;
-  }
-
-  private Map<String, RegionState> rsGroupGetRegionsInTransition(String groupName)
-    throws IOException {
-    Map<String, RegionState> rit = Maps.newTreeMap();
-    Set<TableName> tablesInGroupCache = new HashSet<>();
-    for (RegionStateNode regionNode : master.getAssignmentManager().getRegionsInTransition()) {
-      TableName tn = regionNode.getTable();
-      if (isTableInGroup(tn, groupName, tablesInGroupCache)) {
-        rit.put(regionNode.getRegionInfo().getEncodedName(), regionNode.toRegionState());
-      }
-    }
-    return rit;
-  }
-
-  private Map<TableName, Map<ServerName, List<RegionInfo>>>
-    getRSGroupAssignmentsByTable(String groupName) throws IOException {
-    Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();
-    Set<TableName> tablesInGroupCache = new HashSet<>();
-    for (Map.Entry<RegionInfo, ServerName> entry : master.getAssignmentManager().getRegionStates()
-      .getRegionAssignments().entrySet()) {
-      RegionInfo region = entry.getKey();
-      TableName tn = region.getTable();
-      ServerName server = entry.getValue();
-      if (isTableInGroup(tn, groupName, tablesInGroupCache)) {
-        result.computeIfAbsent(tn, k -> new HashMap<>())
-          .computeIfAbsent(server, k -> new ArrayList<>()).add(region);
-      }
-    }
-    RSGroupInfo rsGroupInfo = getRSGroupInfo(groupName);
-    for (ServerName serverName : master.getServerManager().getOnlineServers().keySet()) {
-      if (rsGroupInfo.containsServer(serverName.getAddress())) {
-        for (Map<ServerName, List<RegionInfo>> map : result.values()) {
-          map.computeIfAbsent(serverName, k -> Collections.emptyList());
-        }
-      }
-    }
-
-    return result;
-  }
-
-  /**
-   * Check if the set of servers are belong to dead servers list or online servers list.
-   * @param servers servers to remove
-   */
-  private void checkForDeadOrOnlineServers(Set<Address> servers) throws ConstraintException {
-    // This uglyness is because we only have Address, not ServerName.
-    Set<Address> onlineServers = new HashSet<>();
-    List<ServerName> drainingServers = master.getServerManager().getDrainingServersList();
-    for (ServerName server : master.getServerManager().getOnlineServers().keySet()) {
-      // Only online but not decommissioned servers are really online
-      if (!drainingServers.contains(server)) {
-        onlineServers.add(server.getAddress());
-      }
-    }
-
-    Set<Address> deadServers = new HashSet<>();
-    for(ServerName server: master.getServerManager().getDeadServers().copyServerNames()) {
-      deadServers.add(server.getAddress());
-    }
-
-    for (Address address: servers) {
-      if (onlineServers.contains(address)) {
-        throw new ConstraintException(
-            "Server " + address + " is an online server, not allowed to remove.");
-      }
-      if (deadServers.contains(address)) {
-        throw new ConstraintException(
-            "Server " + address + " is on the dead servers list,"
-                + " Maybe it will come back again, not allowed to remove.");
-      }
-    }
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServiceImpl.java
index 749d353..a00deec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServiceImpl.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
-import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
 import org.apache.hadoop.hbase.net.Address;
@@ -63,62 +62,34 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGro
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.AccessChecker;
-import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /**
  * Implementation of RSGroupAdminService defined in RSGroupAdmin.proto. This class calls
- * {@link RSGroupAdminServer} for actual work, converts result to protocol buffer response, handles
- * exceptions if any occurred and then calls the {@code RpcCallback} with the response.
+ * {@link RSGroupInfoManagerImpl} for actual work, converts result to protocol buffer response,
+ * handles exceptions if any occurred and then calls the {@code RpcCallback} with the response.
+ *
+ * @deprecated Keep it here only for compatibility with {@link RSGroupAdminClient},
+ *     using {@link org.apache.hadoop.hbase.master.MasterRpcServices} instead.
  */
+@Deprecated
 class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
 
   private static final Logger LOG = LoggerFactory.getLogger(RSGroupAdminServiceImpl.class);
 
   private MasterServices master;
 
-  private RSGroupAdminServer groupAdminServer;
-
-  private AccessChecker accessChecker;
-
-  /** Provider for mapping principal names to Users */
-  private UserProvider userProvider;
+  private RSGroupInfoManager rsGroupInfoManager;
 
   RSGroupAdminServiceImpl() {
   }
 
-  void initialize(MasterServices master, RSGroupAdminServer groupAdminServer,
-      AccessChecker accessChecker, UserProvider userProvider) {
-    this.master = master;
-    this.groupAdminServer = groupAdminServer;
-    this.accessChecker = accessChecker;
-    this.userProvider = userProvider;
-  }
-
-  @VisibleForTesting
-  void checkPermission(String request) throws IOException {
-    accessChecker.requirePermission(getActiveUser(), request, null, Action.ADMIN);
-  }
-
-  /**
-   * Returns the active user to which authorization checks should be applied. If we are in the
-   * context of an RPC call, the remote user is used, otherwise the currently logged in user is
-   * used.
-   */
-  private User getActiveUser() throws IOException {
-    // for non-rpc handling, fallback to system user
-    Optional<User> optionalUser = RpcServer.getRequestUser();
-    if (optionalUser.isPresent()) {
-      return optionalUser.get();
-    }
-    return userProvider.getCurrent();
+  void initialize(MasterServices masterServices){
+    this.master = masterServices;
+    this.rsGroupInfoManager = masterServices.getRSGroupInfoManager();
   }
 
   // for backward compatible
@@ -137,8 +108,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preGetRSGroupInfo(groupName);
       }
-      checkPermission("getRSGroupInfo");
-      RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(groupName);
+      RSGroupInfo rsGroupInfo = rsGroupInfoManager.getRSGroup(groupName);
       if (rsGroupInfo != null) {
         builder.setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(fillTables(rsGroupInfo)));
       }
@@ -162,14 +132,13 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName);
       }
-      checkPermission("getRSGroupInfoOfTable");
       Optional<RSGroupInfo> optGroup =
-        RSGroupUtil.getRSGroupInfo(master, groupAdminServer.rsGroupInfoManager, tableName);
+        RSGroupUtil.getRSGroupInfo(master, rsGroupInfoManager, tableName);
       if (optGroup.isPresent()) {
         builder.setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(fillTables(optGroup.get())));
       } else {
         if (master.getTableStateManager().isTablePresent(tableName)) {
-          RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+          RSGroupInfo rsGroupInfo = rsGroupInfoManager.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
           builder.setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(fillTables(rsGroupInfo)));
         }
       }
@@ -197,8 +166,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preMoveServers(hostPorts, request.getTargetGroup());
       }
-      checkPermission("moveServers");
-      groupAdminServer.moveServers(hostPorts, request.getTargetGroup());
+      rsGroupInfoManager.moveServers(hostPorts, request.getTargetGroup());
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postMoveServers(hostPorts, request.getTargetGroup());
       }
@@ -243,7 +211,6 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preMoveTables(tables, request.getTargetGroup());
       }
-      checkPermission("moveTables");
       moveTablesAndWait(tables, request.getTargetGroup());
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postMoveTables(tables, request.getTargetGroup());
@@ -263,8 +230,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preAddRSGroup(request.getRSGroupName());
       }
-      checkPermission("addRSGroup");
-      groupAdminServer.addRSGroup(request.getRSGroupName());
+      rsGroupInfoManager.addRSGroup(new RSGroupInfo(request.getRSGroupName()));
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postAddRSGroup(request.getRSGroupName());
       }
@@ -283,8 +249,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preRemoveRSGroup(request.getRSGroupName());
       }
-      checkPermission("removeRSGroup");
-      groupAdminServer.removeRSGroup(request.getRSGroupName());
+      rsGroupInfoManager.removeRSGroup(request.getRSGroupName());
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postRemoveRSGroup(request.getRSGroupName());
       }
@@ -304,8 +269,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preBalanceRSGroup(request.getRSGroupName());
       }
-      checkPermission("balanceRSGroup");
-      boolean balancerRan = groupAdminServer.balanceRSGroup(request.getRSGroupName());
+      boolean balancerRan = rsGroupInfoManager.balanceRSGroup(request.getRSGroupName());
       builder.setBalanceRan(balancerRan);
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postBalanceRSGroup(request.getRSGroupName(), balancerRan);
@@ -326,8 +290,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preListRSGroups();
       }
-      checkPermission("listRSGroup");
-      List<RSGroupInfo> rsGroupInfos = groupAdminServer.listRSGroups().stream()
+      List<RSGroupInfo> rsGroupInfos = rsGroupInfoManager.listRSGroups().stream()
           .map(RSGroupInfo::new).collect(Collectors.toList());
       Map<String, RSGroupInfo> name2Info = new HashMap<>();
       for (RSGroupInfo rsGroupInfo : rsGroupInfos) {
@@ -364,8 +327,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp);
       }
-      checkPermission("getRSGroupInfoOfServer");
-      RSGroupInfo info = groupAdminServer.getRSGroupOfServer(hp);
+      RSGroupInfo info = rsGroupInfoManager.getRSGroupOfServer(hp);
       if (info != null) {
         builder.setRSGroupInfo(ProtobufUtil.toProtoGroupInfo(fillTables(info)));
       }
@@ -397,8 +359,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
         master.getMasterCoprocessorHost().preMoveServersAndTables(hostPorts, tables,
           request.getTargetGroup());
       }
-      checkPermission("moveServersAndTables");
-      groupAdminServer.moveServers(hostPorts, request.getTargetGroup());
+      rsGroupInfoManager.moveServers(hostPorts, request.getTargetGroup());
       moveTablesAndWait(tables, request.getTargetGroup());
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postMoveServersAndTables(hostPorts, tables,
@@ -424,8 +385,7 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().preRemoveServers(servers);
       }
-      checkPermission("removeServers");
-      groupAdminServer.removeServers(servers);
+      rsGroupInfoManager.removeServers(servers);
       if (master.getMasterCoprocessorHost() != null) {
         master.getMasterCoprocessorHost().postRemoveServers(servers);
       }
@@ -434,4 +394,5 @@ class RSGroupAdminServiceImpl extends RSGroupAdminProtos.RSGroupAdminService {
     }
     done.run(builder.build());
   }
+
 }
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index 0eb15e9..c1b03a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -48,13 +48,8 @@ public interface RSGroupInfoManager {
 
   /**
    * Move servers to a new group.
-   * @param servers list of servers, must be part of the same group
-   * @param srcGroup groupName being moved from
-   * @param dstGroup groupName being moved to
-   * @return Set of servers moved (May be a subset of {@code servers}).
    */
-  Set<Address> moveServers(Set<Address> servers, String srcGroup, String dstGroup)
-      throws IOException;
+  void moveServers(Set<Address> servers, String targetGroupName) throws IOException;
 
   /**
    * Gets the group info of server.
@@ -85,10 +80,7 @@ public interface RSGroupInfoManager {
 
   /**
    * Get {@code RSGroupInfo} for the given table.
-   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for compatibility, where we upgrade
-   *             from a version that stores table names for a rs group in the {@code RSGroupInfo}.
    */
-  @Deprecated
   RSGroupInfo getRSGroupForTable(TableName tableName) throws IOException;
 
   static RSGroupInfoManager create(MasterServices master) throws IOException {
@@ -98,4 +90,14 @@ public interface RSGroupInfoManager {
       return new DisabledRSGroupInfoManager(master.getServerManager());
     }
   }
+
+  /**
+   * Balance a region server group.
+   */
+  boolean balanceRSGroup(String groupName) throws IOException;
+
+  /**
+   * Set group for tables.
+   */
+  void setRSGroup(Set<TableName> tables, String groupName) throws IOException;
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 7224869..83aa12e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -24,15 +24,21 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.OptionalLong;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.function.Function;
 import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -45,6 +51,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -52,11 +59,17 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
@@ -83,15 +96,13 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-
 /**
  * This is an implementation of {@link RSGroupInfoManager} which makes use of an HBase table as the
  * persistence store for the group information. It also makes use of zookeeper to store group
  * information needed for bootstrapping during offline mode.
  * <h2>Concurrency</h2> RSGroup state is kept locally in Maps. There is a rsgroup name to cached
- * RSGroupInfo Map at {@link #rsGroupMap} and a Map of tables to the name of the rsgroup they belong
- * too (in {@link #tableMap}). These Maps are persisted to the hbase:rsgroup table (and cached in
- * zk) on each modification.
+ * RSGroupInfo Map at {@link #rsGroupMap}. These Maps are persisted to the hbase:rsgroup table
+ * (and cached in zk) on each modification.
  * <p/>
  * Mutations on state are synchronized but reads can continue without having to wait on an instance
  * monitor, mutations do wholesale replace of the Maps on update -- Copy-On-Write; the local Maps of
@@ -113,6 +124,18 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   static final TableName RSGROUP_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup");
 
+  @VisibleForTesting
+  static final String KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE = "should keep at least " +
+      "one server in 'default' RSGroup.";
+
+  /** Define the config key of retries threshold when movements failed */
+  @VisibleForTesting
+  static final String FAILED_MOVE_MAX_RETRY = "hbase.rsgroup.move.max.retry";
+
+  /** Define the default number of retries */
+  @VisibleForTesting
+  static final int DEFAULT_MAX_RETRY_VALUE = 50;
+
   private static final String RS_GROUP_ZNODE = "rsgroup";
 
   @VisibleForTesting
@@ -176,7 +199,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   // contains list of groups that were last flushed to persistent store
   private Set<String> prevRSGroups = new HashSet<>();
 
-  private RSGroupInfoManagerImpl(MasterServices masterServices) throws IOException {
+  private RSGroupInfoManagerImpl(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.watcher = masterServices.getZooKeeper();
     this.conn = masterServices.getAsyncClusterConnection();
@@ -214,8 +237,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
     migrate();
   }
 
-  static RSGroupInfoManager getInstance(MasterServices master) throws IOException {
-    RSGroupInfoManagerImpl instance = new RSGroupInfoManagerImpl(master);
+  static RSGroupInfoManager getInstance(MasterServices masterServices) throws IOException {
+    RSGroupInfoManagerImpl instance = new RSGroupInfoManagerImpl(masterServices);
     instance.init();
     return instance;
   }
@@ -231,17 +254,17 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
     Map<String, RSGroupInfo> rsGroupMap = holder.groupName2Group;
     if (rsGroupMap.get(rsGroupInfo.getName()) != null ||
       rsGroupInfo.getName().equals(RSGroupInfo.DEFAULT_GROUP)) {
-      throw new DoNotRetryIOException("Group already exists: " + rsGroupInfo.getName());
+      throw new ConstraintException("Group already exists: " + rsGroupInfo.getName());
     }
     Map<String, RSGroupInfo> newGroupMap = Maps.newHashMap(rsGroupMap);
     newGroupMap.put(rsGroupInfo.getName(), rsGroupInfo);
     flushConfig(newGroupMap);
   }
 
-  private RSGroupInfo getRSGroupInfo(final String groupName) throws DoNotRetryIOException {
-    RSGroupInfo rsGroupInfo = getRSGroup(groupName);
+  private RSGroupInfo getRSGroupInfo(final String groupName) throws ConstraintException {
+    RSGroupInfo rsGroupInfo = holder.groupName2Group.get(groupName);
     if (rsGroupInfo == null) {
-      throw new DoNotRetryIOException("RSGroup " + groupName + " does not exist");
+      throw new ConstraintException("RSGroup " + groupName + " does not exist");
     }
     return rsGroupInfo;
   }
@@ -254,7 +277,6 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
       .map(ServerName::getAddress).collect(Collectors.toSet());
   }
 
-  @Override
   public synchronized Set<Address> moveServers(Set<Address> servers, String srcGroup,
       String dstGroup) throws IOException {
     RSGroupInfo src = getRSGroupInfo(srcGroup);
@@ -284,7 +306,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   }
 
   @Override
-  public RSGroupInfo getRSGroupOfServer(Address serverHostPort) throws IOException {
+  public RSGroupInfo getRSGroupOfServer(Address serverHostPort) {
     for (RSGroupInfo info : holder.groupName2Group.values()) {
       if (info.containsServer(serverHostPort)) {
         return info;
@@ -300,9 +322,30 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
 
   @Override
   public synchronized void removeRSGroup(String groupName) throws IOException {
+    RSGroupInfo rsGroupInfo = getRSGroupInfo(groupName);
+    int serverCount = rsGroupInfo.getServers().size();
+    if (serverCount > 0) {
+      throw new ConstraintException("RSGroup " + groupName + " has " + serverCount +
+          " servers; you must remove these servers from the RSGroup before" +
+          " the RSGroup can be removed.");
+    }
+    for (TableDescriptor td : masterServices.getTableDescriptors().getAll().values()) {
+      if (td.getRegionServerGroup().map(groupName::equals).orElse(false)) {
+        throw new ConstraintException("RSGroup " + groupName + " is already referenced by " +
+            td.getTableName() + "; you must remove all the tables from the rsgroup before " +
+            "the rsgroup can be removed.");
+      }
+    }
+    for (NamespaceDescriptor ns : masterServices.getClusterSchema().getNamespaces()) {
+      String nsGroup = ns.getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP);
+      if (nsGroup != null && nsGroup.equals(groupName)) {
+        throw new ConstraintException(
+            "RSGroup " + groupName + " is referenced by namespace: " + ns.getName());
+      }
+    }
     Map<String, RSGroupInfo> rsGroupMap = holder.groupName2Group;
     if (!rsGroupMap.containsKey(groupName) || groupName.equals(RSGroupInfo.DEFAULT_GROUP)) {
-      throw new DoNotRetryIOException(
+      throw new ConstraintException(
         "Group " + groupName + " does not exist or is a reserved " + "group");
     }
     Map<String, RSGroupInfo> newGroupMap = Maps.newHashMap(rsGroupMap);
@@ -322,6 +365,13 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
 
   @Override
   public synchronized void removeServers(Set<Address> servers) throws IOException {
+    if (servers == null || servers.isEmpty()) {
+      throw new ConstraintException("The set of servers to remove cannot be null or empty.");
+    }
+
+    // check the set of servers
+    checkForDeadOrOnlineServers(servers);
+
     Map<String, RSGroupInfo> rsGroupInfos = new HashMap<String, RSGroupInfo>();
     for (Address el : servers) {
       RSGroupInfo rsGroupInfo = getRSGroupOfServer(el);
@@ -344,6 +394,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
       newGroupMap.putAll(rsGroupInfos);
       flushConfig(newGroupMap);
     }
+    LOG.info("Remove decommissioned servers {} from RSGroup done", servers);
   }
 
   private List<RSGroupInfo> retrieveGroupListFromGroupTable() throws IOException {
@@ -575,7 +626,6 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
     // Make changes visible after having been persisted to the source of truth
     resetRSGroupMap(newGroupMap);
     saveRSGroupMapToZK(newGroupMap);
-
     updateCacheOfRSGroups(newGroupMap.keySet());
   }
 
@@ -764,9 +814,371 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
     }
   }
 
-
   @Override
   public RSGroupInfo getRSGroupForTable(TableName tableName) throws IOException {
     return holder.tableName2Group.get(tableName);
   }
+
+
+  /**
+   * Check if the set of servers are belong to dead servers list or online servers list.
+   * @param servers servers to remove
+   */
+  private void checkForDeadOrOnlineServers(Set<Address> servers) throws IOException {
+    // This uglyness is because we only have Address, not ServerName.
+    Set<Address> onlineServers = new HashSet<>();
+    List<ServerName> drainingServers = masterServices.getServerManager().getDrainingServersList();
+    for (ServerName server : masterServices.getServerManager().getOnlineServers().keySet()) {
+      // Only online but not decommissioned servers are really online
+      if (!drainingServers.contains(server)) {
+        onlineServers.add(server.getAddress());
+      }
+    }
+
+    Set<Address> deadServers = new HashSet<>();
+    for(ServerName server: masterServices.getServerManager().getDeadServers().copyServerNames()) {
+      deadServers.add(server.getAddress());
+    }
+
+    for (Address address: servers) {
+      if (onlineServers.contains(address)) {
+        throw new DoNotRetryIOException(
+            "Server " + address + " is an online server, not allowed to remove.");
+      }
+      if (deadServers.contains(address)) {
+        throw new DoNotRetryIOException(
+            "Server " + address + " is on the dead servers list,"
+                + " Maybe it will come back again, not allowed to remove.");
+      }
+    }
+  }
+
+  private void checkOnlineServersOnly(Set<Address> servers) throws IOException {
+    // This uglyness is because we only have Address, not ServerName.
+    // Online servers are keyed by ServerName.
+    Set<Address> onlineServers = new HashSet<>();
+    for(ServerName server: masterServices.getServerManager().getOnlineServers().keySet()) {
+      onlineServers.add(server.getAddress());
+    }
+    for (Address address: servers) {
+      if (!onlineServers.contains(address)) {
+        throw new DoNotRetryIOException("Server " + address +
+            " is not an online server in 'default' RSGroup.");
+      }
+    }
+  }
+
+  /**
+   * @return List of Regions associated with this <code>server</code>.
+   */
+  private List<RegionInfo> getRegions(final Address server) {
+    LinkedList<RegionInfo> regions = new LinkedList<>();
+    for (Map.Entry<RegionInfo, ServerName> el :
+        masterServices.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
+      if (el.getValue() == null) {
+        continue;
+      }
+
+      if (el.getValue().getAddress().equals(server)) {
+        addRegion(regions, el.getKey());
+      }
+    }
+    for (RegionStateNode state : masterServices.getAssignmentManager().getRegionsInTransition()) {
+      if (state.getRegionLocation() != null &&
+          state.getRegionLocation().getAddress().equals(server)) {
+        addRegion(regions, state.getRegionInfo());
+      }
+    }
+    return regions;
+  }
+
+  private void addRegion(final LinkedList<RegionInfo> regions, RegionInfo hri) {
+    // If meta, move it last otherwise other unassigns fail because meta is not
+    // online for them to update state in. This is dodgy. Needs to be made more
+    // robust. See TODO below.
+    if (hri.isMetaRegion()) {
+      regions.addLast(hri);
+    } else {
+      regions.addFirst(hri);
+    }
+  }
+
+  /**
+   * Move every region from servers which are currently located on these servers, but should not be
+   * located there.
+   * @param servers the servers that will move to new group
+   * @param targetGroupName the target group name
+   * @throws IOException if moving the server and tables fail
+   */
+  private void moveServerRegionsFromGroup(Set<Address> servers, String targetGroupName)
+      throws IOException {
+    moveRegionsBetweenGroups(servers, targetGroupName, rs -> getRegions(rs), info -> {
+      try {
+        String groupName = RSGroupUtil.getRSGroupInfo(masterServices, this, info.getTable())
+            .map(RSGroupInfo::getName).orElse(RSGroupInfo.DEFAULT_GROUP);
+        return groupName.equals(targetGroupName);
+      } catch (IOException e) {
+        LOG.warn("Failed to test group for region {} and target group {}", info, targetGroupName);
+        return false;
+      }
+    }, rs -> rs.getHostname());
+  }
+
+  private <T> void moveRegionsBetweenGroups(Set<T> regionsOwners, String targetGroupName,
+      Function<T, List<RegionInfo>> getRegionsInfo, Function<RegionInfo, Boolean> validation,
+      Function<T, String> getOwnerName) throws IOException {
+    boolean hasRegionsToMove;
+    int retry = 0;
+    Set<T> allOwners = new HashSet<>(regionsOwners);
+    Set<String> failedRegions = new HashSet<>();
+    IOException toThrow = null;
+    do {
+      hasRegionsToMove = false;
+      for (Iterator<T> iter = allOwners.iterator(); iter.hasNext(); ) {
+        T owner = iter.next();
+        // Get regions that are associated with this server and filter regions by group tables.
+        for (RegionInfo region : getRegionsInfo.apply(owner)) {
+          if (!validation.apply(region)) {
+            LOG.info("Moving region {}, which do not belong to RSGroup {}",
+                region.getShortNameToLog(), targetGroupName);
+            try {
+              this.masterServices.getAssignmentManager().move(region);
+              failedRegions.remove(region.getRegionNameAsString());
+            } catch (IOException ioe) {
+              LOG.debug("Move region {} from group failed, will retry, current retry time is {}",
+                  region.getShortNameToLog(), retry, ioe);
+              toThrow = ioe;
+              failedRegions.add(region.getRegionNameAsString());
+            }
+            if (masterServices.getAssignmentManager().getRegionStates().
+                getRegionState(region).isFailedOpen()) {
+              continue;
+            }
+            hasRegionsToMove = true;
+          }
+        }
+
+        if (!hasRegionsToMove) {
+          LOG.info("No more regions to move from {} to RSGroup", getOwnerName.apply(owner));
+          iter.remove();
+        }
+      }
+
+      retry++;
+      try {
+        wait(1000);
+      } catch (InterruptedException e) {
+        LOG.warn("Sleep interrupted", e);
+        Thread.currentThread().interrupt();
+      }
+    } while (hasRegionsToMove && retry <=
+        masterServices.getConfiguration().getInt(FAILED_MOVE_MAX_RETRY, DEFAULT_MAX_RETRY_VALUE));
+
+    //has up to max retry time or there are no more regions to move
+    if (hasRegionsToMove) {
+      // print failed moved regions, for later process conveniently
+      String msg = String
+          .format("move regions for group %s failed, failed regions: %s", targetGroupName,
+              failedRegions);
+      LOG.error(msg);
+      throw new DoNotRetryIOException(
+          msg + ", just record the last failed region's cause, more details in server log",
+          toThrow);
+    }
+  }
+
+  private boolean isTableInGroup(TableName tableName, String groupName,
+      Set<TableName> tablesInGroupCache) throws IOException {
+    if (tablesInGroupCache.contains(tableName)) {
+      return true;
+    }
+    if (RSGroupUtil.getRSGroupInfo(masterServices, this, tableName)
+        .map(RSGroupInfo::getName)
+        .orElse(RSGroupInfo.DEFAULT_GROUP).equals(groupName)) {
+      tablesInGroupCache.add(tableName);
+      return true;
+    }
+    return false;
+  }
+
+  private Map<String, RegionState> rsGroupGetRegionsInTransition(String groupName)
+      throws IOException {
+    Map<String, RegionState> rit = Maps.newTreeMap();
+    Set<TableName> tablesInGroupCache = new HashSet<>();
+    for (RegionStateNode regionNode :
+        masterServices.getAssignmentManager().getRegionsInTransition()) {
+      TableName tn = regionNode.getTable();
+      if (isTableInGroup(tn, groupName, tablesInGroupCache)) {
+        rit.put(regionNode.getRegionInfo().getEncodedName(), regionNode.toRegionState());
+      }
+    }
+    return rit;
+  }
+
+  private Map<TableName, Map<ServerName, List<RegionInfo>>> getRSGroupAssignmentsByTable(
+      String groupName) throws IOException {
+    Map<TableName, Map<ServerName, List<RegionInfo>>> result = Maps.newHashMap();
+    Set<TableName> tablesInGroupCache = new HashSet<>();
+    for (Map.Entry<RegionInfo, ServerName> entry :
+        masterServices.getAssignmentManager().getRegionStates()
+        .getRegionAssignments().entrySet()) {
+      RegionInfo region = entry.getKey();
+      TableName tn = region.getTable();
+      ServerName server = entry.getValue();
+      if (isTableInGroup(tn, groupName, tablesInGroupCache)) {
+        result.computeIfAbsent(tn, k -> new HashMap<>())
+            .computeIfAbsent(server, k -> new ArrayList<>()).add(region);
+      }
+    }
+    RSGroupInfo rsGroupInfo = getRSGroupInfo(groupName);
+    for (ServerName serverName : masterServices.getServerManager().getOnlineServers().keySet()) {
+      if (rsGroupInfo.containsServer(serverName.getAddress())) {
+        for (Map<ServerName, List<RegionInfo>> map : result.values()) {
+          map.computeIfAbsent(serverName, k -> Collections.emptyList());
+        }
+      }
+    }
+
+    return result;
+  }
+
+  @Override
+  public boolean balanceRSGroup(String groupName) throws IOException {
+    ServerManager serverManager = masterServices.getServerManager();
+    LoadBalancer balancer = masterServices.getLoadBalancer();
+    getRSGroupInfo(groupName);
+
+    synchronized (balancer) {
+      // If balance not true, don't run balancer.
+      if (!masterServices.isBalancerOn()) {
+        return false;
+      }
+      // Only allow one balance run at at time.
+      Map<String, RegionState> groupRIT = rsGroupGetRegionsInTransition(groupName);
+      if (groupRIT.size() > 0) {
+        LOG.debug("Not running balancer because {} region(s) in transition: {}", groupRIT.size(),
+            StringUtils.abbreviate(masterServices.getAssignmentManager().getRegionStates()
+                    .getRegionsInTransition().toString(),
+                256));
+        return false;
+      }
+      if (serverManager.areDeadServersInProgress()) {
+        LOG.debug("Not running balancer because processing dead regionserver(s): {}",
+            serverManager.getDeadServers());
+        return false;
+      }
+
+      // We balance per group instead of per table
+      List<RegionPlan> plans = new ArrayList<>();
+      Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
+          getRSGroupAssignmentsByTable(groupName);
+      for (Map.Entry<TableName, Map<ServerName, List<RegionInfo>>> tableMap : assignmentsByTable
+          .entrySet()) {
+        LOG.info("Creating partial plan for table {} : {}", tableMap.getKey(), tableMap.getValue());
+        List<RegionPlan> partialPlans = balancer.balanceCluster(tableMap.getValue());
+        LOG.info("Partial plan for table {} : {}", tableMap.getKey(), partialPlans);
+        if (partialPlans != null) {
+          plans.addAll(partialPlans);
+        }
+      }
+      boolean balancerRan = !plans.isEmpty();
+      if (balancerRan) {
+        LOG.info("RSGroup balance {} starting with plan count: {}", groupName, plans.size());
+        masterServices.executeRegionPlansWithThrottling(plans);
+        LOG.info("RSGroup balance " + groupName + " completed");
+      }
+      return balancerRan;
+    }
+  }
+
+  private void moveTablesAndWait(Set<TableName> tables, String targetGroup) throws IOException {
+    List<Long> procIds = new ArrayList<Long>();
+    for (TableName tableName : tables) {
+      TableDescriptor oldTd = masterServices.getTableDescriptors().get(tableName);
+      if (oldTd == null) {
+        continue;
+      }
+      TableDescriptor newTd =
+          TableDescriptorBuilder.newBuilder(oldTd).setRegionServerGroup(targetGroup).build();
+      procIds.add(masterServices.modifyTable(tableName, newTd, HConstants.NO_NONCE,
+          HConstants.NO_NONCE));
+    }
+    for (long procId : procIds) {
+      Procedure<?> proc = masterServices.getMasterProcedureExecutor().getProcedure(procId);
+      if (proc == null) {
+        continue;
+      }
+      ProcedureSyncWait.waitForProcedureToCompleteIOE(masterServices.getMasterProcedureExecutor(),
+          proc, Long.MAX_VALUE);
+    }
+  }
+
+  @Override
+  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
+    getRSGroupInfo(groupName);
+    moveTablesAndWait(tables, groupName);
+  }
+
+  public void moveServers(Set<Address> servers, String targetGroupName) throws IOException {
+    if (servers == null) {
+      throw new ConstraintException("The list of servers to move cannot be null.");
+    }
+    if (servers.isEmpty()) {
+      // For some reason this difference between null servers and isEmpty is important distinction.
+      // TODO. Why? Stuff breaks if I equate them.
+      return;
+    }
+    if (StringUtils.isEmpty(targetGroupName)) {
+      throw new ConstraintException("RSGroup cannot be null.");
+    }
+    getRSGroupInfo(targetGroupName);
+
+    // Hold a lock on the manager instance while moving servers to prevent
+    // another writer changing our state while we are working.
+    synchronized (this) {
+      // Presume first server's source group. Later ensure all servers are from this group.
+      Address firstServer = servers.iterator().next();
+      RSGroupInfo srcGrp = getRSGroupOfServer(firstServer);
+      if (srcGrp == null) {
+        // Be careful. This exception message is tested for in TestRSGroupsBase...
+        throw new ConstraintException("Source RSGroup for server " + firstServer
+            + " does not exist.");
+      }
+
+      // Only move online servers (when moving from 'default') or servers from other
+      // groups. This prevents bogus servers from entering groups
+      if (RSGroupInfo.DEFAULT_GROUP.equals(srcGrp.getName())) {
+        if (srcGrp.getServers().size() <= servers.size()) {
+          throw new ConstraintException(KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE);
+        }
+        checkOnlineServersOnly(servers);
+      }
+      // Ensure all servers are of same rsgroup.
+      for (Address server: servers) {
+        String tmpGroup = getRSGroupOfServer(server).getName();
+        if (!tmpGroup.equals(srcGrp.getName())) {
+          throw new ConstraintException("Move server request should only come from one source " +
+              "RSGroup. Expecting only " + srcGrp.getName() + " but contains " + tmpGroup);
+        }
+      }
+      if (srcGrp.getServers().size() <= servers.size()) {
+        // check if there are still tables reference this group
+        for (TableDescriptor td : masterServices.getTableDescriptors().getAll().values()) {
+          Optional<String> optGroupName = td.getRegionServerGroup();
+          if (optGroupName.isPresent() && optGroupName.get().equals(srcGrp.getName())) {
+            throw new ConstraintException(
+                "Cannot leave a RSGroup " + srcGrp.getName() + " that contains tables('" +
+                    td.getTableName() + "' at least) without servers to host them.");
+          }
+        }
+      }
+
+      // MovedServers may be < passed in 'servers'.
+      Set<Address> movedServers = moveServers(servers, srcGrp.getName(),
+          targetGroupName);
+      moveServerRegionsFromGroup(movedServers, targetGroupName);
+      LOG.info("Move servers done: {} => {}", srcGrp.getName(), targetGroupName);
+    }
+  }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
index d1b3751..fa07108 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupMajorCompactionTTL.java
@@ -19,11 +19,11 @@
 package org.apache.hadoop.hbase.rsgroup;
 
 import java.util.Arrays;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.util.compaction.MajorCompactorTTL;
@@ -31,6 +31,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
@@ -58,9 +59,9 @@ public class RSGroupMajorCompactionTTL extends MajorCompactorTTL {
       throws Exception {
 
     Connection conn = ConnectionFactory.createConnection(conf);
-    RSGroupAdmin rsGroupAdmin = new RSGroupAdminClient(conn);
+    Admin admin = conn.getAdmin();
 
-    RSGroupInfo rsGroupInfo = rsGroupAdmin.getRSGroupInfo(rsgroup);
+    RSGroupInfo rsGroupInfo = admin.getRSGroup(rsgroup);
     if (rsGroupInfo == null) {
       LOG.error("Invalid rsgroup specified: " + rsgroup);
       throw new IllegalArgumentException("Invalid rsgroup specified: " + rsgroup);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupUtil.java
index af30049..8aef0a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupUtil.java
@@ -27,7 +27,7 @@ import org.slf4j.LoggerFactory;
  * Helper class for RSGroup implementation
  */
 @InterfaceAudience.Private
-final class RSGroupUtil {
+public final class RSGroupUtil {
 
   private static final Logger LOG = LoggerFactory.getLogger(RSGroupUtil.class);
 
@@ -38,8 +38,8 @@ final class RSGroupUtil {
    * Will try to get the rsgroup from {@link TableDescriptor} first, and then try to get the rsgroup
    * from the {@link NamespaceDescriptor}. If still not present, return empty.
    */
-  static Optional<RSGroupInfo> getRSGroupInfo(MasterServices master, RSGroupInfoManager manager,
-      TableName tableName) throws IOException {
+  public static Optional<RSGroupInfo> getRSGroupInfo(MasterServices master,
+      RSGroupInfoManager manager, TableName tableName) throws IOException {
     TableDescriptor td = master.getTableDescriptors().get(tableName);
     if (td == null) {
       return Optional.empty();
@@ -79,7 +79,7 @@ final class RSGroupUtil {
    * Fill the tables field for {@link RSGroupInfo}, for backward compatibility.
    */
   @SuppressWarnings("deprecation")
-  static RSGroupInfo fillTables(RSGroupInfo rsGroupInfo, Collection<TableDescriptor> tds) {
+  public static RSGroupInfo fillTables(RSGroupInfo rsGroupInfo, Collection<TableDescriptor> tds) {
     RSGroupInfo newRsGroupInfo = new RSGroupInfo(rsGroupInfo);
     Predicate<TableDescriptor> filter;
     if (rsGroupInfo.getName().equals(RSGroupInfo.DEFAULT_GROUP)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 44363fa..d46b88d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
@@ -2588,4 +2589,84 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       }
     }
   }
+
+  @Override
+  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "moveServersAndTables",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<Address> servers, String targetGroup) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "moveServers",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<TableName> tables, String targetGroup) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "moveTables",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String name) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "addRSGroup",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String name) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "removeRSGroup",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String groupName) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "balanceRSGroup",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preRemoveServers(
+      ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<Address> servers) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "removeServers",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preGetRSGroupInfo(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String groupName) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfo",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preGetRSGroupInfoOfTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfoOfTable",
+        null, Permission.Action.ADMIN);
+    //todo: should add check for table existence
+  }
+
+  @Override
+  public void preListRSGroups(ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "listRSGroups",
+        null, Permission.Action.ADMIN);
+  }
+
+  @Override
+  public void preGetRSGroupInfoOfServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Address server) throws IOException {
+    accessChecker.requirePermission(getActiveUser(ctx), "getRSGroupInfoOfServer",
+        null, Permission.Action.ADMIN);
+  }
+
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 71f8a91..c2640c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -496,4 +496,9 @@ public class MockNoopMasterServices implements MasterServices {
   public RSGroupInfoManager getRSGroupInfoManager() {
     return null;
   }
+
+  @Override
+  public boolean isBalancerOn() {
+    return false;
+  }
 }
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestMigrateRSGroupInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestMigrateRSGroupInfo.java
index f61e18a..122abc4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestMigrateRSGroupInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestMigrateRSGroupInfo.java
@@ -41,15 +41,18 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Testcase for HBASE-22819
  */
+@RunWith(Parameterized.class)
 @Category({ MediumTests.class })
 public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
 
@@ -63,8 +66,8 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
 
   private static byte[] FAMILY = Bytes.toBytes("family");
 
-  @BeforeClass
-  public static void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     TEST_UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, HMasterForTest.class,
       HMaster.class);
     setUpTestBeforeClass();
@@ -73,8 +76,8 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
     }
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     tearDownAfterClass();
   }
 
@@ -90,7 +93,7 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
     public TableDescriptors getTableDescriptors() {
       if (RESUME != null) {
         for (StackTraceElement element : Thread.currentThread().getStackTrace()) {
-          if (element.getClassName().contains("RSGroupInfoManagerImpl")) {
+          if (element.getMethodName().equals("migrate")) {
             try {
               RESUME.await();
             } catch (InterruptedException e) {
@@ -106,9 +109,10 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
 
   @Test
   public void testMigrate() throws IOException, InterruptedException {
-    String groupName = name.getMethodName();
+    setAdmin();
+    String groupName = getNameWithoutIndex(name.getMethodName());
     addGroup(groupName, TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().size() - 1);
-    RSGroupInfo rsGroupInfo = rsGroupAdmin.getRSGroupInfo(groupName);
+    RSGroupInfo rsGroupInfo = rsGroupAdmin.getRSGroup(groupName);
     assertTrue(rsGroupInfo.getTables().isEmpty());
     for (int i = 0; i < NUM_TABLES; i++) {
       rsGroupInfo.addTable(TableName.valueOf(TABLE_NAME_PREFIX + i));
@@ -126,7 +130,7 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
     // wait until we can get the rs group info for a table
     TEST_UTIL.waitFor(30000, () -> {
       try {
-        rsGroupAdmin.getRSGroupInfoOfTable(TableName.valueOf(TABLE_NAME_PREFIX + 0));
+        rsGroupAdmin.getRSGroup(TableName.valueOf(TABLE_NAME_PREFIX + 0));
         return true;
       } catch (IOException e) {
         return false;
@@ -135,7 +139,7 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
     // confirm that before migrating, we could still get the correct rs group for a table.
     for (int i = 0; i < NUM_TABLES; i++) {
       RSGroupInfo info =
-        rsGroupAdmin.getRSGroupInfoOfTable(TableName.valueOf(TABLE_NAME_PREFIX + i));
+        rsGroupAdmin.getRSGroup(TableName.valueOf(TABLE_NAME_PREFIX + i));
       assertEquals(rsGroupInfo.getName(), info.getName());
       assertEquals(NUM_TABLES, info.getTables().size());
     }
@@ -171,7 +175,7 @@ public class TestMigrateRSGroupInfo extends TestRSGroupsBase {
     // make sure we could still get the correct rs group info after migration
     for (int i = 0; i < NUM_TABLES; i++) {
       RSGroupInfo info =
-        rsGroupAdmin.getRSGroupInfoOfTable(TableName.valueOf(TABLE_NAME_PREFIX + i));
+        rsGroupAdmin.getRSGroup(TableName.valueOf(TABLE_NAME_PREFIX + i));
       assertEquals(rsGroupInfo.getName(), info.getName());
       assertEquals(NUM_TABLES, info.getTables().size());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
index aff591f..614b6c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
@@ -42,11 +42,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
-import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.net.Address;
-import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -57,11 +54,14 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
+@RunWith(Parameterized.class)
 @Category({ MediumTests.class })
 public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
@@ -112,9 +112,9 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
   @Test
   public void testBogusArgs() throws Exception {
-    assertNull(rsGroupAdmin.getRSGroupInfoOfTable(TableName.valueOf("nonexistent")));
-    assertNull(rsGroupAdmin.getRSGroupOfServer(Address.fromParts("bogus", 123)));
-    assertNull(rsGroupAdmin.getRSGroupInfo("bogus"));
+    assertNull(rsGroupAdmin.getRSGroup(TableName.valueOf("nonexistent")));
+    assertNull(rsGroupAdmin.getRSGroup(Address.fromParts("bogus", 123)));
+    assertNull(rsGroupAdmin.getRSGroup("bogus"));
 
     try {
       rsGroupAdmin.removeRSGroup("bogus");
@@ -124,14 +124,15 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
     }
 
     try {
-      rsGroupAdmin.moveTables(Sets.newHashSet(TableName.valueOf("bogustable")), "bogus");
-      fail("Expected move with bogus group to fail");
+      rsGroupAdmin.setRSGroup(Sets.newHashSet(TableName.valueOf("bogustable")), "bogus");
+      fail("Expected set table to bogus group fail");
     } catch (ConstraintException | TableNotFoundException ex) {
       // expected
     }
 
     try {
-      rsGroupAdmin.moveServers(Sets.newHashSet(Address.fromParts("bogus", 123)), "bogus");
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(Address.fromParts("bogus", 123)),
+          "bogus");
       fail("Expected move with bogus group to fail");
     } catch (ConstraintException ex) {
       // expected
@@ -158,8 +159,8 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
     admin.createNamespace(NamespaceDescriptor.create(nsName)
       .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, groupName).build());
-    RSGroupInfo rsGroupInfo = rsGroupAdmin.getRSGroupInfo(groupName);
-    rsGroupAdmin.moveServers(rsGroupInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    RSGroupInfo rsGroupInfo = rsGroupAdmin.getRSGroup(groupName);
+    rsGroupAdmin.moveToRSGroup(rsGroupInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
     // test removing a referenced group
     try {
       rsGroupAdmin.removeRSGroup(groupName);
@@ -198,16 +199,20 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
   @Test
   public void testGetRSGroupInfoCPHookCalled() throws Exception {
-    rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
-    assertTrue(observer.preGetRSGroupInfoCalled);
-    assertTrue(observer.postGetRSGroupInfoCalled);
+    rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
+    if (rsGroupAdmin instanceof RSGroupAdminClient) {
+      assertTrue(observer.preGetRSGroupInfoCalled);
+      assertTrue(observer.postGetRSGroupInfoCalled);
+    }
   }
 
   @Test
   public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception {
-    rsGroupAdmin.getRSGroupInfoOfTable(TableName.META_TABLE_NAME);
-    assertTrue(observer.preGetRSGroupInfoOfTableCalled);
-    assertTrue(observer.postGetRSGroupInfoOfTableCalled);
+    rsGroupAdmin.getRSGroup(TableName.META_TABLE_NAME);
+    if (rsGroupAdmin instanceof RSGroupAdminClient) {
+      assertTrue(observer.preGetRSGroupInfoOfTableCalled);
+      assertTrue(observer.postGetRSGroupInfoOfTableCalled);
+    }
   }
 
   @Test
@@ -220,9 +225,11 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
   @Test
   public void testGetRSGroupInfoOfServerCPHookCalled() throws Exception {
     ServerName masterServerName = ((MiniHBaseCluster) cluster).getMaster().getServerName();
-    rsGroupAdmin.getRSGroupOfServer(masterServerName.getAddress());
-    assertTrue(observer.preGetRSGroupInfoOfServerCalled);
-    assertTrue(observer.postGetRSGroupInfoOfServerCalled);
+    rsGroupAdmin.getRSGroup(masterServerName.getAddress());
+    if (rsGroupAdmin instanceof RSGroupAdminClient) {
+      assertTrue(observer.preGetRSGroupInfoOfServerCalled);
+      assertTrue(observer.postGetRSGroupInfoOfServerCalled);
+    }
   }
 
   @Test
@@ -230,8 +237,8 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
     int initNumGroups = rsGroupAdmin.listRSGroups().size();
     addGroup("bar", 3);
     TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
-    rsGroupAdmin.moveTables(Sets.newHashSet(tableName), "bar");
-    RSGroupInfo barGroup = rsGroupAdmin.getRSGroupInfo("bar");
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), "bar");
+    RSGroupInfo barGroup = rsGroupAdmin.getRSGroup("bar");
     // group is not empty therefore it should fail
     try {
       rsGroupAdmin.removeRSGroup(barGroup.getName());
@@ -240,19 +247,19 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
     }
     // group cannot lose all it's servers therefore it should fail
     try {
-      rsGroupAdmin.moveServers(barGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
+      rsGroupAdmin.moveToRSGroup(barGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
       fail("Expected move servers to fail");
     } catch (IOException e) {
     }
 
-    rsGroupAdmin.moveTables(barGroup.getTables(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.setRSGroup(barGroup.getTables(), RSGroupInfo.DEFAULT_GROUP);
     try {
       rsGroupAdmin.removeRSGroup(barGroup.getName());
       fail("Expected move servers to fail");
     } catch (IOException e) {
     }
 
-    rsGroupAdmin.moveServers(barGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(barGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
     rsGroupAdmin.removeRSGroup(barGroup.getName());
 
     Assert.assertEquals(initNumGroups, rsGroupAdmin.listRSGroups().size());
@@ -260,10 +267,12 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
   @Test
   public void testMultiTableMove() throws Exception {
-    final TableName tableNameA = TableName.valueOf(tablePrefix + name.getMethodName() + "A");
-    final TableName tableNameB = TableName.valueOf(tablePrefix + name.getMethodName() + "B");
+    final TableName tableNameA = TableName.valueOf(tablePrefix +
+        getNameWithoutIndex(name.getMethodName()) + "A");
+    final TableName tableNameB = TableName.valueOf(tablePrefix +
+        getNameWithoutIndex(name.getMethodName()) + "B");
     final byte[] familyNameBytes = Bytes.toBytes("f");
-    String newGroupName = getGroupName(name.getMethodName());
+    String newGroupName = getGroupName(getNameWithoutIndex(name.getMethodName()));
     final RSGroupInfo newGroup = addGroup(newGroupName, 1);
 
     TEST_UTIL.createTable(tableNameA, familyNameBytes);
@@ -284,30 +293,30 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
       }
     });
 
-    RSGroupInfo tableGrpA = rsGroupAdmin.getRSGroupInfoOfTable(tableNameA);
+    RSGroupInfo tableGrpA = rsGroupAdmin.getRSGroup(tableNameA);
     assertTrue(tableGrpA.getName().equals(RSGroupInfo.DEFAULT_GROUP));
 
-    RSGroupInfo tableGrpB = rsGroupAdmin.getRSGroupInfoOfTable(tableNameB);
+    RSGroupInfo tableGrpB = rsGroupAdmin.getRSGroup(tableNameB);
     assertTrue(tableGrpB.getName().equals(RSGroupInfo.DEFAULT_GROUP));
     // change table's group
     LOG.info("Moving table [" + tableNameA + "," + tableNameB + "] to " + newGroup.getName());
-    rsGroupAdmin.moveTables(Sets.newHashSet(tableNameA, tableNameB), newGroup.getName());
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableNameA, tableNameB), newGroup.getName());
 
     // verify group change
     Assert.assertEquals(newGroup.getName(),
-      rsGroupAdmin.getRSGroupInfoOfTable(tableNameA).getName());
+      rsGroupAdmin.getRSGroup(tableNameA).getName());
 
     Assert.assertEquals(newGroup.getName(),
-      rsGroupAdmin.getRSGroupInfoOfTable(tableNameB).getName());
+      rsGroupAdmin.getRSGroup(tableNameB).getName());
 
     // verify tables' not exist in old group
     Set<TableName> DefaultTables =
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getTables();
     assertFalse(DefaultTables.contains(tableNameA));
     assertFalse(DefaultTables.contains(tableNameB));
 
     // verify tables' exist in new group
-    Set<TableName> newGroupTables = rsGroupAdmin.getRSGroupInfo(newGroupName).getTables();
+    Set<TableName> newGroupTables = rsGroupAdmin.getRSGroup(newGroupName).getTables();
     assertTrue(newGroupTables.contains(tableNameA));
     assertTrue(newGroupTables.contains(tableNameB));
   }
@@ -315,7 +324,7 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
   @Test
   public void testTableMoveTruncateAndDrop() throws Exception {
     final byte[] familyNameBytes = Bytes.toBytes("f");
-    String newGroupName = getGroupName(name.getMethodName());
+    String newGroupName = getGroupName(getNameWithoutIndex(name.getMethodName()));
     final RSGroupInfo newGroup = addGroup(newGroupName, 2);
 
     TEST_UTIL.createMultiRegionTable(tableName, familyNameBytes, 5);
@@ -331,16 +340,17 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
       }
     });
 
-    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroupInfoOfTable(tableName);
+    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroup(tableName);
+    LOG.info("got table group info is {}", tableGrp);
     assertTrue(tableGrp.getName().equals(RSGroupInfo.DEFAULT_GROUP));
 
     // change table's group
     LOG.info("Moving table " + tableName + " to " + newGroup.getName());
-    rsGroupAdmin.moveTables(Sets.newHashSet(tableName), newGroup.getName());
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), newGroup.getName());
 
     // verify group change
     Assert.assertEquals(newGroup.getName(),
-      rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+      rsGroupAdmin.getRSGroup(tableName).getName());
 
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
@@ -361,22 +371,20 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
     // test truncate
     admin.disableTable(tableName);
     admin.truncateTable(tableName, true);
-    Assert.assertEquals(1, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().size());
+    Assert.assertEquals(1, rsGroupAdmin.getRSGroup(newGroup.getName()).getTables().size());
     Assert.assertEquals(tableName,
-      rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().first());
+      rsGroupAdmin.getRSGroup(newGroup.getName()).getTables().first());
 
     // verify removed table is removed from group
     TEST_UTIL.deleteTable(tableName);
-    Assert.assertEquals(0, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().size());
+    Assert.assertEquals(0, rsGroupAdmin.getRSGroup(newGroup.getName()).getTables().size());
 
-    assertTrue(observer.preMoveTablesCalled);
-    assertTrue(observer.postMoveTablesCalled);
   }
 
   @Test
   public void testDisabledTableMove() throws Exception {
     final byte[] familyNameBytes = Bytes.toBytes("f");
-    String newGroupName = getGroupName(name.getMethodName());
+    String newGroupName = getGroupName(getNameWithoutIndex(name.getMethodName()));
     final RSGroupInfo newGroup = addGroup(newGroupName, 2);
 
     TEST_UTIL.createMultiRegionTable(tableName, familyNameBytes, 5);
@@ -391,7 +399,7 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
       }
     });
 
-    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroupInfoOfTable(tableName);
+    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroup(tableName);
     assertTrue(tableGrp.getName().equals(RSGroupInfo.DEFAULT_GROUP));
 
     // test disable table
@@ -399,18 +407,18 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
     // change table's group
     LOG.info("Moving table " + tableName + " to " + newGroup.getName());
-    rsGroupAdmin.moveTables(Sets.newHashSet(tableName), newGroup.getName());
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), newGroup.getName());
 
     // verify group change
     Assert.assertEquals(newGroup.getName(),
-      rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+      rsGroupAdmin.getRSGroup(tableName).getName());
   }
 
   @Test
   public void testNonExistentTableMove() throws Exception {
-    TableName tableName = TableName.valueOf(tablePrefix + name.getMethodName());
-
-    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroupInfoOfTable(tableName);
+    TableName tableName = TableName.valueOf(tablePrefix +
+        getNameWithoutIndex(name.getMethodName()));
+    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroup(tableName);
     assertNull(tableGrp);
 
     // test if table exists already.
@@ -419,21 +427,22 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
 
     LOG.info("Moving table " + tableName + " to " + RSGroupInfo.DEFAULT_GROUP);
     try {
-      rsGroupAdmin.moveTables(Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
+      rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
       fail("Table " + tableName + " shouldn't have been successfully moved.");
     } catch (IOException ex) {
       assertTrue(ex instanceof TableNotFoundException);
     }
 
     try {
-      rsGroupAdmin.moveServersAndTables(Sets.newHashSet(Address.fromParts("bogus", 123)),
-        Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
+      rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(Address.fromParts("bogus", 123)),
+          RSGroupInfo.DEFAULT_GROUP);
       fail("Table " + tableName + " shouldn't have been successfully moved.");
     } catch (IOException ex) {
       assertTrue(ex instanceof TableNotFoundException);
     }
     // verify group change
-    assertNull(rsGroupAdmin.getRSGroupInfoOfTable(tableName));
+    assertNull(rsGroupAdmin.getRSGroup(tableName));
   }
 
   @Test
@@ -512,20 +521,10 @@ public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
       }
     });
     SortedSet<TableName> tables
-        = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+        = rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getTables();
     assertTrue("Table 't1' must be in 'default' rsgroup", tables.contains(tn1));
 
     // Cleanup
     TEST_UTIL.deleteTable(tn1);
   }
-
-  private void toggleQuotaCheckAndRestartMiniCluster(boolean enable) throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, enable);
-    TEST_UTIL.startMiniCluster(NUM_SLAVES_BASE - 1);
-    TEST_UTIL.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
-      NUM_SLAVES_BASE - 1);
-    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
-    initialize();
-  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
index 076362e..eb808e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.rsgroup;
 
-import static org.apache.hadoop.hbase.rsgroup.RSGroupAdminServer.DEFAULT_MAX_RETRY_VALUE;
 import static org.apache.hadoop.hbase.util.Threads.sleep;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -55,12 +54,15 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
+@RunWith(Parameterized.class)
 @Category({ LargeTests.class })
 public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
@@ -128,7 +130,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     }
     final ServerName targetServer = tmpTargetServer;
     // move target server to group
-    rsGroupAdmin.moveServers(Sets.newHashSet(targetServer.getAddress()), newGroup.getName());
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(targetServer.getAddress()), newGroup.getName());
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -162,14 +164,14 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     int initNumGroups = rsGroupAdmin.listRSGroups().size();
     RSGroupInfo appInfo = addGroup(getGroupName(name.getMethodName()), 1);
     RSGroupInfo adminInfo = addGroup(getGroupName(name.getMethodName()), 1);
-    RSGroupInfo dInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    RSGroupInfo dInfo = rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
     Assert.assertEquals(initNumGroups + 2, rsGroupAdmin.listRSGroups().size());
     assertEquals(1, adminInfo.getServers().size());
     assertEquals(1, appInfo.getServers().size());
     assertEquals(getNumServers() - 2, dInfo.getServers().size());
-    rsGroupAdmin.moveServers(appInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(appInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
     rsGroupAdmin.removeRSGroup(appInfo.getName());
-    rsGroupAdmin.moveServers(adminInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(adminInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
     rsGroupAdmin.removeRSGroup(adminInfo.getName());
     Assert.assertEquals(rsGroupAdmin.listRSGroups().size(), initNumGroups);
   }
@@ -180,14 +182,14 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     addGroup("bar", 3);
     rsGroupAdmin.addRSGroup("foo");
 
-    RSGroupInfo barGroup = rsGroupAdmin.getRSGroupInfo("bar");
-    RSGroupInfo fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    RSGroupInfo barGroup = rsGroupAdmin.getRSGroup("bar");
+    RSGroupInfo fooGroup = rsGroupAdmin.getRSGroup("foo");
     assertEquals(3, barGroup.getServers().size());
     assertEquals(0, fooGroup.getServers().size());
 
     // test fail bogus server move
     try {
-      rsGroupAdmin.moveServers(Sets.newHashSet(Address.fromString("foo:9999")), "foo");
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(Address.fromString("foo:9999")), "foo");
       fail("Bogus servers shouldn't have been successfully moved.");
     } catch (IOException ex) {
       String exp = "Source RSGroup for server foo:9999 does not exist.";
@@ -197,34 +199,34 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // test success case
     LOG.info("moving servers " + barGroup.getServers() + " to group foo");
-    rsGroupAdmin.moveServers(barGroup.getServers(), fooGroup.getName());
+    rsGroupAdmin.moveToRSGroup(barGroup.getServers(), fooGroup.getName());
 
-    barGroup = rsGroupAdmin.getRSGroupInfo("bar");
-    fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    barGroup = rsGroupAdmin.getRSGroup("bar");
+    fooGroup = rsGroupAdmin.getRSGroup("foo");
     assertEquals(0, barGroup.getServers().size());
     assertEquals(3, fooGroup.getServers().size());
 
     LOG.info("moving servers " + fooGroup.getServers() + " to group default");
-    rsGroupAdmin.moveServers(fooGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(fooGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
 
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return getNumServers() == rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP)
+        return getNumServers() == rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP)
           .getServers().size();
       }
     });
 
-    fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    fooGroup = rsGroupAdmin.getRSGroup("foo");
     assertEquals(0, fooGroup.getServers().size());
 
     // test group removal
     LOG.info("Remove group " + barGroup.getName());
     rsGroupAdmin.removeRSGroup(barGroup.getName());
-    Assert.assertEquals(null, rsGroupAdmin.getRSGroupInfo(barGroup.getName()));
+    Assert.assertEquals(null, rsGroupAdmin.getRSGroup(barGroup.getName()));
     LOG.info("Remove group " + fooGroup.getName());
     rsGroupAdmin.removeRSGroup(fooGroup.getName());
-    Assert.assertEquals(null, rsGroupAdmin.getRSGroupInfo(fooGroup.getName()));
+    Assert.assertEquals(null, rsGroupAdmin.getRSGroup(fooGroup.getName()));
   }
 
   @Test
@@ -236,7 +238,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // remove online servers
     try {
-      rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));
+      rsGroupAdmin.removeRSGroup(Sets.newHashSet(targetServer.getAddress()));
       fail("Online servers shouldn't have been successfully removed.");
     } catch (IOException ex) {
       String exp =
@@ -267,7 +269,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     });
 
     try {
-      rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));
+      rsGroupAdmin.removeRSGroup(Sets.newHashSet(targetServer.getAddress()));
       fail("Dead servers shouldn't have been successfully removed.");
     } catch (IOException ex) {
       String exp = "Server " + targetServer.getAddress() + " is on the dead servers list," +
@@ -287,8 +289,8 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     assertEquals(1, admin.listDecommissionedRegionServers().size());
 
     assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
-    rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));
-    Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    rsGroupAdmin.removeRSGroup(Sets.newHashSet(targetServer.getAddress()));
+    Set<Address> newGroupServers = rsGroupAdmin.getRSGroup(newGroup.getName()).getServers();
     assertFalse(newGroupServers.contains(targetServer.getAddress()));
     assertEquals(2, newGroupServers.size());
 
@@ -320,7 +322,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     for (ServerName server : admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
       .getLiveServerMetrics().keySet()) {
       if (!newGroup.containsServer(server.getAddress()) &&
-        !rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
+        !rsGroupAdmin.getRSGroup("master").containsServer(server.getAddress())) {
         targetServer = server;
         break;
       }
@@ -328,14 +330,15 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     LOG.debug("Print group info : " + rsGroupAdmin.listRSGroups());
     int oldDefaultGroupServerSize =
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size();
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().size();
     int oldDefaultGroupTableSize =
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables().size();
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getTables().size();
 
     // test fail bogus server move
     try {
-      rsGroupAdmin.moveServersAndTables(Sets.newHashSet(Address.fromString("foo:9999")),
-        Sets.newHashSet(tableName), newGroup.getName());
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(Address.fromString("foo:9999")),
+          newGroup.getName());
+      rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), newGroup.getName());
       fail("Bogus servers shouldn't have been successfully moved.");
     } catch (IOException ex) {
       String exp = "Source RSGroup for server foo:9999 does not exist.";
@@ -344,18 +347,19 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     }
 
     // test move when src = dst
-    rsGroupAdmin.moveServersAndTables(Sets.newHashSet(targetServer.getAddress()),
-      Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(targetServer.getAddress()),
+        RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
 
     // verify default group info
     Assert.assertEquals(oldDefaultGroupServerSize,
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size());
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().size());
     Assert.assertEquals(oldDefaultGroupTableSize,
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables().size());
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getTables().size());
 
     // verify new group info
-    Assert.assertEquals(1, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers().size());
-    Assert.assertEquals(0, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().size());
+    Assert.assertEquals(1, rsGroupAdmin.getRSGroup(newGroup.getName()).getServers().size());
+    Assert.assertEquals(0, rsGroupAdmin.getRSGroup(newGroup.getName()).getTables().size());
 
     // get all region to move targetServer
     List<String> regionList = getTableRegionMap().get(tableName);
@@ -381,38 +385,39 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // move targetServer and table to newGroup
     LOG.info("moving server and table to newGroup");
-    rsGroupAdmin.moveServersAndTables(Sets.newHashSet(targetServer.getAddress()),
-      Sets.newHashSet(tableName), newGroup.getName());
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(targetServer.getAddress()),
+        newGroup.getName());
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(tableName), newGroup.getName());
 
     // verify group change
     Assert.assertEquals(newGroup.getName(),
-      rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+      rsGroupAdmin.getRSGroup(tableName).getName());
 
     // verify servers' not exist in old group
     Set<Address> defaultServers =
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers();
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers();
     assertFalse(defaultServers.contains(targetServer.getAddress()));
 
     // verify servers' exist in new group
-    Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    Set<Address> newGroupServers = rsGroupAdmin.getRSGroup(newGroup.getName()).getServers();
     assertTrue(newGroupServers.contains(targetServer.getAddress()));
 
     // verify tables' not exist in old group
     Set<TableName> defaultTables =
-      rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+      rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getTables();
     assertFalse(defaultTables.contains(tableName));
 
     // verify tables' exist in new group
-    Set<TableName> newGroupTables = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables();
+    Set<TableName> newGroupTables = rsGroupAdmin.getRSGroup(newGroup.getName()).getTables();
     assertTrue(newGroupTables.contains(tableName));
 
     // verify that all region still assgin on targetServer
     // TODO: uncomment after we reimplement moveServersAndTables, now the implementation is
-    // moveServers first and then moveTables, so the region will be moved to other region servers.
+    // moveToRSGroup first and then moveTables, so the region will be moved to other region servers.
     // Assert.assertEquals(5, getTableServerRegionMap().get(tableName).get(targetServer).size());
 
-    assertTrue(observer.preMoveServersAndTables);
-    assertTrue(observer.postMoveServersAndTables);
+    assertTrue(observer.preMoveServersCalled);
+    assertTrue(observer.postMoveServersCalled);
   }
 
   @Test
@@ -420,17 +425,17 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     // create groups and assign servers
     rsGroupAdmin.addRSGroup("foo");
 
-    RSGroupInfo fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    RSGroupInfo fooGroup = rsGroupAdmin.getRSGroup("foo");
     assertEquals(0, fooGroup.getServers().size());
-    RSGroupInfo defaultGroup = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    RSGroupInfo defaultGroup = rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
 
     // test remove all servers from default
     try {
-      rsGroupAdmin.moveServers(defaultGroup.getServers(), fooGroup.getName());
-      fail(RSGroupAdminServer.KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE);
+      rsGroupAdmin.moveToRSGroup(defaultGroup.getServers(), fooGroup.getName());
+      fail(RSGroupInfoManagerImpl.KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE);
     } catch (ConstraintException ex) {
       assertTrue(
-        ex.getMessage().contains(RSGroupAdminServer.KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE));
+        ex.getMessage().contains(RSGroupInfoManagerImpl.KEEP_ONE_SERVER_IN_DEFAULT_ERROR_MESSAGE));
     }
 
     // test success case, remove one server from default ,keep at least one server
@@ -438,35 +443,35 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
       Address serverInDefaultGroup = defaultGroup.getServers().iterator().next();
       LOG.info("moving server " + serverInDefaultGroup + " from group default to group " +
         fooGroup.getName());
-      rsGroupAdmin.moveServers(Sets.newHashSet(serverInDefaultGroup), fooGroup.getName());
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(serverInDefaultGroup), fooGroup.getName());
     }
 
-    fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    fooGroup = rsGroupAdmin.getRSGroup("foo");
     LOG.info("moving servers " + fooGroup.getServers() + " to group default");
-    rsGroupAdmin.moveServers(fooGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(fooGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
 
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return getNumServers() == rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP)
+        return getNumServers() == rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP)
           .getServers().size();
       }
     });
 
-    fooGroup = rsGroupAdmin.getRSGroupInfo("foo");
+    fooGroup = rsGroupAdmin.getRSGroup("foo");
     assertEquals(0, fooGroup.getServers().size());
 
     // test group removal
     LOG.info("Remove group " + fooGroup.getName());
     rsGroupAdmin.removeRSGroup(fooGroup.getName());
-    Assert.assertEquals(null, rsGroupAdmin.getRSGroupInfo(fooGroup.getName()));
+    Assert.assertEquals(null, rsGroupAdmin.getRSGroup(fooGroup.getName()));
   }
 
   @Test
   public void testFailedMoveBeforeRetryExhaustedWhenMoveServer() throws Exception {
     String groupName = getGroupName(name.getMethodName());
     rsGroupAdmin.addRSGroup(groupName);
-    final RSGroupInfo newGroup = rsGroupAdmin.getRSGroupInfo(groupName);
+    final RSGroupInfo newGroup = rsGroupAdmin.getRSGroup(groupName);
     Pair<ServerName, RegionStateNode> gotPair = createTableWithRegionSplitting(newGroup,
         10);
 
@@ -482,7 +487,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     Thread t2 = new Thread(() -> {
       LOG.info("thread2 start running, to move regions");
       try {
-        rsGroupAdmin.moveServers(Sets.newHashSet(movedServer.getAddress()), newGroup.getName());
+        rsGroupAdmin.moveToRSGroup(Sets.newHashSet(movedServer.getAddress()), newGroup.getName());
       } catch (IOException e) {
         LOG.error("move server error", e);
       }
@@ -512,7 +517,8 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
       // wait until there is only left the region we changed state and recover its state.
       // wait time is set according to the number of max retries, all except failed regions will be
       // moved in one retry, and will sleep 1s until next retry.
-      while (System.currentTimeMillis() - current <= DEFAULT_MAX_RETRY_VALUE * 1000) {
+      while (System.currentTimeMillis() - current <=
+          RSGroupInfoManagerImpl.DEFAULT_MAX_RETRY_VALUE * 1000) {
         List<RegionInfo> regions = getRegions.apply(owner);
         LOG.debug("server table region size is:{}", regions.size());
         assert regions.size() >= 1;
@@ -600,7 +606,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
   @Test
   public void testFailedMoveServersAndRepair() throws Exception{
-    // This UT calls moveServers() twice to test the idempotency of it.
+    // This UT calls moveToRSGroup() twice to test the idempotency of it.
     // The first time, movement fails because a region is made in SPLITTING state
     // which will not be moved.
     // The second time, the region state is OPEN and check if all
@@ -616,7 +622,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // move server to newGroup and check regions
     try {
-      rsGroupAdmin.moveServers(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
       fail("should get IOException when retry exhausted but there still exists failed moved "
           + "regions");
     }catch (Exception e){
@@ -632,7 +638,7 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // retry move server to newGroup and check if all regions on srcServer was moved
     rsn.setState(RegionState.State.OPEN);
-    rsGroupAdmin.moveServers(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
     assertEquals(master.getAssignmentManager().getRegionsOnServer(srcServer).size(), 0);
   }
 
@@ -661,8 +667,8 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
 
     // move server and table to newGroup and check regions
     try {
-      rsGroupAdmin.moveServersAndTables(Sets.newHashSet(srcServer.getAddress()),
-          Sets.newHashSet(table2), newGroup.getName());
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
+      rsGroupAdmin.setRSGroup(Sets.newHashSet(table2), newGroup.getName());
       fail("should get IOException when retry exhausted but there still exists failed moved "
           + "regions");
     }catch (Exception e){
@@ -679,8 +685,8 @@ public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
     // retry moveServersAndTables to newGroup and check if all regions on srcServer belongs to
     // table2
     rsn.setState(RegionState.State.OPEN);
-    rsGroupAdmin.moveServersAndTables(Sets.newHashSet(srcServer.getAddress()),
-        Sets.newHashSet(table2), newGroup.getName());
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(srcServer.getAddress()), newGroup.getName());
+    rsGroupAdmin.setRSGroup(Sets.newHashSet(table2), newGroup.getName());
     for(RegionInfo regionsInfo : master.getAssignmentManager().getRegionsOnServer(srcServer)){
       assertEquals(regionsInfo.getTable(), table2);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
index 8d10850..be93186 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
@@ -42,9 +42,12 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@RunWith(Parameterized.class)
 @Category({ MediumTests.class })
 public class TestRSGroupsBalance extends TestRSGroupsBase {
 
@@ -80,7 +83,8 @@ public class TestRSGroupsBalance extends TestRSGroupsBase {
     String newGroupName = getGroupName(name.getMethodName());
     addGroup(newGroupName, 3);
 
-    final TableName tableName = TableName.valueOf(tablePrefix + "_ns", name.getMethodName());
+    final TableName tableName = TableName.valueOf(tablePrefix + "_ns",
+        getNameWithoutIndex(name.getMethodName()));
     admin.createNamespace(NamespaceDescriptor.create(tableName.getNamespaceAsString())
       .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, newGroupName).build());
     final TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
@@ -151,13 +155,12 @@ public class TestRSGroupsBalance extends TestRSGroupsBase {
 
   @Test
   public void testMisplacedRegions() throws Exception {
-    String namespace = tablePrefix + "_" + name.getMethodName();
+    String namespace = tablePrefix + "_" + getNameWithoutIndex(name.getMethodName());
     TEST_UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(namespace).build());
-    final TableName tableName =
-        TableName.valueOf(namespace, tablePrefix + "_" + name.getMethodName());
-    LOG.info(name.getMethodName());
+    final TableName tableName = TableName.valueOf(namespace, tablePrefix + "_" +
+        getNameWithoutIndex(name.getMethodName()));
 
-    final RSGroupInfo rsGroupInfo = addGroup(name.getMethodName(), 1);
+    final RSGroupInfo rsGroupInfo = addGroup(getGroupName(name.getMethodName()), 1);
 
     TEST_UTIL.createMultiRegionTable(tableName, new byte[] { 'f' }, 15);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index b9885be..6626fba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -30,6 +31,7 @@ import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
@@ -54,8 +56,10 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.junit.Rule;
 import org.junit.rules.TestName;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -74,7 +78,6 @@ public abstract class TestRSGroupsBase {
   protected static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static Admin admin;
   protected static HBaseCluster cluster;
-  protected static RSGroupAdminClient rsGroupAdmin;
   protected static HMaster master;
   protected boolean INIT = false;
   protected static RSGroupAdminEndpoint rsGroupAdminEndpoint;
@@ -89,6 +92,48 @@ public abstract class TestRSGroupsBase {
   public TestName name = new TestName();
   protected TableName tableName;
 
+  protected Admin rsGroupAdmin;
+
+  @Parameterized.Parameter
+  public Supplier<Object> getAdmin;
+
+  private static RSGroupAdminClient getRSGroupAdmin(){
+    try {
+      return new VerifyingRSGroupAdminClient(
+          new RSGroupAdminClient(TEST_UTIL.getConnection()), TEST_UTIL.getConfiguration());
+    } catch (IOException e) {
+      LOG.error("Get group admin failed", e);
+      return null;
+    }
+  }
+
+  private static Admin getAdmin(){
+    try {
+      return TEST_UTIL.getAdmin();
+    } catch (IOException e) {
+      LOG.error("Get hbase admin failed", e);
+      return null;
+    }
+  }
+
+  public static Object resetAdminConnection(Object admin) {
+    if(admin instanceof RSGroupAdminClient) {
+      return getRSGroupAdmin();
+    }else {
+      return getAdmin();
+    }
+  }
+
+  public static String getNameWithoutIndex(String name) {
+    return name.split("\\[")[0];
+  }
+
+  @Parameterized.Parameters
+  public static List<Object[]> params() {
+    return Arrays.asList(new Supplier<?>[] { TestRSGroupsBase::getRSGroupAdmin },
+        new Supplier<?>[] { TestRSGroupsBase::getAdmin });
+  }
+
   public static void setUpTestBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setFloat(
             "hbase.master.balancer.stochastic.tableSkewCost", 6000);
@@ -107,6 +152,10 @@ public abstract class TestRSGroupsBase {
     initialize();
   }
 
+  public void setAdmin(){
+    rsGroupAdmin = (Admin) getAdmin.get();
+  }
+
   protected static void initialize() throws Exception {
     admin = TEST_UTIL.getAdmin();
     cluster = TEST_UTIL.getHBaseCluster();
@@ -121,8 +170,6 @@ public abstract class TestRSGroupsBase {
       }
     });
     admin.balancerSwitch(false, true);
-    rsGroupAdmin = new VerifyingRSGroupAdminClient(
-        new RSGroupAdminClient(TEST_UTIL.getConnection()), TEST_UTIL.getConfiguration());
     MasterCoprocessorHost host = master.getMasterCoprocessorHost();
     observer = (CPMasterObserver) host.findCoprocessor(CPMasterObserver.class.getName());
     rsGroupAdminEndpoint = (RSGroupAdminEndpoint)
@@ -134,8 +181,9 @@ public abstract class TestRSGroupsBase {
   }
 
   public void setUpBeforeMethod() throws Exception {
+    setAdmin();
     LOG.info(name.getMethodName());
-    tableName = TableName.valueOf(tablePrefix + "_" + name.getMethodName());
+    tableName = TableName.valueOf(tablePrefix + "_" + name.getMethodName().split("\\[")[0]);
     if (!INIT) {
       INIT = true;
       tearDownAfterMethod();
@@ -164,7 +212,8 @@ public abstract class TestRSGroupsBase {
         ((MiniHBaseCluster)cluster).getMaster().getServerName();
 
     try {
-      rsGroupAdmin.moveServers(Sets.newHashSet(masterServerName.getAddress()), "master");
+      rsGroupAdmin.moveToRSGroup(Sets.newHashSet(masterServerName.getAddress()),
+          "master");
     } catch (Exception ex) {
       LOG.warn("Got this on setup, FYI", ex);
     }
@@ -176,7 +225,7 @@ public abstract class TestRSGroupsBase {
         //Might be greater since moving servers back to default
         //is after starting a server
 
-        return rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size()
+        return rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().size()
             == NUM_SLAVES_BASE;
       }
     });
@@ -184,7 +233,7 @@ public abstract class TestRSGroupsBase {
 
   protected RSGroupInfo addGroup(String groupName, int serverCount)
       throws IOException, InterruptedException {
-    RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
     rsGroupAdmin.addRSGroup(groupName);
     Set<Address> set = new HashSet<>();
     for (Address server : defaultInfo.getServers()) {
@@ -193,15 +242,15 @@ public abstract class TestRSGroupsBase {
       }
       set.add(server);
     }
-    rsGroupAdmin.moveServers(set, groupName);
-    RSGroupInfo result = rsGroupAdmin.getRSGroupInfo(groupName);
+    rsGroupAdmin.moveToRSGroup(set, groupName);
+    RSGroupInfo result = rsGroupAdmin.getRSGroup(groupName);
     return result;
   }
 
-  protected void removeGroup(String groupName) throws IOException {
-    RSGroupInfo groupInfo = rsGroupAdmin.getRSGroupInfo(groupName);
-    rsGroupAdmin.moveTables(groupInfo.getTables(), RSGroupInfo.DEFAULT_GROUP);
-    rsGroupAdmin.moveServers(groupInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
+  public void removeGroup(String groupName) throws IOException {
+    RSGroupInfo groupInfo = rsGroupAdmin.getRSGroup(groupName);
+    rsGroupAdmin.setRSGroup(groupInfo.getTables(), RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.moveToRSGroup(groupInfo.getServers(), RSGroupInfo.DEFAULT_GROUP);
     rsGroupAdmin.removeRSGroup(groupName);
   }
 
@@ -224,7 +273,7 @@ public abstract class TestRSGroupsBase {
     RSGroupAdminClient groupAdmin = new RSGroupAdminClient(TEST_UTIL.getConnection());
     for(RSGroupInfo group: groupAdmin.listRSGroups()) {
       if(!group.getName().equals(RSGroupInfo.DEFAULT_GROUP)) {
-        groupAdmin.moveTables(group.getTables(), RSGroupInfo.DEFAULT_GROUP);
+        groupAdmin.setRSGroup(group.getTables(), RSGroupInfo.DEFAULT_GROUP);
         groupAdmin.moveServers(group.getServers(), RSGroupInfo.DEFAULT_GROUP);
         groupAdmin.removeRSGroup(group.getName());
       }
@@ -276,8 +325,9 @@ public abstract class TestRSGroupsBase {
     return count;
   }
 
-  protected String getGroupName(String baseName) {
-    return groupPrefix + "_" + baseName + "_" + rand.nextInt(Integer.MAX_VALUE);
+  public String getGroupName(String baseName) {
+    return groupPrefix + "_" + getNameWithoutIndex(baseName) + "_" +
+        rand.nextInt(Integer.MAX_VALUE);
   }
 
   /**
@@ -290,6 +340,17 @@ public abstract class TestRSGroupsBase {
       .findFirst().get();
   }
 
+  protected void toggleQuotaCheckAndRestartMiniCluster(boolean enable) throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, enable);
+    TEST_UTIL.startMiniCluster(NUM_SLAVES_BASE - 1);
+    TEST_UTIL.getConfiguration().setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
+        NUM_SLAVES_BASE - 1);
+    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    initialize();
+    rsGroupAdmin = (Admin) resetAdminConnection(rsGroupAdmin);
+  }
+
   public static class CPMasterObserver implements MasterCoprocessor, MasterObserver {
     boolean preBalanceRSGroupCalled = false;
     boolean postBalanceRSGroupCalled = false;
@@ -313,6 +374,8 @@ public abstract class TestRSGroupsBase {
     boolean postListRSGroupsCalled = false;
     boolean preGetRSGroupInfoOfServerCalled = false;
     boolean postGetRSGroupInfoOfServerCalled = false;
+    boolean preSetRSGroupForTablesCalled = false;
+    boolean postSetRSGroupForTablesCalled = false;
 
     public void resetFlags() {
       preBalanceRSGroupCalled = false;
@@ -337,6 +400,8 @@ public abstract class TestRSGroupsBase {
       postListRSGroupsCalled = false;
       preGetRSGroupInfoOfServerCalled = false;
       postGetRSGroupInfoOfServerCalled = false;
+      preSetRSGroupForTablesCalled = false;
+      postSetRSGroupForTablesCalled = false;
     }
 
     @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
index 67ed2a9..45d2a2e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
-import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -45,11 +44,14 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
+@RunWith(Parameterized.class)
 @Category({ MediumTests.class })
 public class TestRSGroupsBasics extends TestRSGroupsBase {
 
@@ -81,10 +83,12 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
 
   @Test
   public void testBasicStartUp() throws IOException {
-    RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP);
     assertEquals(NUM_SLAVES_BASE, defaultInfo.getServers().size());
     // Assignment of meta and rsgroup regions.
     int count = master.getAssignmentManager().getRegionStates().getRegionAssignments().size();
+    LOG.info("regions assignments are" +
+        master.getAssignmentManager().getRegionStates().getRegionAssignments().toString());
     // 2 (meta and rsgroup)
     assertEquals(2, count);
   }
@@ -165,6 +169,7 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
 
     // clone
     admin.cloneSnapshot(snapshotName, clonedTableName);
+    admin.deleteSnapshot(snapshotName);
   }
 
   @Test
@@ -206,7 +211,7 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
     assertEquals(0, notClearedServers.size());
 
     // the stopped region server gets cleared and removed from the group
-    Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    Set<Address> newGroupServers = rsGroupAdmin.getRSGroup(newGroup.getName()).getServers();
     assertFalse(newGroupServers.contains(serverToStop.getAddress()));
     assertEquals(serverCountToMoveToNewGroup - 1 /* 1 stopped */, newGroupServers.size());
   }
@@ -244,34 +249,20 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
     assertEquals(serverCountToMoveToDeadServerGroup, notClearedServers.size());
 
     Set<Address> ServersInDeadServerGroup =
-        rsGroupAdmin.getRSGroupInfo(deadServerGroup.getName()).getServers();
+        rsGroupAdmin.getRSGroup(deadServerGroup.getName()).getServers();
     assertEquals(serverCountToMoveToDeadServerGroup, ServersInDeadServerGroup.size());
     assertTrue(ServersInDeadServerGroup.contains(serverToStop.getAddress()));
   }
 
   @Test
   public void testRSGroupsWithHBaseQuota() throws Exception {
-    TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
-    restartHBaseCluster();
-    try {
-      TEST_UTIL.waitFor(90000, new Waiter.Predicate<Exception>() {
-        @Override
-        public boolean evaluate() throws Exception {
-          return admin.isTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
-        }
-      });
-    } finally {
-      TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, false);
-      restartHBaseCluster();
-    }
-  }
-
-  private void restartHBaseCluster() throws Exception {
-    LOG.info("\n\nShutting down cluster");
-    TEST_UTIL.shutdownMiniHBaseCluster();
-    LOG.info("\n\nSleeping a bit");
-    Thread.sleep(2000);
-    TEST_UTIL.restartHBaseCluster(NUM_SLAVES_BASE - 1);
-    initialize();
+    toggleQuotaCheckAndRestartMiniCluster(true);
+    TEST_UTIL.waitFor(90000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return admin.isTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
+      }
+    });
+    toggleQuotaCheckAndRestartMiniCluster(false);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java
index 539f351..2593216 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java
@@ -51,11 +51,14 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
+@RunWith(Parameterized.class)
 @Category({ MediumTests.class })
 public class TestRSGroupsKillRS extends TestRSGroupsBase {
 
@@ -88,7 +91,8 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
   @Test
   public void testKillRS() throws Exception {
     RSGroupInfo appInfo = addGroup("appInfo", 1);
-    final TableName tableName = TableName.valueOf(tablePrefix + "_ns", name.getMethodName());
+    final TableName tableName = TableName.valueOf(tablePrefix + "_ns",
+        getNameWithoutIndex(name.getMethodName()));
     admin.createNamespace(NamespaceDescriptor.create(tableName.getNamespaceAsString())
       .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, appInfo.getName()).build());
     final TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
@@ -128,8 +132,8 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
     });
     Set<Address> newServers = Sets.newHashSet();
     newServers
-      .add(rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().iterator().next());
-    rsGroupAdmin.moveServers(newServers, appInfo.getName());
+      .add(rsGroupAdmin.getRSGroup(RSGroupInfo.DEFAULT_GROUP).getServers().iterator().next());
+    rsGroupAdmin.moveToRSGroup(newServers, appInfo.getName());
 
     // Make sure all the table's regions get reassigned
     // disabling the table guarantees no conflicting assign/unassign (ie SSH) happens
@@ -161,12 +165,12 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
     TEST_UTIL.loadTable(t, Bytes.toBytes("f"));
     Set<TableName> toAddTables = new HashSet<>();
     toAddTables.add(tableName);
-    rsGroupAdmin.moveTables(toAddTables, groupName);
-    assertTrue(rsGroupAdmin.getRSGroupInfo(groupName).getTables().contains(tableName));
+    rsGroupAdmin.setRSGroup(toAddTables, groupName);
+    assertTrue(rsGroupAdmin.getRSGroup(groupName).getTables().contains(tableName));
     TEST_UTIL.waitTableAvailable(tableName, 30000);
 
     // check my_group servers and table regions
-    Set<Address> servers = rsGroupAdmin.getRSGroupInfo(groupName).getServers();
+    Set<Address> servers = rsGroupAdmin.getRSGroup(groupName).getServers();
     assertEquals(2, servers.size());
     LOG.debug("group servers {}", servers);
     for (RegionInfo tr :
@@ -197,7 +201,7 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
 
     // case 1: stop all the regionservers in my_group, and restart a regionserver in my_group,
     // and then check if all table regions are online
-    for(Address addr : rsGroupAdmin.getRSGroupInfo(groupName).getServers()) {
+    for(Address addr : rsGroupAdmin.getRSGroup(groupName).getServers()) {
       TEST_UTIL.getMiniHBaseCluster().stopRegionServer(getServerName(addr));
     }
     // better wait for a while for region reassign
@@ -224,7 +228,7 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
     assertEquals(NUM_SLAVES_BASE - gsn.size(),
         TEST_UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size());
     ServerName newServer = master.getServerManager().getOnlineServersList().get(0);
-    rsGroupAdmin.moveServers(Sets.newHashSet(newServer.getAddress()), groupName);
+    rsGroupAdmin.moveToRSGroup(Sets.newHashSet(newServer.getAddress()), groupName);
     // wait and check if table regions are online
     TEST_UTIL.waitTableAvailable(tableName, 30000);
   }
@@ -240,14 +244,14 @@ public class TestRSGroupsKillRS extends TestRSGroupsBase {
     tableName = TableName.META_TABLE_NAME;
     Set<TableName> toAddTables = new HashSet<>();
     toAddTables.add(tableName);
-    rsGroupAdmin.moveTables(toAddTables, groupName);
-    assertTrue(rsGroupAdmin.getRSGroupInfo(groupName).getTables().contains(tableName));
+    rsGroupAdmin.setRSGroup(toAddTables, groupName);
+    assertTrue(rsGroupAdmin.getRSGroup(groupName).getTables().contains(tableName));
     TEST_UTIL.waitTableAvailable(tableName, 30000);
 
     // restart the regionserver in meta_group, and lower its version
     String originVersion = "";
     Set<Address> servers = new HashSet<>();
-    for(Address addr : rsGroupAdmin.getRSGroupInfo(groupName).getServers()) {
+    for(Address addr : rsGroupAdmin.getRSGroup(groupName).getServers()) {
       servers.add(addr);
       TEST_UTIL.getMiniHBaseCluster().stopRegionServer(getServerName(addr));
       originVersion = master.getRegionServerVersion(getServerName(addr));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
index 7ac1a49..cfc721f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
@@ -56,7 +56,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 // online. In new master, RSGroupInfoManagerImpl gets the data from zk and waits for the expected
 // assignment with a timeout.
 @Category(MediumTests.class)
-public class TestRSGroupsOfflineMode {
+public class TestRSGroupsOfflineMode extends TestRSGroupsBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -106,7 +106,7 @@ public class TestRSGroupsOfflineMode {
   @Test
   public void testOffline() throws Exception, InterruptedException {
     // Table should be after group table name so it gets assigned later.
-    final TableName failoverTable = TableName.valueOf(name.getMethodName());
+    final TableName failoverTable = TableName.valueOf(getNameWithoutIndex(name.getMethodName()));
     TEST_UTIL.createTable(failoverTable, Bytes.toBytes("f"));
     final HRegionServer killRS = ((MiniHBaseCluster) cluster).getRegionServer(0);
     final HRegionServer groupRS = ((MiniHBaseCluster) cluster).getRegionServer(1);
@@ -139,7 +139,7 @@ public class TestRSGroupsOfflineMode {
       }
     });
     // Move table to group and wait.
-    groupAdmin.moveTables(Sets.newHashSet(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME), newGroup);
+    groupAdmin.setRSGroup(Sets.newHashSet(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME), newGroup);
     LOG.info("Waiting for move table...");
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
index c1c157a..06c4e4e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
@@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
+import java.util.Optional;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -31,7 +33,11 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.PermissionStorage;
@@ -92,6 +98,9 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
 
   private static RSGroupAdminEndpoint rsGroupAdminEndpoint;
+  private static HMaster master;
+  private static AccessChecker accessChecker;
+  private static UserProvider userProvider;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -131,6 +140,22 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
 
     systemUserConnection = TEST_UTIL.getConnection();
     setUpTableAndUserPermissions();
+    master = TEST_UTIL.getHBaseCluster().getMaster();
+    accessChecker = master.getAccessChecker();
+    userProvider = UserProvider.instantiate(TEST_UTIL.getConfiguration());
+  }
+
+  private void checkPermission(String request) throws IOException {
+    accessChecker.requirePermission(getActiveUser(), request, null, Permission.Action.ADMIN);
+  }
+
+  private User getActiveUser() throws IOException {
+    // for non-rpc handling, fallback to system user
+    Optional<User> optionalUser = RpcServer.getRequestUser();
+    if (optionalUser.isPresent()) {
+      return optionalUser.get();
+    }
+    return userProvider.getCurrent();
   }
 
   private static void setUpTableAndUserPermissions() throws Exception {
@@ -204,7 +229,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testGetRSGroupInfo() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("getRSGroupInfo");
+      checkPermission("getRSGroupInfo");
       return null;
     };
 
@@ -214,7 +239,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testGetRSGroupInfoOfTable() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("getRSGroupInfoOfTable");
+      checkPermission("getRSGroupInfoOfTable");
       return null;
     };
 
@@ -224,7 +249,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testMoveServers() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("moveServers");
+      checkPermission("moveServers");
       return null;
     };
 
@@ -234,7 +259,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testMoveTables() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("moveTables");
+      checkPermission("moveTables");
       return null;
     };
 
@@ -244,7 +269,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testAddRSGroup() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("addRSGroup");
+      checkPermission("addRSGroup");
       return null;
     };
 
@@ -254,7 +279,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testRemoveRSGroup() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("removeRSGroup");
+      checkPermission("removeRSGroup");
       return null;
     };
 
@@ -264,7 +289,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testBalanceRSGroup() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("balanceRSGroup");
+      checkPermission("balanceRSGroup");
       return null;
     };
 
@@ -274,7 +299,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testListRSGroup() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("listRSGroup");
+      checkPermission("listRSGroup");
       return null;
     };
 
@@ -284,7 +309,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testGetRSGroupInfoOfServer() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("getRSGroupInfoOfServer");
+      checkPermission("getRSGroupInfoOfServer");
       return null;
     };
 
@@ -294,7 +319,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testMoveServersAndTables() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("moveServersAndTables");
+      checkPermission("moveServersAndTables");
       return null;
     };
 
@@ -304,7 +329,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil {
   @Test
   public void testRemoveServers() throws Exception {
     AccessTestAction action = () -> {
-      rsGroupAdminEndpoint.getGroupAdminService().checkPermission("removeServers");
+      checkPermission("removeServers");
       return null;
     };
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
index 28131a9..b189697 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
@@ -73,24 +73,18 @@ public class VerifyingRSGroupAdminClient extends RSGroupAdminClient {
   }
 
   @Override
-  public RSGroupInfo getRSGroupInfo(String groupName) throws IOException {
+  public RSGroupInfo getRSGroup(String groupName) throws IOException {
     return wrapped.getRSGroupInfo(groupName);
   }
 
   @Override
-  public RSGroupInfo getRSGroupInfoOfTable(TableName tableName) throws IOException {
-    return wrapped.getRSGroupInfoOfTable(tableName);
+  public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
+    return wrapped.getRSGroup(tableName);
   }
 
   @Override
-  public void moveServers(Set<Address> servers, String targetGroup) throws IOException {
-    wrapped.moveServers(servers, targetGroup);
-    verify();
-  }
-
-  @Override
-  public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
-    wrapped.moveTables(tables, targetGroup);
+  public void moveToRSGroup(Set<Address> servers, String targetGroup) throws IOException {
+    wrapped.moveToRSGroup(servers, targetGroup);
     verify();
   }
 
@@ -111,20 +105,19 @@ public class VerifyingRSGroupAdminClient extends RSGroupAdminClient {
   }
 
   @Override
-  public RSGroupInfo getRSGroupOfServer(Address hostPort) throws IOException {
-    return wrapped.getRSGroupOfServer(hostPort);
+  public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
+    return wrapped.getRSGroup(hostPort);
   }
 
   @Override
-  public void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String targetGroup)
-          throws IOException {
-    wrapped.moveServersAndTables(servers, tables, targetGroup);
+  public void removeRSGroup(Set<Address> servers) throws IOException {
+    wrapped.removeRSGroup(servers);
     verify();
   }
 
   @Override
-  public void removeServers(Set<Address> servers) throws IOException {
-    wrapped.removeServers(servers);
+  public void setRSGroup(Set<TableName> tables, String groupName) throws IOException{
+    wrapped.setRSGroup(tables, groupName);
     verify();
   }
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index f3fdde7..52d032e 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -50,12 +50,14 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -1158,6 +1160,56 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
+  public RSGroupInfo getRSGroup(String groupName) {
+    throw new NotImplementedException("getRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public void moveToRSGroup(Set<Address> servers, String targetGroup) {
+    throw new NotImplementedException("moveToRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public void addRSGroup(String groupName) {
+    throw new NotImplementedException("addRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public void removeRSGroup(String groupName) {
+    throw new NotImplementedException("removeRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public boolean balanceRSGroup(String groupName) {
+    throw new NotImplementedException("balanceRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public List<RSGroupInfo> listRSGroups() {
+    throw new NotImplementedException("listRSGroups not supported in ThriftAdmin");
+  }
+
+  @Override
+  public RSGroupInfo getRSGroup(Address hostPort) {
+    throw new NotImplementedException("getRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public void removeRSGroup(Set<Address> servers) {
+    throw new NotImplementedException("removeRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public RSGroupInfo getRSGroup(TableName tableName) {
+    throw new NotImplementedException("getRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
+  public void setRSGroup(Set<TableName> tables, String groupName) {
+    throw new NotImplementedException("setRSGroup not supported in ThriftAdmin");
+  }
+
+  @Override
   public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
     return splitRegionAsync(regionName, null);
   }