You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/07/19 23:21:56 UTC

[20/22] hbase git commit: HBASE-17758 [RSGROUP] Add shell command to move servers and tables at the same time (Guangxu Cheng)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index e71470e..ee30e15 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -68,6 +68,8 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.GetRSGroupI
 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.ListRSGroupInfosResponse;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersAndTablesRequest;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersAndTablesResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveServersResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.MoveTablesRequest;
@@ -206,6 +208,26 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
   }
 
   @Override
+  public void moveServersAndTables(RpcController controller, MoveServersAndTablesRequest request,
+      RpcCallback<MoveServersAndTablesResponse> done) {
+    MoveServersAndTablesResponse.Builder builder = MoveServersAndTablesResponse.newBuilder();
+    try {
+      Set<HostAndPort> hostPorts = Sets.newHashSet();
+      for (HBaseProtos.ServerName el : request.getServersList()) {
+        hostPorts.add(HostAndPort.fromParts(el.getHostName(), el.getPort()));
+      }
+      Set<TableName> tables = new HashSet<>(request.getTableNameList().size());
+      for (HBaseProtos.TableName tableName : request.getTableNameList()) {
+        tables.add(ProtobufUtil.toTableName(tableName));
+      }
+      groupAdminServer.moveServersAndTables(hostPorts, tables, request.getTargetGroup());
+    } catch (IOException e) {
+      ResponseConverter.setControllerException(controller, e);
+    }
+    done.run(builder.build());
+  }
+
+  @Override
   public void addRSGroup(RpcController controller,
                        AddRSGroupRequest request,
                        RpcCallback<AddRSGroupResponse> done) {
@@ -953,6 +975,16 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
   }
 
   @Override
+  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
   public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
                             String name) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index 1069ac0..863b71e 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -501,6 +501,19 @@ public class RSGroupAdminServer extends RSGroupAdmin {
   }
 
   @Override
