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 2018/10/05 00:20:15 UTC

[9/9] hbase git commit: HBASE-21265 Split up TestRSGroups

HBASE-21265 Split up TestRSGroups

Signed-off-by: Ted Yu <ty...@apache.org>


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

Branch: refs/heads/branch-1.4
Commit: 54d89e1a2b2beda1b9c2a43003ea4bcc96d22e7f
Parents: 7242650
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Oct 4 16:37:44 2018 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Oct 4 16:37:44 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/rsgroup/TestEnableRSGroup.java |  98 --
 .../hbase/rsgroup/TestEnableRSGroups.java       |  98 ++
 .../hadoop/hbase/rsgroup/TestRSGroups.java      | 521 ----------
 .../hbase/rsgroup/TestRSGroupsAdmin1.java       | 434 ++++++++
 .../hbase/rsgroup/TestRSGroupsAdmin2.java       | 417 ++++++++
 .../hbase/rsgroup/TestRSGroupsBalance.java      | 183 ++++
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  | 987 ++++---------------
 .../hbase/rsgroup/TestRSGroupsBasics.java       | 215 ++++
 .../hbase/rsgroup/TestRSGroupsKillRS.java       | 141 +++
 9 files changed, 1697 insertions(+), 1397 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroup.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroup.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroup.java
