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 2015/08/01 18:36:13 UTC

[1/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Repository: hbase
Updated Branches:
  refs/heads/hbase-6721 [created] 16f65badc


http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestGroupBasedLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestGroupBasedLoadBalancer.java
new file mode 100644
index 0000000..72bf887
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestGroupBasedLoadBalancer.java
@@ -0,0 +1,588 @@
+/**
+ * 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.master.balancer;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.group.GroupBasedLoadBalancer;
+import org.apache.hadoop.hbase.group.GroupInfo;
+import org.apache.hadoop.hbase.group.GroupInfoManager;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+//TODO use stochastic based load balancer instead
+@Category(SmallTests.class)
+public class TestGroupBasedLoadBalancer {
+
+  private static final Log LOG = LogFactory.getLog(TestGroupBasedLoadBalancer.class);
+  private static GroupBasedLoadBalancer loadBalancer;
+  private static SecureRandom rand;
+
+  static String[]  groups = new String[] { GroupInfo.DEFAULT_GROUP, "dg2", "dg3",
+      "dg4" };
+  static TableName[] tables =
+      new TableName[] { TableName.valueOf("dt1"),
+          TableName.valueOf("dt2"),
+          TableName.valueOf("dt3"),
+          TableName.valueOf("dt4")};
+  static List<ServerName> servers;
+  static Map<String, GroupInfo> groupMap;
+  static Map<TableName, String> tableMap;
+  static List<HTableDescriptor> tableDescs;
+  int[] regionAssignment = new int[] { 2, 5, 7, 10, 4, 3, 1 };
+  static int regionId = 0;
+
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    rand = new SecureRandom();
+    servers = generateServers(7);
+    groupMap = constructGroupInfo(servers, groups);
+    tableMap = new HashMap<TableName, String>();
+    tableDescs = constructTableDesc();
+    Configuration conf = HBaseConfiguration.create();
+    conf.set("hbase.regions.slop", "0");
+    conf.set("hbase.group.grouploadbalancer.class", SimpleLoadBalancer.class.getCanonicalName());
+    loadBalancer = new GroupBasedLoadBalancer(getMockedGroupInfoManager());
+    loadBalancer.setMasterServices(getMockedMaster());
+    loadBalancer.setConf(conf);
+    loadBalancer.initialize();
+  }
+
+  /**
+   * Test the load balancing algorithm.
+   *
+   * Invariant is that all servers of the group should be hosting either floor(average) or
+   * ceiling(average)
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testBalanceCluster() throws Exception {
+    Map<ServerName, List<HRegionInfo>> servers = mockClusterServers();
+    ArrayListMultimap<String, ServerAndLoad> list = convertToGroupBasedMap(servers);
+    LOG.info("Mock Cluster :  " + printStats(list));
+    List<RegionPlan> plans = loadBalancer.balanceCluster(servers);
+    ArrayListMultimap<String, ServerAndLoad> balancedCluster = reconcile(
+        list, plans);
+    LOG.info("Mock Balance : " + printStats(balancedCluster));
+    assertClusterAsBalanced(balancedCluster);
+  }
+
+  /**
+   * Invariant is that all servers of a group have load between floor(avg) and
+   * ceiling(avg) number of regions.
+   */
+  private void assertClusterAsBalanced(
+      ArrayListMultimap<String, ServerAndLoad> groupLoadMap) {
+    for (String gName : groupLoadMap.keySet()) {
+      List<ServerAndLoad> groupLoad = groupLoadMap.get(gName);
+      int numServers = groupLoad.size();
+      int numRegions = 0;
+      int maxRegions = 0;
+      int minRegions = Integer.MAX_VALUE;
+      for (ServerAndLoad server : groupLoad) {
+        int nr = server.getLoad();
+        if (nr > maxRegions) {
+          maxRegions = nr;
+        }
+        if (nr < minRegions) {
+          minRegions = nr;
+        }
+        numRegions += nr;
+      }
+      if (maxRegions - minRegions < 2) {
+        // less than 2 between max and min, can't balance
+        return;
+      }
+      int min = numRegions / numServers;
+      int max = numRegions % numServers == 0 ? min : min + 1;
+
+      for (ServerAndLoad server : groupLoad) {
+        assertTrue(server.getLoad() <= max);
+        assertTrue(server.getLoad() >= min);
+      }
+    }
+  }
+
+  /**
+   * Tests immediate assignment.
+   *
+   * Invariant is that all regions have an assignment.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testImmediateAssignment() throws Exception {
+    List<HRegionInfo> regions = randomRegions(20);
+    Map<HRegionInfo, ServerName> assignments = loadBalancer
+        .immediateAssignment(regions, servers);
+    assertImmediateAssignment(regions, servers, assignments);
+  }
+
+  /**
+   * All regions have an assignment.
+   *
+   * @param regions
+   * @param servers
+   * @param assignments
+   * @throws java.io.IOException
+   * @throws java.io.FileNotFoundException
+   */
+  private void assertImmediateAssignment(List<HRegionInfo> regions,
+                                         List<ServerName> servers, Map<HRegionInfo, ServerName> assignments)
+      throws FileNotFoundException, IOException {
+    for (HRegionInfo region : regions) {
+      assertTrue(assignments.containsKey(region));
+      ServerName server = assignments.get(region);
+      TableName tableName = region.getTable();
+
+      String groupName =
+          loadBalancer.getGroupInfoManager().getGroupOfTable(tableName);
+      assertTrue(StringUtils.isNotEmpty(groupName));
+      GroupInfo gInfo = getMockedGroupInfoManager().getGroup(groupName);
+      assertTrue("Region is not correctly assigned to group servers.",
+          gInfo.containsServer(server.getHostPort()));
+    }
+  }
+
+  /**
+   * Tests the bulk assignment used during cluster startup.
+   *
+   * Round-robin. Should yield a balanced cluster so same invariant as the
+   * load balancer holds, all servers holding either floor(avg) or
+   * ceiling(avg).
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testBulkAssignment() throws Exception {
+    List<HRegionInfo> regions = randomRegions(25);
+    Map<ServerName, List<HRegionInfo>> assignments = loadBalancer
+        .roundRobinAssignment(regions, servers);
+    //test empty region/servers scenario
+    //this should not throw an NPE
+    loadBalancer.roundRobinAssignment(regions,
+        Collections.EMPTY_LIST);
+    //test regular scenario
+    assertTrue(assignments.keySet().size() == servers.size());
+    for (ServerName sn : assignments.keySet()) {
+      List<HRegionInfo> regionAssigned = assignments.get(sn);
+      for (HRegionInfo region : regionAssigned) {
+        TableName tableName = region.getTable();
+        String groupName =
+            getMockedGroupInfoManager().getGroupOfTable(tableName);
+        assertTrue(StringUtils.isNotEmpty(groupName));
+        GroupInfo gInfo = getMockedGroupInfoManager().getGroup(
+            groupName);
+        assertTrue(
+            "Region is not correctly assigned to group servers.",
+            gInfo.containsServer(sn.getHostPort()));
+      }
+    }
+    ArrayListMultimap<String, ServerAndLoad> loadMap = convertToGroupBasedMap(assignments);
+    assertClusterAsBalanced(loadMap);
+  }
+
+  /**
+   * Test the cluster startup bulk assignment which attempts to retain
+   * assignment info.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testRetainAssignment() throws Exception {
+    // Test simple case where all same servers are there
+    Map<ServerName, List<HRegionInfo>> currentAssignments = mockClusterServers();
+    Map<HRegionInfo, ServerName> inputForTest = new HashMap<HRegionInfo, ServerName>();
+    for (ServerName sn : currentAssignments.keySet()) {
+      for (HRegionInfo region : currentAssignments.get(sn)) {
+        inputForTest.put(region, sn);
+      }
+    }
+    //verify region->null server assignment is handled
+    inputForTest.put(randomRegions(1).get(0), null);
+    Map<ServerName, List<HRegionInfo>> newAssignment = loadBalancer
+        .retainAssignment(inputForTest, servers);
+    assertRetainedAssignment(inputForTest, servers, newAssignment);
+  }
+
+  /**
+   * Asserts a valid retained assignment plan.
+   * <p>
+   * Must meet the following conditions:
+   * <ul>
+   * <li>Every input region has an assignment, and to an online server
+   * <li>If a region had an existing assignment to a server with the same
+   * address a a currently online server, it will be assigned to it
+   * </ul>
+   *
+   * @param existing
+   * @param assignment
+   * @throws java.io.IOException
+   * @throws java.io.FileNotFoundException
+   */
+  private void assertRetainedAssignment(
+      Map<HRegionInfo, ServerName> existing, List<ServerName> servers,
+      Map<ServerName, List<HRegionInfo>> assignment)
+      throws FileNotFoundException, IOException {
+    // Verify condition 1, every region assigned, and to online server
+    Set<ServerName> onlineServerSet = new TreeSet<ServerName>(servers);
+    Set<HRegionInfo> assignedRegions = new TreeSet<HRegionInfo>();
+    for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+      assertTrue(
+          "Region assigned to server that was not listed as online",
+          onlineServerSet.contains(a.getKey()));
+      for (HRegionInfo r : a.getValue())
+        assignedRegions.add(r);
+    }
+    assertEquals(existing.size(), assignedRegions.size());
+
+    // Verify condition 2, every region must be assigned to correct server.
+    Set<String> onlineHostNames = new TreeSet<String>();
+    for (ServerName s : servers) {
+      onlineHostNames.add(s.getHostname());
+    }
+
+    for (Map.Entry<ServerName, List<HRegionInfo>> a : assignment.entrySet()) {
+      ServerName currentServer = a.getKey();
+      for (HRegionInfo r : a.getValue()) {
+        ServerName oldAssignedServer = existing.get(r);
+        TableName tableName = r.getTable();
+        String groupName =
+            getMockedGroupInfoManager().getGroupOfTable(tableName);
+        assertTrue(StringUtils.isNotEmpty(groupName));
+        GroupInfo gInfo = getMockedGroupInfoManager().getGroup(
+            groupName);
+        assertTrue(
+            "Region is not correctly assigned to group servers.",
+            gInfo.containsServer(currentServer.getHostPort()));
+        if (oldAssignedServer != null
+            && onlineHostNames.contains(oldAssignedServer
+            .getHostname())) {
+          // this region was previously assigned somewhere, and that
+          // host is still around, then the host must have been is a
+          // different group.
+          if (!oldAssignedServer.getHostPort().equals(currentServer.getHostPort())) {
+            assertFalse(gInfo.containsServer(oldAssignedServer.getHostPort()));
+          }
+        }
+      }
+    }
+  }
+
+  private String printStats(
+      ArrayListMultimap<String, ServerAndLoad> groupBasedLoad) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("\n");
+    for (String groupName : groupBasedLoad.keySet()) {
+      sb.append("Stats for group: " + groupName);
+      sb.append("\n");
+      sb.append(groupMap.get(groupName).getServers());
+      sb.append("\n");
+      List<ServerAndLoad> groupLoad = groupBasedLoad.get(groupName);
+      int numServers = groupLoad.size();
+      int totalRegions = 0;
+      sb.append("Per Server Load: \n");
+      for (ServerAndLoad sLoad : groupLoad) {
+        sb.append("Server :" + sLoad.getServerName() + " Load : "
+            + sLoad.getLoad() + "\n");
+        totalRegions += sLoad.getLoad();
+      }
+      sb.append(" Group Statistics : \n");
+      float average = (float) totalRegions / numServers;
+      int max = (int) Math.ceil(average);
+      int min = (int) Math.floor(average);
+      sb.append("[srvr=" + numServers + " rgns=" + totalRegions + " avg="
+          + average + " max=" + max + " min=" + min + "]");
+      sb.append("\n");
+      sb.append("===============================");
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
+  private ArrayListMultimap<String, ServerAndLoad> convertToGroupBasedMap(
+      final Map<ServerName, List<HRegionInfo>> serversMap) throws IOException {
+    ArrayListMultimap<String, ServerAndLoad> loadMap = ArrayListMultimap
+        .create();
+    for (GroupInfo gInfo : getMockedGroupInfoManager().listGroups()) {
+      Set<HostPort> groupServers = gInfo.getServers();
+      for (HostPort hostPort : groupServers) {
+        ServerName actual = null;
+        for(ServerName entry: servers) {
+          if(entry.getHostPort().equals(hostPort)) {
+            actual = entry;
+            break;
+          }
+        }
+        List<HRegionInfo> regions = serversMap.get(actual);
+        assertTrue("No load for " + actual, regions != null);
+        loadMap.put(gInfo.getName(),
+            new ServerAndLoad(actual, regions.size()));
+      }
+    }
+    return loadMap;
+  }
+
+  private ArrayListMultimap<String, ServerAndLoad> reconcile(
+      ArrayListMultimap<String, ServerAndLoad> previousLoad,
+      List<RegionPlan> plans) {
+    ArrayListMultimap<String, ServerAndLoad> result = ArrayListMultimap
+        .create();
+    result.putAll(previousLoad);
+    if (plans != null) {
+      for (RegionPlan plan : plans) {
+        ServerName source = plan.getSource();
+        updateLoad(result, source, -1);
+        ServerName destination = plan.getDestination();
+        updateLoad(result, destination, +1);
+      }
+    }
+    return result;
+  }
+
+  private void updateLoad(
+      ArrayListMultimap<String, ServerAndLoad> previousLoad,
+      final ServerName sn, final int diff) {
+    for (String groupName : previousLoad.keySet()) {
+      ServerAndLoad newSAL = null;
+      ServerAndLoad oldSAL = null;
+      for (ServerAndLoad sal : previousLoad.get(groupName)) {
+        if (ServerName.isSameHostnameAndPort(sn, sal.getServerName())) {
+          oldSAL = sal;
+          newSAL = new ServerAndLoad(sn, sal.getLoad() + diff);
+          break;
+        }
+      }
+      if (newSAL != null) {
+        previousLoad.remove(groupName, oldSAL);
+        previousLoad.put(groupName, newSAL);
+        break;
+      }
+    }
+  }
+
+  private Map<ServerName, List<HRegionInfo>> mockClusterServers() throws IOException {
+    assertTrue(servers.size() == regionAssignment.length);
+    Map<ServerName, List<HRegionInfo>> assignment = new TreeMap<ServerName, List<HRegionInfo>>();
+    for (int i = 0; i < servers.size(); i++) {
+      int numRegions = regionAssignment[i];
+      List<HRegionInfo> regions = assignedRegions(numRegions, servers.get(i));
+      assignment.put(servers.get(i), regions);
+    }
+    return assignment;
+  }
+
+  /**
+   * Generate a list of regions evenly distributed between the tables.
+   *
+   * @param numRegions The number of regions to be generated.
+   * @return List of HRegionInfo.
+   */
+  private List<HRegionInfo> randomRegions(int numRegions) {
+    List<HRegionInfo> regions = new ArrayList<HRegionInfo>(numRegions);
+    byte[] start = new byte[16];
+    byte[] end = new byte[16];
+    rand.nextBytes(start);
+    rand.nextBytes(end);
+    int regionIdx = rand.nextInt(tables.length);
+    for (int i = 0; i < numRegions; i++) {
+      Bytes.putInt(start, 0, numRegions << 1);
+      Bytes.putInt(end, 0, (numRegions << 1) + 1);
+      int tableIndex = (i + regionIdx) % tables.length;
+      HRegionInfo hri = new HRegionInfo(
+          tables[tableIndex], start, end, false, regionId++);
+      regions.add(hri);
+    }
+    return regions;
+  }
+
+  /**
+   * Generate assigned regions to a given server using group information.
+   *
+   * @param numRegions the num regions to generate
+   * @param sn the servername
+   * @return the list of regions
+   * @throws java.io.IOException Signals that an I/O exception has occurred.
+   */
+  private List<HRegionInfo> assignedRegions(int numRegions, ServerName sn) throws IOException {
+    List<HRegionInfo> regions = new ArrayList<HRegionInfo>(numRegions);
+    byte[] start = new byte[16];
+    byte[] end = new byte[16];
+    Bytes.putInt(start, 0, numRegions << 1);
+    Bytes.putInt(end, 0, (numRegions << 1) + 1);
+    for (int i = 0; i < numRegions; i++) {
+      TableName tableName = getTableName(sn);
+      HRegionInfo hri = new HRegionInfo(
+          tableName, start, end, false,
+          regionId++);
+      regions.add(hri);
+    }
+    return regions;
+  }
+
+  private static List<ServerName> generateServers(int numServers) {
+    List<ServerName> servers = new ArrayList<ServerName>(numServers);
+    for (int i = 0; i < numServers; i++) {
+      String host = "server" + rand.nextInt(100000);
+      int port = rand.nextInt(60000);
+      servers.add(ServerName.valueOf(host, port, -1));
+    }
+    return servers;
+  }
+
+  /**
+   * Construct group info, with each group having at least one server.
+   *
+   * @param servers the servers
+   * @param groups the groups
+   * @return the map
+   */
+  private static Map<String, GroupInfo> constructGroupInfo(
+      List<ServerName> servers, String[] groups) {
+    assertTrue(servers != null);
+    assertTrue(servers.size() >= groups.length);
+    int index = 0;
+    Map<String, GroupInfo> groupMap = new HashMap<String, GroupInfo>();
+    for (String grpName : groups) {
+      GroupInfo groupInfo = new GroupInfo(grpName);
+      groupInfo.addServer(servers.get(index).getHostPort());
+      groupMap.put(grpName, groupInfo);
+      index++;
+    }
+    while (index < servers.size()) {
+      int grpIndex = rand.nextInt(groups.length);
+      groupMap.get(groups[grpIndex]).addServer(
+          servers.get(index).getHostPort());
+      index++;
+    }
+    return groupMap;
+  }
+
+  /**
+   * Construct table descriptors evenly distributed between the groups.
+   *
+   * @return the list
+   */
+  private static List<HTableDescriptor> constructTableDesc() {
+    List<HTableDescriptor> tds = Lists.newArrayList();
+    int index = rand.nextInt(groups.length);
+    for (int i = 0; i < tables.length; i++) {
+      HTableDescriptor htd = new HTableDescriptor(tables[i]);
+      int grpIndex = (i + index) % groups.length ;
+      String groupName = groups[grpIndex];
+      tableMap.put(tables[i], groupName);
+      tds.add(htd);
+    }
+    return tds;
+  }
+
+  private static MasterServices getMockedMaster() throws IOException {
+    TableDescriptors tds = Mockito.mock(TableDescriptors.class);
+    Mockito.when(tds.get(tables[0])).thenReturn(tableDescs.get(0));
+    Mockito.when(tds.get(tables[1])).thenReturn(tableDescs.get(1));
+    Mockito.when(tds.get(tables[2])).thenReturn(tableDescs.get(2));
+    Mockito.when(tds.get(tables[3])).thenReturn(tableDescs.get(3));
+    MasterServices services = Mockito.mock(HMaster.class);
+    Mockito.when(services.getTableDescriptors()).thenReturn(tds);
+    AssignmentManager am = Mockito.mock(AssignmentManager.class);
+    Mockito.when(services.getAssignmentManager()).thenReturn(am);
+    return services;
+  }
+
+  private static GroupInfoManager getMockedGroupInfoManager() throws IOException {
+    GroupInfoManager gm = Mockito.mock(GroupInfoManager.class);
+    Mockito.when(gm.getGroup(groups[0])).thenReturn(
+        groupMap.get(groups[0]));
+    Mockito.when(gm.getGroup(groups[1])).thenReturn(
+        groupMap.get(groups[1]));
+    Mockito.when(gm.getGroup(groups[2])).thenReturn(
+        groupMap.get(groups[2]));
+    Mockito.when(gm.getGroup(groups[3])).thenReturn(
+        groupMap.get(groups[3]));
+    Mockito.when(gm.listGroups()).thenReturn(
+        Lists.newLinkedList(groupMap.values()));
+    Mockito.when(gm.isOnline()).thenReturn(true);
+    Mockito.when(gm.getGroupOfTable(Mockito.any(TableName.class)))
+        .thenAnswer(new Answer<String>() {
+          @Override
+          public String answer(InvocationOnMock invocation) throws Throwable {
+            return tableMap.get(invocation.getArguments()[0]);
+          }
+        });
+    return gm;
+  }
+
+  private TableName getTableName(ServerName sn) throws IOException {
+    TableName tableName = null;
+    GroupInfoManager gm = getMockedGroupInfoManager();
+    GroupInfo groupOfServer = null;
+    for(GroupInfo gInfo : gm.listGroups()){
+      if(gInfo.containsServer(sn.getHostPort())){
+        groupOfServer = gInfo;
+        break;
+      }
+    }
+
+    for(HTableDescriptor desc : tableDescs){
+      if(gm.getGroupOfTable(desc.getTableName()).endsWith(groupOfServer.getName())){
+        tableName = desc.getTableName();
+      }
+    }
+    return tableName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index b474dd2..405c5a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -2629,4 +2629,129 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(replicateLogEntriesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER,
       USER_GROUP_READ, USER_GROUP_ADMIN, USER_GROUP_CREATE);
   }
+
+  @Test
+  public void testMoveServers() throws Exception {
+    AccessTestAction action1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preMoveServers(ObserverContext.createAndPrepare(CP_ENV, null),
+            null, null);
+        return null;
+      }
+    };
+    AccessTestAction action2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.postMoveServers(ObserverContext.createAndPrepare(CP_ENV, null),
+            null, null);
+        return null;
+      }
+    };
+
+    verifyAllowed(action1, SUPERUSER, USER_ADMIN);
+    verifyAllowed(action2, SUPERUSER, USER_ADMIN);
+    verifyDenied(action1, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+    verifyDenied(action2, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
+  public void testMoveTables() throws Exception {
+    AccessTestAction action1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preMoveTables(ObserverContext.createAndPrepare(CP_ENV, null),
+            null, null);
+        return null;
+      }
+    };
+    AccessTestAction action2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.postMoveTables(ObserverContext.createAndPrepare(CP_ENV, null),
+            null, null);
+        return null;
+      }
+    };
+
+    verifyAllowed(action1, SUPERUSER, USER_ADMIN);
+    verifyAllowed(action2, SUPERUSER, USER_ADMIN);
+    verifyDenied(action1, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+    verifyDenied(action2, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
+  public void testAddGroup() throws Exception {
+    AccessTestAction action1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preAddGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null);
+        return null;
+      }
+    };
+    AccessTestAction action2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.postAddGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null);
+        return null;
+      }
+    };
+
+    verifyAllowed(action1, SUPERUSER, USER_ADMIN);
+    verifyAllowed(action2, SUPERUSER, USER_ADMIN);
+    verifyDenied(action1, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+    verifyDenied(action2, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
+  public void testRemoveGroup() throws Exception {
+    AccessTestAction action1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preRemoveGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null);
+        return null;
+      }
+    };
+    AccessTestAction action2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.postRemoveGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null);
+        return null;
+      }
+    };
+
+    verifyAllowed(action1, SUPERUSER, USER_ADMIN);
+    verifyAllowed(action2, SUPERUSER, USER_ADMIN);
+    verifyDenied(action1, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+    verifyDenied(action2, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
+  public void testBalanceGroup() throws Exception {
+    AccessTestAction action1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBalanceGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null);
+        return null;
+      }
+    };
+    AccessTestAction action2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.postBalanceGroup(ObserverContext.createAndPrepare(CP_ENV, null),
+            null, false);
+        return null;
+      }
+    };
+
+    verifyAllowed(action1, SUPERUSER, USER_ADMIN);
+    verifyAllowed(action2, SUPERUSER, USER_ADMIN);
+    verifyDenied(action1, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+    verifyDenied(action2, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/hbase.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase.rb b/hbase-shell/src/main/ruby/hbase.rb
index aca1006..f995536 100644
--- a/hbase-shell/src/main/ruby/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase.rb
@@ -102,5 +102,6 @@ require 'hbase/quotas'
 require 'hbase/replication_admin'
 require 'hbase/security'
 require 'hbase/visibility_labels'
+require 'hbase/group_admin'
 
 include HBaseQuotasConstants

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 86b88f0..d6a05a7 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -414,6 +414,27 @@ module Hbase
       locator.close()
 
       table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
+
+      #clone group
+      if(groups_available?(conf))
+      	group_admin =  org.apache.hadoop.hbase.group.GroupAdminClient.new(@conf)
+      	group_info = group_admin.getGroupInfoOfTable(table_name)
+        exp_group = group_info.getName
+        if(exp_group == "default")
+          exp_group = nil;
+        end
+        ns =
+            @admin.getNamespaceDescriptor(
+                org.apache.hadoop.hbase.TableName.valueOf(table_name).getNamespaceAsString)
+        ns_group =
+          ns.getValue(org.apache.hadoop.hbase.group.GroupInfo::NAMESPACEDESC_PROP_GROUP)
+        if(!exp_group.nil? && ns_group.nil?|| (ns_group != exp_group))
+          yield " - Preserving explicit group assignment to #{exp_group}" if block_given?
+          table_description.setValue(org.apache.hadoop.hbase.group.GroupInfo::TABLEDESC_PROP_GROUP,
+          group_info.getName())
+        end
+      end
+
       yield 'Disabling table...' if block_given?
       disable(table_name)
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/hbase/group_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/group_admin.rb b/hbase-shell/src/main/ruby/hbase/group_admin.rb
new file mode 100644
index 0000000..4532031
--- /dev/null
+++ b/hbase-shell/src/main/ruby/hbase/group_admin.rb
@@ -0,0 +1,121 @@
+#
+# 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.
+#
+
+include Java
+java_import org.apache.hadoop.hbase.util.Pair
+
+# Wrapper for org.apache.hadoop.hbase.group.GroupAdminClient
+# Which is an API to manage region server groups
+
+module Hbase
+  class GroupAdmin
+    include HBaseConstants
+
+    def initialize(configuration, formatter)
+      @admin = org.apache.hadoop.hbase.group.GroupAdminClient.new(configuration)
+      @conf = configuration
+      @formatter = formatter
+    end
+
+    #----------------------------------------------------------------------------------------------
+    # Returns a list of groups in hbase
+    def listGroups
+      @admin.listGroups.map { |g| g.getName }
+    end
+    #----------------------------------------------------------------------------------------------
+    # get a group's information
+    def getGroup(group_name)
+      group = @admin.getGroupInfo(group_name)
+      res = {}
+      if block_given?
+        yield("Servers:")
+      else
+        res += v
+      end
+      group.getServers.each do |v|
+        if block_given?
+          yield(v)
+        else
+          res += v
+        end
+      end
+      if block_given?
+        yield("Tables:")
+      else
+        res += v
+      end
+      group.getTables.each do |v|
+        if block_given?
+          yield(v.toString)
+        else
+          res += v.toString
+        end
+      end
+    end
+    #----------------------------------------------------------------------------------------------
+    # add a group
+    def addGroup(group_name)
+      @admin.addGroup(group_name)
+    end
+    #----------------------------------------------------------------------------------------------
+    # remove a group
+    def removeGroup(group_name)
+      @admin.removeGroup(group_name)
+    end
+    #----------------------------------------------------------------------------------------------
+    # balance a group
+    def balanceGroup(group_name)
+      @admin.balanceGroup(group_name)
+    end
+    #----------------------------------------------------------------------------------------------
+    # move server to a group
+    def moveServers(dest, *args)
+      servers = java.util.HashSet.new()
+      args[0].each do |s|
+        servers.add(org.apache.hadoop.hbase.HostPort.valueOf(s))
+      end
+      @admin.moveServers(servers, dest)
+    end
+    #----------------------------------------------------------------------------------------------
+    # move server to a group
+    def moveTables(dest, *args)
+      tables = java.util.HashSet.new();
+      args[0].each do |s|
+        tables.add(org.apache.hadoop.hbase.TableName.valueOf(s))
+      end
+      @admin.moveTables(tables,dest)
+    end
+    #----------------------------------------------------------------------------------------------
+    # get group of server
+    def getGroupOfServer(server)
+      @admin.getGroupOfServer(org.apache.hadoop.hbase.HostPort.valueOf(server))
+    end
+    #----------------------------------------------------------------------------------------------
+    # get group of server
+    def getGroupOfTable(table)
+      @admin.getGroupInfoOfTable(org.apache.hadoop.hbase.TableName.valueOf(table))
+    end
+    #----------------------------------------------------------------------------------------------
+    # get list tables of groups
+    def listTablesOfGroup(group_name)
+      @admin.listTablesOfGroup(group_name)
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/hbase/hbase.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/hbase.rb b/hbase-shell/src/main/ruby/hbase/hbase.rb
index 135e1d5..8477bbd 100644
--- a/hbase-shell/src/main/ruby/hbase/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase/hbase.rb
@@ -47,6 +47,10 @@ module Hbase
       ::Hbase::Admin.new(@connection.getAdmin, formatter)
     end
 
+    def group_admin(formatter)
+      ::Hbase::GroupAdmin.new(configuration, formatter)
+    end
+
     # Create new one each time
     def table(table, shell)
       ::Hbase::Table.new(@connection.getTable(table), shell)

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 ab5f44d..3cc1f06 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -87,6 +87,10 @@ module Shell
       @hbase_admin ||= hbase.admin(formatter)
     end
 
+    def group_admin
+      @group_admin ||= hbase.group_admin(formatter)
+    end
+
     def hbase_table(name)
       hbase.table(name, self)
     end
@@ -415,3 +419,22 @@ Shell.load_command_group(
     set_visibility
   ]
 )
+
+Shell.load_command_group(
+  'group',
+  :full_name => 'Groups',
+  :comment => "NOTE: Above commands are only applicable if running with the Groups setup",
+  :commands => %w[
+    list_groups
+    get_group
+    add_group
+    remove_group
+    balance_group
+    move_group_servers
+    move_group_tables
+    get_server_group
+    get_table_group
+    list_group_tables
+    list_group_server_transitions
+  ]
+)

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index 4ad04cd..4ecd9e8 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -54,6 +54,10 @@ module Shell
         @shell.hbase_admin
       end
 
+      def group_admin
+        @shell.group_admin
+      end
+
       def table(name)
         @shell.hbase_table(name)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/add_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_group.rb b/hbase-shell/src/main/ruby/shell/commands/add_group.rb
new file mode 100644
index 0000000..7f91ee5
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/add_group.rb
@@ -0,0 +1,39 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class AddGroup < Command
+      def help
+        return <<-EOF
+Create a new region server group.
+
+Example:
+
+  hbase> add_group 'my_group'
+EOF
+      end
+
+      def command(group_name)
+        group_admin.addGroup(group_name)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/balance_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/balance_group.rb b/hbase-shell/src/main/ruby/shell/commands/balance_group.rb
new file mode 100644
index 0000000..4c59f63
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/balance_group.rb
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class BalanceGroup < Command
+      def help
+        return <<-EOF
+Balance a region server group
+
+  hbase> group_balance 'my_group'
+EOF
+      end
+
+      def command(group_name)
+        group_admin.balanceGroup(group_name)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/get_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/get_group.rb b/hbase-shell/src/main/ruby/shell/commands/get_group.rb
new file mode 100644
index 0000000..5ed8226
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/get_group.rb
@@ -0,0 +1,44 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class GetGroup < Command
+      def help
+        return <<-EOF
+Get a region server group's information.
+
+Example:
+
+  hbase> get_group 'default'
+EOF
+      end
+
+      def command(group_name)
+        now = Time.now
+        formatter.header([ "GROUP INFORMATION" ])
+        group_admin.getGroup(group_name) do |s|
+          formatter.row([ s ])
+        end
+        formatter.footer(now)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb b/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb
new file mode 100644
index 0000000..c78d4d2
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb
@@ -0,0 +1,40 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class GetServerGroup < Command
+      def help
+        return <<-EOF
+Get the group name the given region server is a member of.
+
+  hbase> get_server_group 'server1:port1'
+EOF
+      end
+
+      def command(server)
+        now = Time.now
+        groupName = group_admin.getGroupOfServer(server).getName
+        formatter.row([ groupName ])
+        formatter.footer(now,1)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb b/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb
new file mode 100644
index 0000000..dd8766d
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb
@@ -0,0 +1,41 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class GetTableGroup < Command
+      def help
+        return <<-EOF
+Get the group name the given table is a member of.
+
+  hbase> get_table_group 'myTable'
+EOF
+      end
+
+      def command(table)
+        now = Time.now
+        groupName =
+            group_admin.getGroupOfTable(table).getName
+        formatter.row([ groupName ])
+        formatter.footer(now,1)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/list_group_server_transitions.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_group_server_transitions.rb b/hbase-shell/src/main/ruby/shell/commands/list_group_server_transitions.rb
new file mode 100644
index 0000000..313873f
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_group_server_transitions.rb
@@ -0,0 +1,44 @@
+#
+# 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.
+#
+
+#TODO make this command name sho
+module Shell
+  module Commands
+    class ListGroupServerTransitions < Command
+      def help
+        return <<-EOF
+List region servers in transition.
+
+Example:
+
+  hbase> list_group_server_transitions 'default'
+EOF
+      end
+      def command()
+        now = Time.now
+        formatter.header(["Server", "Destination"])
+        count = group_admin.listServersInTransition do |server, dest|
+          formatter.row([ server, dest ])
+        end
+        formatter.footer(now, count)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/list_group_tables.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_group_tables.rb b/hbase-shell/src/main/ruby/shell/commands/list_group_tables.rb
new file mode 100644
index 0000000..ae0862c
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_group_tables.rb
@@ -0,0 +1,45 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListGroupTables < Command
+      def help
+        return <<-EOF
+List member tables of a given region server group in hbase.
+
+Example:
+
+  hbase> list_group_tables 'default'
+EOF
+      end
+
+      def command(group_name)
+        now = Time.now
+        formatter.header([ "TABLES" ])
+        list = group_admin.listTablesOfGroup(group_name)
+        list.each do |table|
+          formatter.row([ table.toString ])
+        end
+        formatter.footer(now, list.size)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/list_groups.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_groups.rb b/hbase-shell/src/main/ruby/shell/commands/list_groups.rb
new file mode 100644
index 0000000..2e7dd08
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_groups.rb
@@ -0,0 +1,50 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class ListGroups < Command
+      def help
+        return <<-EOF
+List all region server groups. Optional regular expression parameter could
+be used to filter the output.
+
+Example:
+
+  hbase> list_groups
+  hbase> list_groups 'abc.*'
+EOF
+      end
+
+      def command(regex = ".*")
+        now = Time.now
+        formatter.header([ "GROUPS" ])
+
+        regex = /#{regex}/ unless regex.is_a?(Regexp)
+        list = group_admin.listGroups.grep(regex)
+        list.each do |group|
+          formatter.row([ group ])
+        end
+
+        formatter.footer(now, list.size)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb b/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb
new file mode 100644
index 0000000..5e5c850
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class MoveGroupServers < Command
+      def help
+        return <<-EOF
+Reassign a region server from one group to another.
+
+  hbase> move_group_servers 'dest',['server1:port','server2:port']
+EOF
+      end
+
+      def command(dest, *servers)
+        group_admin.moveServers(dest, *servers)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb b/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb
new file mode 100644
index 0000000..f495f2c
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class MoveGroupTables < Command
+      def help
+        return <<-EOF
+Reassign tables from one group to another.
+
+  hbase> move_group_tables 'dest',['table1','table2']
+EOF
+      end
+
+      def command(dest, *servers)
+        group_admin.moveTables(dest, *servers)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-shell/src/main/ruby/shell/commands/remove_group.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_group.rb b/hbase-shell/src/main/ruby/shell/commands/remove_group.rb
new file mode 100644
index 0000000..66863a4
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/remove_group.rb
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class RemoveGroup < Command
+      def help
+        return <<-EOF
+Remove a group.
+
+  hbase> remove_group 'my_group'
+EOF
+      end
+
+      def command(group_name)
+        group_admin.removeGroup(group_name)
+      end
+    end
+  end
+end


[2/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 302d215..999da30 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -30,6 +33,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
@@ -43,6 +47,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.ipc.QosPriority;
+import org.apache.hadoop.hbase.group.GroupInfo;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.mob.MobUtils;
@@ -64,8 +69,12 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Re
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
@@ -94,6 +103,12 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusR
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfServerRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfServerResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
@@ -116,6 +131,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshot
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListGroupInfosRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListGroupInfosResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
@@ -134,8 +151,14 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableReques
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveServersRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveServersResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveTablesRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveTablesResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RemoveGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RemoveGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
@@ -270,8 +293,8 @@ public class MasterRpcServices extends RSRpcServices
       MasterService.newReflectiveBlockingService(this),
       MasterService.BlockingInterface.class));
     bssi.add(new BlockingServiceAndInterface(
-      RegionServerStatusService.newReflectiveBlockingService(this),
-      RegionServerStatusService.BlockingInterface.class));
+        RegionServerStatusService.newReflectiveBlockingService(this),
+        RegionServerStatusService.BlockingInterface.class));
     bssi.addAll(super.getServices());
     return bssi;
   }