+  public void moveServersAndTables(Set<HostAndPort> servers, Set<TableName> tables,
+      String targetGroup) throws IOException {
+    if (servers == null || servers.isEmpty() ) {
+      throw new ConstraintException("The list of servers to move cannot be null or empty.");
+    }
+    if (tables == null || tables.isEmpty()) {
+      throw new ConstraintException("The list of tables to move cannot be null or empty.");
+    }
+    moveServers(servers, targetGroup);
+    moveTables(tables, targetGroup);
+  }
+
+  @Override
   public void close() throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index 5b5563e..e11cb57 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -107,6 +107,16 @@ public interface RSGroupInfoManager {
   void moveTables(Set<TableName> tableNames, String groupName) throws IOException;
 
   /**
+   * Move servers and tables to a new group.
+   * @param servers list of servers, must be part of the same group
+   * @param tables set of tables to move
+   * @param srcGroup groupName being moved from
+   * @param dstGroup groupName being moved to
+   */
+  void moveServersAndTables(Set<HostAndPort> servers, Set<TableName> tables,
+      String srcGroup, String dstGroup) throws IOException;
+
+  /**
    * List the groups
    *
    * @return list of RSGroupInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 2b360e0..1d33cc3 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -780,4 +780,28 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene
     }
   }
 
+  @Override
+  public void moveServersAndTables(Set<HostAndPort> servers, Set<TableName> tables, String srcGroup,
+      String dstGroup) throws IOException {
+    //get server's group
+    RSGroupInfo srcGroupInfo = getRSGroup(srcGroup);
+    RSGroupInfo dstGroupInfo = getRSGroup(dstGroup);
+
+    //move servers
+    for (HostAndPort el: servers) {
+      srcGroupInfo.removeServer(el);
+      dstGroupInfo.addServer(el);
+    }
+    //move tables
+    for(TableName tableName: tables) {
+      srcGroupInfo.removeTable(tableName);
+      dstGroupInfo.addTable(tableName);
+    }
+
+    //flush changed groupinfo
+    Map<String,RSGroupInfo> newGroupMap = Maps.newHashMap(rsGroupMap);
+    newGroupMap.put(srcGroupInfo.getName(), srcGroupInfo);
+    newGroupMap.put(dstGroupInfo.getName(), dstGroupInfo);
+    flushConfig(newGroupMap);    
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 52f576d..3fc12fa 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -701,4 +701,115 @@ public abstract class TestRSGroupsBase {
     assertTrue(newGroupTables.contains(tableNameA));
     assertTrue(newGroupTables.contains(tableNameB));
   }
+
+  @Test
+  public void testMoveServersAndTables() throws Exception {
+    final TableName tableName = TableName.valueOf(tablePrefix + "_testMoveServersAndTables");
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, getGroupName("testMoveServersAndTables"), 1);
+
+    //create table
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    TEST_UTIL.createMultiRegionTable(tableName, familyNameBytes, 5);
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        List<String> regions = getTableRegionMap().get(tableName);
+        if (regions == null)
+          return false;
+        return getTableRegionMap().get(tableName).size() >= 5;
+      }
+    });
+
+    //get server which is not a member of new group
+    ServerName targetServer = null;
+    for(ServerName server : admin.getClusterStatus().getServers()) {
+      if(!newGroup.containsServer(server.getHostPort()) && 
+           !rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getHostPort())) {
+        targetServer = server;
+        break;
+      }
+    }
+
+    LOG.debug("Print group info : " + rsGroupAdmin.listRSGroups());
+    int oldDefaultGroupServerSize =
+            rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size();
+    int oldDefaultGroupTableSize =
+            rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables().size();
+
+    //test fail bogus server move
+    try {
+      rsGroupAdmin.moveServersAndTables(Sets.newHashSet(HostAndPort.fromString("foo:9999")),
+              Sets.newHashSet(tableName), newGroup.getName());
+      fail("Bogus servers shouldn't have been successfully moved.");
+    } catch(IOException ex) {
+    }
+
+    //test fail server move
+    try {
+      rsGroupAdmin.moveServersAndTables(Sets.newHashSet(targetServer.getHostPort()),
+              Sets.newHashSet(tableName), RSGroupInfo.DEFAULT_GROUP);
+      fail("servers shouldn't have been successfully moved.");
+    } catch(IOException ex) {
+    }
+
+    //verify default group info
+    Assert.assertEquals(oldDefaultGroupServerSize,
+            rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size());
+    Assert.assertEquals(oldDefaultGroupTableSize,
+            rsGroupAdmin.getRSGroupInfo(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());
+
+    //get all region to move targetServer
+    List<String> regionList = getTableRegionMap().get(tableName);
+    for(String region : regionList) {
+      // Lets move this region to the targetServer
+      admin.move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(region))),
+              Bytes.toBytes(targetServer.getServerName()));
+    }
+
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return getTableRegionMap().get(tableName) != null &&
+                getTableRegionMap().get(tableName).size() == 5 &&
+                getTableServerRegionMap().get(tableName).size() == 1 &&
+                admin.getClusterStatus().getRegionsInTransition().size() < 1;
+      }
+    });
+
+    //verify that all region move to targetServer
+    Assert.assertNotNull(getTableServerRegionMap().get(tableName));
+    Assert.assertNotNull(getTableServerRegionMap().get(tableName).get(targetServer));
+    Assert.assertEquals(5, getTableServerRegionMap().get(tableName).get(targetServer).size());
+
+    //move targetServer and table to newGroup
+    LOG.info("moving server and table to newGroup");
+    rsGroupAdmin.moveServersAndTables(Sets.newHashSet(targetServer.getHostPort()),
+            Sets.newHashSet(tableName), newGroup.getName());
+
+    //verify group change
+    Assert.assertEquals(newGroup.getName(),
+            rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+
+    //verify servers' not exist in old group
+    Set<HostAndPort> defaultServers = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers();
+    assertFalse(defaultServers.contains(targetServer.getHostPort()));
+
+    //verify servers' exist in new group
+    Set<HostAndPort> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    assertTrue(newGroupServers.contains(targetServer.getHostPort()));
+
+    //verify tables' not exist in old group
+    Set<TableName> defaultTables = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getTables();
+    assertFalse(defaultTables.contains(tableName));
+
+    //verify tables' exist in new group
+    Set<TableName> newGroupTables = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables();
+    assertTrue(newGroupTables.contains(tableName));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
index d1f4898..87daaf2 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java
@@ -144,6 +144,13 @@ public class VerifyingRSGroupAdminClient extends RSGroupAdmin {
   }
 
   @Override
+  public void moveServersAndTables(Set<HostAndPort> servers, Set<TableName> tables,
+      String targetGroup) throws IOException {
+    wrapped.moveServersAndTables(servers, tables, targetGroup);
+    verify();
+  }
+
+  @Override
   public void close() throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
index 213ad24..3fa9540 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java
@@ -606,6 +606,16 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
   }
 
   @Override
+  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
   public void postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
       throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
index e6770bb..8cee711 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
@@ -605,6 +605,16 @@ public class BaseMasterObserver implements MasterObserver {
   }
 
   @Override
+  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
   public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
       throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 620ce0f..3d8fbca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -1053,7 +1053,6 @@ public interface MasterObserver extends Coprocessor {
    * @param ctx the environment to interact with the framework and master
    * @param servers set of servers to move
    * @param targetGroup destination group
-   * @throws IOException on failure
    */
   void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
                       Set<HostAndPort> servers, String targetGroup) throws IOException;