deleted file mode 100644
index 45dc5d3..0000000
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroup.java
+++ /dev/null
@@ -1,98 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test enable RSGroup
- */
-@Category({ MediumTests.class })
-public class TestEnableRSGroup {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(TestEnableRSGroup.class);
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf = TEST_UTIL.getConfiguration();
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    TEST_UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testEnableRSGroup() throws IOException, InterruptedException {
-    TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
-    LOG.info("stopped master...");
-    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, RSGroupAdminEndpoint.class.getName());
-    conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, RSGroupBasedLoadBalancer.class.getName());
-    TEST_UTIL.getMiniHBaseCluster().setConf(conf);
-
-    TEST_UTIL.getMiniHBaseCluster().startMaster();
-    TEST_UTIL.getMiniHBaseCluster().waitForActiveAndReadyMaster(60000);
-    LOG.info("started master...");
-
-    // check if master started successfully
-    Waiter.waitFor(TEST_UTIL.getConfiguration(), 60000, new ExplainingPredicate<IOException>() {
-      @Override
-      public boolean evaluate() throws IOException {
-        return TEST_UTIL.getMiniHBaseCluster().getMaster() != null;
-      }
-
-      @Override
-      public String explainFailure() throws IOException {
-        return "Master failed to start up";
-      }
-    });
-
-    // wait RSGroupBasedLoadBalancer online
-    Waiter.waitFor(TEST_UTIL.getConfiguration(), 60000, new ExplainingPredicate<IOException>() {
-      @Override
-      public boolean evaluate() throws IOException {
-        RSGroupBasedLoadBalancer loadBalancer =
-            (RSGroupBasedLoadBalancer) TEST_UTIL.getMiniHBaseCluster().getMaster().getLoadBalancer();
-        return loadBalancer != null && loadBalancer.isOnline();
-      }
-
-      @Override
-      public String explainFailure() throws IOException {
-        return "RSGroupBasedLoadBalancer failed to come online";
-      }
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroups.java
new file mode 100644
index 0000000..27eb6e1
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestEnableRSGroups.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.rsgroup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test enable RSGroup
+ */
+@Category({ MediumTests.class })
+public class TestEnableRSGroups {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(TestEnableRSGroups.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = TEST_UTIL.getConfiguration();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testEnableRSGroups() throws IOException, InterruptedException {
+    TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
+    LOG.info("stopped master...");
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, RSGroupAdminEndpoint.class.getName());
+    conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, RSGroupBasedLoadBalancer.class.getName());
+    TEST_UTIL.getMiniHBaseCluster().setConf(conf);
+
+    TEST_UTIL.getMiniHBaseCluster().startMaster();
+    TEST_UTIL.getMiniHBaseCluster().waitForActiveAndReadyMaster(60000);
+    LOG.info("started master...");
+
+    // check if master started successfully
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 60000, new ExplainingPredicate<IOException>() {
+      @Override
+      public boolean evaluate() throws IOException {
+        return TEST_UTIL.getMiniHBaseCluster().getMaster() != null;
+      }
+
+      @Override
+      public String explainFailure() throws IOException {
+        return "Master failed to start up";
+      }
+    });
+
+    // wait RSGroupBasedLoadBalancer online
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 60000, new ExplainingPredicate<IOException>() {
+      @Override
+      public boolean evaluate() throws IOException {
+        RSGroupBasedLoadBalancer loadBalancer =
+            (RSGroupBasedLoadBalancer) TEST_UTIL.getMiniHBaseCluster().getMaster().getLoadBalancer();
+        return loadBalancer != null && loadBalancer.isOnline();
+      }
+
+      @Override
+      public String explainFailure() throws IOException {
+        return "RSGroupBasedLoadBalancer failed to come online";
+      }
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
deleted file mode 100644
index 9831ef5..0000000
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ /dev/null
@@ -1,521 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * 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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.Predicate;
-import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-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.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-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;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Sets;
-
-@Category({MediumTests.class})
-public class TestRSGroups extends TestRSGroupsBase {
-  protected static final Log LOG = LogFactory.getLog(TestRSGroups.class);
-  private static HMaster master;
-  private static boolean init = false;
-  private static RSGroupAdminEndpoint RSGroupAdminEndpoint;
-  private static CPMasterObserver observer;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    TEST_UTIL = new HBaseTestingUtility();
-    TEST_UTIL.getConfiguration().setFloat(
-            "hbase.master.balancer.stochastic.tableSkewCost", 6000);
-    TEST_UTIL.getConfiguration().set(
-        HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
-        RSGroupBasedLoadBalancer.class.getName());
-    TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
-        RSGroupAdminEndpoint.class.getName() + "," + CPMasterObserver.class.getName());
-    TEST_UTIL.getConfiguration().setBoolean(
-        HConstants.ZOOKEEPER_USEMULTI,
-        true);
-    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();
-  }
-
-  private static void initialize() throws Exception {
-    admin = TEST_UTIL.getHBaseAdmin();
-    cluster = TEST_UTIL.getHBaseCluster();
-    master = ((MiniHBaseCluster)cluster).getMaster();
-
-    //wait for balancer to come online
-    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        return master.isInitialized() &&
-            ((RSGroupBasedLoadBalancer) master.getLoadBalancer()).isOnline();
-      }
-    });
-    admin.setBalancerRunning(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 =
-        host.findCoprocessors(RSGroupAdminEndpoint.class).get(0);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void beforeMethod() throws Exception {
-    if(!init) {
-      init = true;
-      afterMethod();
-    }
-    observer.resetFlags();
-  }
-
-  @After
-  public void afterMethod() throws Exception {
-    deleteTableIfNecessary();
-    deleteNamespaceIfNecessary();
-    deleteGroups();
-
-    int missing = NUM_SLAVES_BASE - getNumServers();
-    LOG.info("Restoring servers: "+missing);
-    for(int i=0; i<missing; i++) {
-      ((MiniHBaseCluster)cluster).startRegionServer();
-    }
-
-    rsGroupAdmin.addRSGroup("master");
-    ServerName masterServerName =
-        ((MiniHBaseCluster)cluster).getMaster().getServerName();
-
-    try {
-      rsGroupAdmin.moveServers(
-          Sets.newHashSet(masterServerName.getAddress()),
-          "master");
-    } catch (Exception ex) {
-      // ignore
-    }
-
-    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        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()
-            == NUM_SLAVES_BASE;
-      }
-    });
-  }
-
-  @Test
-  public void testBasicStartUp() throws IOException {
-    RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
-    assertEquals(4, defaultInfo.getServers().size());
-    // Assignment of root and meta regions.
-    int count = master.getAssignmentManager().getRegionStates().getRegionAssignments().size();
-    //3 meta,namespace, group
-    assertEquals(3, count);
-  }
-
-  @Test
-  public void testNamespaceCreateAndAssign() throws Exception {
-    LOG.info("testNamespaceCreateAndAssign");
-    String nsName = tablePrefix+"_foo";
-    final TableName tableName = TableName.valueOf(nsName, tablePrefix + "_testCreateAndAssign");
-    RSGroupInfo appInfo = addGroup(rsGroupAdmin, "appInfo", 1);
-    admin.createNamespace(NamespaceDescriptor.create(nsName)
-        .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "appInfo").build());
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor("f"));
-    admin.createTable(desc);
-    //wait for created table to be assigned
-    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        return getTableRegionMap().get(desc.getTableName()) != null;
-      }
-    });
-    ServerName targetServer =
-        ServerName.parseServerName(appInfo.getServers().iterator().next().toString());
-    AdminProtos.AdminService.BlockingInterface rs = admin.getConnection().getAdmin(targetServer);
-    //verify it was assigned to the right group
-    Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(rs).size());
-  }
-
-  @Test
-  public void testDefaultNamespaceCreateAndAssign() throws Exception {
-    LOG.info("testDefaultNamespaceCreateAndAssign");
-    final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
-    admin.modifyNamespace(NamespaceDescriptor.create("default")
-        .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "default").build());
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor("f"));
-    admin.createTable(desc);
-    //wait for created table to be assigned
-    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        return getTableRegionMap().get(desc.getTableName()) != null;
-      }
-    });
-  }
-
-  @Test
-  public void testNamespaceConstraint() throws Exception {
-    String nsName = tablePrefix+"_foo";
-    String groupName = tablePrefix+"_foo";
-    LOG.info("testNamespaceConstraint");
-    rsGroupAdmin.addRSGroup(groupName);
-    admin.createNamespace(NamespaceDescriptor.create(nsName)
-        .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, groupName)
-        .build());
-    //test removing a referenced group
-    try {
-      rsGroupAdmin.removeRSGroup(groupName);
-      fail("Expected a constraint exception");
-    } catch (IOException ex) {
-    }
-    //test modify group
-    //changing with the same name is fine
-    admin.modifyNamespace(
-        NamespaceDescriptor.create(nsName)
-          .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, groupName)
-          .build());
-    String anotherGroup = tablePrefix+"_anotherGroup";
-    rsGroupAdmin.addRSGroup(anotherGroup);
-    //test add non-existent group
-    admin.deleteNamespace(nsName);
-    rsGroupAdmin.removeRSGroup(groupName);
-    try {
-      admin.createNamespace(NamespaceDescriptor.create(nsName)
-          .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "foo")
-          .build());
-      fail("Expected a constraint exception");
-    } catch (IOException ex) {
-    }
-  }
-
-  @Test
-  public void testGroupInfoMultiAccessing() throws Exception {
-    RSGroupInfoManager manager = RSGroupAdminEndpoint.getGroupInfoManager();
-    final RSGroupInfo defaultGroup = manager.getRSGroup("default");
-    // getRSGroup updates default group's server list
-    // this process must not affect other threads iterating the list
-    Iterator<Address> it = defaultGroup.getServers().iterator();
-    manager.getRSGroup("default");
-    it.next();
-  }
-
-  public static class CPMasterObserver extends BaseMasterObserver {
-    boolean preBalanceRSGroupCalled = false;
-    boolean postBalanceRSGroupCalled = false;
-    boolean preMoveServersCalled = false;
-    boolean postMoveServersCalled = false;
-    boolean preMoveTablesCalled = false;
-    boolean postMoveTablesCalled = false;
-    boolean preAddRSGroupCalled = false;
-    boolean postAddRSGroupCalled = false;
-    boolean preRemoveRSGroupCalled = false;
-    boolean postRemoveRSGroupCalled = false;
-    boolean preRemoveServersCalled = false;
-    boolean postRemoveServersCalled = false;
-    boolean preMoveServersAndTables = false;
-    boolean postMoveServersAndTables = false;
-
-    void resetFlags() {
-      preBalanceRSGroupCalled = false;
-      postBalanceRSGroupCalled = false;
-      preMoveServersCalled = false;
-      postMoveServersCalled = false;
-      preMoveTablesCalled = false;
-      postMoveTablesCalled = false;
-      preAddRSGroupCalled = false;
-      postAddRSGroupCalled = false;
-      preRemoveRSGroupCalled = false;
-      postRemoveRSGroupCalled = false;
-      preRemoveServersCalled = false;
-      postRemoveServersCalled = false;
-      preMoveServersAndTables = false;
-      postMoveServersAndTables = false;
-    }
-
-    @Override
-    public void preMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
-      preMoveServersAndTables = true;
-    }
-    @Override
-    public void postMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
-      postMoveServersAndTables = true;
-    }
-    @Override
-    public void preRemoveServers(
-        final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers) throws IOException {
-      preRemoveServersCalled = true;
-    }
-    @Override
-    public void postRemoveServers(
-        final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers) throws IOException {
-      postRemoveServersCalled = true;
-    }
-    @Override
-    public void preRemoveRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String name) throws IOException {
-      preRemoveRSGroupCalled = true;
-    }
-    @Override
-    public void postRemoveRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String name) throws IOException {
-      postRemoveRSGroupCalled = true;
-    }
-    @Override
-    public void preAddRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String name) throws IOException {
-      preAddRSGroupCalled = true;
-    }
-    @Override
-    public void postAddRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String name) throws IOException {
-      postAddRSGroupCalled = true;
-    }
-    @Override
-    public void preMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<TableName> tables, String targetGroup) throws IOException {
-      preMoveTablesCalled = true;
-    }
-    @Override
-    public void postMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<TableName> tables, String targetGroup) throws IOException {
-      postMoveTablesCalled = true;
-    }
-    @Override
-    public void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers, String targetGroup) throws IOException {
-      preMoveServersCalled = true;
-    }
-
-    @Override
-    public void postMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        Set<Address> servers, String targetGroup) throws IOException {
-      postMoveServersCalled = true;
-    }
-    @Override
-    public void preBalanceRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String groupName) throws IOException {
-      preBalanceRSGroupCalled = true;
-    }
-    @Override
-    public void postBalanceRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        String groupName, boolean balancerRan) throws IOException {
-      postBalanceRSGroupCalled = true;
-    }
-  }
-  @Test
-  public void testMoveServersAndTables() throws Exception {
-    super.testMoveServersAndTables();
-    assertTrue(observer.preMoveServersAndTables);
-    assertTrue(observer.postMoveServersAndTables);
-  }
-  @Test
-  public void testTableMoveTruncateAndDrop() throws Exception {
-    super.testTableMoveTruncateAndDrop();
-    assertTrue(observer.preMoveTablesCalled);
-    assertTrue(observer.postMoveTablesCalled);
-  }
-
-  @Test
-  public void testRemoveServers() throws Exception {
-    super.testRemoveServers();
-    assertTrue(observer.preRemoveServersCalled);
-  }
-
-  @Test
-  public void testGroupBalance() throws Exception {
-    super.testGroupBalance();
-    assertTrue(observer.preBalanceRSGroupCalled);
-    assertTrue(observer.postBalanceRSGroupCalled);
-  }
-
-  @Test
-  public void testMisplacedRegions() throws Exception {
-    final TableName tableName = TableName.valueOf(tablePrefix+"_testMisplacedRegions");
-    LOG.info("testMisplacedRegions");
-
-    final RSGroupInfo RSGroupInfo = addGroup(rsGroupAdmin, "testMisplacedRegions", 1);
-
-    TEST_UTIL.createMultiRegionTable(tableName, new byte[]{'f'}, 15);
-    TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-
-    RSGroupAdminEndpoint.getGroupInfoManager()
-        .moveTables(Sets.newHashSet(tableName), RSGroupInfo.getName());
-
-    assertTrue(rsGroupAdmin.balanceRSGroup(RSGroupInfo.getName()));
-
-    TEST_UTIL.waitFor(60000, new Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        ServerName serverName =
-            ServerName.valueOf(RSGroupInfo.getServers().iterator().next().toString(), 1);
-        return admin.getConnection().getAdmin()
-            .getOnlineRegions(serverName).size() == 15;
-      }
-    });
-  }
-
-  @Test
-  public void testRSGroupBalancerSwitch() throws IOException {
-    //Balancer is initially off in the test, set to true and check
-    assertFalse(admin.setBalancerRunning(true, true));
-    assertTrue(admin.isBalancerEnabled());
-    //Set balancer off and check if it actually turned off
-    assertTrue(admin.setBalancerRunning(false,true));
-    assertFalse(admin.isBalancerEnabled());
-  }
-
-  @Test
-  public void testCloneSnapshot() throws Exception {
-    final TableName tableName = TableName.valueOf(tablePrefix+"_testCloneSnapshot");
-    LOG.info("testCloneSnapshot");
-
-    byte[] FAMILY = Bytes.toBytes("test");
-    String snapshotName = tableName.getNameAsString() + "_snap";
-    TableName clonedTableName = TableName.valueOf(tableName.getNameAsString() + "_clone");
-
-    // create base table
-    TEST_UTIL.createTable(tableName, FAMILY);
-
-    // create snapshot
-    admin.snapshot(snapshotName, tableName);
-
-    // clone
-    admin.cloneSnapshot(snapshotName, clonedTableName);
-  }
-
-  @Test
-  public void testRSGroupListDoesNotContainFailedTableCreation() throws Exception {
-    toggleQuotaCheckAndRestartMiniCluster(true);
-    String nsp = "np1";
-    NamespaceDescriptor nspDesc =
-        NamespaceDescriptor.create(nsp).addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "5")
-            .addConfiguration(TableNamespaceManager.KEY_MAX_TABLES, "2").build();
-    admin.createNamespace(nspDesc);
-    assertEquals(3, admin.listNamespaceDescriptors().length);
-    HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
-    HTableDescriptor tableDescOne =
-        new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
-    tableDescOne.addFamily(fam1);
-    admin.createTable(tableDescOne);
-
-    HTableDescriptor tableDescTwo =
-        new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
-    tableDescTwo.addFamily(fam1);
-    boolean constraintViolated = false;
-
-    try {
-      admin.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"),
-          6);
-      Assert.fail("Creation table should fail because of quota violation.");
-    } catch (Exception exp) {
-      assertTrue(exp instanceof IOException);
-      constraintViolated = true;
-    } finally {
-      assertTrue("Constraint not violated for table " + tableDescTwo.getTableName(),
-          constraintViolated);
-    }
-    List<RSGroupInfo> rsGroupInfoList = rsGroupAdmin.listRSGroups();
-    boolean foundTable2 = false;
-    boolean foundTable1 = false;
-    for (int i = 0; i < rsGroupInfoList.size(); i++){
-      if(rsGroupInfoList.get(i).getTables().contains(tableDescTwo.getTableName())){
-        foundTable2 = true;
-      }
-      if(rsGroupInfoList.get(i).getTables().contains(tableDescOne.getTableName())){
-        foundTable1 = true;
-      }
-    }
-    assertFalse("Found table2 in rsgroup list.", foundTable2);
-    assertTrue("Did not find table1 in rsgroup list", foundTable1);
-
-    TEST_UTIL.deleteTable(tableDescOne.getTableName());
-    admin.deleteNamespace(nspDesc.getName());
-    toggleQuotaCheckAndRestartMiniCluster(false);
-
-  }
-
-  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();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
new file mode 100644
index 0000000..1bfc06b
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin1.java
@@ -0,0 +1,434 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+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;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Sets;
+
+@Category({MediumTests.class})
+public class TestRSGroupsAdmin1 extends TestRSGroupsBase {
+  protected static final Log LOG = LogFactory.getLog(TestRSGroupsAdmin1.class);
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    setUpTestBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    tearDownAfterClass();
+  }
+
+  @Before
+  public void beforeMethod() throws Exception {
+    setUpBeforeMethod();
+  }
+
+  @After
+  public void afterMethod() throws Exception {
+    tearDownAfterMethod();
+  }
+
+  @Test
+  public void testValidGroupNames() throws IOException {
+    String[] badNames = {"foo*","foo@","-"};
+    String[] goodNames = {"foo_123"};
+
+    for(String entry: badNames) {
+      try {
+        rsGroupAdmin.addRSGroup(entry);
+        fail("Expected a constraint exception for: "+entry);
+      } catch(ConstraintException ex) {
+        //expected
+      }
+    }
+
+    for(String entry: goodNames) {
+      rsGroupAdmin.addRSGroup(entry);
+    }
+  }
+
+  @Test
+  public void testBogusArgs() throws Exception {
+    assertNull(rsGroupAdmin.getRSGroupInfoOfTable(TableName.valueOf("nonexistent")));
+    assertNull(rsGroupAdmin.getRSGroupOfServer(Address.fromParts("bogus",123)));
+    assertNull(rsGroupAdmin.getRSGroupInfo("bogus"));
+
+    try {
+      rsGroupAdmin.removeRSGroup("bogus");
+      fail("Expected removing bogus group to fail");
+    } catch(ConstraintException ex) {
+      //expected
+    }
+
+    try {
+      rsGroupAdmin.moveTables(Sets.newHashSet(TableName.valueOf("bogustable")), "bogus");
+      fail("Expected move with bogus group to fail");
+    } catch(ConstraintException ex) {
+      //expected
+    }
+
+    try {
+      rsGroupAdmin.moveServers(Sets.newHashSet(Address.fromParts("bogus",123)), "bogus");
+      fail("Expected move with bogus group to fail");
+    } catch(ConstraintException ex) {
+      //expected
+    }
+
+    try {
+      rsGroupAdmin.balanceRSGroup("bogus");
+      fail("Expected move with bogus group to fail");
+    } catch(ConstraintException ex) {
+      //expected
+    }
+  }
+
+  @Test
+  public void testNamespaceConstraint() throws Exception {
+    String nsName = tablePrefix+"_foo";
+    String groupName = tablePrefix+"_foo";
+    LOG.info("testNamespaceConstraint");
+    rsGroupAdmin.addRSGroup(groupName);
+    admin.createNamespace(NamespaceDescriptor.create(nsName)
+        .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, groupName)
+        .build());
+    //test removing a referenced group
+    try {
+      rsGroupAdmin.removeRSGroup(groupName);
+      fail("Expected a constraint exception");
+    } catch (IOException ex) {
+    }
+    //test modify group
+    //changing with the same name is fine
+    admin.modifyNamespace(
+        NamespaceDescriptor.create(nsName)
+          .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, groupName)
+          .build());
+    String anotherGroup = tablePrefix+"_anotherGroup";
+    rsGroupAdmin.addRSGroup(anotherGroup);
+    //test add non-existent group
+    admin.deleteNamespace(nsName);
+    rsGroupAdmin.removeRSGroup(groupName);
+    try {
+      admin.createNamespace(NamespaceDescriptor.create(nsName)
+          .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "foo")
+          .build());
+      fail("Expected a constraint exception");
+    } catch (IOException ex) {
+    }
+  }
+
+  @Test
+  public void testGroupInfoMultiAccessing() throws Exception {
+    RSGroupInfoManager manager = RSGroupAdminEndpoint.getGroupInfoManager();
+    final RSGroupInfo defaultGroup = manager.getRSGroup("default");
+    // getRSGroup updates default group's server list
+    // this process must not affect other threads iterating the list
+    Iterator<Address> it = defaultGroup.getServers().iterator();
+    manager.getRSGroup("default");
+    it.next();
+  }
+
+  @Test
+  public void testFailRemoveGroup() throws IOException, InterruptedException {
+    LOG.info("testFailRemoveGroup");
+
+    int initNumGroups = rsGroupAdmin.listRSGroups().size();
+    addGroup(rsGroupAdmin, "bar", 3);
+    TableName tableName = TableName.valueOf(tablePrefix+"_my_table");
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
+    rsGroupAdmin.moveTables(Sets.newHashSet(tableName), "bar");
+    RSGroupInfo barGroup = rsGroupAdmin.getRSGroupInfo("bar");
+    //group is not empty therefore it should fail
+    try {
+      rsGroupAdmin.removeRSGroup(barGroup.getName());
+      fail("Expected remove group to fail");
+    } catch(IOException e) {
+    }
+    //group cannot lose all it's servers therefore it should fail
+    try {
+      rsGroupAdmin.moveServers(barGroup.getServers(), RSGroupInfo.DEFAULT_GROUP);
+      fail("Expected move servers to fail");
+    } catch(IOException e) {
+    }
+
+    rsGroupAdmin.moveTables(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.removeRSGroup(barGroup.getName());
+
+    Assert.assertEquals(initNumGroups, rsGroupAdmin.listRSGroups().size());
+  }
+
+  @Test
+  public void testMultiTableMove() throws Exception {
+    LOG.info("testMultiTableMove");
+
+    final TableName tableNameA = TableName.valueOf(tablePrefix + "_testMultiTableMoveA");
+    final TableName tableNameB = TableName.valueOf(tablePrefix + "_testMultiTableMoveB");
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    String newGroupName = getGroupName("testMultiTableMove");
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, newGroupName, 1);
+
+    TEST_UTIL.createTable(tableNameA, familyNameBytes);
+    TEST_UTIL.createTable(tableNameB, familyNameBytes);
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        List<String> regionsA = getTableRegionMap().get(tableNameA);
+        if (regionsA == null) {
+          return false;
+        }
+        List<String> regionsB = getTableRegionMap().get(tableNameB);
+        if (regionsB == null) {
+          return false;
+        }
+        return getTableRegionMap().get(tableNameA).size() >= 1
+                && getTableRegionMap().get(tableNameB).size() >= 1;
+      }
+    });
+
+    RSGroupInfo tableGrpA = rsGroupAdmin.getRSGroupInfoOfTable(tableNameA);
+    assertTrue(tableGrpA.getName().equals(RSGroupInfo.DEFAULT_GROUP));
+
+    RSGroupInfo tableGrpB = rsGroupAdmin.getRSGroupInfoOfTable(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());
+
+    //verify group change
+    Assert.assertEquals(newGroup.getName(),
+            rsGroupAdmin.getRSGroupInfoOfTable(tableNameA).getName());
+
+    Assert.assertEquals(newGroup.getName(),
+            rsGroupAdmin.getRSGroupInfoOfTable(tableNameB).getName());
+
+    //verify tables' not exist in old group
+    Set<TableName> DefaultTables =
+        rsGroupAdmin.getRSGroupInfo(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();
+    assertTrue(newGroupTables.contains(tableNameA));
+    assertTrue(newGroupTables.contains(tableNameB));
+  }
+
+  @Test
+  public void testTableMoveTruncateAndDrop() throws Exception {
+    LOG.info("testTableMove");
+
+    final TableName tableName = TableName.valueOf(tablePrefix + "_testTableMoveAndDrop");
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    String newGroupName = getGroupName("testTableMove");
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, newGroupName, 2);
+
+    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;
+      }
+    });
+
+    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroupInfoOfTable(tableName);
+    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());
+
+    //verify group change
+    Assert.assertEquals(newGroup.getName(),
+        rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Map<ServerName, List<String>> serverMap = getTableServerRegionMap().get(tableName);
+        int count = 0;
+        if (serverMap != null) {
+          for (ServerName rs : serverMap.keySet()) {
+            if (newGroup.containsServer(rs.getAddress())) {
+              count += serverMap.get(rs).size();
+            }
+          }
+        }
+        return count == 5;
+      }
+    });
+
+    //test truncate
+    admin.disableTable(tableName);
+    admin.truncateTable(tableName, true);
+    Assert.assertEquals(1, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().size());
+    Assert.assertEquals(tableName, rsGroupAdmin.getRSGroupInfo(
+        newGroup.getName()).getTables().first());
+
+    //verify removed table is removed from group
+    TEST_UTIL.deleteTable(tableName);
+    Assert.assertEquals(0, rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getTables().size());
+
+    assertTrue(observer.preMoveTablesCalled);
+    assertTrue(observer.postMoveTablesCalled);
+  }
+
+  @Test
+  public void testDisabledTableMove() throws Exception {
+    final TableName tableName = TableName.valueOf(tablePrefix + "_testDisabledTableMove");
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    String newGroupName = getGroupName("testDisabledTableMove");
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, newGroupName, 2);
+
+    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;
+      }
+    });
+
+    RSGroupInfo tableGrp = rsGroupAdmin.getRSGroupInfoOfTable(tableName);
+    assertTrue(tableGrp.getName().equals(RSGroupInfo.DEFAULT_GROUP));
+
+    //test disable table
+    admin.disableTable(tableName);
+
+    //change table's group
+    LOG.info("Moving table "+ tableName + " to " + newGroup.getName());
+    rsGroupAdmin.moveTables(Sets.newHashSet(tableName), newGroup.getName());
+
+    //verify group change
+    Assert.assertEquals(newGroup.getName(),
+        rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+  }
+
+  @Test
+  public void testRSGroupListDoesNotContainFailedTableCreation() throws Exception {
+    toggleQuotaCheckAndRestartMiniCluster(true);
+    String nsp = "np1";
+    NamespaceDescriptor nspDesc =
+        NamespaceDescriptor.create(nsp).addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "5")
+            .addConfiguration(TableNamespaceManager.KEY_MAX_TABLES, "2").build();
+    admin.createNamespace(nspDesc);
+    assertEquals(3, admin.listNamespaceDescriptors().length);
+    HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
+    HTableDescriptor tableDescOne =
+        new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table1"));
+    tableDescOne.addFamily(fam1);
+    admin.createTable(tableDescOne);
+
+    HTableDescriptor tableDescTwo =
+        new HTableDescriptor(TableName.valueOf(nsp + TableName.NAMESPACE_DELIM + "table2"));
+    tableDescTwo.addFamily(fam1);
+    boolean constraintViolated = false;
+
+    try {
+      admin.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"),
+          6);
+      Assert.fail("Creation table should fail because of quota violation.");
+    } catch (Exception exp) {
+      assertTrue(exp instanceof IOException);
+      constraintViolated = true;
+    } finally {
+      assertTrue("Constraint not violated for table " + tableDescTwo.getTableName(),
+          constraintViolated);
+    }
+    List<RSGroupInfo> rsGroupInfoList = rsGroupAdmin.listRSGroups();
+    boolean foundTable2 = false;
+    boolean foundTable1 = false;
+    for (int i = 0; i < rsGroupInfoList.size(); i++){
+      if(rsGroupInfoList.get(i).getTables().contains(tableDescTwo.getTableName())){
+        foundTable2 = true;
+      }
+      if(rsGroupInfoList.get(i).getTables().contains(tableDescOne.getTableName())){
+        foundTable1 = true;
+      }
+    }
+    assertFalse("Found table2 in rsgroup list.", foundTable2);
+    assertTrue("Did not find table1 in rsgroup list", foundTable1);
+
+    TEST_UTIL.deleteTable(tableDescOne.getTableName());
+    admin.deleteNamespace(nspDesc.getName());
+    toggleQuotaCheckAndRestartMiniCluster(false);
+
+  }
+
+  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();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
new file mode 100644
index 0000000..b026daf
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java
@@ -0,0 +1,417 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.net.Address;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MediumTests.class})
+public class TestRSGroupsAdmin2 extends TestRSGroupsBase {
+  protected static final Log LOG = LogFactory.getLog(TestRSGroupsAdmin2.class);
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    setUpTestBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    tearDownAfterClass();
+  }
+
+  @Before
+  public void beforeMethod() throws Exception {
+    setUpBeforeMethod();
+  }
+
+  @After
+  public void afterMethod() throws Exception {
+    tearDownAfterMethod();
+  }
+
+
+  @Test
+  public void testRegionMove() throws Exception {
+    LOG.info("testRegionMove");
+
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, getGroupName("testRegionMove"), 1);
+    final TableName tableName = TableName.valueOf(tablePrefix + rand.nextInt());
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    // All the regions created below will be assigned to the default group.
+    TEST_UTIL.createMultiRegionTable(tableName, familyNameBytes, 6);
+    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() >= 6;
+      }
+    });
+
+    //get target region to move
+    Map<ServerName,List<String>> assignMap =
+        getTableServerRegionMap().get(tableName);
+    String targetRegion = null;
+    for(ServerName server : assignMap.keySet()) {
+      targetRegion = assignMap.get(server).size() > 0 ? assignMap.get(server).get(0) : null;
+      if(targetRegion != null) {
+        break;
+      }
+    }
+    //get server which is not a member of new group
+    ServerName targetServer = null;
+    for(ServerName server : admin.getClusterStatus().getServers()) {
+      if(!newGroup.containsServer(server.getAddress())) {
+        targetServer = server;
+        break;
+      }
+    }
+    assertNotNull(targetServer);
+
+    final AdminProtos.AdminService.BlockingInterface targetRS =
+        admin.getConnection().getAdmin(targetServer);
+
+    //move target server to group
+    rsGroupAdmin.moveServers(Sets.newHashSet(targetServer.getAddress()),
+        newGroup.getName());
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return ProtobufUtil.getOnlineRegions(targetRS).size() <= 0;
+      }
+    });
+
+    // Lets move this region to the new group.
+    TEST_UTIL.getHBaseAdmin().move(Bytes.toBytes(HRegionInfo.encodeRegionName(Bytes.toBytes(targetRegion))),
+        Bytes.toBytes(targetServer.getServerName()));
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        List<String> regions = getTableRegionMap().get(tableName);
+        Set<RegionState> regionsInTransition = admin.getClusterStatus().getRegionsInTransition();
+        return (regions != null && getTableRegionMap().get(tableName).size() == 6) &&
+           ( regionsInTransition == null || regionsInTransition.size() < 1);
+      }
+    });
+
+    //verify that targetServer didn't open it
+    for (HRegionInfo region: ProtobufUtil.getOnlineRegions(targetRS)) {
+      if (targetRegion.equals(region.getRegionNameAsString())) {
+        fail("Target server opened region");
+      }
+    }
+  }
+
+  @Test
+  public void testRegionServerMove() throws IOException,
+      InterruptedException {
+    LOG.info("testRegionServerMove");
+
+    int initNumGroups = rsGroupAdmin.listRSGroups().size();
+    RSGroupInfo appInfo = addGroup(rsGroupAdmin, getGroupName("testRegionServerMove"), 1);
+    RSGroupInfo adminInfo = addGroup(rsGroupAdmin, getGroupName("testRegionServerMove"), 1);
+    RSGroupInfo dInfo = rsGroupAdmin.getRSGroupInfo(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.removeRSGroup(appInfo.getName());
+    rsGroupAdmin.moveServers(adminInfo.getServers(),
+        RSGroupInfo.DEFAULT_GROUP);
+    rsGroupAdmin.removeRSGroup(adminInfo.getName());
+    Assert.assertEquals(rsGroupAdmin.listRSGroups().size(), initNumGroups);
+  }
+
+  @Test
+  public void testMoveServers() throws Exception {
+    LOG.info("testMoveServers");
+
+    //create groups and assign servers
+    addGroup(rsGroupAdmin, "bar", 3);
+    rsGroupAdmin.addRSGroup("foo");
+
+    RSGroupInfo barGroup = rsGroupAdmin.getRSGroupInfo("bar");
+    RSGroupInfo fooGroup = rsGroupAdmin.getRSGroupInfo("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");
+      fail("Bogus servers shouldn't have been successfully moved.");
+    } catch(IOException ex) {
+      String exp = "Server foo:9999 does not have a group.";
+      String msg = "Expected '"+exp+"' in exception message: ";
+      assertTrue(msg+" "+ex.getMessage(), ex.getMessage().contains(exp));
+    }
+
+    //test success case
+    LOG.info("moving servers "+barGroup.getServers()+" to group foo");
+    rsGroupAdmin.moveServers(barGroup.getServers(), fooGroup.getName());
+
+    barGroup = rsGroupAdmin.getRSGroupInfo("bar");
+    fooGroup = rsGroupAdmin.getRSGroupInfo("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);
+
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return getNumServers() ==
+        rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers().size();
+      }
+    });
+
+    fooGroup = rsGroupAdmin.getRSGroupInfo("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()));
+    LOG.info("Remove group "+fooGroup.getName());
+    rsGroupAdmin.removeRSGroup(fooGroup.getName());
+    Assert.assertEquals(null, rsGroupAdmin.getRSGroupInfo(fooGroup.getName()));
+  }
+
+  @Test
+  public void testRemoveServers() throws Exception {
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, "testRemoveServers", 3);
+    ServerName targetServer = ServerName.parseServerName(
+        newGroup.getServers().iterator().next().toString());
+    try {
+      rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));
+      fail("Online servers shouldn't have been successfully removed.");
+    } catch(IOException ex) {
+      String exp = "Server " + targetServer.getAddress()
+          + " is an online server, not allowed to remove.";
+      String msg = "Expected '" + exp + "' in exception message: ";
+      assertTrue(msg + " " + ex.getMessage(), ex.getMessage().contains(exp));
+    }
+    assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
+
+    AdminProtos.AdminService.BlockingInterface targetRS =
+        ((ClusterConnection) admin.getConnection()).getAdmin(targetServer);
+    try {
+      targetServer = ProtobufUtil.toServerName(targetRS.getServerInfo(null,
+          GetServerInfoRequest.newBuilder().build()).getServerInfo().getServerName());
+      //stopping may cause an exception
+      //due to the connection loss
+      targetRS.stopServer(null,
+          AdminProtos.StopServerRequest.newBuilder().setReason("Die").build());
+    } catch(Exception e) {
+    }
+
+    final HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+    //wait for stopped regionserver to dead server list
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return !master.getServerManager().areDeadServersInProgress()
+            && cluster.getClusterStatus().getDeadServerNames().size() > 0;
+      }
+    });
+
+    try {
+      rsGroupAdmin.removeServers(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,"
+          + " Maybe it will come back again, not allowed to remove.";
+      String msg = "Expected '" + exp + "' in exception message: ";
+      assertTrue(msg + " " + ex.getMessage(), ex.getMessage().contains(exp));
+    }
+    assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
+
+    ServerName sn = TEST_UTIL.getHBaseClusterInterface().getClusterStatus().getMaster();
+    TEST_UTIL.getHBaseClusterInterface().stopMaster(sn);
+    TEST_UTIL.getHBaseClusterInterface().waitForMasterToStop(sn, 60000);
+    TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
+    TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster(60000);
+
+    assertEquals(3, cluster.getClusterStatus().getServersSize());
+    assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
+    assertFalse(cluster.getClusterStatus().getDeadServerNames().contains(targetServer));
+    assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
+
+    rsGroupAdmin.removeServers(Sets.newHashSet(targetServer.getAddress()));
+    Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    assertFalse(newGroupServers.contains(targetServer.getAddress()));
+    assertEquals(2, newGroupServers.size());
+
+    assertTrue(observer.preRemoveServersCalled);
+  }
+
+  @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.getAddress()) &&
+           !rsGroupAdmin.getRSGroupInfo("master").containsServer(server.getAddress())) {
+        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(Address.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.getAddress()),
+              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 {
+        List<String> regions = getTableRegionMap().get(tableName);
+        Map<ServerName, List<String>> serverMap = getTableServerRegionMap().get(tableName);
+        Set<RegionState> regionsInTransition = admin.getClusterStatus().getRegionsInTransition();
+        return (regions != null && regions.size() == 5) &&
+          (serverMap != null && serverMap.size() == 1) &&
+          (regionsInTransition == null || regionsInTransition.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.getAddress()),
+            Sets.newHashSet(tableName), newGroup.getName());
+
+    //verify group change
+    Assert.assertEquals(newGroup.getName(),
+            rsGroupAdmin.getRSGroupInfoOfTable(tableName).getName());
+
+    //verify servers' not exist in old group
+    Set<Address> defaultServers = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP).getServers();
+    assertFalse(defaultServers.contains(targetServer.getAddress()));
+
+    //verify servers' exist in new group
+    Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
+    assertTrue(newGroupServers.contains(targetServer.getAddress()));
+
+    //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));
+
+    assertTrue(observer.preMoveServersAndTables);
+    assertTrue(observer.postMoveServersAndTables);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/54d89e1a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
new file mode 100644
index 0000000..425a385
--- /dev/null
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBalance.java
@@ -0,0 +1,183 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Sets;
+
+@Category({MediumTests.class})
+public class TestRSGroupsBalance extends TestRSGroupsBase {
+  protected static final Log LOG = LogFactory.getLog(TestRSGroupsBalance.class);
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    setUpTestBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    tearDownAfterClass();
+  }
+
+  @Before
+  public void beforeMethod() throws Exception {
+    setUpBeforeMethod();
+  }
+
+  @After
+  public void afterMethod() throws Exception {
+    tearDownAfterMethod();
+  }
+
+  @Test
+  public void testRSGroupBalancerSwitch() throws IOException {
+    //Balancer is initially off in the test, set to true and check
+    assertFalse(admin.setBalancerRunning(true, true));
+    assertTrue(admin.isBalancerEnabled());
+    //Set balancer off and check if it actually turned off
+    assertTrue(admin.setBalancerRunning(false,true));
+    assertFalse(admin.isBalancerEnabled());
+  }
+
+  @Test
+  public void testGroupBalance() throws Exception {
+    LOG.info("testGroupBalance");
+    String newGroupName = getGroupName("testGroupBalance");
+    final RSGroupInfo newGroup = addGroup(rsGroupAdmin, newGroupName, 3);
+
+    final TableName tableName = TableName.valueOf(tablePrefix+"_ns", "testGroupBalance");
+    admin.createNamespace(
+        NamespaceDescriptor.create(tableName.getNamespaceAsString())
+            .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, newGroupName).build());
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor("f"));
+    byte [] startKey = Bytes.toBytes("aaaaa");
+    byte [] endKey = Bytes.toBytes("zzzzz");
+    admin.createTable(desc, startKey, endKey, 6);
+    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 regions.size() >= 6;
+      }
+    });
+
+    //make assignment uneven, move all regions to one server
+    Map<ServerName,List<String>> assignMap =
+        getTableServerRegionMap().get(tableName);
+    final ServerName first = assignMap.entrySet().iterator().next().getKey();
+    for(HRegionInfo region: admin.getTableRegions(tableName)) {
+      if(!assignMap.get(first).contains(region.getRegionNameAsString())) {
+        admin.move(region.getEncodedNameAsBytes(), Bytes.toBytes(first.getServerName()));
+      }
+    }
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Map<ServerName, List<String>> map = getTableServerRegionMap().get(tableName);
+        if (map == null) {
+          return true;
+        }
+        List<String> regions = map.get(first);
+        if (regions == null) {
+          return true;
+        }
+        return regions.size() >= 6;
+      }
+    });
+
+    //balance the other group and make sure it doesn't affect the new group
+    rsGroupAdmin.balanceRSGroup(RSGroupInfo.DEFAULT_GROUP);
+    assertEquals(6, getTableServerRegionMap().get(tableName).get(first).size());
+
+    rsGroupAdmin.balanceRSGroup(newGroupName);
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        for (List<String> regions : getTableServerRegionMap().get(tableName).values()) {
+          if (2 != regions.size()) {
+            return false;
+          }
+        }
+        return true;
+      }
+    });
+
+    assertTrue(observer.preBalanceRSGroupCalled);
+    assertTrue(observer.postBalanceRSGroupCalled);
+  }
+
+  @Test
+  public void testMisplacedRegions() throws Exception {
+    final TableName tableName = TableName.valueOf(tablePrefix+"_testMisplacedRegions");
+    LOG.info("testMisplacedRegions");
+
+    final RSGroupInfo RSGroupInfo = addGroup(rsGroupAdmin, "testMisplacedRegions", 1);
+
+    TEST_UTIL.createMultiRegionTable(tableName, new byte[]{'f'}, 15);
+    TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
+
+    RSGroupAdminEndpoint.getGroupInfoManager()
+        .moveTables(Sets.newHashSet(tableName), RSGroupInfo.getName());
+
+    assertTrue(rsGroupAdmin.balanceRSGroup(RSGroupInfo.getName()));
+
+    TEST_UTIL.waitFor(60000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ServerName serverName =
+            ServerName.valueOf(RSGroupInfo.getServers().iterator().next().toString(), 1);
+        return admin.getConnection().getAdmin()
+            .getOnlineRegions(serverName).size() == 15;
+      }
+    });
+  }
+
+}
\ No newline at end of file