@@ -355,10 +378,10 @@ public class MasterRpcServices extends RSRpcServices
       AddColumnRequest req) throws ServiceException {
     try {
       master.addColumn(
-        ProtobufUtil.toTableName(req.getTableName()),
-        HColumnDescriptor.convert(req.getColumnFamilies()),
-        req.getNonceGroup(),
-        req.getNonce());
+          ProtobufUtil.toTableName(req.getTableName()),
+          HColumnDescriptor.convert(req.getColumnFamilies()),
+          req.getNonceGroup(),
+          req.getNonce());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -438,10 +461,10 @@ public class MasterRpcServices extends RSRpcServices
       DeleteColumnRequest req) throws ServiceException {
     try {
       master.deleteColumn(
-        ProtobufUtil.toTableName(req.getTableName()),
-        req.getColumnName().toByteArray(),
-        req.getNonceGroup(),
-        req.getNonce());
+          ProtobufUtil.toTableName(req.getTableName()),
+          req.getColumnName().toByteArray(),
+          req.getNonceGroup(),
+          req.getNonce());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -691,7 +714,7 @@ public class MasterRpcServices extends RSRpcServices
       // to complete
       long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
       return ExecProcedureResponse.newBuilder().setExpectedTimeout(
-        waitTime).build();
+          waitTime).build();
     } catch (ForeignException e) {
       throw new ServiceException(e.getCause());
     } catch (IOException e) {
@@ -711,14 +734,14 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       ProcedureDescription desc = request.getProcedure();
       MasterProcedureManager mpm = master.mpmHost.getProcedureManager(
-        desc.getSignature());
+          desc.getSignature());
       if (mpm == null) {
         throw new ServiceException("The procedure is not registered: "
           + desc.getSignature());
       }
 
       LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
-        + desc.getSignature());
+          + desc.getSignature());
 
       byte[] data = mpm.execProcedureWithRet(desc);
 