@@ -1089,6 +1088,25 @@ public interface MasterObserver extends Coprocessor {
                       Set<TableName> tables, String targetGroup) throws IOException;
 
   /**
+   * Called before servers are moved to target region server group
+   * @param ctx the environment to interact with the framework and master
+   * @param servers set of servers to move
+   * @param targetGroup destination group
+   * @throws IOException on failure
+   */
+  void preMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException;
+
+  /**
+   * Called after servers are moved to target region server group
+   * @param ctx the environment to interact with the framework and master
+   * @param servers set of servers to move
+   * @param targetGroup name of group
+   */
+  void postMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException;
+
+  /**
    * Called before a new region server group is added
    * @param ctx the environment to interact with the framework and master
    * @param name group name

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
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 43dbbdb..6757e7a 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
@@ -1231,6 +1231,32 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preMoveServersAndTables(final Set<HostAndPort> servers, final Set<TableName> tables,
+      final String targetGroup) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+                       ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
+          oserver.preMoveServersAndTables(ctx, servers, tables, targetGroup);
+        }
+      }
+    });
+  }
+
+  public void postMoveServersAndTables(final Set<HostAndPort> servers, final Set<TableName> tables,
+      final String targetGroup) throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+                       ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        if(((MasterEnvironment)ctx.getEnvironment()).supportGroupCPs) {
+          oserver.postMoveServersAndTables(ctx, servers, tables, targetGroup);
+        }
+      }
+    });
+  }
+
   public void preAddRSGroup(final String name)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
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 ae09676..3b43379 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
@@ -2691,6 +2691,12 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
+  public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+    requirePermission("moveServersAndTables", Action.ADMIN);
+  }
+
+  @Override
   public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              Set<HostAndPort> servers, String targetGroup) throws IOException {
     requirePermission("moveServers", Action.ADMIN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 38087d8..224b775 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -1282,6 +1282,16 @@ public class TestMasterObserver {
     }
 
     @Override
+    public void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+    }
+
+    @Override
+    public void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        Set<HostAndPort> servers, Set<TableName> tables, String targetGroup) throws IOException {
+    }
+
+    @Override
     public void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
         throws IOException {
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb b/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
index 51a4efb..6fd29bf 100644
--- a/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/rsgroup_admin.rb
@@ -146,5 +146,19 @@ module Hbase
       res
     end
 
+    #--------------------------------------------------------------------------
+    # move server and table to a group
+    def move_servers_tables(dest, *args)
+      servers = java.util.HashSet.new
+      tables = java.util.HashSet.new;
+      args[0].each do |s|
+        servers.add(com.google.common.net.HostAndPort.fromString(s))
+      end
+      args[1].each do |t|
+        tables.add(org.apache.hadoop.hbase.TableName.valueOf(t))
+      end
+      @admin.moveServersAndTables(servers, tables, dest)
+    end
+
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 4479eae..39bd3cb 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -460,6 +460,7 @@ Shell.load_command_group(
     balance_rsgroup
     move_servers_rsgroup
     move_tables_rsgroup
+    move_servers_tables_rsgroup
     get_server_rsgroup
     get_table_rsgroup
   ]

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23322ea/hbase-shell/src/main/ruby/shell/commands/move_servers_tables_rsgroup.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/move_servers_tables_rsgroup.rb b/hbase-shell/src/main/ruby/shell/commands/move_servers_tables_rsgroup.rb
new file mode 100644
index 0000000..5337141
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/move_servers_tables_rsgroup.rb
@@ -0,0 +1,37 @@
+# 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.
+#
+
+module Shell
+  module Commands
+    class MoveServersTablesRsgroup < Command
+      def help
+        return <<-EOF
+Reassign RegionServers and Tables from one group to another.
+
+Example:
+
+  hbase> move_servers_tables_rsgroup 'dest',['server1:port','server2:port'],['table1','table2']
+
+EOF
+      end
+
+      def command(dest, servers, tables)
+        rsgroup_admin.move_servers_tables(dest, servers, tables)
+      end
+    end
+  end
+end