@@ -1092,10 +1115,10 @@ public class MasterRpcServices extends RSRpcServices
       ModifyColumnRequest req) throws ServiceException {
     try {
       master.modifyColumn(
-        ProtobufUtil.toTableName(req.getTableName()),
-        HColumnDescriptor.convert(req.getColumnFamilies()),
-        req.getNonceGroup(),
-        req.getNonce());
+          ProtobufUtil.toTableName(req.getTableName()),
+          HColumnDescriptor.convert(req.getColumnFamilies()),
+          req.getNonceGroup(),
+          req.getNonce());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -1107,7 +1130,7 @@ public class MasterRpcServices extends RSRpcServices
       ModifyNamespaceRequest request) throws ServiceException {
     try {
       master.modifyNamespace(
-        ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
+          ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
       return ModifyNamespaceResponse.getDefaultInstance();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1119,10 +1142,10 @@ public class MasterRpcServices extends RSRpcServices
       ModifyTableRequest req) throws ServiceException {
     try {
       master.modifyTable(
-        ProtobufUtil.toTableName(req.getTableName()),
-        HTableDescriptor.convert(req.getTableSchema()),
-        req.getNonceGroup(),
-        req.getNonce());
+          ProtobufUtil.toTableName(req.getTableName()),
+          HTableDescriptor.convert(req.getTableSchema()),
+          req.getNonceGroup(),
+          req.getNonce());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -1505,4 +1528,149 @@ public class MasterRpcServices extends RSRpcServices
     response.setEnabled(master.isBalancerOn());
     return response.build();
   }
+
+  @Override
+  public GetGroupInfoResponse getGroupInfo(RpcController controller, GetGroupInfoRequest request) throws ServiceException {
+    MasterProtos.GetGroupInfoResponse response = null;
+    try {
+      MasterProtos.GetGroupInfoResponse.Builder builder =
+          MasterProtos.GetGroupInfoResponse.newBuilder();
+      GroupInfo groupInfo = master.getGroupAdminServer().getGroupInfo(request.getGroupName());
+      if(groupInfo != null) {
+        builder.setGroupInfo(ProtobufUtil.toProtoGroupInfo(groupInfo));
+      }
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public GetGroupInfoOfTableResponse getGroupInfoOfTable(RpcController controller,
+                                                                      GetGroupInfoOfTableRequest request) throws ServiceException {
+    MasterProtos.GetGroupInfoOfTableResponse response = null;
+    try {
+      MasterProtos.GetGroupInfoOfTableResponse.Builder builder =
+          MasterProtos.GetGroupInfoOfTableResponse.newBuilder();
+      GroupInfo groupInfo = master.getGroupAdminServer()
+          .getGroupInfoOfTable(ProtobufUtil.toTableName(request.getTableName()));
+      response = builder.setGroupInfo(ProtobufUtil.toProtoGroupInfo(groupInfo)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public MoveServersResponse moveServers(RpcController controller, MoveServersRequest request) throws ServiceException {
+    MasterProtos.MoveServersResponse response = null;
+    try {
+      MasterProtos.MoveServersResponse.Builder builder =
+          MasterProtos.MoveServersResponse.newBuilder();
+      Set<HostPort> hostPorts = Sets.newHashSet();
+      for(HBaseProtos.HostPort el: request.getServersList()) {
+        hostPorts.add(new HostPort(el.getHostName(), el.getPort()));
+      }
+      master.getGroupAdminServer()
+          .moveServers(hostPorts, request.getTargetGroup());
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public MoveTablesResponse moveTables(RpcController controller, MoveTablesRequest request) throws ServiceException {
+    MasterProtos.MoveTablesResponse response = null;
+    try {
+      MasterProtos.MoveTablesResponse.Builder builder =
+          MasterProtos.MoveTablesResponse.newBuilder();
+      Set<TableName> tables = new HashSet<TableName>(request.getTableNameList().size());
+      for(HBaseProtos.TableName tableName: request.getTableNameList()) {
+        tables.add(ProtobufUtil.toTableName(tableName));
+      }
+      master.getGroupAdminServer().moveTables(tables, request.getTargetGroup());
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public AddGroupResponse addGroup(RpcController controller, AddGroupRequest request) throws ServiceException {
+    MasterProtos.AddGroupResponse response = null;
+    try {
+      MasterProtos.AddGroupResponse.Builder builder =
+          MasterProtos.AddGroupResponse.newBuilder();
+      master.getGroupAdminServer().addGroup(request.getGroupName());
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public RemoveGroupResponse removeGroup(RpcController controller, RemoveGroupRequest request) throws ServiceException {
+    MasterProtos.RemoveGroupResponse response = null;
+    try {
+      MasterProtos.RemoveGroupResponse.Builder builder =
+          MasterProtos.RemoveGroupResponse.newBuilder();
+      master.getGroupAdminServer().removeGroup(request.getGroupName());
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public BalanceGroupResponse balanceGroup(RpcController controller, BalanceGroupRequest request) throws ServiceException {
+    MasterProtos.BalanceGroupResponse response = null;
+    try {
+      MasterProtos.BalanceGroupResponse.Builder builder =
+          MasterProtos.BalanceGroupResponse.newBuilder();
+      builder.setBalanceRan(master.getGroupAdminServer().balanceGroup(request.getGroupName()));
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public ListGroupInfosResponse listGroupInfos(RpcController controller,
+                                               ListGroupInfosRequest request) throws ServiceException {
+    MasterProtos.ListGroupInfosResponse response = null;
+    try {
+      MasterProtos.ListGroupInfosResponse.Builder builder =
+          MasterProtos.ListGroupInfosResponse.newBuilder();
+      for(GroupInfo groupInfo: master.getGroupAdminServer().listGroups()) {
+        builder.addGroupInfo(ProtobufUtil.toProtoGroupInfo(groupInfo));
+      }
+      response = builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
+
+  @Override
+  public GetGroupInfoOfServerResponse getGroupInfoOfServer(RpcController controller,
+                                                           GetGroupInfoOfServerRequest request) throws ServiceException {
+    MasterProtos.GetGroupInfoOfServerResponse response = null;
+    try {
+      MasterProtos.GetGroupInfoOfServerResponse.Builder builder =
+          MasterProtos.GetGroupInfoOfServerResponse.newBuilder();
+      GroupInfo groupInfo = master.getGroupAdminServer().getGroupOfServer(
+          new HostPort(request.getServer().getHostName(), request.getServer().getPort()));
+      response = builder.setGroupInfo(ProtobufUtil.toProtoGroupInfo(groupInfo)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return response;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 7d70dc1..e70b247 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.group.GroupAdminServer;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
 import com.google.protobuf.Service;
@@ -334,4 +335,14 @@ public interface MasterServices extends Server {
    * @throws IOException
    */
   public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException;
+
+  /**
+   * @return load balancer
+   */
+  public LoadBalancer getLoadBalancer();
+
+  /**
+   * @return load balancer
+   */
+  GroupAdminServer getGroupAdminServer();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index 553972f..623f9e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.group.GroupAdminServer;
+import org.apache.hadoop.hbase.group.GroupInfoManager;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
@@ -292,7 +294,13 @@ public class CreateTableProcedure
       ProcedureSyncWait.getMasterQuotaManager(env)
         .checkNamespaceTableAndRegionQuota(getTableName(), newRegions.size());
     }
-
+    GroupAdminServer groupAdminServer = env.getMasterServices().getGroupAdminServer();
+    //If master is not initialized and a create table is spawned then it is
+    //a special table and group affilition should be taken care of explicitly
+    if (groupAdminServer != null &&
+        env.getMasterServices().isInitialized()) {
+      groupAdminServer.prepareGroupForTable(hTableDescriptor);
+    }
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       final HRegionInfo[] regions = newRegions == null ? null :

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index c0a5091..a2ea83d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -431,5 +431,9 @@ public class DeleteTableProcedure
     if (!tableName.isSystemTable()) {
       ProcedureSyncWait.getMasterQuotaManager(env).removeTableFromNamespaceQuota(tableName);
     }
+    if (env.getMasterServices().getGroupAdminServer() != null) {
+      LOG.debug("Removing " + tableName + " from group.");
+      env.getMasterServices().getGroupAdminServer().cleanupGroupForTable(tableName);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 6ff8e36..756f507 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
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -414,6 +415,7 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if user has no authorization
    */
+
   private void requirePermission(String request, TableName tableName, byte[] family,
       byte[] qualifier, Action... permissions) throws IOException {
     User user = getActiveUser();
@@ -2558,4 +2560,34 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void postReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
       List<WALEntry> entries, CellScanner cells) throws IOException {
   }
+
+  @Override
+  public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             Set<HostPort> servers, String targetGroup) throws IOException {
+    requirePermission("moveServers", Action.ADMIN);
+  }
+
+  @Override
+  public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                            Set<TableName> tables, String targetGroup) throws IOException {
+    requirePermission("moveTables", Action.ADMIN);
+  }
+
+  @Override
+  public void preAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                          String name) throws IOException {
+    requirePermission("addGroup", Action.ADMIN);
+  }
+
+  @Override
+  public void preRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             String name) throws IOException {
+    requirePermission("removeGroup", Action.ADMIN);
+  }
+
+  @Override
+  public void preBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              String groupName) throws IOException {
+    requirePermission("balanceGroup", Action.ADMIN);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 dbba63b..c02b935 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
@@ -29,6 +29,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.logging.Log;
@@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -1240,6 +1242,56 @@ public class TestMasterObserver {
     public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final String namespace, final Quotas quotas) throws IOException {
     }
+
+    @Override
+    public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                               Set<HostPort> servers, String targetGroup) throws IOException {
+    }
+
+    @Override
+    public void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                                Set<HostPort> servers, String targetGroup) throws IOException {
+    }
+
+    @Override
+    public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              Set<TableName> tables, String targetGroup) throws IOException {
+    }
+
+    @Override
+    public void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                               Set<TableName> tables, String targetGroup) throws IOException {
+    }
+
+    @Override
+    public void preAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                            String name) throws IOException {
+    }
+
+    @Override
+    public void postAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             String name) throws IOException {
+    }
+
+    @Override
+    public void preRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                               String name) throws IOException {
+    }
+
+    @Override
+    public void postRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                                String name) throws IOException {
+    }
+
+    @Override
+    public void preBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                                String groupName) throws IOException {
+    }
+
+    @Override
+    public void postBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                                 String groupName, boolean balancerRan) throws IOException {
+    }
   }
 
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroups.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroups.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroups.java
new file mode 100644
index 0000000..aa7ef1f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroups.java
@@ -0,0 +1,366 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Sets;
+
+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.HostPort;
+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.master.HMaster;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+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 javax.management.MBeanServer;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+
+@Category({MediumTests.class})
+public class TestGroups extends TestGroupsBase {
+  protected static final Log LOG = LogFactory.getLog(TestGroups.class);
+  private static HMaster master;
+  private static boolean init = false;
+
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.getConfiguration().set(
+        HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
+        GroupBasedLoadBalancer.class.getName());
+    TEST_UTIL.getConfiguration().setBoolean(
+        HConstants.ZOOKEEPER_USEMULTI,
+        true);
+    TEST_UTIL.startMiniCluster(NUM_SLAVES_BASE);
+    TEST_UTIL.getConfiguration().set(
+        ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART,
+        ""+NUM_SLAVES_BASE);
+
+    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() &&
+            ((GroupBasedLoadBalancer) master.getLoadBalancer()).isOnline();
+      }
+    });
+    admin.setBalancerRunning(false,true);
+    groupAdmin = new VerifyingGroupAdminClient(admin.getConnection().getGroupAdmin(),
+        TEST_UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void beforeMethod() throws Exception {
+    if(!init) {
+      init = true;
+      afterMethod();
+    }
+
+  }
+
+  @After
+  public void afterMethod() throws Exception {
+    deleteTableIfNecessary();
+    deleteNamespaceIfNecessary();
+    deleteGroups();
+
+    int missing = NUM_SLAVES_BASE + 1 - cluster.getClusterStatus().getServers().size();
+    LOG.info("Restoring servers: "+missing);
+    for(int i=0; i<missing; i++) {
+      ((MiniHBaseCluster)cluster).startRegionServer();
+    }
+
+    groupAdmin.addGroup("master");
+    ServerName masterServerName =
+        ((MiniHBaseCluster)cluster).getMaster().getServerName();
+
+    groupAdmin.moveServers(
+        Sets.newHashSet(masterServerName.getHostPort()),
+        "master");
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        LOG.info("Waiting for cleanup to finish "+groupAdmin.listGroups());
+        //Might be greater since moving servers back to default
+        //is after starting a server
+
+        return groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP).getServers().size()
+           == NUM_SLAVES_BASE;
+      }
+    });
+  }
+
+  @Test
+  public void testJmx() throws Exception {
+    MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
+    Iterator<ObjectName> it = mBeanServer.queryNames(new ObjectName("hadoop:name=Group,service=Group"), null).iterator();
+    //verify it was loaded properly
+    assertEquals("hadoop:name=Group,service=Group", it.next().getCanonicalName());
+
+    final MXBeanImpl info = MXBeanImpl.init(groupAdmin, master);
+    GroupInfo defaultGroup = groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP);
+    assertEquals(2, info.getGroups().size());
+    assertEquals(defaultGroup.getName(), info.getGroups().get(0).getName());
+    assertEquals(defaultGroup.getServers(), Sets.newTreeSet(info.getGroups().get(0).getServers()));
+    assertEquals(defaultGroup.getServers(), Sets.newTreeSet(info.getServersByGroup().get(GroupInfo.DEFAULT_GROUP)));
+
+    GroupInfo barGroup = addGroup(groupAdmin, "bar", 3);
+    TableName tableName1 = TableName.valueOf(tablePrefix+"_testJmx1");
+    TableName tableName2 = TableName.valueOf(tablePrefix+"_testJmx2");
+    TEST_UTIL.createTable(tableName1, Bytes.toBytes("f"));
+    TEST_UTIL.createTable(tableName2, Bytes.toBytes("f"));
+    groupAdmin.moveTables(Sets.newHashSet(tableName2), barGroup.getName());
+    assertEquals(3, info.getGroups().size());
+
+    int defaultIndex = -1;
+    int barIndex = -1;
+
+    for(int i=0; i<info.getGroups().size(); i++) {
+      MXBean.GroupInfoBean bean = info.getGroups().get(i);
+      if(bean.getName().equals(defaultGroup.getName())) {
+        defaultIndex = i;
+      }
+      else if(bean.getName().equals(barGroup.getName())) {
+        barIndex = i;
+      }
+    }
+
+    defaultGroup = groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP);
+    assertEquals(defaultGroup.getName(),
+        info.getGroups().get(defaultIndex).getName());
+    for(TableName entry: defaultGroup.getTables()) {
+      assertTrue(info.getGroups().get(defaultIndex).getTables().contains(entry));
+    }
+    assertEquals(defaultGroup.getTables().size(),
+        info.getGroups().get(defaultIndex).getTables().size());
+    assertEquals(defaultGroup.getServers(),
+        Sets.newTreeSet(info.getGroups().get(defaultIndex).getServers()));
+
+    barGroup = groupAdmin.getGroupInfo(barGroup.getName());
+    assertEquals(barGroup.getName(),
+        info.getGroups().get(barIndex).getName());
+    for(TableName entry: barGroup.getTables()) {
+      assertTrue(info.getGroups().get(barIndex).getTables().contains(entry));
+    }
+    assertEquals(barGroup.getTables().size(),
+        info.getGroups().get(barIndex).getTables().size());
+    assertEquals(barGroup.getServers(),
+        Sets.newTreeSet(info.getGroups().get(barIndex).getServers()));
+  }
+
+  @Test
+  public void testBasicStartUp() throws IOException {
+    GroupInfo defaultInfo = groupAdmin.getGroupInfo(GroupInfo.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");
+    GroupInfo appInfo = addGroup(groupAdmin, "appInfo", 1);
+    admin.createNamespace(NamespaceDescriptor.create(nsName)
+        .addConfiguration(GroupInfo.NAMESPACEDESC_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
+    assertEquals(1, ProtobufUtil.getOnlineRegions(rs).size());
+    //verify prop was not stored as part of the schema
+    assertNull(admin.getTableDescriptor(tableName).getValue(GroupInfo.TABLEDESC_PROP_GROUP));
+  }
+
+  @Test
+  public void testDefaultNamespaceCreateAndAssign() throws Exception {
+    LOG.info("testDefaultNamespaceCreateAndAssign");
+    final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
+    admin.modifyNamespace(NamespaceDescriptor.create("default")
+        .addConfiguration(GroupInfo.NAMESPACEDESC_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");
+    groupAdmin.addGroup(groupName);
+    admin.createNamespace(NamespaceDescriptor.create(nsName)
+        .addConfiguration(GroupInfo.NAMESPACEDESC_PROP_GROUP, groupName)
+        .build());
+    //test removing a referenced group
+    try {
+      groupAdmin.removeGroup(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(GroupInfo.NAMESPACEDESC_PROP_GROUP, groupName)
+          .build());
+    String anotherGroup = tablePrefix+"_anotherGroup";
+    groupAdmin.addGroup(anotherGroup);
+    //test add non-existent group
+    admin.deleteNamespace(nsName);
+    groupAdmin.removeGroup(groupName);
+    try {
+      admin.createNamespace(NamespaceDescriptor.create(nsName)
+          .addConfiguration(GroupInfo.NAMESPACEDESC_PROP_GROUP, "foo")
+          .build());
+      fail("Expected a constraint exception");
+    } catch (IOException ex) {
+    }
+  }
+
+  @Test
+  public void testGroupInfoMultiAccessing() throws Exception {
+    GroupInfoManager manager = master.getGroupAdminServer().getGroupInfoManager();
+    final GroupInfo defaultGroup = manager.getGroup("default");
+    // getGroup updates default group's server list
+    // this process must not affect other threads iterating the list
+    Iterator<HostPort> it = defaultGroup.getServers().iterator();
+    manager.getGroup("default");
+    it.next();
+  }
+
+  @Test
+  public void testTracker() throws IOException, InterruptedException {
+    LOG.info("testTracker");
+    ZooKeeperWatcher watcher =
+        new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "testTracker", null);
+    GroupTracker tracker = new GroupTracker(watcher, null);
+    try {
+      final Map<String, GroupInfo> groupMap = new ConcurrentHashMap<String, GroupInfo>();
+      final AtomicBoolean stateChanged = new AtomicBoolean(false);
+      GroupTracker.Listener listener = new GroupTracker.Listener() {
+
+        @Override
+        public void groupMapChanged(Map<String, GroupInfo> map) {
+          groupMap.clear();
+          groupMap.putAll(map);
+          stateChanged.set(true);
+        }
+      };
+      tracker.addListener(listener);
+      tracker.start();
+
+      //wait for tracker to retrieve initial info
+      tracker.blockUntilReady(0);
+      int tries = 60000/100;
+      while(groupMap.size() < 1 && tries > 0) {
+        Thread.sleep(100);
+        tries--;
+      }
+      assertNotSame(0, tries);
+      assertNotNull(groupAdmin.getGroupInfo("default"));
+
+      stateChanged.set(false);
+      groupAdmin.addGroup("foo");
+      while(!stateChanged.get()) {
+        Thread.sleep(100);
+      }
+      stateChanged.set(false);
+      assertEquals(3, groupMap.size());
+      assertNotNull(tracker.getGroup("foo"));
+      assertEquals(0, tracker.getGroup("foo").getServers().size());
+
+      addGroup(groupAdmin, "bar", 1);
+      while(!stateChanged.get()) {
+        Thread.sleep(100);
+      }
+      stateChanged.set(false);
+      assertEquals(4, groupMap.size());
+      assertNotNull(tracker.getGroup("bar"));
+      assertEquals(1, tracker.getGroup("bar").getServers().size());
+    } finally {
+      if(tracker != null) {
+        tracker.stop();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsBase.java
new file mode 100644
index 0000000..f2834ce
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsBase.java
@@ -0,0 +1,561 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseCluster;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.security.SecureRandom;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public abstract class TestGroupsBase {
+  protected static final Log LOG = LogFactory.getLog(TestGroupsBase.class);
+
+  //shared
+  protected final static String groupPrefix = "Group";
+  protected final static String tablePrefix = "Group";
+  protected final static SecureRandom rand = new SecureRandom();
+
+  //shared, cluster type specific
+  protected static HBaseTestingUtility TEST_UTIL;
+  protected static HBaseAdmin admin;
+  protected static HBaseCluster cluster;
+  protected static GroupAdmin groupAdmin;
+
+  public final static long WAIT_TIMEOUT = 60000*5;
+  public final static int NUM_SLAVES_BASE = 4; //number of slaves for the smallest cluster
+
+
+
+  protected GroupInfo addGroup(GroupAdmin gAdmin, String groupName,
+                               int serverCount) throws IOException, InterruptedException {
+    GroupInfo defaultInfo = gAdmin
+        .getGroupInfo(GroupInfo.DEFAULT_GROUP);
+    assertTrue(defaultInfo != null);
+    assertTrue(defaultInfo.getServers().size() >= serverCount);
+    gAdmin.addGroup(groupName);
+
+    Set<HostPort> set = new HashSet<HostPort>();
+    for(HostPort server: defaultInfo.getServers()) {
+      if(set.size() == serverCount) {
+        break;
+      }
+      set.add(server);
+    }
+    gAdmin.moveServers(set, groupName);
+    GroupInfo result = gAdmin.getGroupInfo(groupName);
+    assertTrue(result.getServers().size() >= serverCount);
+    return result;
+  }
+
+  static void removeGroup(GroupAdminClient groupAdmin, String groupName) throws IOException {
+    GroupInfo groupInfo = groupAdmin.getGroupInfo(groupName);
+    groupAdmin.moveTables(groupInfo.getTables(), GroupInfo.DEFAULT_GROUP);
+    groupAdmin.moveServers(groupInfo.getServers(), GroupInfo.DEFAULT_GROUP);
+    groupAdmin.removeGroup(groupName);
+  }
+
+  protected void deleteTableIfNecessary() throws IOException {
+    for (HTableDescriptor desc : TEST_UTIL.getHBaseAdmin().listTables(tablePrefix+".*")) {
+      TEST_UTIL.deleteTable(desc.getName());
+    }
+  }
+
+  protected void deleteNamespaceIfNecessary() throws IOException {
+    for (NamespaceDescriptor desc : TEST_UTIL.getHBaseAdmin().listNamespaceDescriptors()) {
+      if(desc.getName().startsWith(tablePrefix)) {
+        admin.deleteNamespace(desc.getName());
+      }
+    }
+  }
+
+  protected void deleteGroups() throws IOException {
+    GroupAdminClient groupAdmin = new GroupAdminClient(TEST_UTIL.getConfiguration());
+    for(GroupInfo group: groupAdmin.listGroups()) {
+      if(!group.getName().equals(GroupInfo.DEFAULT_GROUP)) {
+        groupAdmin.moveTables(group.getTables(), GroupInfo.DEFAULT_GROUP);
+        groupAdmin.moveServers(group.getServers(), GroupInfo.DEFAULT_GROUP);
+        groupAdmin.removeGroup(group.getName());
+      }
+    }
+  }
+
+  public Map<TableName, List<String>> getTableRegionMap() throws IOException {
+    Map<TableName, List<String>> map = Maps.newTreeMap();
+    Map<TableName, Map<ServerName, List<String>>> tableServerRegionMap
+        = getTableServerRegionMap();
+    for(TableName tableName : tableServerRegionMap.keySet()) {
+      if(!map.containsKey(tableName)) {
+        map.put(tableName, new LinkedList<String>());
+      }
+      for(List<String> subset: tableServerRegionMap.get(tableName).values()) {
+        map.get(tableName).addAll(subset);
+      }
+    }
+    return map;
+  }
+
+  public Map<TableName, Map<ServerName, List<String>>> getTableServerRegionMap()
+      throws IOException {
+    Map<TableName, Map<ServerName, List<String>>> map = Maps.newTreeMap();
+    ClusterStatus status = TEST_UTIL.getHBaseClusterInterface().getClusterStatus();
+    for(ServerName serverName : status.getServers()) {
+      for(RegionLoad rl : status.getLoad(serverName).getRegionsLoad().values()) {
+        TableName tableName = HRegionInfo.getTable(rl.getName());
+        if(!map.containsKey(tableName)) {
+          map.put(tableName, new TreeMap<ServerName, List<String>>());
+        }
+        if(!map.get(tableName).containsKey(serverName)) {
+          map.get(tableName).put(serverName, new LinkedList<String>());
+        }
+        map.get(tableName).get(serverName).add(rl.getNameAsString());
+      }
+    }
+    return map;
+  }
+
+  @Test(expected = ConstraintException.class)
+  public void testGroupInfoOfTableNonExistent() throws Exception {
+    groupAdmin.getGroupInfoOfTable(TableName.valueOf("nonexistent"));
+  }
+
+  @Test
+  public void testCreateMultiRegion() throws IOException {
+    LOG.info("testCreateMultiRegion");
+    byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateMultiRegion");
+    byte[] end = {1,3,5,7,9};
+    byte[] start = {0,2,4,6,8};
+    byte[][] f = {Bytes.toBytes("f")};
+    TEST_UTIL.createTable(tableName, f,1,start,end,10);
+  }
+
+  @Test
+  public void testCreateAndDrop() throws Exception {
+    LOG.info("testCreateAndDrop");
+
+    final TableName tableName = TableName.valueOf(tablePrefix + "_testCreateAndDrop");
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("cf"));
+    //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(tableName) != null;
+      }
+    });
+    TEST_UTIL.deleteTable(tableName);
+  }
+
+
+  @Test
+  public void testSimpleRegionServerMove() throws IOException,
+      InterruptedException {
+    LOG.info("testSimpleRegionServerMove");
+
+    GroupInfo appInfo = addGroup(groupAdmin, groupPrefix + rand.nextInt(), 1);
+    GroupInfo adminInfo = addGroup(groupAdmin, groupPrefix + rand.nextInt(), 1);
+    GroupInfo dInfo = groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP);
+    assertEquals(4, groupAdmin.listGroups().size());
+    assertEquals(1, adminInfo.getServers().size());
+    assertEquals(1, appInfo.getServers().size());
+    assertEquals(admin.getClusterStatus().getServers().size() - 3, dInfo.getServers().size());
+    groupAdmin.moveServers(appInfo.getServers(),
+        GroupInfo.DEFAULT_GROUP);
+    groupAdmin.removeGroup(appInfo.getName());
+    groupAdmin.moveServers(adminInfo.getServers(),
+        GroupInfo.DEFAULT_GROUP);
+    groupAdmin.removeGroup(adminInfo.getName());
+    assertEquals(groupAdmin.listGroups().size(), 2);
+  }
+
+  @Test
+  public void testMoveServers() throws Exception {
+    LOG.info("testMoveServers");
+
+    //create groups and assign servers
+    addGroup(groupAdmin, "bar", 3);
+    groupAdmin.addGroup("foo");
+
+    GroupInfo barGroup = groupAdmin.getGroupInfo("bar");
+    GroupInfo fooGroup = groupAdmin.getGroupInfo("foo");
+    assertEquals(3, barGroup.getServers().size());
+    assertEquals(0, fooGroup.getServers().size());
+
+    //test fail bogus server move
+    try {
+      groupAdmin.moveServers(Sets.newHashSet(HostPort.valueOf("foo:9999")),"foo");
+      fail("Bogus servers shouldn't have been successfully moved.");
+    } catch(IOException ex) {
+      String exp = "Server foo:9999 is not an online server in default 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");
+    groupAdmin.moveServers(barGroup.getServers(), fooGroup.getName());
+
+    barGroup = groupAdmin.getGroupInfo("bar");
+    fooGroup = groupAdmin.getGroupInfo("foo");
+    assertEquals(0,barGroup.getServers().size());
+    assertEquals(3,fooGroup.getServers().size());
+
+    LOG.info("moving servers "+fooGroup.getServers()+" to group default");
+    groupAdmin.moveServers(fooGroup.getServers(), GroupInfo.DEFAULT_GROUP);
+
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return admin.getClusterStatus().getServers().size() -1 ==
+            groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP).getServers().size();
+      }
+    });
+
+    fooGroup = groupAdmin.getGroupInfo("foo");
+    assertEquals(0,fooGroup.getServers().size());
+
+    //test group removal
+    LOG.info("Remove group "+barGroup.getName());
+    groupAdmin.removeGroup(barGroup.getName());
+    assertEquals(null, groupAdmin.getGroupInfo(barGroup.getName()));
+    LOG.info("Remove group "+fooGroup.getName());
+    groupAdmin.removeGroup(fooGroup.getName());
+    assertEquals(null, groupAdmin.getGroupInfo(fooGroup.getName()));
+  }
+
+  @Test
+  public void testTableMoveAndDrop() throws Exception {
+    LOG.info("testTableMove");
+
+    final TableName tableName = TableName.valueOf(tablePrefix + "_testTableMoveAndDrop");
+    final byte[] familyNameBytes = Bytes.toBytes("f");
+    String newGroupName = "g_" + rand.nextInt();
+    final GroupInfo newGroup = addGroup(groupAdmin, 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;
+      }
+    });
+
+    GroupInfo tableGrp = groupAdmin.getGroupInfoOfTable(tableName);
+    assertTrue(tableGrp.getName().equals(GroupInfo.DEFAULT_GROUP));
+
+    //change table's group
+    LOG.info("Moving table "+tableName+" to "+newGroup.getName());
+    groupAdmin.moveTables(Sets.newHashSet(tableName), newGroup.getName());
+
+    //verify group change
+    assertEquals(newGroup.getName(),
+        groupAdmin.getGroupInfoOfTable(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.getHostPort())) {
+              count += serverMap.get(rs).size();
+            }
+          }
+        }
+        return count == 5;
+      }
+    });
+
+    //verify removed table is removed from group
+    TEST_UTIL.deleteTable(tableName);
+    assertEquals(0, groupAdmin.getGroupInfo(newGroup.getName()).getTables().size());
+  }
+
+  @Test
+  public void testGroupBalance() throws Exception {
+    LOG.info("testGroupBalance");
+    String newGroupName = "g_" + rand.nextInt();
+    final GroupInfo newGroup = addGroup(groupAdmin, newGroupName, 3);
+
+    final TableName tableName = TableName.valueOf(tablePrefix+"_ns", "testGroupBalance");
+    admin.createNamespace(
+        NamespaceDescriptor.create(tableName.getNamespaceAsString())
+            .addConfiguration(GroupInfo.NAMESPACEDESC_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)) {
+        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
+    groupAdmin.balanceGroup(GroupInfo.DEFAULT_GROUP);
+    assertEquals(6, getTableServerRegionMap().get(tableName).get(first).size());
+
+    groupAdmin.balanceGroup(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;
+      }
+    });
+  }
+
+  @Test
+  public void testRegionMove() throws Exception {
+    LOG.info("testRegionMove");
+
+    final GroupInfo newGroup = addGroup(groupAdmin, "g_" + rand.nextInt(), 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.getHostPort())) {
+        targetServer = server;
+        break;
+      }
+    }
+
+    final AdminProtos.AdminService.BlockingInterface targetRS =
+        admin.getConnection().getAdmin(targetServer);
+
+    //move target server to group
+    groupAdmin.moveServers(Sets.newHashSet(targetServer.getHostPort()),
+        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 {
+        return
+            getTableRegionMap().get(tableName) != null &&
+                getTableRegionMap().get(tableName).size() == 6 &&
+                admin.getClusterStatus().getRegionsInTransition().size() < 1;
+      }
+    });
+
+    //verify that targetServer didn't open it
+    assertFalse(ProtobufUtil.getOnlineRegions(targetRS).contains(targetRegion));
+  }
+
+  @Test
+  public void testFailRemoveGroup() throws IOException, InterruptedException {
+    LOG.info("testFailRemoveGroup");
+
+    addGroup(groupAdmin, "bar", 3);
+    TableName tableName = TableName.valueOf(tablePrefix+"_my_table");
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
+    groupAdmin.moveTables(Sets.newHashSet(tableName), "bar");
+    GroupInfo barGroup = groupAdmin.getGroupInfo("bar");
+    //group is not empty therefore it should fail
+    try {
+      groupAdmin.removeGroup(barGroup.getName());
+      fail("Expected remove group to fail");
+    } catch(IOException e) {
+    }
+    //group cannot lose all it's servers therefore it should fail
+    try {
+      groupAdmin.moveServers(barGroup.getServers(), GroupInfo.DEFAULT_GROUP);
+      fail("Expected move servers to fail");
+    } catch(IOException e) {
+    }
+
+    groupAdmin.moveTables(barGroup.getTables(), GroupInfo.DEFAULT_GROUP);
+    try {
+      groupAdmin.removeGroup(barGroup.getName());
+      fail("Expected move servers to fail");
+    } catch(IOException e) {
+    }
+
+    groupAdmin.moveServers(barGroup.getServers(), GroupInfo.DEFAULT_GROUP);
+    groupAdmin.removeGroup(barGroup.getName());
+
+    assertEquals(2, groupAdmin.listGroups().size());
+  }
+
+  @Test
+  public void testKillRS() throws Exception {
+    LOG.info("testKillRS");
+    GroupInfo appInfo = addGroup(groupAdmin, "appInfo", 1);
+
+
+    final TableName tableName = TableName.valueOf(tablePrefix+"_ns", "_testKillRS");
+    admin.createNamespace(
+        NamespaceDescriptor.create(tableName.getNamespaceAsString())
+            .addConfiguration(GroupInfo.NAMESPACEDESC_PROP_GROUP, appInfo.getName()).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().first().toString());
+    AdminProtos.AdminService.BlockingInterface targetRS =
+        admin.getConnection().getAdmin(targetServer);
+    HRegionInfo targetRegion = ProtobufUtil.getOnlineRegions(targetRS).get(0);
+    assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
+
+    try {
+      //stopping may cause an exception
+      //due to the connection loss
+      targetRS.stopServer(null,
+          AdminProtos.StopServerRequest.newBuilder().setReason("Die").build());
+    } catch(Exception e) {
+    }
+    assertFalse(cluster.getClusterStatus().getServers().contains(targetServer));
+
+    //wait for created table to be assigned
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return cluster.getClusterStatus().getRegionsInTransition().size() == 0;
+      }
+    });
+    TreeSet<HostPort> newServers = Sets.newTreeSet();
+    newServers.add(groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP).getServers().first());
+    groupAdmin.moveServers(newServers, appInfo.getName());
+    admin.assign(targetRegion.getRegionName());
+
+    //wait for region to be assigned
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return cluster.getClusterStatus().getRegionsInTransition().size() == 0;
+      }
+    });
+
+    targetServer = ServerName.parseServerName(newServers.first().toString());
+    targetRS =
+        admin.getConnection().getAdmin(targetServer);
+    assertEquals(1, ProtobufUtil.getOnlineRegions(targetRS).size());
+    assertEquals(tableName,
+        ProtobufUtil.getOnlineRegions(targetRS).get(0).getTable());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/group/VerifyingGroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/group/VerifyingGroupAdminClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/VerifyingGroupAdminClient.java
new file mode 100644
index 0000000..fb8973e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/VerifyingGroupAdminClient.java
@@ -0,0 +1,159 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Assert;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+
+public class VerifyingGroupAdminClient implements GroupAdmin {
+  private Table table;
+  private ZooKeeperWatcher zkw;
+  private GroupSerDe serDe;
+  private GroupAdmin wrapped;
+
+  public VerifyingGroupAdminClient(GroupAdmin groupAdmin, Configuration conf)
+      throws IOException {
+    wrapped = groupAdmin;
+    table = ConnectionFactory.createConnection(conf).getTable(GroupInfoManager.GROUP_TABLE_NAME);
+    zkw = new ZooKeeperWatcher(conf, this.getClass().getSimpleName(), null);
+    serDe = new GroupSerDe();
+  }
+
+  @Override
+  public void addGroup(String groupName) throws IOException {
+    wrapped.addGroup(groupName);
+    verify();
+  }
+
+  @Override
+  public GroupInfo getGroupInfo(String groupName) throws IOException {
+    return wrapped.getGroupInfo(groupName);
+  }
+
+  @Override
+  public GroupInfo getGroupInfoOfTable(TableName tableName) throws IOException {
+    return wrapped.getGroupInfoOfTable(tableName);
+  }
+
+  @Override
+  public void moveServers(Set<HostPort> servers, String targetGroup) throws IOException {
+    wrapped.moveServers(servers, targetGroup);
+    verify();
+  }
+
+  @Override
+  public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
+    wrapped.moveTables(tables, targetGroup);
+    verify();
+  }
+
+  @Override
+  public void removeGroup(String name) throws IOException {
+    wrapped.removeGroup(name);
+    verify();
+  }
+
+  @Override
+  public boolean balanceGroup(String name) throws IOException {
+    return wrapped.balanceGroup(name);
+  }
+
+  @Override
+  public List<GroupInfo> listGroups() throws IOException {
+    return wrapped.listGroups();
+  }
+
+  @Override
+  public GroupInfo getGroupOfServer(HostPort hostPort) throws IOException {
+    return wrapped.getGroupOfServer(hostPort);
+  }
+
+  public void verify() throws IOException {
+    Get get = new Get(GroupInfoManager.ROW_KEY);
+    get.addFamily(GroupInfoManager.META_FAMILY_BYTES);
+    Map<String, GroupInfo> groupMap = Maps.newHashMap();
+    Set<GroupInfo> zList = Sets.newHashSet();
+
+    Result result = table.get(get);
+    if(!result.isEmpty()) {
+      NavigableMap<byte[],NavigableMap<byte[],byte[]>> dataMap =
+          result.getNoVersionMap();
+      for(byte[] groupNameBytes:
+          dataMap.get(GroupInfoManager.META_FAMILY_BYTES).keySet()) {
+        RSGroupProtos.GroupInfo proto =
+            RSGroupProtos.GroupInfo.parseFrom(
+                dataMap.get(GroupInfoManager.META_FAMILY_BYTES).get(groupNameBytes));
+        GroupInfo groupInfo = ProtobufUtil.toGroupInfo(proto);
+        groupMap.put(groupInfo.getName(), groupInfo);
+      }
+    }
+    Assert.assertEquals(Sets.newHashSet(groupMap.values()),
+        Sets.newHashSet(wrapped.listGroups()));
+    try {
+      String groupBasePath = ZKUtil.joinZNode(zkw.baseZNode, "groupInfo");
+      for(String znode: ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) {
+        byte[] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(groupBasePath, znode));
+        if(data.length > 0) {
+          ProtobufUtil.expectPBMagicPrefix(data);
+          ByteArrayInputStream bis = new ByteArrayInputStream(
+              data, ProtobufUtil.lengthOfPBMagic(), data.length);
+          zList.add(ProtobufUtil.toGroupInfo(RSGroupProtos.GroupInfo.parseFrom(bis)));
+        }
+      }
+      Assert.assertEquals(zList.size(), groupMap.size());
+      for(GroupInfo groupInfo: zList) {
+        Assert.assertTrue(groupMap.get(groupInfo.getName()).equals(groupInfo));
+      }
+    } catch (KeeperException e) {
+      throw new IOException("ZK verification failed", e);
+    } catch (DeserializationException e) {
+      throw new IOException("ZK verification failed", e);
+    } catch (InterruptedException e) {
+      throw new IOException("ZK verification failed", e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index 51861d6..b342b0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -27,12 +27,16 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -52,6 +56,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
@@ -537,7 +542,7 @@ public class TestAssignmentManagerOnCluster {
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
 
-      MyLoadBalancer.controledRegion = hri.getEncodedName();
+      MyLoadBalancer.controledRegion = hri;
 
       HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
       master.assignRegion(hri);
@@ -561,6 +566,105 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.deleteTable(table);
     }
   }
+  
+  /**
+   * This tests round-robin assignment failed due to no bulkplan
+   */
+  @Test (timeout=60000)
+  public void testRoundRobinAssignmentFailed() throws Exception {
+    String table = "testRoundRobinAssignmentFailed";
+    try {
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      desc.addFamily(new HColumnDescriptor(FAMILY));
+      admin.createTable(desc);
+
+      Table meta = admin.getConnection().getTable(TableName.META_TABLE_NAME);
+      HRegionInfo hri = new HRegionInfo(
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+      MetaTableAccessor.addRegionToMeta(meta, hri);
+
+      MyLoadBalancer.controledRegion = hri;
+
+      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+      AssignmentManager am = master.getAssignmentManager();
+      // round-robin assignment but balancer cannot find a plan
+      // assignment should fail
+      am.assign(Arrays.asList(hri));
+
+      // if bulk assignment cannot update region state to online
+      // or failed_open this waits until timeout
+      assertFalse(am.waitForAssignment(hri));
+      RegionState state = am.getRegionStates().getRegionState(hri);
+      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
+      // Failed to open since no plan, so it's on no server
+      assertNull(state.getServerName());
+
+      // try again with valid plan
+      MyLoadBalancer.controledRegion = null;
+      am.assign(Arrays.asList(hri));
+      assertTrue(am.waitForAssignment(hri));
+
+      ServerName serverName = master.getAssignmentManager().
+        getRegionStates().getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
+    } finally {
+      MyLoadBalancer.controledRegion = null;
+      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+    }
+  }
+  
+  /**
+   * This tests retain assignment failed due to no bulkplan
+   */
+  @Test (timeout=60000)
+  public void testRetainAssignmentFailed() throws Exception {
+    String table = "testRetainAssignmentFailed";
+    try {
+      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      desc.addFamily(new HColumnDescriptor(FAMILY));
+      admin.createTable(desc);
+
+      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+      HRegionInfo hri = new HRegionInfo(
+        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
+      MetaTableAccessor.addRegionToMeta(meta, hri);
+
+      MyLoadBalancer.controledRegion = hri;
+
+      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+      AssignmentManager am = master.getAssignmentManager();
+
+      Map<HRegionInfo, ServerName> regions = new HashMap<HRegionInfo, ServerName>();
+      ServerName dest = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
+      regions.put(hri, dest);
+      // retainAssignment but balancer cannot find a plan
+      // assignment should fail
+      am.assign(regions);
+
+      // if retain assignment cannot update region state to online
+      // or failed_open this waits until timeout
+      assertFalse(am.waitForAssignment(hri));
+      RegionState state = am.getRegionStates().getRegionState(hri);
+      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
+      // Failed to open since no plan, so it's on no server
+      assertNull(state.getServerName());
+
+      // try retainAssigment again with valid plan
+      MyLoadBalancer.controledRegion = null;
+      am.assign(regions);
+      assertTrue(am.waitForAssignment(hri));
+
+      ServerName serverName = master.getAssignmentManager().
+        getRegionStates().getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
+      
+      // it retains on same server as specified
+      assertEquals(serverName, dest);
+    } finally {
+      MyLoadBalancer.controledRegion = null;
+      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+    }
+  }
 
   /**
    * This tests region open failure which is not recoverable
@@ -1170,7 +1274,7 @@ public class TestAssignmentManagerOnCluster {
 
   static class MyLoadBalancer extends StochasticLoadBalancer {
     // For this region, if specified, always assign to nowhere
-    static volatile String controledRegion = null;
+    static volatile HRegionInfo controledRegion = null;
 
     static volatile Integer countRegionServers = null;
     static AtomicInteger counter = new AtomicInteger(0);
@@ -1178,7 +1282,7 @@ public class TestAssignmentManagerOnCluster {
     @Override
     public ServerName randomAssignment(HRegionInfo regionInfo,
         List<ServerName> servers) {
-      if (regionInfo.getEncodedName().equals(controledRegion)) {
+      if (regionInfo.equals(controledRegion)) {
         return null;
       }
       return super.randomAssignment(regionInfo, servers);
@@ -1196,8 +1300,26 @@ public class TestAssignmentManagerOnCluster {
           return null;
         }
       }
+      if (regions.get(0).equals(controledRegion)) {
+        Map<ServerName, List<HRegionInfo>> m = Maps.newHashMap();
+        m.put(LoadBalancer.BOGUS_SERVER_NAME, regions);
+        return m;
+      }
       return super.roundRobinAssignment(regions, servers);
     }
+
+    @Override
+    public Map<ServerName, List<HRegionInfo>> retainAssignment(
+        Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
+      for (HRegionInfo hri : regions.keySet()) {
+        if (hri.equals(controledRegion)) {
+          Map<ServerName, List<HRegionInfo>> m = Maps.newHashMap();
+          m.put(LoadBalancer.BOGUS_SERVER_NAME, Lists.newArrayList(regions.keySet()));
+          return m;
+        }
+      }
+      return super.retainAssignment(regions, servers);
+    }
   }
 
   public static class MyMaster extends HMaster {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index bc9af45..35f0695 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -60,6 +60,8 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.group.GroupAdmin;
+import org.apache.hadoop.hbase.group.GroupAdminServer;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -396,32 +398,29 @@ public class TestCatalogJanitor {
 
     @Override
     public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
     }
 
     @Override
     public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
     }
 
     @Override
     public void deleteNamespace(String name) throws IOException {
-      //To change body of implemented methods use File | Settings | File Templates.
     }
 
     @Override
     public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
+      return null;
     }
 
     @Override
     public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
+      return null;
     }
 
     @Override
     public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
-      return null;  //To change body of implemented methods use File | Settings | File Templates.
+      return null;
     }
 
     @Override
@@ -436,6 +435,9 @@ public class TestCatalogJanitor {
         final long nonce) throws IOException {
       return -1;
     }
+    public LoadBalancer getLoadBalancer() {
+      return null;
+    }
 
     @Override
     public long truncateTable(
@@ -446,6 +448,10 @@ public class TestCatalogJanitor {
       return -1;
     }
 
+    public GroupAdminServer getGroupAdminServer() {
+      return null;
+    }
+
 
     @Override
     public long modifyTable(


[4/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RSGroupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RSGroupProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RSGroupProtos.java
new file mode 100644
index 0000000..a39e48a
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RSGroupProtos.java
@@ -0,0 +1,1331 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: RSGroup.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class RSGroupProtos {
+  private RSGroupProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface GroupInfoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string name = 1;
+    /**
+     * <code>required string name = 1;</code>
+     */
+    boolean hasName();
+    /**
+     * <code>required string name = 1;</code>
+     */
+    java.lang.String getName();
+    /**
+     * <code>required string name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNameBytes();
+
+    // repeated .hbase.pb.HostPort servers = 4;
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> 
+        getServersList();
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort getServers(int index);
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    int getServersCount();
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder> 
+        getServersOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder getServersOrBuilder(
+        int index);
+
+    // repeated .hbase.pb.TableName tables = 3;
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
+        getTablesList();
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index);
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    int getTablesCount();
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTablesOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GroupInfo}
+   */
+  public static final class GroupInfo extends
+      com.google.protobuf.GeneratedMessage
+      implements GroupInfoOrBuilder {
+    // Use GroupInfo.newBuilder() to construct.
+    private GroupInfo(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GroupInfo(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GroupInfo defaultInstance;
+    public static GroupInfo getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GroupInfo getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GroupInfo(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              name_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                tables_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              tables_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                servers_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              servers_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          tables_ = java.util.Collections.unmodifiableList(tables_);
+        }
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          servers_ = java.util.Collections.unmodifiableList(servers_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.internal_static_hbase_pb_GroupInfo_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.internal_static_hbase_pb_GroupInfo_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.class, org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GroupInfo> PARSER =
+        new com.google.protobuf.AbstractParser<GroupInfo>() {
+      public GroupInfo parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GroupInfo(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GroupInfo> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string name = 1;
+    public static final int NAME_FIELD_NUMBER = 1;
+    private java.lang.Object name_;
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public boolean hasName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public java.lang.String getName() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          name_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNameBytes() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        name_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .hbase.pb.HostPort servers = 4;
+    public static final int SERVERS_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> servers_;
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> getServersList() {
+      return servers_;
+    }
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder> 
+        getServersOrBuilderList() {
+      return servers_;
+    }
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    public int getServersCount() {
+      return servers_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort getServers(int index) {
+      return servers_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder getServersOrBuilder(
+        int index) {
+      return servers_.get(index);
+    }
+
+    // repeated .hbase.pb.TableName tables = 3;
+    public static final int TABLES_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tables_;
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTablesList() {
+      return tables_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTablesOrBuilderList() {
+      return tables_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    public int getTablesCount() {
+      return tables_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index) {
+      return tables_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName tables = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
+        int index) {
+      return tables_.get(index);
+    }
+
+    private void initFields() {
+      name_ = "";
+      servers_ = java.util.Collections.emptyList();
+      tables_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getServersCount(); i++) {
+        if (!getServers(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getTablesCount(); i++) {
+        if (!getTables(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getNameBytes());
+      }
+      for (int i = 0; i < tables_.size(); i++) {
+        output.writeMessage(3, tables_.get(i));
+      }
+      for (int i = 0; i < servers_.size(); i++) {
+        output.writeMessage(4, servers_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getNameBytes());
+      }
+      for (int i = 0; i < tables_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, tables_.get(i));
+      }
+      for (int i = 0; i < servers_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, servers_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo other = (org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo) obj;
+
+      boolean result = true;
+      result = result && (hasName() == other.hasName());
+      if (hasName()) {
+        result = result && getName()
+            .equals(other.getName());
+      }
+      result = result && getServersList()
+          .equals(other.getServersList());
+      result = result && getTablesList()
+          .equals(other.getTablesList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasName()) {
+        hash = (37 * hash) + NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getName().hashCode();
+      }
+      if (getServersCount() > 0) {
+        hash = (37 * hash) + SERVERS_FIELD_NUMBER;
+        hash = (53 * hash) + getServersList().hashCode();
+      }
+      if (getTablesCount() > 0) {
+        hash = (37 * hash) + TABLES_FIELD_NUMBER;
+        hash = (53 * hash) + getTablesList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GroupInfo}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.internal_static_hbase_pb_GroupInfo_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.internal_static_hbase_pb_GroupInfo_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.class, org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getServersFieldBuilder();
+          getTablesFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        name_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (serversBuilder_ == null) {
+          servers_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          serversBuilder_.clear();
+        }
+        if (tablesBuilder_ == null) {
+          tables_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          tablesBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.internal_static_hbase_pb_GroupInfo_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo build() {
+        org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo result = new org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.name_ = name_;
+        if (serversBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            servers_ = java.util.Collections.unmodifiableList(servers_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.servers_ = servers_;
+        } else {
+          result.servers_ = serversBuilder_.build();
+        }
+        if (tablesBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            tables_ = java.util.Collections.unmodifiableList(tables_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.tables_ = tables_;
+        } else {
+          result.tables_ = tablesBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.getDefaultInstance()) return this;
+        if (other.hasName()) {
+          bitField0_ |= 0x00000001;
+          name_ = other.name_;
+          onChanged();
+        }
+        if (serversBuilder_ == null) {
+          if (!other.servers_.isEmpty()) {
+            if (servers_.isEmpty()) {
+              servers_ = other.servers_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureServersIsMutable();
+              servers_.addAll(other.servers_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.servers_.isEmpty()) {
+            if (serversBuilder_.isEmpty()) {
+              serversBuilder_.dispose();
+              serversBuilder_ = null;
+              servers_ = other.servers_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              serversBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getServersFieldBuilder() : null;
+            } else {
+              serversBuilder_.addAllMessages(other.servers_);
+            }
+          }
+        }
+        if (tablesBuilder_ == null) {
+          if (!other.tables_.isEmpty()) {
+            if (tables_.isEmpty()) {
+              tables_ = other.tables_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureTablesIsMutable();
+              tables_.addAll(other.tables_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tables_.isEmpty()) {
+            if (tablesBuilder_.isEmpty()) {
+              tablesBuilder_.dispose();
+              tablesBuilder_ = null;
+              tables_ = other.tables_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              tablesBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTablesFieldBuilder() : null;
+            } else {
+              tablesBuilder_.addAllMessages(other.tables_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasName()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getServersCount(); i++) {
+          if (!getServers(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        for (int i = 0; i < getTablesCount(); i++) {
+          if (!getTables(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string name = 1;
+      private java.lang.Object name_ = "";
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public boolean hasName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public java.lang.String getName() {
+        java.lang.Object ref = name_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          name_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNameBytes() {
+        java.lang.Object ref = name_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          name_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder clearName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        name_ = getDefaultInstance().getName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .hbase.pb.HostPort servers = 4;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> servers_ =
+        java.util.Collections.emptyList();
+      private void ensureServersIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          servers_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort>(servers_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder> serversBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> getServersList() {
+        if (serversBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(servers_);
+        } else {
+          return serversBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public int getServersCount() {
+        if (serversBuilder_ == null) {
+          return servers_.size();
+        } else {
+          return serversBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort getServers(int index) {
+        if (serversBuilder_ == null) {
+          return servers_.get(index);
+        } else {
+          return serversBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder setServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort value) {
+        if (serversBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServersIsMutable();
+          servers_.set(index, value);
+          onChanged();
+        } else {
+          serversBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder setServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder builderForValue) {
+        if (serversBuilder_ == null) {
+          ensureServersIsMutable();
+          servers_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          serversBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder addServers(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort value) {
+        if (serversBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServersIsMutable();
+          servers_.add(value);
+          onChanged();
+        } else {
+          serversBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder addServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort value) {
+        if (serversBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServersIsMutable();
+          servers_.add(index, value);
+          onChanged();
+        } else {
+          serversBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder addServers(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder builderForValue) {
+        if (serversBuilder_ == null) {
+          ensureServersIsMutable();
+          servers_.add(builderForValue.build());
+          onChanged();
+        } else {
+          serversBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder addServers(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder builderForValue) {
+        if (serversBuilder_ == null) {
+          ensureServersIsMutable();
+          servers_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          serversBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder addAllServers(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort> values) {
+        if (serversBuilder_ == null) {
+          ensureServersIsMutable();
+          super.addAll(values, servers_);
+          onChanged();
+        } else {
+          serversBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder clearServers() {
+        if (serversBuilder_ == null) {
+          servers_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          serversBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public Builder removeServers(int index) {
+        if (serversBuilder_ == null) {
+          ensureServersIsMutable();
+          servers_.remove(index);
+          onChanged();
+        } else {
+          serversBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder getServersBuilder(
+          int index) {
+        return getServersFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder getServersOrBuilder(
+          int index) {
+        if (serversBuilder_ == null) {
+          return servers_.get(index);  } else {
+          return serversBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder> 
+           getServersOrBuilderList() {
+        if (serversBuilder_ != null) {
+          return serversBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(servers_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder addServersBuilder() {
+        return getServersFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder addServersBuilder(
+          int index) {
+        return getServersFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.HostPort servers = 4;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder> 
+           getServersBuilderList() {
+        return getServersFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder> 
+          getServersFieldBuilder() {
+        if (serversBuilder_ == null) {
+          serversBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder>(
+                  servers_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          servers_ = null;
+        }
+        return serversBuilder_;
+      }
+
+      // repeated .hbase.pb.TableName tables = 3;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tables_ =
+        java.util.Collections.emptyList();
+      private void ensureTablesIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          tables_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>(tables_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tablesBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTablesList() {
+        if (tablesBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(tables_);
+        } else {
+          return tablesBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public int getTablesCount() {
+        if (tablesBuilder_ == null) {
+          return tables_.size();
+        } else {
+          return tablesBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index) {
+        if (tablesBuilder_ == null) {
+          return tables_.get(index);
+        } else {
+          return tablesBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder setTables(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tablesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTablesIsMutable();
+          tables_.set(index, value);
+          onChanged();
+        } else {
+          tablesBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder setTables(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tablesBuilder_ == null) {
+          ensureTablesIsMutable();
+          tables_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          tablesBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder addTables(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tablesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTablesIsMutable();
+          tables_.add(value);
+          onChanged();
+        } else {
+          tablesBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder addTables(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tablesBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTablesIsMutable();
+          tables_.add(index, value);
+          onChanged();
+        } else {
+          tablesBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder addTables(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tablesBuilder_ == null) {
+          ensureTablesIsMutable();
+          tables_.add(builderForValue.build());
+          onChanged();
+        } else {
+          tablesBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder addTables(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tablesBuilder_ == null) {
+          ensureTablesIsMutable();
+          tables_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          tablesBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder addAllTables(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> values) {
+        if (tablesBuilder_ == null) {
+          ensureTablesIsMutable();
+          super.addAll(values, tables_);
+          onChanged();
+        } else {
+          tablesBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder clearTables() {
+        if (tablesBuilder_ == null) {
+          tables_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          tablesBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public Builder removeTables(int index) {
+        if (tablesBuilder_ == null) {
+          ensureTablesIsMutable();
+          tables_.remove(index);
+          onChanged();
+        } else {
+          tablesBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTablesBuilder(
+          int index) {
+        return getTablesFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
+          int index) {
+        if (tablesBuilder_ == null) {
+          return tables_.get(index);  } else {
+          return tablesBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+           getTablesOrBuilderList() {
+        if (tablesBuilder_ != null) {
+          return tablesBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(tables_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTablesBuilder() {
+        return getTablesFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTablesBuilder(
+          int index) {
+        return getTablesFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName tables = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder> 
+           getTablesBuilderList() {
+        return getTablesFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTablesFieldBuilder() {
+        if (tablesBuilder_ == null) {
+          tablesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tables_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          tables_ = null;
+        }
+        return tablesBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GroupInfo)
+    }
+
+    static {
+      defaultInstance = new GroupInfo(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GroupInfo)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_GroupInfo_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_GroupInfo_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\rRSGroup.proto\022\010hbase.pb\032\013HBase.proto\"c" +
+      "\n\tGroupInfo\022\014\n\004name\030\001 \002(\t\022#\n\007servers\030\004 \003" +
+      "(\0132\022.hbase.pb.HostPort\022#\n\006tables\030\003 \003(\0132\023" +
+      ".hbase.pb.TableNameBC\n*org.apache.hadoop" +
+      ".hbase.protobuf.generatedB\rRSGroupProtos" +
+      "H\001\210\001\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_hbase_pb_GroupInfo_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_hbase_pb_GroupInfo_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_GroupInfo_descriptor,
+              new java.lang.String[] { "Name", "Servers", "Tables", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
index d09598a..cd0d083 100644
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ b/hbase-protocol/src/main/protobuf/HBase.proto
@@ -237,3 +237,9 @@ message NamespaceDescriptor {
 message RegionServerInfo {
   optional int32 infoPort = 1;
 }
+
+message HostPort {
+  required string host_name = 1;
+  required uint32 port = 2;
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index 10a7854..3e58492 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -31,6 +31,7 @@ import "Client.proto";
 import "ClusterStatus.proto";
 import "ErrorHandling.proto";
 import "Quota.proto";
+import "RSGroup.proto";
 
 /* Column-level protobufs */
 
@@ -450,6 +451,92 @@ message MajorCompactionTimestampResponse {
   required int64 compaction_timestamp = 1;
 }
 
+/** Group level protobufs */
+
+message ListTablesOfGroupRequest {
+  required string group_name = 1;
+}
+
+message ListTablesOfGroupResponse {
+  repeated TableName table_name = 1;
+}
+
+message GetGroupInfoRequest {
+  required string group_name = 1;
+}
+
+message GetGroupInfoResponse {
+  optional GroupInfo group_info = 1;
+}
+
+message GetGroupInfoOfTableRequest {
+  required TableName table_name = 1;
+}
+
+message GetGroupInfoOfTableResponse {
+  optional GroupInfo group_info = 1;
+}
+
+message MoveServersRequest {
+  required string target_group = 1;
+  repeated HostPort servers = 2;
+}
+
+message MoveServersResponse {
+}
+
+message MoveTablesRequest {
+  required string target_group = 1;
+  repeated TableName table_name = 2;
+}
+
+message MoveTablesResponse {
+}
+
+message AddGroupRequest {
+  required string group_name = 1;
+}
+
+message AddGroupResponse {
+}
+
+message RemoveGroupRequest {
+  required string group_name = 1;
+}
+
+message RemoveGroupResponse {
+}
+
+message BalanceGroupRequest {
+  required string group_name = 1;
+}
+
+message BalanceGroupResponse {
+  required bool balanceRan = 1;
+}
+
+message ListGroupInfosRequest {
+}
+
+message ListGroupInfosResponse {
+  repeated GroupInfo  group_info = 1;
+}
+
+message GetGroupInfoOfServerRequest {
+  required HostPort server = 1;
+}
+
+message GetGroupInfoOfServerResponse {
+  optional GroupInfo group_info = 1;
+}
+
+message ListServersInTransitionRequest {
+}
+
+message ListServersInTransitionResponse {
+  repeated NameStringPair transitions = 1;
+}
+
 service MasterService {
   /** Used by the client to get the number of regions that have received the updated schema */
   rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -681,4 +768,31 @@ service MasterService {
 
   rpc getProcedureResult(GetProcedureResultRequest)
     returns(GetProcedureResultResponse);
+
+  rpc GetGroupInfo(GetGroupInfoRequest)
+    returns (GetGroupInfoResponse);
+
+  rpc GetGroupInfoOfTable(GetGroupInfoOfTableRequest)
+    returns (GetGroupInfoOfTableResponse);
+
+  rpc GetGroupInfoOfServer(GetGroupInfoOfServerRequest)
+    returns (GetGroupInfoOfServerResponse);
+
+  rpc MoveServers(MoveServersRequest)
+    returns (MoveServersResponse);
+
+  rpc MoveTables(MoveTablesRequest)
+    returns (MoveTablesResponse);
+
+  rpc AddGroup(AddGroupRequest)
+    returns (AddGroupResponse);
+
+  rpc RemoveGroup(RemoveGroupRequest)
+    returns (RemoveGroupResponse);
+
+  rpc BalanceGroup(BalanceGroupRequest)
+    returns (BalanceGroupResponse);
+
+  rpc ListGroupInfos(ListGroupInfosRequest)
+    returns (ListGroupInfosResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/protobuf/RSGroup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/RSGroup.proto b/hbase-protocol/src/main/protobuf/RSGroup.proto
new file mode 100644
index 0000000..920313c
--- /dev/null
+++ b/hbase-protocol/src/main/protobuf/RSGroup.proto
@@ -0,0 +1,33 @@
+/**
+ * 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 hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "RSGroupProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+message GroupInfo {
+  required string name = 1;
+  repeated HostPort servers = 4;
+  repeated TableName tables = 3;
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 8cf5954..d11ee38 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
@@ -21,12 +21,14 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -595,4 +597,55 @@ public abstract class BaseMasterAndRegionObserver extends BaseRegionObserver
   public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final String namespace, final Quotas quotas) throws IOException {
   }
+
+  @Override
+  public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             Set<HostPort> servers, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              Set<HostPort> servers, String targetGroup) throws IOException {
+
+  }
+
+  @Override
+  public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                            Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void preAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                          String name) throws IOException {
+  }
+
+  @Override
+  public void postAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                           String name) throws IOException {
+  }
+
+  @Override
+  public void preRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             String name) throws IOException {
+  }
+
+  @Override
+  public void postRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              String name) throws IOException {
+  }
+
+  @Override
+  public void preBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              String groupName) throws IOException {
+  }
+
+  @Override
+  public void postBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                               String groupName, boolean balancerRan) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 23a098c..95885d9 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
@@ -21,12 +21,14 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -588,4 +590,54 @@ public class BaseMasterObserver implements MasterObserver {
   public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final String namespace, final Quotas quotas) throws IOException {
   }
+
+  @Override
+  public void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             Set<HostPort> servers, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              Set<HostPort> servers, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                            Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             Set<TableName> tables, String targetGroup) throws IOException {
+  }
+
+  @Override
+  public void preAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                          String name) throws IOException {
+  }
+
+  @Override
+  public void postAddGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                           String name) throws IOException {
+  }
+
+  @Override
+  public void preRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             String name) throws IOException {
+  }
+
+  @Override
+  public void postRemoveGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              String name) throws IOException {
+  }
+
+  @Override
+  public void preBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              String groupName) throws IOException {
+  }
+
+  @Override
+  public void postBalanceGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                               String groupName, boolean balancerRan) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 3246d96..63202cc 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
@@ -21,12 +21,14 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -1180,4 +1182,98 @@ public interface MasterObserver extends Coprocessor {
    */
   void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final String namespace, final Quotas quotas) 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
+   * @param targetGroup
+   * @throws IOException
+   */
+  void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                      Set<HostPort> servers, 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
+   * @param targetGroup
+   * @throws IOException
+   */
+  void postMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                       Set<HostPort> servers, String targetGroup) throws IOException;
+
+  /**
+   * Called before tables are moved to target region server group
+   * @param ctx the environment to interact with the framework and master
+   * @param tables
+   * @param targetGroup
+   * @throws IOException
+   */
+  void preMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                     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 tables
+   * @param targetGroup
+   * @throws IOException
+   */
+  void postMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                      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
+   * @throws IOException
+   */
+  void preAddGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                   String name) throws IOException;
+
+  /**
+   * Called after a new region server group is added
+   * @param ctx the environment to interact with the framework and master
+   * @param name group name
+   * @throws IOException
+   */
+  void postAddGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                    String name) throws IOException;
+
+  /**
+   * Called before a region server group is removed
+   * @param ctx the environment to interact with the framework and master
+   * @param name group name
+   * @throws IOException
+   */
+  void preRemoveGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                      String name) throws IOException;
+
+  /**
+   * Called after a region server group is removed
+   * @param ctx the environment to interact with the framework and master
+   * @param name group name
+   * @throws IOException
+   */
+  void postRemoveGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                       String name) throws IOException;
+
+  /**
+   * Called before a region server group is removed
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName group name
+   * @throws IOException
+   */
+  void preBalanceGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                       String groupName) throws IOException;
+
+  /**
+   * Called after a region server group is removed
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName group name
+   * @throws IOException
+   */
+  void postBalanceGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+                        String groupName, boolean balancerRan) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupAdminServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupAdminServer.java
new file mode 100644
index 0000000..a0ab98f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupAdminServer.java
@@ -0,0 +1,486 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.metrics.util.MBeanUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Service to support Region Server Grouping (HBase-6721)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class GroupAdminServer implements GroupAdmin {
+  private static final Log LOG = LogFactory.getLog(GroupAdminServer.class);
+
+    private MasterServices master;
+  //List of servers that are being moved from one group to another
+  //Key=host:port,Value=targetGroup
+  ConcurrentMap<HostPort,String> serversInTransition =
+      new ConcurrentHashMap<HostPort, String>();
+  private GroupInfoManagerImpl groupInfoManager;
+
+  public GroupAdminServer(MasterServices master) throws IOException {
+    this.master = master;
+    groupInfoManager = new GroupInfoManagerImpl(master);
+    registerMBean();
+  }
+
+  @Override
+  public GroupInfo getGroupInfo(String groupName) throws IOException {
+    return getGroupInfoManager().getGroup(groupName);
+  }
+
+
+  @Override
+  public GroupInfo getGroupInfoOfTable(TableName tableName) throws IOException {
+    String groupName = getGroupInfoManager().getGroupOfTable(tableName);
+    if (groupName == null) {
+      if(master.getTableDescriptors().get(tableName) == null) {
+        throw new ConstraintException("Table "+tableName+" does not exist");
+      }
+      throw new ConstraintException("Table "+tableName+" has no group");
+    }
+    return getGroupInfoManager().getGroup(groupName);
+  }
+
+  @Override
+  public void moveServers(Set<HostPort> servers, String targetGroupName)
+      throws IOException {
+    if (servers == null) {
+      throw new DoNotRetryIOException(
+          "The list of servers cannot be null.");
+    }
+    if (StringUtils.isEmpty(targetGroupName)) {
+      throw new DoNotRetryIOException("The target group cannot be null.");
+    }
+    if(servers.size() < 1) {
+      return;
+    }
+
+    GroupInfo targetGrp = getGroupInfo(targetGroupName);
+    GroupInfoManager manager = getGroupInfoManager();
+    synchronized (manager) {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preMoveServers(servers, targetGroupName);
+      }
+      //we only allow a move from a single source group
+      //so this should be ok
+      GroupInfo srcGrp = manager.getGroupOfServer(servers.iterator().next());
+      //only move online servers (from default)
+      //or servers from other groups
+      //this prevents bogus servers from entering groups
+      if(GroupInfo.DEFAULT_GROUP.equals(srcGrp.getName())) {
+        Set<HostPort> onlineServers = new HashSet<HostPort>();
+        for(ServerName server: master.getServerManager().getOnlineServers().keySet()) {
+          onlineServers.add(server.getHostPort());
+        }
+        for(HostPort el: servers) {
+          if(!onlineServers.contains(el)) {
+            throw new DoNotRetryIOException(
+                "Server "+el+" is not an online server in default group.");
+          }
+        }
+      }
+
+      if(srcGrp.getServers().size() <= servers.size() &&
+          srcGrp.getTables().size() > 0) {
+        throw new DoNotRetryIOException("Cannot leave a group "+srcGrp.getName()+
+            " that contains tables " +"without servers.");
+      }
+
+      String sourceGroupName =
+          getGroupInfoManager().getGroupOfServer(srcGrp.getServers().iterator().next()).getName();
+      if(getGroupInfo(targetGroupName) == null) {
+        throw new ConstraintException("Target group does not exist: "+targetGroupName);
+      }
+
+      for(HostPort server: servers) {
+        if (serversInTransition.containsKey(server)) {
+          throw new DoNotRetryIOException(
+              "Server list contains a server that is already being moved: "+server);
+        }
+        String tmpGroup = getGroupInfoManager().getGroupOfServer(server).getName();
+        if (sourceGroupName != null && !tmpGroup.equals(sourceGroupName)) {
+          throw new DoNotRetryIOException(
+              "Move server request should only come from one source group. "+
+              "Expecting only "+sourceGroupName+" but contains "+tmpGroup);
+        }
+      }
+
+      if(sourceGroupName.equals(targetGroupName)) {
+        throw new ConstraintException(
+            "Target group is the same as source group: "+targetGroupName);
+      }
+
+      //update the servers as in transition
+      for(HostPort server: servers) {
+        serversInTransition.put(server, targetGroupName);
+      }
+
+      getGroupInfoManager().moveServers(servers, sourceGroupName, targetGroupName);
+      boolean found;
+      List<HostPort> tmpServers = Lists.newArrayList(servers);
+      do {
+        found = false;
+        for(Iterator<HostPort> iter = tmpServers.iterator();
+            iter.hasNext(); ) {
+          HostPort rs = iter.next();
+          //get online regions
+          List<HRegionInfo> regions = new LinkedList<HRegionInfo>();
+          for(Map.Entry<HRegionInfo, ServerName> el:
+              master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
+            if (el.getValue().getHostPort().equals(rs)) {
+              regions.add(el.getKey());
+            }
+          }
+          for(RegionState state :
+              master.getAssignmentManager().getRegionStates().getRegionsInTransition().values()) {
+            if (state.getServerName().getHostPort().equals(rs)) {
+              regions.add(state.getRegion());
+            }
+          }
+
+          //unassign regions for a server
+          LOG.info("Unassigning "+regions.size()+
+              " regions from server "+rs+" for move to "+targetGroupName);
+          if(regions.size() > 0) {
+            //TODO bulk unassign or throttled unassign?
+            for(HRegionInfo region: regions) {
+              //regions might get assigned from tables of target group
+              //so we need to filter
+              if(!targetGrp.containsTable(region.getTable())) {
+                master.getAssignmentManager().unassign(region);
+                found = true;
+              }
+            }
+          }
+          if(!found) {
+            iter.remove();
+            serversInTransition.remove(rs);
+          }
+        }
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          LOG.warn("Sleep interrupted", e);
+        }
+      } while(found);
+
+      LOG.info("Move server done: "+sourceGroupName+"->"+targetGroupName);
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postMoveServers(servers, targetGroupName);
+      }
+    }
+  }
+
+  @Override
+  public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
+    if (tables == null) {
+      throw new ConstraintException(
+          "The list of servers cannot be null.");
+    }
+    if(tables.size() < 1) {
+      LOG.debug("moveTables() passed an empty set. Ignoring.");
+      return;
+    }
+    GroupInfoManager manager = getGroupInfoManager();
+    synchronized (manager) {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preMoveTables(tables, targetGroup);
+      }
+
+      if(targetGroup != null) {
+        GroupInfo destGroup = manager.getGroup(targetGroup);
+        if(destGroup == null) {
+          throw new ConstraintException("Target group does not exist: "+targetGroup);
+        }
+        if(destGroup.getServers().size() < 1) {
+          throw new ConstraintException("Target group must have at least one server.");
+        }
+      }
+
+      for(TableName table : tables) {
+        String srcGroup = manager.getGroupOfTable(table);
+        if(srcGroup != null && srcGroup.equals(targetGroup)) {
+          throw new ConstraintException("Source group is the same as target group for table "+table+" :"+srcGroup);
+        }
+      }
+      manager.moveTables(tables, targetGroup);
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postMoveTables(tables, targetGroup);
+      }
+    }
+    for(TableName table: tables) {
+      for(HRegionInfo region:
+          master.getAssignmentManager().getRegionStates().getRegionsOfTable(table)) {
+        master.getAssignmentManager().unassign(region);
+      }
+    }
+  }
+
+  @Override
+  public void addGroup(String name) throws IOException {
+    if (master.getMasterCoprocessorHost() != null) {
+      master.getMasterCoprocessorHost().preAddGroup(name);
+    }
+    getGroupInfoManager().addGroup(new GroupInfo(name));
+    if (master.getMasterCoprocessorHost() != null) {
+      master.getMasterCoprocessorHost().postAddGroup(name);
+    }
+  }
+
+  @Override
+  public void removeGroup(String name) throws IOException {
+    GroupInfoManager manager = getGroupInfoManager();
+    synchronized (manager) {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preRemoveGroup(name);
+      }
+      GroupInfo groupInfo = getGroupInfoManager().getGroup(name);
+      if(groupInfo == null) {
+        throw new DoNotRetryIOException("Group "+name+" does not exist");
+      }
+      int tableCount = groupInfo.getTables().size();
+      if (tableCount > 0) {
+        throw new DoNotRetryIOException("Group "+name+" must have no associated tables: "+tableCount);
+      }
+      int serverCount = groupInfo.getServers().size();
+      if(serverCount > 0) {
+        throw new DoNotRetryIOException("Group "+name+" must have no associated servers: "+serverCount);
+      }
+      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
+        String nsGroup = ns.getConfigurationValue(GroupInfo.NAMESPACEDESC_PROP_GROUP);
+        if(nsGroup != null &&  nsGroup.equals(name)) {
+          throw new DoNotRetryIOException("Group "+name+" is referenced by namespace: "+ns.getName());
+        }
+      }
+      manager.removeGroup(name);
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postRemoveGroup(name);
+      }
+    }
+  }
+
+  @Override
+  public boolean balanceGroup(String groupName) throws IOException {
+    ServerManager serverManager = master.getServerManager();
+    AssignmentManager assignmentManager = master.getAssignmentManager();
+    LoadBalancer balancer = master.getLoadBalancer();
+
+    boolean balancerRan;
+    synchronized (balancer) {
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().preBalanceGroup(groupName);
+      }
+      // Only allow one balance run at at time.
+      Map<String, RegionState> groupRIT = groupGetRegionsInTransition(groupName);
+      if (groupRIT.size() > 0) {
+        LOG.debug("Not running balancer because " +
+          groupRIT.size() +
+          " region(s) in transition: " +
+          StringUtils.abbreviate(
+              master.getAssignmentManager().getRegionStates().getRegionsInTransition().toString(),
+              256));
+        return false;
+      }
+      if (serverManager.areDeadServersInProgress()) {
+        LOG.debug("Not running balancer because processing dead regionserver(s): " +
+            serverManager.getDeadServers());
+        return false;
+      }
+
+      //We balance per group instead of per table
+      List<RegionPlan> plans = new ArrayList<RegionPlan>();
+      for(Map.Entry<TableName, Map<ServerName, List<HRegionInfo>>> tableMap:
+          getGroupAssignmentsByTable(groupName).entrySet()) {
+        LOG.info("Creating partial plan for table "+tableMap.getKey()+": "+tableMap.getValue());
+        List<RegionPlan> partialPlans = balancer.balanceCluster(tableMap.getValue());
+        LOG.info("Partial plan for table "+tableMap.getKey()+": "+partialPlans);
+        if (partialPlans != null) {
+          plans.addAll(partialPlans);
+        }
+      }
+      long startTime = System.currentTimeMillis();
+      balancerRan = plans != null;
+      if (plans != null && !plans.isEmpty()) {
+        LOG.info("Group balance "+groupName+" starting with plan count: "+plans.size());
+        for (RegionPlan plan: plans) {
+          LOG.info("balance " + plan);
+          assignmentManager.balance(plan);
+        }
+        LOG.info("Group balance "+groupName+" completed after "+(System.currentTimeMillis()-startTime)+" seconds");
+      }
+      if (master.getMasterCoprocessorHost() != null) {
+        master.getMasterCoprocessorHost().postBalanceGroup(groupName, balancerRan);
+      }
+    }
+    return balancerRan;
+  }
+
+  @Override
+  public List<GroupInfo> listGroups() throws IOException {
+    return getGroupInfoManager().listGroups();
+  }
+
+  @Override
+  public GroupInfo getGroupOfServer(HostPort hostPort) throws IOException {
+    return getGroupInfoManager().getGroupOfServer(hostPort);
+  }
+
+  @InterfaceAudience.Private
+  public GroupInfoManager getGroupInfoManager() throws IOException {
+    return groupInfoManager;
+  }
+
+  private Map<String, RegionState> groupGetRegionsInTransition(String groupName)
+      throws IOException {
+    Map<String, RegionState> rit = Maps.newTreeMap();
+    AssignmentManager am = master.getAssignmentManager();
+    GroupInfo groupInfo = getGroupInfo(groupName);
+    for(TableName tableName : groupInfo.getTables()) {
+      for(HRegionInfo regionInfo: am.getRegionStates().getRegionsOfTable(tableName)) {
+        RegionState state =
+            master.getAssignmentManager().getRegionStates().getRegionTransitionState(regionInfo);
+        if(state != null) {
+          rit.put(regionInfo.getEncodedName(), state);
+        }
+      }
+    }
+    return rit;
+  }
+
+  private Map<TableName, Map<ServerName, List<HRegionInfo>>>
+      getGroupAssignmentsByTable(String groupName) throws IOException {
+    Map<TableName, Map<ServerName, List<HRegionInfo>>> result = Maps.newHashMap();
+    GroupInfo groupInfo = getGroupInfo(groupName);
+    Map<TableName, Map<ServerName, List<HRegionInfo>>> assignments = Maps.newHashMap();
+    for(Map.Entry<HRegionInfo, ServerName> entry:
+        master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
+      TableName currTable = entry.getKey().getTable();
+      ServerName currServer = entry.getValue();
+      HRegionInfo currRegion = entry.getKey();
+      if(groupInfo.getTables().contains(currTable)) {
+        if(!assignments.containsKey(entry.getKey().getTable())) {
+          assignments.put(currTable, new HashMap<ServerName, List<HRegionInfo>>());
+        }
+        if(!assignments.get(currTable).containsKey(currServer)) {
+          assignments.get(currTable).put(currServer, new ArrayList<HRegionInfo>());
+        }
+        assignments.get(currTable).get(currServer).add(currRegion);
+      }
+    }
+
+    Map<ServerName, List<HRegionInfo>> serverMap = Maps.newHashMap();
+    for(ServerName serverName: master.getServerManager().getOnlineServers().keySet()) {
+      if(groupInfo.getServers().contains(serverName.getHostPort())) {
+        serverMap.put(serverName, Collections.EMPTY_LIST);
+      }
+    }
+
+    //add all tables that are members of the group
+    for(TableName tableName : groupInfo.getTables()) {
+      if(assignments.containsKey(tableName)) {
+        result.put(tableName, new HashMap<ServerName, List<HRegionInfo>>());
+        result.get(tableName).putAll(serverMap);
+        result.get(tableName).putAll(assignments.get(tableName));
+        LOG.debug("Adding assignments for "+tableName+": "+assignments.get(tableName));
+      }
+    }
+
+    return result;
+  }
+
+  void registerMBean() {
+    MXBeanImpl mxBeanInfo =
+        MXBeanImpl.init(this, master);
+    MBeanUtil.registerMBean("Group", "Group", mxBeanInfo);
+    LOG.info("Registered Group MXBean");
+  }
+
+  public void prepareGroupForTable(HTableDescriptor desc) throws IOException {
+    String groupName =
+        master.getNamespaceDescriptor(desc.getTableName().getNamespaceAsString())
+                .getConfigurationValue(GroupInfo.NAMESPACEDESC_PROP_GROUP);
+    if (groupName == null) {
+      groupName = GroupInfo.DEFAULT_GROUP;
+    }
+    GroupInfo groupInfo = getGroupInfo(groupName);
+    if (groupInfo == null) {
+      throw new ConstraintException("Group " + groupName + " does not exist.");
+    }
+    if (!groupInfo.containsTable(desc.getTableName())) {
+      LOG.debug("Pre-moving table " + desc.getTableName() + " to group " + groupName);
+      moveTables(Sets.newHashSet(desc.getTableName()), groupName);
+    }
+  }
+
+  public void cleanupGroupForTable(TableName tableName) throws IOException {
+    try {
+      GroupInfo group = getGroupInfoOfTable(tableName);
+      if (group != null) {
+        LOG.debug("Removing deleted table from table group " + group.getName());
+        moveTables(Sets.newHashSet(tableName), null);
+      }
+    } catch (ConstraintException ex) {
+      LOG.debug("Failed to perform group information cleanup for table: " + tableName, ex);
+    } catch (IOException ex) {
+      LOG.debug("Failed to perform group information cleanup for table: " + tableName, ex);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}


[3/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupBasedLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupBasedLoadBalancer.java
new file mode 100644
index 0000000..ae5fcac
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupBasedLoadBalancer.java
@@ -0,0 +1,433 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * GroupBasedLoadBalancer, used when Region Server Grouping is configured (HBase-6721)
+ * It does region balance based on a table's group membership.
+ *
+ * Most assignment methods contain two exclusive code paths: Online - when the group
+ * table is online and Offline - when it is unavailable.
+ *
+ * During Offline, assignments are assigned based on cached information in zookeeper.
+ * If unavailable (ie bootstrap) then regions are assigned randombly.
+ *
+ * Once the GROUP table has been assigned, the balancer switches to Online and will then
+ * start providing appropriate assignments for user tables.
+ *
+ */
+@InterfaceAudience.Public
+public class GroupBasedLoadBalancer implements GroupableBalancer, LoadBalancer {
+  /** Config for pluggable load balancers */
+  public static final String HBASE_GROUP_LOADBALANCER_CLASS = "hbase.group.grouploadbalancer.class";
+
+  private static final Log LOG = LogFactory.getLog(GroupBasedLoadBalancer.class);
+
+  private Configuration config;
+  private ClusterStatus clusterStatus;
+  private MasterServices masterServices;
+  private GroupInfoManager groupManager;
+  private LoadBalancer internalBalancer;
+
+  //used during reflection by LoadBalancerFactory
+  @InterfaceAudience.Private
+  public GroupBasedLoadBalancer() {
+  }
+
+  //This constructor should only be used for unit testing
+  @InterfaceAudience.Private
+  public GroupBasedLoadBalancer(GroupInfoManager groupManager) {
+    this.groupManager = groupManager;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.config = conf;
+  }
+
+  @Override
+  public void setClusterStatus(ClusterStatus st) {
+    this.clusterStatus = st;
+  }
+
+  @Override
+  public void setMasterServices(MasterServices masterServices) {
+    this.masterServices = masterServices;
+  }
+
+  @Override
+  public List<RegionPlan> balanceCluster(Map<ServerName, List<HRegionInfo>> clusterState)
+      throws HBaseIOException {
+
+    if (!isOnline()) {
+      throw new IllegalStateException(GroupInfoManager.GROUP_TABLE_NAME+
+          " is not online, unable to perform balance");
+    }
+
+    Map<ServerName,List<HRegionInfo>> correctedState = correctAssignments(clusterState);
+    List<RegionPlan> regionPlans = new ArrayList<RegionPlan>();
+    try {
+      for (GroupInfo info : groupManager.listGroups()) {
+        Map<ServerName, List<HRegionInfo>> groupClusterState = new HashMap<ServerName, List<HRegionInfo>>();
+        for (HostPort sName : info.getServers()) {
+          for(ServerName curr: clusterState.keySet()) {
+            if(curr.getHostPort().equals(sName)) {
+              groupClusterState.put(curr, correctedState.get(curr));
+            }
+          }
+        }
+        List<RegionPlan> groupPlans = this.internalBalancer
+            .balanceCluster(groupClusterState);
+        if (groupPlans != null) {
+          regionPlans.addAll(groupPlans);
+        }
+      }
+    } catch (IOException exp) {
+      LOG.warn("Exception while balancing cluster.", exp);
+      regionPlans.clear();
+    }
+    return regionPlans;
+  }
+
+  @Override
+  public Map<ServerName, List<HRegionInfo>> roundRobinAssignment (
+      List<HRegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
+    Map<ServerName, List<HRegionInfo>> assignments = Maps.newHashMap();
+    ListMultimap<String,HRegionInfo> regionMap = LinkedListMultimap.create();
+    ListMultimap<String,ServerName> serverMap = LinkedListMultimap.create();
+    generateGroupMaps(regions, servers, regionMap, serverMap);
+    for(String groupKey : regionMap.keySet()) {
+      if (regionMap.get(groupKey).size() > 0) {
+        Map<ServerName, List<HRegionInfo>> result =
+            this.internalBalancer.roundRobinAssignment(
+                regionMap.get(groupKey),
+                serverMap.get(groupKey));
+        if(result != null) {
+          assignments.putAll(result);
+        }
+      }
+    }
+    return assignments;
+  }
+
+  @Override
+  public Map<ServerName, List<HRegionInfo>> retainAssignment(
+      Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+    if (!isOnline()) {
+      return offlineRetainAssignment(regions, servers);
+    }
+    return onlineRetainAssignment(regions, servers);
+  }
+
+  public Map<ServerName, List<HRegionInfo>> offlineRetainAssignment(
+      Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+      //We will just keep assignments even if they are incorrect.
+      //Chances are most will be assigned correctly.
+      //Then we just use balance to correct the misplaced few.
+      //we need to correct catalog and group table assignment anyway.
+      return internalBalancer.retainAssignment(regions, servers);
+  }
+
+  public Map<ServerName, List<HRegionInfo>> onlineRetainAssignment(
+      Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+    try {
+      Map<ServerName, List<HRegionInfo>> assignments = new TreeMap<ServerName, List<HRegionInfo>>();
+      ListMultimap<String, HRegionInfo> groupToRegion = ArrayListMultimap.create();
+      List<HRegionInfo> misplacedRegions = getMisplacedRegions(regions);
+      for (HRegionInfo region : regions.keySet()) {
+        if (!misplacedRegions.contains(region)) {
+          String groupName = groupManager.getGroupOfTable(region.getTable());
+          groupToRegion.put(groupName, region);
+        }
+      }
+      // Now the "groupToRegion" map has only the regions which have correct
+      // assignments.
+      for (String key : groupToRegion.keySet()) {
+        Map<HRegionInfo, ServerName> currentAssignmentMap = new TreeMap<HRegionInfo, ServerName>();
+        List<HRegionInfo> regionList = groupToRegion.get(key);
+        GroupInfo info = groupManager.getGroup(key);
+        List<ServerName> candidateList = filterOfflineServers(info, servers);
+        for (HRegionInfo region : regionList) {
+          currentAssignmentMap.put(region, regions.get(region));
+        }
+        assignments.putAll(this.internalBalancer.retainAssignment(
+            currentAssignmentMap, candidateList));
+      }
+
+      for (HRegionInfo region : misplacedRegions) {
+        String groupName = groupManager.getGroupOfTable(
+            region.getTable());
+        GroupInfo info = groupManager.getGroup(groupName);
+        List<ServerName> candidateList = filterOfflineServers(info, servers);
+        ServerName server = this.internalBalancer.randomAssignment(region,
+            candidateList);
+        if (server != null && !assignments.containsKey(server)) {
+          assignments.put(server, new ArrayList<HRegionInfo>());
+        } else if (server != null) {
+          assignments.get(server).add(region);
+        } else {
+          //if not server is available assign to bogus so it ends up in RIT
+          if(!assignments.containsKey(BOGUS_SERVER_NAME)) {
+            assignments.put(BOGUS_SERVER_NAME, new ArrayList<HRegionInfo>());
+          }
+          assignments.get(BOGUS_SERVER_NAME).add(region);
+        }
+      }
+      return assignments;
+    } catch (IOException e) {
+      throw new HBaseIOException("Failed to do online retain assignment", e);
+    }
+  }
+
+  @Override
+  public Map<HRegionInfo, ServerName> immediateAssignment(
+      List<HRegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
+    Map<HRegionInfo,ServerName> assignments = Maps.newHashMap();
+    ListMultimap<String,HRegionInfo> regionMap = LinkedListMultimap.create();
+    ListMultimap<String,ServerName> serverMap = LinkedListMultimap.create();
+    generateGroupMaps(regions, servers, regionMap, serverMap);
+    for(String groupKey : regionMap.keySet()) {
+      if (regionMap.get(groupKey).size() > 0) {
+        assignments.putAll(
+            this.internalBalancer.immediateAssignment(
+                regionMap.get(groupKey),
+                serverMap.get(groupKey)));
+      }
+    }
+    return assignments;
+  }
+
+  @Override
+  public ServerName randomAssignment(HRegionInfo region,
+      List<ServerName> servers) throws HBaseIOException {
+    ListMultimap<String,HRegionInfo> regionMap = LinkedListMultimap.create();
+    ListMultimap<String,ServerName> serverMap = LinkedListMultimap.create();
+    generateGroupMaps(Lists.newArrayList(region), servers, regionMap, serverMap);
+    List<ServerName> filteredServers = serverMap.get(regionMap.keySet().iterator().next());
+    return this.internalBalancer.randomAssignment(region, filteredServers);
+  }
+
+  private void generateGroupMaps(
+    List<HRegionInfo> regions,
+    List<ServerName> servers,
+    ListMultimap<String, HRegionInfo> regionMap,
+    ListMultimap<String, ServerName> serverMap) throws HBaseIOException {
+    try {
+      for (HRegionInfo region : regions) {
+        String groupName = groupManager.getGroupOfTable(region.getTable());
+        if(groupName == null) {
+          LOG.warn("Group for table "+region.getTable()+" is null");
+        }
+        regionMap.put(groupName, region);
+      }
+      for (String groupKey : regionMap.keySet()) {
+        GroupInfo info = groupManager.getGroup(groupKey);
+        serverMap.putAll(groupKey, filterOfflineServers(info, servers));
+        if(serverMap.get(groupKey).size() < 1) {
+          serverMap.put(groupKey, BOGUS_SERVER_NAME);
+        }
+      }
+    } catch(IOException e) {
+      throw new HBaseIOException("Failed to generate group maps", e);
+    }
+  }
+
+  private List<ServerName> filterOfflineServers(GroupInfo groupInfo,
+                                                List<ServerName> onlineServers) {
+    if (groupInfo != null) {
+      return filterServers(groupInfo.getServers(), onlineServers);
+    } else {
+      LOG.debug("Group Information found to be null. Some regions might be unassigned.");
+      return Collections.EMPTY_LIST;
+    }
+  }
+
+  /**
+   * Filter servers based on the online servers.
+   *
+   * @param servers
+   *          the servers
+   * @param onlineServers
+   *          List of servers which are online.
+   * @return the list
+   */
+  private List<ServerName> filterServers(Collection<HostPort> servers,
+      Collection<ServerName> onlineServers) {
+    ArrayList<ServerName> finalList = new ArrayList<ServerName>();
+    for (HostPort server : servers) {
+      for(ServerName curr: onlineServers) {
+        if(curr.getHostPort().equals(server)) {
+          finalList.add(curr);
+        }
+      }
+    }
+    return finalList;
+  }
+
+  private ListMultimap<String, HRegionInfo> groupRegions(
+      List<HRegionInfo> regionList) throws IOException {
+    ListMultimap<String, HRegionInfo> regionGroup = ArrayListMultimap
+        .create();
+    for (HRegionInfo region : regionList) {
+      String groupName = groupManager.getGroupOfTable(region.getTable());
+      regionGroup.put(groupName, region);
+    }
+    return regionGroup;
+  }
+
+  private List<HRegionInfo> getMisplacedRegions(
+      Map<HRegionInfo, ServerName> regions) throws IOException {
+    List<HRegionInfo> misplacedRegions = new ArrayList<HRegionInfo>();
+    for (HRegionInfo region : regions.keySet()) {
+      ServerName assignedServer = regions.get(region);
+      GroupInfo info = groupManager.getGroup(groupManager.getGroupOfTable(region.getTable()));
+      if (assignedServer != null &&
+          (info == null || !info.containsServer(assignedServer.getHostPort()))) {
+        LOG.warn("Found misplaced region: "+region.getRegionNameAsString()+
+            " on server: "+assignedServer+
+            " found in group: "+groupManager.getGroupOfServer(assignedServer.getHostPort())+
+            " outside of group: "+info.getName());
+        misplacedRegions.add(region);
+      }
+    }
+    return misplacedRegions;
+  }
+
+  private Map<ServerName, List<HRegionInfo>> correctAssignments(
+       Map<ServerName, List<HRegionInfo>> existingAssignments){
+    Map<ServerName, List<HRegionInfo>> correctAssignments = new TreeMap<ServerName, List<HRegionInfo>>();
+    List<HRegionInfo> misplacedRegions = new LinkedList<HRegionInfo>();
+    for (ServerName sName : existingAssignments.keySet()) {
+      correctAssignments.put(sName, new LinkedList<HRegionInfo>());
+      List<HRegionInfo> regions = existingAssignments.get(sName);
+      for (HRegionInfo region : regions) {
+        GroupInfo info = null;
+        try {
+          info = groupManager.getGroup(groupManager.getGroupOfTable(region.getTable()));
+        }catch(IOException exp){
+          LOG.debug("Group information null for region of table " + region.getTable(),
+              exp);
+        }
+        if ((info == null) || (!info.containsServer(sName.getHostPort()))) {
+          // Misplaced region.
+          misplacedRegions.add(region);
+        } else {
+          correctAssignments.get(sName).add(region);
+        }
+      }
+    }
+
+    //TODO bulk unassign?
+    //unassign misplaced regions, so that they are assigned to correct groups.
+    for(HRegionInfo info: misplacedRegions) {
+      this.masterServices.getAssignmentManager().unassign(info);
+    }
+    return correctAssignments;
+  }
+
+  @Override
+  public void initialize() throws HBaseIOException {
+    // Create the balancer
+    Class<? extends LoadBalancer> balancerKlass = config.getClass(
+        HBASE_GROUP_LOADBALANCER_CLASS,
+        StochasticLoadBalancer.class, LoadBalancer.class);
+    internalBalancer = ReflectionUtils.newInstance(balancerKlass, config);
+    internalBalancer.setClusterStatus(clusterStatus);
+    internalBalancer.setMasterServices(masterServices);
+    internalBalancer.setConf(config);
+    internalBalancer.initialize();
+  }
+
+  public boolean isOnline() {
+    return groupManager != null && groupManager.isOnline();
+  }
+
+  @InterfaceAudience.Private
+  public GroupInfoManager getGroupInfoManager() throws IOException {
+    return groupManager;
+  }
+
+  @Override
+  public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
+  }
+
+  @Override
+  public void regionOffline(HRegionInfo regionInfo) {
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {
+    //DO nothing for now
+  }
+
+  @Override
+  public void stop(String why) {
+  }
+
+  @Override
+  public boolean isStopped() {
+    return false;
+  }
+
+  @Override
+  public void setGroupInfoManager(GroupInfoManager groupInfoManager) throws IOException {
+    this.groupManager = groupInfoManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManager.java
new file mode 100644
index 0000000..4ed7fa8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManager.java
@@ -0,0 +1,129 @@
+/**
+ * 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.group;
+
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Interface used to manage GroupInfo storage. An implementation
+ * has the option to support offline mode.
+ * See {@link GroupBasedLoadBalancer}
+ */
+public interface GroupInfoManager {
+  //Assigned before user tables
+  public static final TableName GROUP_TABLE_NAME =
+      TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR,"rsgroup");
+  public static final byte[] GROUP_TABLE_NAME_BYTES = GROUP_TABLE_NAME.toBytes();
+  public static final String groupZNode = "groupInfo";
+  public static final byte[] META_FAMILY_BYTES = Bytes.toBytes("m");
+  public static final byte[] ROW_KEY = {0};
+
+
+  /**
+   * Adds the group.
+   *
+   * @param groupInfo the group name
+   * @throws java.io.IOException Signals that an I/O exception has occurred.
+   */
+  void addGroup(GroupInfo groupInfo) throws IOException;
+
+  /**
+   * Remove a region server group.
+   *
+   * @param groupName the group name
+   * @throws java.io.IOException Signals that an I/O exception has occurred.
+   */
+  void removeGroup(String groupName) throws IOException;
+
+  /**
+   * move servers to a new group.
+   * @param hostPorts list of servers, must be part of the same group
+   * @param srcGroup
+   * @param dstGroup
+   * @return true if move was successful
+   * @throws java.io.IOException
+   */
+  boolean moveServers(Set<HostPort> hostPorts, String srcGroup, String dstGroup) throws IOException;
+
+  /**
+   * Gets the group info of server.
+   *
+   * @param hostPort the server
+   * @return An instance of GroupInfo
+   */
+  GroupInfo getGroupOfServer(HostPort hostPort) throws IOException;
+
+  /**
+   * Gets the group information.
+   *
+   * @param groupName the group name
+   * @return An instance of GroupInfo
+   */
+  GroupInfo getGroup(String groupName) throws IOException;
+
+  /**
+   * Get the group membership of a table
+   * @param tableName
+   * @return Group name of table
+   * @throws java.io.IOException
+   */
+  String getGroupOfTable(TableName tableName) throws IOException;
+
+  /**
+   * Set the group membership of a set of tables
+   *
+   * @param tableNames
+   * @param groupName
+   * @throws java.io.IOException
+   */
+  void moveTables(Set<TableName> tableNames, String groupName) throws IOException;
+
+  /**
+   * List the groups
+   *
+   * @return list of GroupInfo
+   * @throws java.io.IOException
+   */
+  List<GroupInfo> listGroups() throws IOException;
+
+  /**
+   * Refresh/reload the group information from
+   * the persistent store
+   *
+   * @throws java.io.IOException
+   */
+  void refresh() throws IOException;
+
+  /**
+   * Whether the manager is able to fully
+   * return group metadata
+   *
+   * @return
+   */
+  boolean isOnline();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManagerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManagerImpl.java
new file mode 100644
index 0000000..a0df353
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupInfoManagerImpl.java
@@ -0,0 +1,702 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MetaTableAccessor.DefaultVisitorBase;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
+import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This is an implementation of {@link GroupInfoManager}. Which makes
+ * use of an HBase table as the persistence store for the group information.
+ * It also makes use of zookeeper to store group information needed
+ * for bootstrapping during offline mode.
+ */
+public class GroupInfoManagerImpl implements GroupInfoManager, ServerListener {
+  private static final Log LOG = LogFactory.getLog(GroupInfoManagerImpl.class);
+
+  /** Table descriptor for <code>hbase:rsgroup</code> catalog table */
+  private final static HTableDescriptor GROUP_TABLE_DESC;
+  static {
+    GROUP_TABLE_DESC = new HTableDescriptor(GROUP_TABLE_NAME_BYTES);
+    GROUP_TABLE_DESC.addFamily(new HColumnDescriptor(META_FAMILY_BYTES));
+    GROUP_TABLE_DESC.setRegionSplitPolicyClassName(DisabledRegionSplitPolicy.class.getName());
+  }
+
+  //Access to this map should always be synchronized.
+  private volatile Map<String, GroupInfo> groupMap;
+  private volatile Map<TableName, String> tableMap;
+  private MasterServices master;
+  private Table groupTable;
+  private ClusterConnection conn;
+  private ZooKeeperWatcher watcher;
+  private GroupStartupWorker groupStartupWorker;
+  //contains list of groups that were last flushed to persistent store
+  private volatile Set<String> prevGroups;
+  private GroupSerDe groupSerDe;
+  private DefaultServerUpdater defaultServerUpdater;
+
+
+  public GroupInfoManagerImpl(MasterServices master) throws IOException {
+    this.groupMap = Collections.EMPTY_MAP;
+    this.tableMap = Collections.EMPTY_MAP;
+    groupSerDe = new GroupSerDe();
+    this.master = master;
+    this.watcher = master.getZooKeeper();
+    this.conn = master.getConnection();
+    groupStartupWorker = new GroupStartupWorker(this, master, conn);
+    prevGroups = new HashSet<String>();
+    refresh();
+    groupStartupWorker.start();
+    defaultServerUpdater = new DefaultServerUpdater(this);
+    master.getServerManager().registerListener(this);
+    defaultServerUpdater.start();
+  }
+
+  /**
+   * Adds the group.
+   *
+   * @param groupInfo the group name
+   */
+  @Override
+  public synchronized void addGroup(GroupInfo groupInfo) throws IOException {
+    if (groupMap.get(groupInfo.getName()) != null ||
+        groupInfo.getName().equals(GroupInfo.DEFAULT_GROUP)) {
+      throw new DoNotRetryIOException("Group already exists: "+groupInfo.getName());
+    }
+    Map<String, GroupInfo> newGroupMap = Maps.newHashMap(groupMap);
+    newGroupMap.put(groupInfo.getName(), groupInfo);
+    flushConfig(newGroupMap);
+  }
+
+  @Override
+  public synchronized boolean moveServers(Set<HostPort> hostPorts, String srcGroup,
+                                          String dstGroup) throws IOException {
+    GroupInfo src = new GroupInfo(getGroup(srcGroup));
+    GroupInfo dst = new GroupInfo(getGroup(dstGroup));
+    boolean foundOne = false;
+    for(HostPort el: hostPorts) {
+      foundOne = src.removeServer(el) || foundOne;
+      dst.addServer(el);
+    }
+
+    Map<String,GroupInfo> newGroupMap = Maps.newHashMap(groupMap);
+    newGroupMap.put(src.getName(), src);
+    newGroupMap.put(dst.getName(), dst);
+
+    flushConfig(newGroupMap);
+    return foundOne;
+  }
+
+  /**
+   * Gets the group info of server.
+   *
+   * @param hostPort the server
+   * @return An instance of GroupInfo.
+   */
+  @Override
+  public GroupInfo getGroupOfServer(HostPort hostPort) throws IOException {
+    for (GroupInfo info : groupMap.values()) {
+      if (info.containsServer(hostPort)){
+        return info;
+      }
+    }
+    return getGroup(GroupInfo.DEFAULT_GROUP);
+  }
+
+  /**
+   * Gets the group information.
+   *
+   * @param groupName
+   *          the group name
+   * @return An instance of GroupInfo
+   */
+  @Override
+  public GroupInfo getGroup(String groupName) throws IOException {
+    GroupInfo groupInfo = groupMap.get(groupName);
+    return groupInfo;
+  }
+
+
+
+  @Override
+  public String getGroupOfTable(TableName tableName) throws IOException {
+    return tableMap.get(tableName);
+  }
+
+  @Override
+  public synchronized void moveTables(Set<TableName> tableNames, String groupName) throws IOException {
+    if (groupName != null && !groupMap.containsKey(groupName)) {
+      throw new DoNotRetryIOException("Group "+groupName+" does not exist or is a special group");
+    }
+    Map<String,GroupInfo> newGroupMap = Maps.newHashMap(groupMap);
+    for(TableName tableName: tableNames) {
+      if (tableMap.containsKey(tableName)) {
+        GroupInfo src = new GroupInfo(groupMap.get(tableMap.get(tableName)));
+        src.removeTable(tableName);
+        newGroupMap.put(src.getName(), src);
+      }
+      if(groupName != null) {
+        GroupInfo dst = new GroupInfo(newGroupMap.get(groupName));
+        dst.addTable(tableName);
+        newGroupMap.put(dst.getName(), dst);
+      }
+    }
+
+    flushConfig(newGroupMap);
+  }
+
+
+  /**
+   * Delete a region server group.
+   *
+   * @param groupName the group name
+   * @throws java.io.IOException Signals that an I/O exception has occurred.
+   */
+  @Override
+  public synchronized void removeGroup(String groupName) throws IOException {
+    if (!groupMap.containsKey(groupName) || groupName.equals(GroupInfo.DEFAULT_GROUP)) {
+      throw new DoNotRetryIOException("Group "+groupName+" does not exist or is a reserved group");
+    }
+    Map<String,GroupInfo> newGroupMap = Maps.newHashMap(groupMap);
+    newGroupMap.remove(groupName);
+    flushConfig(newGroupMap);
+  }
+
+  @Override
+  public List<GroupInfo> listGroups() throws IOException {
+    List<GroupInfo> list = Lists.newLinkedList(groupMap.values());
+    return list;
+  }
+
+  @Override
+  public boolean isOnline() {
+    return groupStartupWorker.isOnline();
+  }
+
+  @Override
+  public synchronized void refresh() throws IOException {
+    refresh(false);
+  }
+
+  private synchronized void refresh(boolean forceOnline) throws IOException {
+    List<GroupInfo> groupList = new LinkedList<GroupInfo>();
+
+    //overwrite anything read from zk, group table is source of truth
+    //if online read from GROUP table
+    if (forceOnline || isOnline()) {
+      LOG.debug("Refreshing in Online mode.");
+      if (groupTable == null) {
+        groupTable = conn.getTable(GROUP_TABLE_NAME);
+      }
+      groupList.addAll(groupSerDe.retrieveGroupList(groupTable));
+    } else {
+      LOG.debug("Refershing in Offline mode.");
+      String groupBasePath = ZKUtil.joinZNode(watcher.baseZNode, groupZNode);
+      groupList.addAll(groupSerDe.retrieveGroupList(watcher, groupBasePath));
+    }
+
+    //refresh default group, prune
+    NavigableSet<TableName> orphanTables = new TreeSet<TableName>();
+    for(String entry: master.getTableDescriptors().getAll().keySet()) {
+      orphanTables.add(TableName.valueOf(entry));
+    }
+
+    List<TableName> specialTables;
+    if(!master.isInitialized()) {
+      specialTables = new ArrayList<TableName>();
+      specialTables.add(AccessControlLists.ACL_TABLE_NAME);
+      specialTables.add(TableName.META_TABLE_NAME);
+      specialTables.add(TableName.NAMESPACE_TABLE_NAME);
+      specialTables.add(GroupInfoManager.GROUP_TABLE_NAME);
+    } else {
+      specialTables =
+          master.listTableNamesByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
+    }
+
+    for(TableName table : specialTables) {
+      orphanTables.add(table);
+    }
+    for(GroupInfo group: groupList) {
+      if(!group.getName().equals(GroupInfo.DEFAULT_GROUP)) {
+        orphanTables.removeAll(group.getTables());
+      }
+    }
+
+    //This is added to the last of the list
+    //so it overwrites the default group loaded
+    //from region group table or zk
+    groupList.add(new GroupInfo(GroupInfo.DEFAULT_GROUP,
+        new TreeSet<HostPort>(getDefaultServers()),
+        orphanTables));
+
+
+    //populate the data
+    HashMap<String, GroupInfo> newGroupMap = Maps.newHashMap();
+    HashMap<TableName, String> newTableMap = Maps.newHashMap();
+    for (GroupInfo group : groupList) {
+      newGroupMap.put(group.getName(), group);
+      for(TableName table: group.getTables()) {
+        newTableMap.put(table, group.getName());
+      }
+    }
+    groupMap = Collections.unmodifiableMap(newGroupMap);
+    tableMap = Collections.unmodifiableMap(newTableMap);
+
+    prevGroups.clear();
+    prevGroups.addAll(groupMap.keySet());
+  }
+
+  private synchronized Map<TableName,String> flushConfigTable(Map<String,GroupInfo> newGroupMap) throws IOException {
+    Map<TableName,String> newTableMap = Maps.newHashMap();
+    Put put = new Put(ROW_KEY);
+    Delete delete = new Delete(ROW_KEY);
+
+    //populate deletes
+    for(String groupName : prevGroups) {
+      if(!newGroupMap.containsKey(groupName)) {
+        delete.deleteColumns(META_FAMILY_BYTES, Bytes.toBytes(groupName));
+      }
+    }
+
+    //populate puts
+    for(GroupInfo groupInfo : newGroupMap.values()) {
+      RSGroupProtos.GroupInfo proto = ProtobufUtil.toProtoGroupInfo(groupInfo);
+      put.add(META_FAMILY_BYTES,
+          Bytes.toBytes(groupInfo.getName()),
+          proto.toByteArray());
+      for(TableName entry: groupInfo.getTables()) {
+        newTableMap.put(entry, groupInfo.getName());
+      }
+    }
+
+    RowMutations rowMutations = new RowMutations(ROW_KEY);
+    if(put.size() > 0) {
+      rowMutations.add(put);
+    }
+    if(delete.size() > 0) {
+      rowMutations.add(delete);
+    }
+    if(rowMutations.getMutations().size() > 0) {
+      groupTable.mutateRow(rowMutations);
+    }
+    return newTableMap;
+  }
+
+  private synchronized void flushConfig(Map<String,GroupInfo> newGroupMap) throws IOException {
+    Map<TableName, String> newTableMap;
+    //this should only not enter during startup
+    if(!isOnline()) {
+      LOG.error("Still in Offline mode.");
+      throw new IOException("Still in Offline mode.");
+    }
+
+    newTableMap = flushConfigTable(newGroupMap);
+
+    //make changes visible since it has been
+    //persisted in the source of truth
+    groupMap = Collections.unmodifiableMap(newGroupMap);
+    tableMap = Collections.unmodifiableMap(newTableMap);
+
+
+    try {
+      //Write zk data first since that's what we'll read first
+      String groupBasePath = ZKUtil.joinZNode(watcher.baseZNode, groupZNode);
+      ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC);
+
+      List<ZKUtil.ZKUtilOp> zkOps = new ArrayList<ZKUtil.ZKUtilOp>(newGroupMap.size());
+      for(String groupName : prevGroups) {
+        if(!newGroupMap.containsKey(groupName)) {
+          String znode = ZKUtil.joinZNode(groupBasePath, groupName);
+          zkOps.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(znode));
+        }
+      }
+
+
+      for(GroupInfo groupInfo : newGroupMap.values()) {
+        String znode = ZKUtil.joinZNode(groupBasePath, groupInfo.getName());
+        RSGroupProtos.GroupInfo proto = ProtobufUtil.toProtoGroupInfo(groupInfo);
+        LOG.debug("Updating znode: "+znode);
+        ZKUtil.createAndFailSilent(watcher, znode);
+        zkOps.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(znode));
+        zkOps.add(ZKUtil.ZKUtilOp.createAndFailSilent(znode,
+            ProtobufUtil.prependPBMagic(proto.toByteArray())));
+      }
+      LOG.debug("Writing ZK GroupInfo count: " + zkOps.size());
+
+      ZKUtil.multiOrSequential(watcher, zkOps, false);
+    } catch (KeeperException e) {
+      LOG.error("Failed to write to groupZNode", e);
+      master.abort("Failed to write to groupZNode", e);
+      throw new IOException("Failed to write to groupZNode",e);
+    }
+
+    prevGroups.clear();
+    prevGroups.addAll(newGroupMap.keySet());
+  }
+
+  private List<ServerName> getOnlineRS() throws IOException {
+    if (master != null) {
+      return master.getServerManager().getOnlineServersList();
+    }
+    try {
+      LOG.debug("Reading online RS from zookeeper");
+      List<ServerName> servers = new LinkedList<ServerName>();
+      for (String el: ZKUtil.listChildrenNoWatch(watcher, watcher.rsZNode)) {
+        servers.add(ServerName.parseServerName(el));
+      }
+      return servers;
+    } catch (KeeperException e) {
+      throw new IOException("Failed to retrieve server list from zookeeper", e);
+    }
+  }
+
+  private List<HostPort> getDefaultServers() throws IOException {
+    List<HostPort> defaultServers = new LinkedList<HostPort>();
+    for(ServerName server : getOnlineRS()) {
+      HostPort hostPort = new HostPort(server.getHostname(), server.getPort());
+      boolean found = false;
+      for(GroupInfo groupInfo : groupMap.values()) {
+        if(!GroupInfo.DEFAULT_GROUP.equals(groupInfo.getName()) &&
+            groupInfo.containsServer(hostPort)) {
+          found = true;
+          break;
+        }
+      }
+      if(!found) {
+        defaultServers.add(hostPort);
+      }
+    }
+    return defaultServers;
+  }
+
+  private synchronized void updateDefaultServers(
+      NavigableSet<HostPort> hostPort) throws IOException {
+    if(!isOnline()) {
+      LOG.info("Offline mode. Skipping update of default servers");
+      return;
+    }
+    GroupInfo info = groupMap.get(GroupInfo.DEFAULT_GROUP);
+    GroupInfo newInfo = new GroupInfo(info.getName(), hostPort, info.getTables());
+    HashMap<String, GroupInfo> newGroupMap = Maps.newHashMap(groupMap);
+    newGroupMap.put(newInfo.getName(), newInfo);
+    flushConfig(newGroupMap);
+  }
+
+  @Override
+  public void serverAdded(ServerName serverName) {
+    defaultServerUpdater.serverChanged();
+  }
+
+  @Override
+  public void serverRemoved(ServerName serverName) {
+    defaultServerUpdater.serverChanged();
+  }
+
+  private static class DefaultServerUpdater extends Thread {
+    private static final Log LOG = LogFactory.getLog(DefaultServerUpdater.class);
+    private GroupInfoManagerImpl mgr;
+    private boolean hasChanged = false;
+
+    public DefaultServerUpdater(GroupInfoManagerImpl mgr) {
+      this.mgr = mgr;
+    }
+
+    public void run() {
+      List<HostPort> prevDefaultServers = new LinkedList<HostPort>();
+      while(!mgr.master.isAborted() || !mgr.master.isStopped()) {
+        try {
+          LOG.info("Updating default servers.");
+          List<HostPort> servers = mgr.getDefaultServers();
+          Collections.sort(servers);
+          if(!servers.equals(prevDefaultServers)) {
+            mgr.updateDefaultServers(new TreeSet<HostPort>(servers));
+            prevDefaultServers = servers;
+            LOG.info("Updated with servers: "+servers.size());
+          }
+          try {
+            synchronized (this) {
+              if(!hasChanged) {
+                wait();
+              }
+              hasChanged = false;
+            }
+          } catch (InterruptedException e) {
+          }
+        } catch (IOException e) {
+          LOG.warn("Failed to update default servers", e);
+        }
+      }
+    }
+
+    public void serverChanged() {
+      synchronized (this) {
+        hasChanged = true;
+        this.notify();
+      }
+    }
+  }
+
+
+  private static class GroupStartupWorker extends Thread {
+    private static final Log LOG = LogFactory.getLog(GroupStartupWorker.class);
+
+    private Configuration conf;
+    private volatile boolean isOnline = false;
+    private MasterServices masterServices;
+    private GroupInfoManagerImpl groupInfoManager;
+    private ClusterConnection conn;
+
+    public GroupStartupWorker(GroupInfoManagerImpl groupInfoManager,
+                              MasterServices masterServices,
+                              ClusterConnection conn) {
+      this.conf = masterServices.getConfiguration();
+      this.masterServices = masterServices;
+      this.groupInfoManager = groupInfoManager;
+      this.conn = conn;
+      setName(GroupStartupWorker.class.getName()+"-"+masterServices.getServerName());
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      if(waitForGroupTableOnline()) {
+        LOG.info("GroupBasedLoadBalancer is now online");
+      }
+    }
+
+    public boolean waitForGroupTableOnline() {
+      final List<HRegionInfo> foundRegions = new LinkedList<HRegionInfo>();
+      final List<HRegionInfo> assignedRegions = new LinkedList<HRegionInfo>();
+      final AtomicBoolean found = new AtomicBoolean(false);
+      final TableStateManager tsm = masterServices.getTableStateManager();
+      boolean createSent = false;
+      while (!found.get() && isMasterRunning()) {
+        foundRegions.clear();
+        assignedRegions.clear();
+        found.set(true);
+        try {
+          final Table nsTable = conn.getTable(TableName.NAMESPACE_TABLE_NAME);
+          final Table groupTable = conn.getTable(GroupInfoManager.GROUP_TABLE_NAME);
+          boolean rootMetaFound =
+              masterServices.getMetaTableLocator().verifyMetaRegionLocation(
+                  conn,
+                  masterServices.getZooKeeper(),
+                  1);
+          final AtomicBoolean nsFound = new AtomicBoolean(false);
+          if (rootMetaFound) {
+
+            MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() {
+              @Override
+              public boolean visitInternal(Result row) throws IOException {
+                HRegionInfo info = HRegionInfo.getHRegionInfo(row);
+                if (info != null) {
+                  Cell serverCell =
+                      row.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+                          HConstants.SERVER_QUALIFIER);
+                  if (GROUP_TABLE_NAME.equals(info.getTable()) && serverCell != null) {
+                    ServerName sn =
+                        ServerName.parseVersionedServerName(CellUtil.cloneValue(serverCell));
+                    if (sn == null) {
+                      found.set(false);
+                    } else if (tsm.isTableState(GROUP_TABLE_NAME, TableState.State.ENABLED)) {
+                      try {
+                        HBaseProtos.RegionSpecifier regionSpecifier =
+                            HBaseProtos.RegionSpecifier.newBuilder()
+                                .setValue(ByteString.copyFrom(row.getRow()))
+                                .setType(
+                                    HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
+                                .build();
+                        ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
+                        ClientProtos.GetRequest req =
+                            ClientProtos.GetRequest.newBuilder()
+                                .setRegion(regionSpecifier)
+                                .setGet(ProtobufUtil.toGet(new Get(ROW_KEY))).build();
+                        rs.get(null, req);
+                        assignedRegions.add(info);
+                      } catch(Exception ex) {
+                        LOG.debug("Caught exception while verifying group region", ex);
+                      }
+                    }
+                    foundRegions.add(info);
+                  }
+                  if (TableName.NAMESPACE_TABLE_NAME.equals(info.getTable())) {
+                    Cell cell = row.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+                        HConstants.SERVER_QUALIFIER);
+                    ServerName sn = null;
+                    if(cell != null) {
+                      sn = ServerName.parseVersionedServerName(CellUtil.cloneValue(cell));
+                    }
+                    if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME,
+                        TableState.State.ENABLED)) {
+                      try {
+                        HBaseProtos.RegionSpecifier regionSpecifier =
+                            HBaseProtos.RegionSpecifier.newBuilder()
+                                .setValue(ByteString.copyFrom(row.getRow()))
+                                .setType(
+                                    HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
+                                .build();
+                        ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
+                        ClientProtos.GetRequest req =
+                            ClientProtos.GetRequest.newBuilder()
+                                .setRegion(regionSpecifier)
+                                .setGet(ProtobufUtil.toGet(new Get(ROW_KEY))).build();
+                        rs.get(null, req);
+                        nsFound.set(true);
+                      } catch(Exception ex) {
+                        LOG.debug("Caught exception while verifying group region", ex);
+                      }
+                    }
+                  }
+                }
+                return true;
+              }
+            };
+            MetaTableAccessor.fullScanRegions(conn, visitor);
+            // if no regions in meta then we have to create the table
+            if (foundRegions.size() < 1 && rootMetaFound && !createSent && nsFound.get()) {
+              groupInfoManager.createGroupTable(masterServices);
+              createSent = true;
+            }
+            LOG.info("Group table: " + GROUP_TABLE_NAME + " isOnline: " + found.get()
+                + ", regionCount: " + foundRegions.size() + ", assignCount: "
+                + assignedRegions.size() + ", rootMetaFound: "+rootMetaFound);
+            found.set(found.get() && assignedRegions.size() == foundRegions.size()
+                && foundRegions.size() > 0);
+          } else {
+            LOG.info("Waiting for catalog tables to come online");
+            found.set(false);
+          }
+          if (found.get()) {
+            LOG.debug("With group table online, refreshing cached information.");
+            groupInfoManager.refresh(true);
+            isOnline = true;
+            //flush any inconsistencies between ZK and HTable
+            groupInfoManager.flushConfig(groupInfoManager.groupMap);
+          }
+        } catch(Exception e) {
+          found.set(false);
+          LOG.warn("Failed to perform check", e);
+        }
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          LOG.info("Sleep interrupted", e);
+        }
+      }
+      return found.get();
+    }
+
+    public boolean isOnline() {
+      return isOnline;
+    }
+
+    private boolean isMasterRunning() {
+      return !masterServices.isAborted() && !masterServices.isStopped();
+    }
+  }
+
+  private void createGroupTable(MasterServices masterServices) throws IOException {
+    HRegionInfo newRegions[] = new HRegionInfo[]{
+        new HRegionInfo(GROUP_TABLE_DESC.getTableName(), null, null)};
+    //we need to create the table this way to bypass
+    //checkInitialized
+    masterServices.getExecutorService()
+        .submit(new CreateTableHandler(
+            masterServices,
+            masterServices.getMasterFileSystem(),
+            GROUP_TABLE_DESC,
+            masterServices.getConfiguration(),
+            newRegions,
+            masterServices).prepare());
+    //wait for region to be online
+    int tries = 600;
+    while(masterServices.getAssignmentManager().getRegionStates()
+        .getRegionServerOfRegion(newRegions[0]) == null && tries > 0) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        throw new IOException("Wait interrupted", e);
+      }
+      tries--;
+    }
+    if(tries <= 0) {
+      throw new IOException("Failed to create group table.");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupSerDe.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupSerDe.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupSerDe.java
new file mode 100644
index 0000000..b0262ef
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupSerDe.java
@@ -0,0 +1,97 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+
+//TODO do better encapsulation of SerDe logic from GroupInfoManager and GroupTracker
+public class GroupSerDe {
+  private static final Log LOG = LogFactory.getLog(GroupSerDe.class);
+
+  public GroupSerDe() {
+
+  }
+
+  public List<GroupInfo> retrieveGroupList(Table groupTable) throws IOException {
+    List<GroupInfo> groupInfoList = Lists.newArrayList();
+    Result result = groupTable.get(new Get(GroupInfoManager.ROW_KEY));
+    if(!result.isEmpty()) {
+      NavigableMap<byte[],NavigableMap<byte[],byte[]>> dataMap = result.getNoVersionMap();
+      for(byte[] groupName: dataMap.get(GroupInfoManager.META_FAMILY_BYTES).keySet()) {
+        RSGroupProtos.GroupInfo proto =
+            RSGroupProtos.GroupInfo.parseFrom(
+                dataMap.get(GroupInfoManager.META_FAMILY_BYTES).get(groupName));
+        groupInfoList.add(ProtobufUtil.toGroupInfo(proto));
+      }
+    }
+    return groupInfoList;
+  }
+
+  public List<GroupInfo> retrieveGroupList(ZooKeeperWatcher watcher,
+                                           String groupBasePath) throws IOException {
+    List<GroupInfo> groupInfoList = Lists.newArrayList();
+    //Overwrite any info stored by table, this takes precedence
+    try {
+      if(ZKUtil.checkExists(watcher, groupBasePath) != -1) {
+        for(String znode: ZKUtil.listChildrenAndWatchForNewChildren(watcher, groupBasePath)) {
+          byte[] data = ZKUtil.getData(watcher, ZKUtil.joinZNode(groupBasePath, znode));
+          if(data.length > 0) {
+            ProtobufUtil.expectPBMagicPrefix(data);
+            ByteArrayInputStream bis = new ByteArrayInputStream(
+                data, ProtobufUtil.lengthOfPBMagic(), data.length);
+            groupInfoList.add(ProtobufUtil.toGroupInfo(RSGroupProtos.GroupInfo.parseFrom(bis)));
+          }
+        }
+        LOG.debug("Read ZK GroupInfo count:" + groupInfoList.size());
+      }
+    } catch (KeeperException e) {
+      throw new IOException("Failed to read groupZNode",e);
+    } catch (DeserializationException e) {
+      throw new IOException("Failed to read groupZNode",e);
+    } catch (InterruptedException e) {
+      throw new IOException("Failed to read groupZNode",e);
+    }
+    return groupInfoList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupTracker.java
new file mode 100644
index 0000000..f0c0a8f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupTracker.java
@@ -0,0 +1,341 @@
+/*
+ * Copyright 2010 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.group;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class GroupTracker extends ZooKeeperNodeTracker {
+  private static final Log LOG = LogFactory.getLog(GroupTracker.class);
+
+  private List<Listener> listeners = Collections.synchronizedList(new ArrayList<Listener>());
+  private GroupSerDe groupSerDe = new GroupSerDe();
+  private volatile Map<String, GroupInfo> groupMap = new HashMap<String, GroupInfo>();
+  private volatile Map<HostPort, GroupInfo> serverMap = new HashMap<HostPort, GroupInfo>();
+  private RegionServerTracker rsTracker;
+  private volatile boolean started = false;
+
+  /**
+   * Constructs a new ZK node tracker.
+   * <p/>
+   * <p>After construction, use {@link #start} to kick off tracking.
+   *
+   * @param watcher
+   * @param abortable
+   */
+  public GroupTracker(ZooKeeperWatcher watcher, Abortable abortable) throws IOException {
+    //TODO make period configurable
+    super(watcher,
+        ZKUtil.joinZNode(watcher.baseZNode, GroupInfoManager.groupZNode),
+        abortable!=null?abortable:new PersistentAbortable(10000));
+    if(abortable == null) {
+      ((PersistentAbortable)this.abortable).setGroupTracker(this);
+    }
+    rsTracker = new RegionServerTracker(watcher, abortable, this);
+    try {
+      ZKUtil.listChildrenAndWatchThem(watcher, node);
+      rsTracker.start();
+    } catch (KeeperException e) {
+      throw new IOException("Failed to start RS tracker", e);
+    }
+  }
+
+  public void addListener(Listener listener) {
+    listeners.add(listener);
+  }
+
+  public void removeListener(Listener listener) {
+    listeners.remove(listener);
+  }
+
+  @Override
+  public synchronized void start() {
+    super.start();
+    started = true;
+  }
+
+  @Override
+  public void nodeCreated(String path) {
+    if (path.equals(node)) {
+      refresh();
+    }
+  }
+
+  @Override
+  public void nodeDataChanged(String path) {
+    if (path.equals(node)) {
+      nodeCreated(path);
+    }
+  }
+
+  @Override
+  public void nodeChildrenChanged(String path) {
+    if (path.startsWith(node)) {
+      refresh();
+    }
+  }
+
+  public void blockUntilReady(int timeout) throws InterruptedException, IOException {
+    blockUntilAvailable(timeout, false);
+    if(getData(false) != null) {
+      refresh(false);
+    }
+  }
+
+  private void refresh() {
+    try {
+      refresh(false);
+    } catch (IOException e) {
+      this.abortable.abort("Failed to read group znode", e);
+    }
+  }
+
+  private synchronized void refresh(boolean force) throws IOException {
+    List<ServerName> onlineRS = rsTracker.getOnlineServers();
+    Set<HostPort> hostPorts = new HashSet<HostPort>();
+    for(ServerName entry: onlineRS) {
+      hostPorts.add(new HostPort(entry.getHostname(), entry.getPort()));
+    }
+    Map<String, GroupInfo> tmpGroupMap = new HashMap<String, GroupInfo>();
+    Map<HostPort, GroupInfo> tmpServerMap = new HashMap<HostPort, GroupInfo>();
+    for(GroupInfo groupInfo: listGroups()) {
+      tmpGroupMap.put(groupInfo.getName(), groupInfo);
+      for(HostPort server: groupInfo.getServers()) {
+        tmpServerMap.put(server, groupInfo);
+        hostPorts.remove(server);
+      }
+    }
+    GroupInfo groupInfo = tmpGroupMap.get(GroupInfo.DEFAULT_GROUP);
+    groupInfo.addAllServers(hostPorts);
+    for(HostPort entry: hostPorts) {
+      tmpServerMap.put(entry, groupInfo);
+    }
+
+    //when reading sync on "this" if groupMap<->serverMap
+    //invariant needs to be guaranteed
+    groupMap = tmpGroupMap;
+    serverMap = tmpServerMap;
+
+    Map<String, GroupInfo> map = getGroupMap();
+    for(Listener listener : listeners) {
+      listener.groupMapChanged(map);
+    }
+  }
+
+  private List<GroupInfo> listGroups() throws IOException {
+    return groupSerDe.retrieveGroupList(watcher, node);
+  }
+
+  public GroupInfo getGroup(String name) {
+    GroupInfo groupInfo = groupMap.get(name);
+    return groupInfo;
+  }
+
+  public GroupInfo getGroupOfServer(String hostPort) {
+    GroupInfo groupInfo = serverMap.get(hostPort);
+    return groupInfo;
+  }
+
+  public Map<String, GroupInfo> getGroupMap() {
+    return Collections.unmodifiableMap(groupMap);
+  }
+
+  public interface Listener {
+    public void groupMapChanged(Map<String, GroupInfo> groupMap);
+  }
+
+
+  /**
+   * This class is copied for RegionServerTracker
+   * We need our own since the other one was tied to ServerManager
+   * and thus the master
+   */
+  private static class RegionServerTracker extends ZooKeeperListener {
+    private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
+    private volatile List<ServerName> regionServers = new ArrayList<ServerName>();
+    private Abortable abortable;
+    private GroupTracker groupTracker;
+
+    public RegionServerTracker(ZooKeeperWatcher watcher,
+        Abortable abortable, GroupTracker groupTracker) {
+      super(watcher);
+      this.abortable = abortable;
+      this.groupTracker = groupTracker;
+    }
+
+    public void start() throws KeeperException, IOException {
+      watcher.registerListener(this);
+      refresh();
+    }
+
+    private void add(final List<String> servers) throws IOException {
+      List<ServerName> temp = new ArrayList<ServerName>();
+      for (String n: servers) {
+        ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n));
+        temp.add(sn);
+      }
+      regionServers = temp;
+      //we're refreshing groups, since default membership
+      //is dynamic and new servers may end up as new default group members
+      refreshGroups();
+    }
+
+    private void remove(final ServerName sn) {
+      List<ServerName> temp = new ArrayList<ServerName>();
+      for(ServerName el: regionServers) {
+        if(!sn.equals(el)) {
+          temp.add(el);
+        }
+      }
+      regionServers = temp;
+      refreshGroups();
+    }
+
+    private void refreshGroups() {
+      if(groupTracker.started && groupTracker.getData(false) != null) {
+        groupTracker.refresh();
+      }
+    }
+
+    public void refresh() throws KeeperException, IOException {
+      List<String> servers =
+        ZKUtil.listChildrenAndWatchThem(watcher, watcher.rsZNode);
+      add(servers);
+    }
+
+    @Override
+    public void nodeDeleted(String path) {
+      if (path.startsWith(watcher.rsZNode)) {
+        String serverName = ZKUtil.getNodeName(path);
+        LOG.info("RegionServer ephemeral node deleted, processing expiration [" +
+          serverName + "]");
+        ServerName sn = ServerName.parseServerName(serverName);
+        remove(sn);
+      }
+    }
+
+    @Override
+    public void nodeChildrenChanged(String path) {
+      if (path.equals(watcher.rsZNode)) {
+        try {
+          List<String> servers =
+            ZKUtil.listChildrenAndWatchThem(watcher, watcher.rsZNode);
+          add(servers);
+        } catch (IOException e) {
+          abortable.abort("Unexpected zk exception getting RS nodes", e);
+        } catch (KeeperException e) {
+          abortable.abort("Unexpected zk exception getting RS nodes", e);
+        }
+      }
+    }
+
+    /**
+     * Gets the online servers.
+     * @return list of online servers
+     */
+    public List<ServerName> getOnlineServers() {
+      return regionServers;
+    }
+  }
+
+  private static class Refresher extends Thread {
+    private final static Log LOG = LogFactory.getLog(Refresher.class);
+    private GroupTracker groupTracker;
+    private volatile boolean isRunning = true;
+    private int period;
+
+    public Refresher(GroupTracker groupTracker, int period) {
+      this.groupTracker = groupTracker;
+      this.period = period;
+      this.setDaemon(true);
+    }
+
+    public boolean isRunning() {
+      return isRunning;
+    }
+
+    @Override
+    public void run() {
+      while(true) {
+        try {
+          groupTracker.rsTracker.refresh();
+          groupTracker.refresh(true);
+          LOG.info("Recovery refresh successful");
+          isRunning = false;
+          return;
+        } catch (IOException e) {
+          LOG.warn("Failed to refresh", e);
+        } catch (KeeperException e) {
+          LOG.warn("Failed to refresh", e);
+        }
+        try {
+          Thread.sleep(period);
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+  }
+
+  private static class PersistentAbortable implements Abortable {
+    private final Log LOG = LogFactory.getLog(Abortable.class);
+    private Refresher refresher;
+    private GroupTracker groupTracker;
+    private int period;
+
+
+    public PersistentAbortable(int period) {
+      this.period = period;
+    }
+
+    public void setGroupTracker(GroupTracker groupTracker) {
+      this.groupTracker = groupTracker;
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("Launching referesher because of abort: "+why, e);
+      if(refresher == null || !refresher.isRunning()) {
+        refresher = new Refresher(groupTracker, period);
+      }
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupableBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupableBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupableBalancer.java
new file mode 100644
index 0000000..e696926
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/GroupableBalancer.java
@@ -0,0 +1,12 @@
+package org.apache.hadoop.hbase.group;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+
+import java.io.IOException;
+
+@InterfaceAudience.Private
+public interface GroupableBalancer extends LoadBalancer {
+
+  void setGroupInfoManager(GroupInfoManager groupInfoManager) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBean.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBean.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBean.java
new file mode 100644
index 0000000..a19b24e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBean.java
@@ -0,0 +1,65 @@
+/**
+ * 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.group;
+
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.TableName;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public interface MXBean {
+
+  public Map<String, List<HostPort>> getServersByGroup() throws IOException;
+
+  public List<GroupInfoBean> getGroups() throws IOException;
+
+  public static class GroupInfoBean {
+
+    private String name;
+    private List<HostPort> servers;
+    private List<TableName> tables;
+
+    //Need this to convert NavigableSet to List
+    public GroupInfoBean(GroupInfo groupInfo) {
+      this.name = groupInfo.getName();
+      this.servers = new LinkedList<HostPort>();
+      this.servers.addAll(groupInfo.getServers());
+      this.tables = new LinkedList<TableName>();
+      this.tables.addAll(groupInfo.getTables());
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public List<HostPort> getServers() {
+      return servers;
+    }
+
+    public List<TableName> getTables() {
+      return tables;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBeanImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBeanImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBeanImpl.java
new file mode 100644
index 0000000..b0894eb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/group/MXBeanImpl.java
@@ -0,0 +1,82 @@
+/**
+ * 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.group;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterServices;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class MXBeanImpl implements MXBean {
+  private static final Log LOG = LogFactory.getLog(MXBeanImpl.class);
+
+  private static MXBeanImpl instance = null;
+
+  private GroupAdmin groupAdmin;
+  private MasterServices master;
+
+  public synchronized static MXBeanImpl init(
+      final GroupAdmin groupAdmin,
+      MasterServices master) {
+    if (instance == null) {
+      instance = new MXBeanImpl(groupAdmin, master);
+    }
+    return instance;
+  }
+
+  protected MXBeanImpl(final GroupAdmin groupAdmin,
+      MasterServices master) {
+    this.groupAdmin = groupAdmin;
+    this.master = master;
+  }
+
+  @Override
+  public Map<String, List<HostPort>> getServersByGroup() throws IOException {
+    Map<String, List<HostPort>> data = new HashMap<String, List<HostPort>>();
+    for (final ServerName entry :
+      master.getServerManager().getOnlineServersList()) {
+      GroupInfo groupInfo = groupAdmin.getGroupOfServer(
+          new HostPort(entry.getHostname(), entry.getPort()));
+      if(!data.containsKey(groupInfo.getName())) {
+        data.put(groupInfo.getName(), new LinkedList<HostPort>());
+      }
+      data.get(groupInfo.getName()).add(entry.getHostPort());
+    }
+    return data;
+  }
+
+  @Override
+  public List<GroupInfoBean> getGroups() throws IOException {
+    LinkedList list = new LinkedList();
+    for(GroupInfo group: groupAdmin.listGroups()) {
+      list.add(new GroupInfoBean(group));
+    }
+    return list;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index f7f98fe..248aafc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -1033,7 +1033,7 @@ public class AssignmentManager {
           return;
         }
         LOG.info("Assigning " + region.getRegionNameAsString() +
-            " to " + plan.getDestination().toString());
+            " to " + plan.getDestination());
         // Transition RegionState to PENDING_OPEN
        regionStates.updateRegionState(region,
           State.PENDING_OPEN, plan.getDestination());
@@ -1222,8 +1222,7 @@ public class AssignmentManager {
           || existingPlan.getDestination() == null
           || !destServers.contains(existingPlan.getDestination())) {
         newPlan = true;
-        randomPlan = new RegionPlan(region, null,
-            balancer.randomAssignment(region, destServers));
+        randomPlan = new RegionPlan(region, null, balancer.randomAssignment(region, destServers));
         if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
           List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
           regions.add(region);
@@ -1468,6 +1467,14 @@ public class AssignmentManager {
       throw new IOException("Unable to determine a plan to assign region(s)");
     }
 
+    if (bulkPlan.containsKey(LoadBalancer.BOGUS_SERVER_NAME)) {
+      // Found no plan for some regions, put those regions in RIT
+      for (HRegionInfo hri : bulkPlan.get(LoadBalancer.BOGUS_SERVER_NAME)) {
+        regionStates.updateRegionState(hri, State.FAILED_OPEN);
+      }
+      bulkPlan.remove(LoadBalancer.BOGUS_SERVER_NAME);
+    }
+
     assign(regions.size(), servers.size(),
       "retainAssignment=true", bulkPlan);
   }
@@ -1497,6 +1504,14 @@ public class AssignmentManager {
       throw new IOException("Unable to determine a plan to assign region(s)");
     }
 
+    if (bulkPlan.containsKey(LoadBalancer.BOGUS_SERVER_NAME)) {
+      // Found no plan for some regions, put those regions in RIT
+      for (HRegionInfo hri : bulkPlan.get(LoadBalancer.BOGUS_SERVER_NAME)) {
+        regionStates.updateRegionState(hri, State.FAILED_OPEN);
+      }
+      bulkPlan.remove(LoadBalancer.BOGUS_SERVER_NAME);
+    }
+
     processFavoredNodes(regions);
     assign(regions.size(), servers.size(), "round-robin=true", bulkPlan);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index a4bbe6f..503d9b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -45,6 +45,11 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Service;
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -76,9 +81,13 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
+import org.apache.hadoop.hbase.group.GroupAdminServer;
+import org.apache.hadoop.hbase.group.GroupInfo;
+import org.apache.hadoop.hbase.group.GroupableBalancer;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
@@ -323,6 +332,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   // handle table states
   private TableStateManager tableStateManager;
 
+  private GroupAdminServer groupAdminServer;
+
   /** flag used in test cases in order to simulate RS failures during master initialization */
   private volatile boolean initializationBeforeMetaAssignment = false;
 
@@ -718,6 +729,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       waitForServerOnline();
     }
 
+    if (balancer instanceof GroupableBalancer) {
+      groupAdminServer = new GroupAdminServer(this);
+      ((GroupableBalancer)balancer).setGroupInfoManager(groupAdminServer.getGroupInfoManager());
+    }
+
     //initialize load balancer
     this.balancer.setClusterStatus(getClusterStatus());
     this.balancer.setMasterServices(this);
@@ -1373,11 +1389,17 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       final byte[] destServerName) throws HBaseIOException {
     RegionState regionState = assignmentManager.getRegionStates().
       getRegionState(Bytes.toString(encodedRegionName));
-    if (regionState == null) {
+
+    HRegionInfo hri;
+    if (Bytes.toString(encodedRegionName)
+        .equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+      hri = HRegionInfo.FIRST_META_REGIONINFO;
+    } else if (regionState != null) {
+      hri = regionState.getRegion();
+    } else {
       throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
     }
 
-    HRegionInfo hri = regionState.getRegion();
     ServerName dest;
     if (destServerName == null || destServerName.length == 0) {
       LOG.info("Passed destination servername is null/empty so " +
@@ -1390,7 +1412,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         return;
       }
     } else {
-      dest = ServerName.valueOf(Bytes.toString(destServerName));
+      ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
+      dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
+      if (dest == null) {
+        LOG.debug("Unable to determine a plan to assign " + hri);
+        return;
+      }
       if (dest.equals(serverName) && balancer instanceof BaseLoadBalancer
           && !((BaseLoadBalancer)balancer).shouldBeOnMaster(hri)) {
         // To avoid unnecessary region moving later by balancer. Don't put user
@@ -1453,7 +1480,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
     sanityCheckTableDescriptor(hTableDescriptor);
-
     if (cpHost != null) {
       cpHost.preCreateTable(hTableDescriptor, newRegions);
     }
@@ -1463,16 +1489,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     //       TableExistsException by saying if the schema is the same or not.
     ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
     long procId = this.procedureExecutor.submitProcedure(
-      new CreateTableProcedure(
-        procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch),
-      nonceGroup,
-      nonce);
+        new CreateTableProcedure(
+            procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch),
+        nonceGroup,
+        nonce);
     latch.await();
-
     if (cpHost != null) {
       cpHost.postCreateTable(hTableDescriptor, newRegions);
     }
-
     return procId;
   }
 
@@ -2332,6 +2356,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
     TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
     checkNamespaceManagerReady();
+
+    String group =  descriptor.getConfigurationValue(GroupInfo.NAMESPACEDESC_PROP_GROUP);
+    if(group != null && groupAdminServer.getGroupInfo(group) == null) {
+      throw new ConstraintException("Region server group "+group+" does not exit");
+    }
     if (cpHost != null) {
       if (cpHost.preCreateNamespace(descriptor)) {
         return;
@@ -2348,6 +2377,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
     TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
     checkNamespaceManagerReady();
+
+    String group = descriptor.getConfigurationValue(GroupInfo.NAMESPACEDESC_PROP_GROUP);
+    if(group != null && groupAdminServer.getGroupInfo(group) == null) {
+      throw new ConstraintException("Region server group "+group+" does not exit");
+    }
     if (cpHost != null) {
       if (cpHost.preModifyNamespace(descriptor)) {
         return;
@@ -2667,4 +2701,14 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, LoadBalancerFactory
         .getDefaultLoadBalancerClass().getName());
   }
+
+  @Override
+  public LoadBalancer getLoadBalancer() {
+    return balancer;
+  }
+
+  @Override
+  public GroupAdminServer getGroupAdminServer() {
+    return groupAdminServer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index c4eecfa..e942713 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -51,6 +51,9 @@ import org.apache.hadoop.hbase.Stoppable;
 @InterfaceAudience.Private
 public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObserver {
 
+  //used to signal to the caller that the region(s) cannot be assigned
+  ServerName BOGUS_SERVER_NAME = ServerName.parseServerName("127.0.0.1,1,1");
+
   /**
    * Set the current cluster status.  This allows a LoadBalancer to map host name to a server
    * @param st

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/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 16b8852..7a3b01f 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.lang.ClassUtils;
 import org.apache.commons.logging.Log;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HostPort;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -1101,4 +1103,116 @@ public class MasterCoprocessorHost
     }
     return bypass;
   }
+
+  public void preMoveServers(final Set<HostPort> servers, final String targetGroup)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.preMoveServers(ctx, servers, targetGroup);
+      }
+    });
+  }
+
+  public void postMoveServers(final Set<HostPort> servers, final String targetGroup)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.postMoveServers(ctx, servers, targetGroup);
+      }
+    });
+  }
+
+  public void preMoveTables(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 {
+        oserver.preMoveTables(ctx, tables, targetGroup);
+      }
+    });
+  }
+
+  public void postMoveTables(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 {
+        oserver.postMoveTables(ctx, tables, targetGroup);
+      }
+    });
+  }
+
+  public void preAddGroup(final String name)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.preAddGroup(ctx, name);
+      }
+    });
+  }
+
+  public void postAddGroup(final String name)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.postAddGroup(ctx, name);
+      }
+    });
+  }
+
+  public void preRemoveGroup(final String name)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.preRemoveGroup(ctx, name);
+      }
+    });
+  }
+
+  public void postRemoveGroup(final String name)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.postRemoveGroup(ctx, name);
+      }
+    });
+  }
+
+  public void preBalanceGroup(final String name)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.preBalanceGroup(ctx, name);
+      }
+    });
+  }
+
+  public void postBalanceGroup(final String name, final boolean balanceRan)
+      throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver,
+          ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
+        oserver.postBalanceGroup(ctx, name, balanceRan);
+      }
+    });
+  }
+
+
 }


[6/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Posted by ap...@apache.org.
HBASE-6721 RegionServer group based assignment (Francis Liu)


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

Branch: refs/heads/hbase-6721
Commit: 16f65badc80a2ab42914158d5b658c3f80cad1e9
Parents: 4b6598e
Author: Andrew Purtell <ap...@apache.org>
Authored: Sat Aug 1 09:32:25 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Sat Aug 1 09:35:33 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/Connection.java  |     9 +
 .../hadoop/hbase/client/ConnectionAdapter.java  |     6 +
 .../hbase/client/ConnectionImplementation.java  |    79 +
 .../hadoop/hbase/group/GroupAdminClient.java    |   192 +
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |    34 +
 .../java/org/apache/hadoop/hbase/HostPort.java  |    76 +
 .../org/apache/hadoop/hbase/ServerName.java     |    21 +-
 .../apache/hadoop/hbase/group/GroupAdmin.java   |   108 +
 .../apache/hadoop/hbase/group/GroupInfo.java    |   182 +
 .../hbase/group/IntegrationTestGroup.java       |    94 +
 .../hbase/protobuf/generated/HBaseProtos.java   |   643 +-
 .../hbase/protobuf/generated/MasterProtos.java  | 13466 ++++++++++++++++-
 .../hbase/protobuf/generated/RSGroupProtos.java |  1331 ++
 hbase-protocol/src/main/protobuf/HBase.proto    |     6 +
 hbase-protocol/src/main/protobuf/Master.proto   |   114 +
 hbase-protocol/src/main/protobuf/RSGroup.proto  |    33 +
 .../BaseMasterAndRegionObserver.java            |    53 +
 .../hbase/coprocessor/BaseMasterObserver.java   |    52 +
 .../hbase/coprocessor/MasterObserver.java       |    96 +
 .../hadoop/hbase/group/GroupAdminServer.java    |   486 +
 .../hbase/group/GroupBasedLoadBalancer.java     |   433 +
 .../hadoop/hbase/group/GroupInfoManager.java    |   129 +
 .../hbase/group/GroupInfoManagerImpl.java       |   702 +
 .../apache/hadoop/hbase/group/GroupSerDe.java   |    97 +
 .../apache/hadoop/hbase/group/GroupTracker.java |   341 +
 .../hadoop/hbase/group/GroupableBalancer.java   |    12 +
 .../org/apache/hadoop/hbase/group/MXBean.java   |    65 +
 .../apache/hadoop/hbase/group/MXBeanImpl.java   |    82 +
 .../hadoop/hbase/master/AssignmentManager.java  |    21 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |    64 +-
 .../hadoop/hbase/master/LoadBalancer.java       |     3 +
 .../hbase/master/MasterCoprocessorHost.java     |   114 +
 .../hadoop/hbase/master/MasterRpcServices.java  |   212 +-
 .../hadoop/hbase/master/MasterServices.java     |    11 +
 .../master/procedure/CreateTableProcedure.java  |    10 +-
 .../master/procedure/DeleteTableProcedure.java  |     4 +
 .../hbase/security/access/AccessController.java |    32 +
 .../hbase/coprocessor/TestMasterObserver.java   |    52 +
 .../apache/hadoop/hbase/group/TestGroups.java   |   366 +
 .../hadoop/hbase/group/TestGroupsBase.java      |   561 +
 .../hbase/group/VerifyingGroupAdminClient.java  |   159 +
 .../master/TestAssignmentManagerOnCluster.java  |   128 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |    18 +-
 .../balancer/TestGroupBasedLoadBalancer.java    |   588 +
 .../security/access/TestAccessController.java   |   125 +
 hbase-shell/src/main/ruby/hbase.rb              |     1 +
 hbase-shell/src/main/ruby/hbase/admin.rb        |    21 +
 hbase-shell/src/main/ruby/hbase/group_admin.rb  |   121 +
 hbase-shell/src/main/ruby/hbase/hbase.rb        |     4 +
 hbase-shell/src/main/ruby/shell.rb              |    23 +
 hbase-shell/src/main/ruby/shell/commands.rb     |     4 +
 .../src/main/ruby/shell/commands/add_group.rb   |    39 +
 .../main/ruby/shell/commands/balance_group.rb   |    37 +
 .../src/main/ruby/shell/commands/get_group.rb   |    44 +
 .../ruby/shell/commands/get_server_group.rb     |    40 +
 .../main/ruby/shell/commands/get_table_group.rb |    41 +
 .../commands/list_group_server_transitions.rb   |    44 +
 .../ruby/shell/commands/list_group_tables.rb    |    45 +
 .../src/main/ruby/shell/commands/list_groups.rb |    50 +
 .../ruby/shell/commands/move_group_servers.rb   |    37 +
 .../ruby/shell/commands/move_group_tables.rb    |    37 +
 .../main/ruby/shell/commands/remove_group.rb    |    37 +
 62 files changed, 21607 insertions(+), 428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
index dab4905..0e9d5c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.group.GroupAdmin;
 
 /**
  * A cluster connection encapsulating lower level individual connections to actual servers and
@@ -166,6 +167,14 @@ public interface Connection extends Abortable, Closeable {
    */
   Admin getAdmin() throws IOException;
 
+  /**
+   * Retrieve A GroupAdmin implementation which is used for manage region server groups.
+   * Client is not guaranteed to be thread-safe.
+   * @return GroupAdmin instance
+   * @throws IOException
+   */
+  GroupAdmin getGroupAdmin() throws IOException;
+
   @Override
   public void close() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
index 1d8a793..74d2237 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
+import org.apache.hadoop.hbase.group.GroupAdmin;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
@@ -131,6 +132,11 @@ abstract class ConnectionAdapter implements ClusterConnection {
   }
 
   @Override
+  public GroupAdmin getGroupAdmin() throws IOException {
+    return wrappedConnection.getGroupAdmin();
+  }
+
+  @Override
   public boolean isMasterRunning() throws MasterNotRunningException,
       ZooKeeperConnectionException {
     return wrappedConnection.isMasterRunning();

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 2754997..5dfa2be 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
+import org.apache.hadoop.hbase.group.GroupAdmin;
+import org.apache.hadoop.hbase.group.GroupAdminClient;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
@@ -54,8 +56,26 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddGroupResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceGroupResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfServerRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfServerResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoOfTableResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListGroupInfosRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListGroupInfosResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveServersRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveServersResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveTablesRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveTablesResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RemoveGroupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RemoveGroupResponse;
 import org.apache.hadoop.hbase.quotas.ThrottlingException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.security.User;
@@ -362,6 +382,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     return new HBaseAdmin(this);
   }
 
+  @Override
+  public GroupAdmin getGroupAdmin() throws IOException {
+    return new GroupAdminClient(conf);
+  }
+
   private ExecutorService getBatchPool() {
     if (batchPool == null) {
       synchronized (this) {
@@ -1611,6 +1636,60 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public GetGroupInfoResponse getGroupInfo(RpcController controller,
+          GetGroupInfoRequest request) throws ServiceException {
+        return stub.getGroupInfo(controller, request);
+      }
+
+      @Override
+      public GetGroupInfoOfTableResponse getGroupInfoOfTable(RpcController controller,
+          GetGroupInfoOfTableRequest request) throws ServiceException {
+        return stub.getGroupInfoOfTable(controller, request);
+      }
+
+      @Override
+      public GetGroupInfoOfServerResponse getGroupInfoOfServer(RpcController controller,
+          GetGroupInfoOfServerRequest request) throws ServiceException {
+        return stub.getGroupInfoOfServer(controller, request);
+      }
+
+      @Override
+      public MoveServersResponse moveServers(RpcController controller,
+          MoveServersRequest request) throws ServiceException {
+        return stub.moveServers(controller, request);
+      }
+
+      @Override
+      public MoveTablesResponse moveTables(RpcController controller,
+          MoveTablesRequest request) throws ServiceException {
+        return stub.moveTables(controller, request);
+      }
+
+      @Override
+      public AddGroupResponse addGroup(RpcController controller,
+          AddGroupRequest request) throws ServiceException {
+        return stub.addGroup(controller, request);
+      }
+
+      @Override
+      public RemoveGroupResponse removeGroup(RpcController controller,
+          RemoveGroupRequest request) throws ServiceException {
+        return stub.removeGroup(controller, request);
+      }
+
+      @Override
+      public BalanceGroupResponse balanceGroup(RpcController controller,
+          BalanceGroupRequest request) throws ServiceException {
+        return stub.balanceGroup(controller, request);
+      }
+
+      @Override
+      public ListGroupInfosResponse listGroupInfos(RpcController controller,
+          ListGroupInfosRequest request) throws ServiceException {
+        return stub.listGroupInfos(controller, request);
+      }
+
+      @Override
       public MasterProtos.IsMasterRunningResponse isMasterRunning(
           RpcController controller, MasterProtos.IsMasterRunningRequest request)
           throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-client/src/main/java/org/apache/hadoop/hbase/group/GroupAdminClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/group/GroupAdminClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/group/GroupAdminClient.java
new file mode 100644
index 0000000..691e9dc
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/group/GroupAdminClient.java
@@ -0,0 +1,192 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Sets;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Client used for managing region server group information.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class GroupAdminClient implements GroupAdmin {
+  private MasterProtos.MasterService.BlockingInterface proxy;
+	private static final Log LOG = LogFactory.getLog(GroupAdminClient.class);
+
+  public GroupAdminClient(Configuration conf) throws IOException {
+    proxy = new HBaseAdmin(conf).getConnection().getKeepAliveMasterService();
+  }
+
+  @Override
+  public GroupInfo getGroupInfo(String groupName) throws IOException {
+    try {
+      MasterProtos.GetGroupInfoResponse resp =
+        proxy.getGroupInfo(null,
+            MasterProtos.GetGroupInfoRequest.newBuilder().setGroupName(groupName).build());
+      if(resp.hasGroupInfo()) {
+        return ProtobufUtil.toGroupInfo(resp.getGroupInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public GroupInfo getGroupInfoOfTable(TableName tableName) throws IOException {
+    MasterProtos.GetGroupInfoOfTableRequest request =
+        MasterProtos.GetGroupInfoOfTableRequest.newBuilder()
+            .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
+
+    try {
+      return ProtobufUtil.toGroupInfo(proxy.getGroupInfoOfTable(null, request).getGroupInfo());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void moveServers(Set<HostPort> servers, String targetGroup) throws IOException {
+    Set<HBaseProtos.HostPort> hostPorts = Sets.newHashSet();
+    for(HostPort el: servers) {
+      hostPorts.add(HBaseProtos.HostPort.newBuilder()
+        .setHostName(el.getHostname())
+        .setPort(el.getPort())
+        .build());
+    }
+    MasterProtos.MoveServersRequest request =
+        MasterProtos.MoveServersRequest.newBuilder()
+            .setTargetGroup(targetGroup)
+            .addAllServers(hostPorts).build();
+
+    try {
+      proxy.moveServers(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
+    MasterProtos.MoveTablesRequest.Builder builder =
+        MasterProtos.MoveTablesRequest.newBuilder()
+            .setTargetGroup(targetGroup);
+    for(TableName tableName: tables) {
+      builder.addTableName(ProtobufUtil.toProtoTableName(tableName));
+    }
+    try {
+      proxy.moveTables(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void addGroup(String groupName) throws IOException {
+    MasterProtos.AddGroupRequest request =
+        MasterProtos.AddGroupRequest.newBuilder()
+            .setGroupName(groupName).build();
+    try {
+      proxy.addGroup(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeGroup(String name) throws IOException {
+    MasterProtos.RemoveGroupRequest request =
+        MasterProtos.RemoveGroupRequest.newBuilder()
+            .setGroupName(name).build();
+    try {
+      proxy.removeGroup(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean balanceGroup(String name) throws IOException {
+    MasterProtos.BalanceGroupRequest request =
+        MasterProtos.BalanceGroupRequest.newBuilder()
+            .setGroupName(name).build();
+
+    try {
+      return proxy.balanceGroup(null, request).getBalanceRan();
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public List<GroupInfo> listGroups() throws IOException {
+    try {
+      List<RSGroupProtos.GroupInfo> resp =
+          proxy.listGroupInfos(null, MasterProtos.ListGroupInfosRequest.newBuilder().build())
+              .getGroupInfoList();
+      List<GroupInfo> result = new ArrayList<GroupInfo>(resp.size());
+      for(RSGroupProtos.GroupInfo entry: resp) {
+        result.add(ProtobufUtil.toGroupInfo(entry));
+      }
+      return result;
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public GroupInfo getGroupOfServer(HostPort hostPort) throws IOException {
+    MasterProtos.GetGroupInfoOfServerRequest request =
+        MasterProtos.GetGroupInfoOfServerRequest.newBuilder()
+            .setServer(HBaseProtos.HostPort.newBuilder()
+                .setHostName(hostPort.getHostname())
+                .setPort(hostPort.getPort())
+                .build())
+            .build();
+    try {
+      return ProtobufUtil.toGroupInfo(
+          proxy.getGroupInfoOfServer(null, request).getGroupInfo());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 5bded37..7b7245b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -38,6 +38,8 @@ import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -69,6 +71,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.LimitInputStream;
+import org.apache.hadoop.hbase.group.GroupInfo;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
@@ -120,6 +123,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescripto
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
@@ -3058,4 +3062,34 @@ public final class ProtobufUtil {
     builder.setSrcChecksum(VersionInfo.getSrcChecksum());
     return builder.build();
   }
+
+  public static GroupInfo toGroupInfo(RSGroupProtos.GroupInfo proto) {
+    GroupInfo groupInfo = new GroupInfo(proto.getName());
+    for(HBaseProtos.HostPort el: proto.getServersList()) {
+      groupInfo.addServer(new HostPort(el.getHostName(), el.getPort()));
+    }
+    for(HBaseProtos.TableName pTableName: proto.getTablesList()) {
+      groupInfo.addTable(ProtobufUtil.toTableName(pTableName));
+    }
+    return groupInfo;
+  }
+
+  public static RSGroupProtos.GroupInfo toProtoGroupInfo(GroupInfo pojo) {
+    List<HBaseProtos.TableName> tables =
+        new ArrayList<HBaseProtos.TableName>(pojo.getTables().size());
+    for(TableName arg: pojo.getTables()) {
+      tables.add(ProtobufUtil.toProtoTableName(arg));
+    }
+    List<HBaseProtos.HostPort> hostports =
+        new ArrayList<HBaseProtos.HostPort>(pojo.getServers().size());
+    for(HostPort el: pojo.getServers()) {
+      hostports.add(HBaseProtos.HostPort.newBuilder()
+          .setHostName(el.getHostname())
+          .setPort(el.getPort())
+          .build());
+    }
+    return RSGroupProtos.GroupInfo.newBuilder().setName(pojo.getName())
+        .addAllServers(hostports)
+        .addAllTables(tables).build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-common/src/main/java/org/apache/hadoop/hbase/HostPort.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HostPort.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HostPort.java
new file mode 100644
index 0000000..c047ee0
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HostPort.java
@@ -0,0 +1,76 @@
+/**
+ *
+ * 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;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Addressing;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HostPort implements Comparable<HostPort> {
+  private final String hostnameOnly;
+  private final int port;
+
+  public HostPort(final String hostname, final int port) {
+    this.hostnameOnly = hostname;
+    this.port = port;
+  }
+
+  public String getHostname() {
+    return hostnameOnly;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public static HostPort valueOf(final String hostport) {
+    String splits[] = hostport.split(":",2);
+    if(splits.length < 2)
+      throw new IllegalArgumentException("Server list contains not a valid <HOST>:<PORT> entry");
+    return new HostPort(splits[0], Integer.parseInt(splits[1]));
+  }
+
+  @Override
+  public String toString() {
+    return Addressing.createHostAndPortStr(this.hostnameOnly, this.port);
+  }
+
+  @Override
+  public int compareTo(HostPort other) {
+    int compare = this.getHostname().compareToIgnoreCase(other.getHostname());
+    if (compare != 0) return compare;
+    compare = this.getPort() - other.getPort();
+    return compare;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null) return false;
+    if (!(o instanceof HostPort)) return false;
+    return this.compareTo((HostPort)o) == 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
index 059dcb8..9a13f8e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -88,9 +88,8 @@ public class ServerName implements Comparable<ServerName>, Serializable {
   public static final String UNKNOWN_SERVERNAME = "#unknown#";
 
   private final String servername;
-  private final String hostnameOnly;
-  private final int port;
   private final long startcode;
+  private final HostPort hostPort;
 
   /**
    * Cached versioned bytes of this ServerName instance.
@@ -102,10 +101,9 @@ public class ServerName implements Comparable<ServerName>, Serializable {
   private ServerName(final String hostname, final int port, final long startcode) {
     // Drop the domain is there is one; no need of it in a local cluster.  With it, we get long
     // unwieldy names.
-    this.hostnameOnly = hostname;
-    this.port = port;
+    this.hostPort = new HostPort(hostname, port);
     this.startcode = startcode;
-    this.servername = getServerName(this.hostnameOnly, port, startcode);
+    this.servername = getServerName(hostname, port, startcode);
   }
 
   /**
@@ -189,7 +187,8 @@ public class ServerName implements Comparable<ServerName>, Serializable {
    * in compares, etc.
    */
   public String toShortString() {
-    return Addressing.createHostAndPortStr(getHostNameMinusDomain(this.hostnameOnly), this.port);
+    return Addressing.createHostAndPortStr(
+        getHostNameMinusDomain(hostPort.getHostname()), hostPort.getPort());
   }
 
   /**
@@ -208,11 +207,11 @@ public class ServerName implements Comparable<ServerName>, Serializable {
   }
 
   public String getHostname() {
-    return hostnameOnly;
+    return hostPort.getHostname();
   }
 
   public int getPort() {
-    return port;
+    return hostPort.getPort();
   }
 
   public long getStartcode() {
@@ -256,7 +255,11 @@ public class ServerName implements Comparable<ServerName>, Serializable {
    * {@link Addressing#createHostAndPortStr(String, int)}
    */
   public String getHostAndPort() {
-    return Addressing.createHostAndPortStr(this.hostnameOnly, this.port);
+    return Addressing.createHostAndPortStr(hostPort.getHostname(), hostPort.getPort());
+  }
+
+  public HostPort getHostPort() {
+    return hostPort;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupAdmin.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupAdmin.java
new file mode 100644
index 0000000..822c1ef
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupAdmin.java
@@ -0,0 +1,108 @@
+/**
+ * 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.group;
+
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Group user API interface used between client and server.
+ */
+@InterfaceAudience.Private
+public interface GroupAdmin extends Closeable {
+  /**
+   * Gets the group information.
+   *
+   * @param groupName the group name
+   * @return An instance of GroupInfo
+   */
+  GroupInfo getGroupInfo(String groupName) throws IOException;
+
+  /**
+   * Gets the group info of table.
+   *
+   * @param tableName the table name
+   * @return An instance of GroupInfo.
+   */
+  GroupInfo getGroupInfoOfTable(TableName tableName) throws IOException;
+
+  /**
+   * Move a set of serves to another group
+   *
+   *
+   * @param servers set of servers, must be in the form HOST:PORT
+   * @param targetGroup the target group
+   * @throws java.io.IOException Signals that an I/O exception has occurred.
+   */
+  void moveServers(Set<HostPort> servers, String targetGroup) throws IOException;
+
+  /**
+   * Move tables to a new group.
+   * This will unassign all of a table's region so it can be reassigned to the correct group.
+   * @param tables list of tables to move
+   * @param targetGroup target group
+   * @throws java.io.IOException
+   */
+  void moveTables(Set<TableName> tables, String targetGroup) throws IOException;
+
+  /**
+   * Add a new group
+   * @param name name of the group
+   * @throws java.io.IOException
+   */
+  void addGroup(String name) throws IOException;
+
+  /**
+   * Remove a group
+   * @param name name of the group
+   * @throws java.io.IOException
+   */
+  void removeGroup(String name) throws IOException;
+
+  /**
+   * Balance the regions in a group
+   *
+   * @param name the name of the gorup to balance
+   * @return
+   * @throws java.io.IOException
+   */
+  boolean balanceGroup(String name) throws IOException;
+
+  /**
+   * Lists the existing groups.
+   *
+   * @return Collection of GroupInfo.
+   */
+  List<GroupInfo> listGroups() throws IOException;
+
+  /**
+   * Retrieve the GroupInfo a server is affiliated to
+   * @param hostPort
+   * @return
+   * @throws java.io.IOException
+   */
+  GroupInfo getGroupOfServer(HostPort hostPort) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupInfo.java
new file mode 100644
index 0000000..41a6e2e
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupInfo.java
@@ -0,0 +1,182 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.HostPort;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.NavigableSet;
+
+/**
+ * Stores the group information of region server groups.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class GroupInfo implements Serializable {
+
+  public static final String DEFAULT_GROUP = "default";
+  public static final String NAMESPACEDESC_PROP_GROUP = "hbase.rsgroup.name";
+  public static final String TABLEDESC_PROP_GROUP = "hbase.rsgroup.name";
+  public static final String TRANSITION_GROUP_PREFIX = "_transition_";
+
+  private String name;
+  private NavigableSet<HostPort> servers;
+  private NavigableSet<TableName> tables;
+
+  public GroupInfo(String name) {
+    this(name, Sets.<HostPort>newTreeSet(), Sets.<TableName>newTreeSet());
+  }
+
+  //constructor for jackson
+  @JsonCreator
+  GroupInfo(@JsonProperty("name") String name,
+            @JsonProperty("servers") NavigableSet<HostPort> servers,
+            @JsonProperty("tables") NavigableSet<TableName> tables) {
+    this.name = name;
+    this.servers = servers;
+    this.tables = tables;
+  }
+
+  public GroupInfo(GroupInfo src) {
+    name = src.getName();
+    servers = Sets.newTreeSet(src.getServers());
+    tables = Sets.newTreeSet(src.getTables());
+  }
+
+  /**
+   * Get group name.
+   *
+   * @return
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Adds the server to the group.
+   *
+   * @param hostPort the server
+   */
+  public void addServer(HostPort hostPort){
+    servers.add(hostPort);
+  }
+
+  /**
+   * Adds a group of servers.
+   *
+   * @param hostPort the servers
+   */
+  public void addAllServers(Collection<HostPort> hostPort){
+    servers.addAll(hostPort);
+  }
+
+  /**
+   * @param hostPort
+   * @return true, if a server with hostPort is found
+   */
+  public boolean containsServer(HostPort hostPort) {
+    return servers.contains(hostPort);
+  }
+
+  /**
+   * Get list of servers.
+   *
+   * @return
+   */
+  public NavigableSet<HostPort> getServers() {
+    return servers;
+  }
+
+  /**
+   * Remove a server from this group.
+   *
+   * @param hostPort
+   */
+  public boolean removeServer(HostPort hostPort) {
+    return servers.remove(hostPort);
+  }
+
+  /**
+   * Set of tables that are members of this group
+   * @return
+   */
+  public NavigableSet<TableName> getTables() {
+    return tables;
+  }
+
+  public void addTable(TableName table) {
+    tables.add(table);
+  }
+
+  public void addAllTables(Collection<TableName> arg) {
+    tables.addAll(arg);
+  }
+
+  public boolean containsTable(TableName table) {
+    return tables.contains(table);
+  }
+
+  public boolean removeTable(TableName table) {
+    return tables.remove(table);
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("GroupName:");
+    sb.append(this.name);
+    sb.append(", ");
+    sb.append(" Servers:");
+    sb.append(this.servers);
+    return sb.toString();
+
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    GroupInfo groupInfo = (GroupInfo) o;
+
+    if (!name.equals(groupInfo.name)) return false;
+    if (!servers.equals(groupInfo.servers)) return false;
+    if (!tables.equals(groupInfo.tables)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = servers.hashCode();
+    result = 31 * result + tables.hashCode();
+    result = 31 * result + name.hashCode();
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-it/src/test/java/org/apache/hadoop/hbase/group/IntegrationTestGroup.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/group/IntegrationTestGroup.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/group/IntegrationTestGroup.java
new file mode 100644
index 0000000..22cddd7
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/group/IntegrationTestGroup.java
@@ -0,0 +1,94 @@
+/**
+ * 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.group;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Runs all of the units tests defined in TestGroupBase
+ * as an integration test.
+ * Requires TestGroupBase.NUM_SLAVE_BASE servers to run.
+ */
+@Category(IntegrationTests.class)
+public class IntegrationTestGroup extends TestGroupsBase {
+  //Integration specific
+  private final static Log LOG = LogFactory.getLog(IntegrationTestGroup.class);
+  private static boolean initialized = false;
+
+  @Before
+  public void beforeMethod() throws Exception {
+    if(!initialized) {
+      LOG.info("Setting up IntegrationTestGroup");
+      LOG.info("Initializing cluster with " + NUM_SLAVES_BASE + " servers");
+      TEST_UTIL = new IntegrationTestingUtility();
+      ((IntegrationTestingUtility)TEST_UTIL).initializeCluster(NUM_SLAVES_BASE);
+      //set shared configs
+      admin = TEST_UTIL.getHBaseAdmin();
+      cluster = TEST_UTIL.getHBaseClusterInterface();
+      groupAdmin = new VerifyingGroupAdminClient(admin.getConnection().getGroupAdmin(),
+          TEST_UTIL.getConfiguration());
+      LOG.info("Done initializing cluster");
+      initialized = true;
+      //cluster may not be clean
+      //cleanup when initializing
+      afterMethod();
+    }
+  }
+
+  @After
+  public void afterMethod() throws Exception {
+    LOG.info("Cleaning up previous test run");
+    //cleanup previous artifacts
+    deleteTableIfNecessary();
+    deleteNamespaceIfNecessary();
+    deleteGroups();
+    admin.setBalancerRunning(false,true);
+
+    LOG.info("Restoring the cluster");
+    ((IntegrationTestingUtility)TEST_UTIL).restoreCluster();
+    LOG.info("Done restoring the cluster");
+
+    groupAdmin.addGroup("master");
+    groupAdmin.moveServers(
+        Sets.newHashSet(cluster.getInitialClusterStatus().getMaster().getHostPort()),
+        "master");
+
+    TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        LOG.info("Waiting for cleanup to finish "+groupAdmin.listGroups());
+        //Might be greater since moving servers back to default
+        //is after starting a server
+        return groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP).getServers().size()
+            == NUM_SLAVES_BASE;
+      }
+    });
+    LOG.info("Done cleaning up previous test run");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index bde5403..2422427 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -17652,6 +17652,621 @@ public final class HBaseProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.RegionServerInfo)
   }
 
+  public interface HostPortOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string host_name = 1;
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    boolean hasHostName();
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    java.lang.String getHostName();
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getHostNameBytes();
+
+    // required uint32 port = 2;
+    /**
+     * <code>required uint32 port = 2;</code>
+     */
+    boolean hasPort();
+    /**
+     * <code>required uint32 port = 2;</code>
+     */
+    int getPort();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.HostPort}
+   */
+  public static final class HostPort extends
+      com.google.protobuf.GeneratedMessage
+      implements HostPortOrBuilder {
+    // Use HostPort.newBuilder() to construct.
+    private HostPort(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private HostPort(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final HostPort defaultInstance;
+    public static HostPort getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public HostPort getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private HostPort(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              hostName_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              port_ = input.readUInt32();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_HostPort_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_HostPort_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<HostPort> PARSER =
+        new com.google.protobuf.AbstractParser<HostPort>() {
+      public HostPort parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new HostPort(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<HostPort> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string host_name = 1;
+    public static final int HOST_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object hostName_;
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    public boolean hasHostName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    public java.lang.String getHostName() {
+      java.lang.Object ref = hostName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          hostName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string host_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getHostNameBytes() {
+      java.lang.Object ref = hostName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        hostName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required uint32 port = 2;
+    public static final int PORT_FIELD_NUMBER = 2;
+    private int port_;
+    /**
+     * <code>required uint32 port = 2;</code>
+     */
+    public boolean hasPort() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required uint32 port = 2;</code>
+     */
+    public int getPort() {
+      return port_;
+    }
+
+    private void initFields() {
+      hostName_ = "";
+      port_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasHostName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasPort()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getHostNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt32(2, port_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getHostNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(2, port_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort) obj;
+
+      boolean result = true;
+      result = result && (hasHostName() == other.hasHostName());
+      if (hasHostName()) {
+        result = result && getHostName()
+            .equals(other.getHostName());
+      }
+      result = result && (hasPort() == other.hasPort());
+      if (hasPort()) {
+        result = result && (getPort()
+            == other.getPort());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHostName()) {
+        hash = (37 * hash) + HOST_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getHostName().hashCode();
+      }
+      if (hasPort()) {
+        hash = (37 * hash) + PORT_FIELD_NUMBER;
+        hash = (53 * hash) + getPort();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.HostPort}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPortOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_HostPort_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_HostPort_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        hostName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        port_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_hbase_pb_HostPort_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort build() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.hostName_ = hostName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.port_ = port_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort.getDefaultInstance()) return this;
+        if (other.hasHostName()) {
+          bitField0_ |= 0x00000001;
+          hostName_ = other.hostName_;
+          onChanged();
+        }
+        if (other.hasPort()) {
+          setPort(other.getPort());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasHostName()) {
+          
+          return false;
+        }
+        if (!hasPort()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.HostPort) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string host_name = 1;
+      private java.lang.Object hostName_ = "";
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public boolean hasHostName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public java.lang.String getHostName() {
+        java.lang.Object ref = hostName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          hostName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getHostNameBytes() {
+        java.lang.Object ref = hostName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          hostName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public Builder setHostName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        hostName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public Builder clearHostName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        hostName_ = getDefaultInstance().getHostName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string host_name = 1;</code>
+       */
+      public Builder setHostNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        hostName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 port = 2;
+      private int port_ ;
+      /**
+       * <code>required uint32 port = 2;</code>
+       */
+      public boolean hasPort() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required uint32 port = 2;</code>
+       */
+      public int getPort() {
+        return port_;
+      }
+      /**
+       * <code>required uint32 port = 2;</code>
+       */
+      public Builder setPort(int value) {
+        bitField0_ |= 0x00000002;
+        port_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 port = 2;</code>
+       */
+      public Builder clearPort() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        port_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.HostPort)
+    }
+
+    static {
+      defaultInstance = new HostPort(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.HostPort)
+  }
+
   private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TableName_descriptor;
   private static
@@ -17772,6 +18387,11 @@ public final class HBaseProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_HostPort_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_HostPort_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -17830,14 +18450,15 @@ public final class HBaseProtos {
       "\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014\022/\n\rc" +
       "onfiguration\030\002 \003(\0132\030.hbase.pb.NameString" +
       "Pair\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001" +
-      "(\005*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_E",
-      "QUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREA" +
-      "TER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n" +
-      "\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECO" +
-      "NDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007" +
-      "MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.ap" +
-      "ache.hadoop.hbase.protobuf.generatedB\013HB" +
-      "aseProtosH\001\240\001\001"
+      "(\005\"+\n\010HostPort\022\021\n\thost_name\030\001 \002(\t\022\014\n\004por",
+      "t\030\002 \002(\r*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS" +
+      "_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n" +
+      "\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_O" +
+      "P\020\006*n\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICR" +
+      "OSECONDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020" +
+      "\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*o" +
+      "rg.apache.hadoop.hbase.protobuf.generate" +
+      "dB\013HBaseProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17988,6 +18609,12 @@ public final class HBaseProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionServerInfo_descriptor,
               new java.lang.String[] { "InfoPort", });
+          internal_static_hbase_pb_HostPort_descriptor =
+            getDescriptor().getMessageTypes().get(24);
+          internal_static_hbase_pb_HostPort_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_HostPort_descriptor,
+              new java.lang.String[] { "HostName", "Port", });
           return null;
         }
       };


[5/6] hbase git commit: HBASE-6721 RegionServer group based assignment (Francis Liu)

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/16f65bad/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 0f74b85..beee110 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -50378,6 +50378,11811 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:hbase.pb.MajorCompactionTimestampResponse)
   }
 
+  public interface ListTablesOfGroupRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string group_name = 1;
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    boolean hasGroupName();
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    java.lang.String getGroupName();
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getGroupNameBytes();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ListTablesOfGroupRequest}
+   */
+  public static final class ListTablesOfGroupRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements ListTablesOfGroupRequestOrBuilder {
+    // Use ListTablesOfGroupRequest.newBuilder() to construct.
+    private ListTablesOfGroupRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ListTablesOfGroupRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ListTablesOfGroupRequest defaultInstance;
+    public static ListTablesOfGroupRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ListTablesOfGroupRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ListTablesOfGroupRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              groupName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ListTablesOfGroupRequest> PARSER =
+        new com.google.protobuf.AbstractParser<ListTablesOfGroupRequest>() {
+      public ListTablesOfGroupRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ListTablesOfGroupRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ListTablesOfGroupRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string group_name = 1;
+    public static final int GROUP_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object groupName_;
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public boolean hasGroupName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public java.lang.String getGroupName() {
+      java.lang.Object ref = groupName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          groupName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getGroupNameBytes() {
+      java.lang.Object ref = groupName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        groupName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      groupName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasGroupName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getGroupNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getGroupNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest) obj;
+
+      boolean result = true;
+      result = result && (hasGroupName() == other.hasGroupName());
+      if (hasGroupName()) {
+        result = result && getGroupName()
+            .equals(other.getGroupName());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasGroupName()) {
+        hash = (37 * hash) + GROUP_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getGroupName().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ListTablesOfGroupRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        groupName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.groupName_ = groupName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest.getDefaultInstance()) return this;
+        if (other.hasGroupName()) {
+          bitField0_ |= 0x00000001;
+          groupName_ = other.groupName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasGroupName()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string group_name = 1;
+      private java.lang.Object groupName_ = "";
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public boolean hasGroupName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public java.lang.String getGroupName() {
+        java.lang.Object ref = groupName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          groupName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getGroupNameBytes() {
+        java.lang.Object ref = groupName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          groupName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder setGroupName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        groupName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder clearGroupName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        groupName_ = getDefaultInstance().getGroupName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder setGroupNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        groupName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ListTablesOfGroupRequest)
+    }
+
+    static {
+      defaultInstance = new ListTablesOfGroupRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ListTablesOfGroupRequest)
+  }
+
+  public interface ListTablesOfGroupResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .hbase.pb.TableName table_name = 1;
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
+        getTableNameList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName(int index);
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    int getTableNameCount();
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableNameOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.ListTablesOfGroupResponse}
+   */
+  public static final class ListTablesOfGroupResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements ListTablesOfGroupResponseOrBuilder {
+    // Use ListTablesOfGroupResponse.newBuilder() to construct.
+    private ListTablesOfGroupResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ListTablesOfGroupResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ListTablesOfGroupResponse defaultInstance;
+    public static ListTablesOfGroupResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ListTablesOfGroupResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ListTablesOfGroupResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                tableName_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              tableName_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          tableName_ = java.util.Collections.unmodifiableList(tableName_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ListTablesOfGroupResponse> PARSER =
+        new com.google.protobuf.AbstractParser<ListTablesOfGroupResponse>() {
+      public ListTablesOfGroupResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ListTablesOfGroupResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ListTablesOfGroupResponse> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated .hbase.pb.TableName table_name = 1;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableName_;
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableNameList() {
+      return tableName_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableNameOrBuilderList() {
+      return tableName_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    public int getTableNameCount() {
+      return tableName_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName(int index) {
+      return tableName_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(
+        int index) {
+      return tableName_.get(index);
+    }
+
+    private void initFields() {
+      tableName_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getTableNameCount(); i++) {
+        if (!getTableName(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < tableName_.size(); i++) {
+        output.writeMessage(1, tableName_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < tableName_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, tableName_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse) obj;
+
+      boolean result = true;
+      result = result && getTableNameList()
+          .equals(other.getTableNameList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getTableNameCount() > 0) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableNameList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.ListTablesOfGroupResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          tableNameBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_ListTablesOfGroupResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse(this);
+        int from_bitField0_ = bitField0_;
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            tableName_ = java.util.Collections.unmodifiableList(tableName_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse.getDefaultInstance()) return this;
+        if (tableNameBuilder_ == null) {
+          if (!other.tableName_.isEmpty()) {
+            if (tableName_.isEmpty()) {
+              tableName_ = other.tableName_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensureTableNameIsMutable();
+              tableName_.addAll(other.tableName_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableName_.isEmpty()) {
+            if (tableNameBuilder_.isEmpty()) {
+              tableNameBuilder_.dispose();
+              tableNameBuilder_ = null;
+              tableName_ = other.tableName_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              tableNameBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTableNameFieldBuilder() : null;
+            } else {
+              tableNameBuilder_.addAllMessages(other.tableName_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getTableNameCount(); i++) {
+          if (!getTableName(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTablesOfGroupResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .hbase.pb.TableName table_name = 1;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableName_ =
+        java.util.Collections.emptyList();
+      private void ensureTableNameIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          tableName_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>(tableName_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableNameList() {
+        if (tableNameBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(tableName_);
+        } else {
+          return tableNameBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public int getTableNameCount() {
+        if (tableNameBuilder_ == null) {
+          return tableName_.size();
+        } else {
+          return tableNameBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName(int index) {
+        if (tableNameBuilder_ == null) {
+          return tableName_.get(index);
+        } else {
+          return tableNameBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableNameIsMutable();
+          tableName_.set(index, value);
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          ensureTableNameIsMutable();
+          tableName_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder addTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableNameIsMutable();
+          tableName_.add(value);
+          onChanged();
+        } else {
+          tableNameBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder addTableName(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTableNameIsMutable();
+          tableName_.add(index, value);
+          onChanged();
+        } else {
+          tableNameBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder addTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          ensureTableNameIsMutable();
+          tableName_.add(builderForValue.build());
+          onChanged();
+        } else {
+          tableNameBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder addTableName(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          ensureTableNameIsMutable();
+          tableName_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          tableNameBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder addAllTableName(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> values) {
+        if (tableNameBuilder_ == null) {
+          ensureTableNameIsMutable();
+          super.addAll(values, tableName_);
+          onChanged();
+        } else {
+          tableNameBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder removeTableName(int index) {
+        if (tableNameBuilder_ == null) {
+          ensureTableNameIsMutable();
+          tableName_.remove(index);
+          onChanged();
+        } else {
+          tableNameBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder(
+          int index) {
+        return getTableNameFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(
+          int index) {
+        if (tableNameBuilder_ == null) {
+          return tableName_.get(index);  } else {
+          return tableNameBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+           getTableNameOrBuilderList() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(tableName_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableNameBuilder() {
+        return getTableNameFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableNameBuilder(
+          int index) {
+        return getTableNameFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.TableName table_name = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder> 
+           getTableNameBuilderList() {
+        return getTableNameFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ListTablesOfGroupResponse)
+    }
+
+    static {
+      defaultInstance = new ListTablesOfGroupResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ListTablesOfGroupResponse)
+  }
+
+  public interface GetGroupInfoRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string group_name = 1;
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    boolean hasGroupName();
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    java.lang.String getGroupName();
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getGroupNameBytes();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetGroupInfoRequest}
+   */
+  public static final class GetGroupInfoRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements GetGroupInfoRequestOrBuilder {
+    // Use GetGroupInfoRequest.newBuilder() to construct.
+    private GetGroupInfoRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetGroupInfoRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetGroupInfoRequest defaultInstance;
+    public static GetGroupInfoRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetGroupInfoRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetGroupInfoRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              groupName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetGroupInfoRequest> PARSER =
+        new com.google.protobuf.AbstractParser<GetGroupInfoRequest>() {
+      public GetGroupInfoRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetGroupInfoRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetGroupInfoRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string group_name = 1;
+    public static final int GROUP_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object groupName_;
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public boolean hasGroupName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public java.lang.String getGroupName() {
+      java.lang.Object ref = groupName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          groupName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string group_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getGroupNameBytes() {
+      java.lang.Object ref = groupName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        groupName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      groupName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasGroupName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getGroupNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getGroupNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest) obj;
+
+      boolean result = true;
+      result = result && (hasGroupName() == other.hasGroupName());
+      if (hasGroupName()) {
+        result = result && getGroupName()
+            .equals(other.getGroupName());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasGroupName()) {
+        hash = (37 * hash) + GROUP_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getGroupName().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetGroupInfoRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        groupName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.groupName_ = groupName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest.getDefaultInstance()) return this;
+        if (other.hasGroupName()) {
+          bitField0_ |= 0x00000001;
+          groupName_ = other.groupName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasGroupName()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string group_name = 1;
+      private java.lang.Object groupName_ = "";
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public boolean hasGroupName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public java.lang.String getGroupName() {
+        java.lang.Object ref = groupName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          groupName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getGroupNameBytes() {
+        java.lang.Object ref = groupName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          groupName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder setGroupName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        groupName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder clearGroupName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        groupName_ = getDefaultInstance().getGroupName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string group_name = 1;</code>
+       */
+      public Builder setGroupNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        groupName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GetGroupInfoRequest)
+    }
+
+    static {
+      defaultInstance = new GetGroupInfoRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GetGroupInfoRequest)
+  }
+
+  public interface GetGroupInfoResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .hbase.pb.GroupInfo group_info = 1;
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    boolean hasGroupInfo();
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo getGroupInfo();
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfoOrBuilder getGroupInfoOrBuilder();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GetGroupInfoResponse}
+   */
+  public static final class GetGroupInfoResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements GetGroupInfoResponseOrBuilder {
+    // Use GetGroupInfoResponse.newBuilder() to construct.
+    private GetGroupInfoResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetGroupInfoResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetGroupInfoResponse defaultInstance;
+    public static GetGroupInfoResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetGroupInfoResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetGroupInfoResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = groupInfo_.toBuilder();
+              }
+              groupInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(groupInfo_);
+                groupInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetGroupInfoResponse> PARSER =
+        new com.google.protobuf.AbstractParser<GetGroupInfoResponse>() {
+      public GetGroupInfoResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetGroupInfoResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetGroupInfoResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .hbase.pb.GroupInfo group_info = 1;
+    public static final int GROUP_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo groupInfo_;
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    public boolean hasGroupInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo getGroupInfo() {
+      return groupInfo_;
+    }
+    /**
+     * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfoOrBuilder getGroupInfoOrBuilder() {
+      return groupInfo_;
+    }
+
+    private void initFields() {
+      groupInfo_ = org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (hasGroupInfo()) {
+        if (!getGroupInfo().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, groupInfo_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, groupInfo_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse) obj;
+
+      boolean result = true;
+      result = result && (hasGroupInfo() == other.hasGroupInfo());
+      if (hasGroupInfo()) {
+        result = result && getGroupInfo()
+            .equals(other.getGroupInfo());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasGroupInfo()) {
+        hash = (37 * hash) + GROUP_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getGroupInfo().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GetGroupInfoResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getGroupInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (groupInfoBuilder_ == null) {
+          groupInfo_ = org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.getDefaultInstance();
+        } else {
+          groupInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_GetGroupInfoResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (groupInfoBuilder_ == null) {
+          result.groupInfo_ = groupInfo_;
+        } else {
+          result.groupInfo_ = groupInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse.getDefaultInstance()) return this;
+        if (other.hasGroupInfo()) {
+          mergeGroupInfo(other.getGroupInfo());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasGroupInfo()) {
+          if (!getGroupInfo().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetGroupInfoResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .hbase.pb.GroupInfo group_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo groupInfo_ = org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo, org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfoOrBuilder> groupInfoBuilder_;
+      /**
+       * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+       */
+      public boolean hasGroupInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo getGroupInfo() {
+        if (groupInfoBuilder_ == null) {
+          return groupInfo_;
+        } else {
+          return groupInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.GroupInfo group_info = 1;</code>
+       */
+      public Builder setGroupInfo(org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos.GroupInfo value) {
+        if (groupInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          grou

<TRUNCATED>