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/28 05:58:50 UTC

[2/2] hbase git commit: HBASE-14226 Merge group admin APIs into Admin (Francis Liu)

HBASE-14226 Merge group admin APIs into Admin (Francis Liu)

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroups.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/group/VerifyingGroupAdminClient.java
	hbase-shell/src/main/ruby/hbase.rb
	hbase-shell/src/main/ruby/hbase/admin.rb


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

Branch: refs/heads/hbase-6721-0.98
Commit: b5c9a602fcfac739cf2af244e8ae11b9df01ddc5
Parents: 8726a60
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Aug 27 14:36:20 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Aug 27 20:56:08 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 186 ++++++++++++++++++
 .../apache/hadoop/hbase/client/HConnection.java |   7 -
 .../hadoop/hbase/client/HConnectionManager.java |   7 -
 .../hadoop/hbase/group/GroupAdminClient.java    | 192 -------------------
 .../apache/hadoop/hbase/group/GroupAdmin.java   | 108 -----------
 .../hbase/group/IntegrationTestGroup.java       |   3 +-
 .../hadoop/hbase/group/GroupAdminServer.java    |  16 +-
 .../apache/hadoop/hbase/group/MXBeanImpl.java   |  12 +-
 .../apache/hadoop/hbase/group/TestGroups.java   |  10 +-
 .../hadoop/hbase/group/TestGroupsBase.java      |   8 +-
 .../hbase/group/TestGroupsOfflineMode.java      |   2 +-
 .../hbase/group/VerifyingGroupAdminClient.java  |  22 +--
 .../hadoop/hbase/master/TestCatalogJanitor.java |   1 -
 hbase-shell/src/main/ruby/hbase.rb              |   1 -
 hbase-shell/src/main/ruby/hbase/admin.rb        |  93 +++++++++
 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              |   4 -
 hbase-shell/src/main/ruby/shell/commands.rb     |   4 -
 .../src/main/ruby/shell/commands/add_group.rb   |   2 +-
 .../main/ruby/shell/commands/balance_group.rb   |   2 +-
 .../src/main/ruby/shell/commands/get_group.rb   |   2 +-
 .../ruby/shell/commands/get_server_group.rb     |   2 +-
 .../main/ruby/shell/commands/get_table_group.rb |   2 +-
 .../src/main/ruby/shell/commands/list_groups.rb |   2 +-
 .../ruby/shell/commands/move_group_servers.rb   |   2 +-
 .../ruby/shell/commands/move_group_tables.rb    |   2 +-
 .../main/ruby/shell/commands/remove_group.rb    |   2 +-
 28 files changed, 312 insertions(+), 507 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index f86dd84..d2adb0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -28,10 +28,12 @@ import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hbase.HConstants;
 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.MasterNotRunningException;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -65,6 +68,7 @@ import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.group.GroupInfo;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -91,6 +95,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
@@ -134,6 +139,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
@@ -3583,4 +3589,184 @@ public class HBaseAdmin implements Abortable, Closeable {
       throw e;
     }
   }
+
+  /**
+   * Gets the group information.
+   *
+   * @param groupName the group name
+   * @return An instance of GroupInfo
+   */
+  public GroupInfo getGroupInfo(String groupName) throws IOException {
+    try {
+
+      MasterProtos.GetGroupInfoResponse resp =
+          connection.getMaster().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);
+    }
+  }
+
+  /**
+   * Gets the group info of table.
+   *
+   * @param tableName the table name
+   * @return An instance of GroupInfo.
+   */
+  public GroupInfo getGroupInfoOfTable(TableName tableName) throws IOException {
+    MasterProtos.GetGroupInfoOfTableRequest request =
+        MasterProtos.GetGroupInfoOfTableRequest.newBuilder()
+            .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
+
+    try {
+      return ProtobufUtil.toGroupInfo(
+          connection.getMaster().getGroupInfoOfTable(null, request).getGroupInfo());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * Move a set of servers to another group
+   *
+   * @param servers set of servers, must be in the form HOST:PORT
+   * @param targetGroup the target group
+   * @throws IOException
+   */
+  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 {
+      connection.getMaster().moveServers(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * 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 IOException
+   */
+  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 {
+      connection.getMaster().moveTables(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * Add a new group
+   * @param name name of the group
+   * @throws IOException
+   */
+  public void addGroup(String groupName) throws IOException {
+    MasterProtos.AddGroupRequest request =
+        MasterProtos.AddGroupRequest.newBuilder()
+            .setGroupName(groupName).build();
+    try {
+      connection.getMaster().addGroup(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * Remove a group
+   * @param name name of the group
+   * @throws IOException
+   */
+  public void removeGroup(String name) throws IOException {
+    MasterProtos.RemoveGroupRequest request =
+        MasterProtos.RemoveGroupRequest.newBuilder()
+            .setGroupName(name).build();
+    try {
+      connection.getMaster().removeGroup(null, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * Balance the regions in a group
+   *
+   * @param name the name of the gorup to balance
+   * @throws IOException
+   */
+  public boolean balanceGroup(String name) throws IOException {
+    MasterProtos.BalanceGroupRequest request =
+        MasterProtos.BalanceGroupRequest.newBuilder()
+            .setGroupName(name).build();
+
+    try {
+      return connection.getMaster().balanceGroup(null, request).getBalanceRan();
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
+
+  /**
+   * Lists the existing groups.
+   *
+   * @return Collection of GroupInfo.
+   */
+  public List<GroupInfo> listGroups() throws IOException {
+    try {
+      List<RSGroupProtos.GroupInfo> resp =
+          connection.getMaster().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);
+    }
+  }
+
+  /**
+   * Retrieve the GroupInfo a server is affiliated to
+   * @param hostPort
+   * @throws IOException
+   */
+  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(
+          connection.getMaster().getGroupInfoOfServer(null, request).getGroupInfo());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
index 9087198..e267c50 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.group.GroupAdmin;
-import org.apache.hadoop.hbase.group.GroupAdminClient;
 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;
@@ -557,9 +555,4 @@ public interface HConnection extends Abortable, Closeable {
    * @return the configured client backoff policy
    */
   ClientBackoffPolicy getBackoffPolicy();
-
-  /**
-   * @return client for region server group apis
-   */
-  GroupAdmin getGroupAdmin() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
index 6fc2e4a..8a7d328 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
@@ -71,8 +71,6 @@ 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.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -2562,11 +2560,6 @@ public class HConnectionManager {
       return this.backoffPolicy;
     }
 
-    @Override
-    public GroupAdmin getGroupAdmin() throws IOException {
-      return new GroupAdminClient(conf);
-    }
-
     /*
      * Return the number of cached region for a table. It will only be called
      * from a unit test.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
deleted file mode 100644
index 691e9dc..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/group/GroupAdminClient.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.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/b5c9a602/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
deleted file mode 100644
index 822c1ef..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/group/GroupAdmin.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Copyright The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.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/b5c9a602/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
index 62f4f8a..8ea9732 100644
--- 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
@@ -51,8 +51,7 @@ public class IntegrationTestGroup extends TestGroupsBase {
       //set shared configs
       admin = TEST_UTIL.getHBaseAdmin();
       cluster = TEST_UTIL.getHBaseClusterInterface();
-      groupAdmin = new VerifyingGroupAdminClient(admin.getConnection().getGroupAdmin(),
-          TEST_UTIL.getConfiguration());
+      groupAdmin = new VerifyingGroupAdminClient(TEST_UTIL.getConfiguration());
       LOG.info("Done initializing cluster");
       initialized = true;
       //cluster may not be clean

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 551aa6a..a745c5e 100644
--- 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
@@ -61,7 +61,7 @@ import java.util.concurrent.ConcurrentMap;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class GroupAdminServer implements GroupAdmin {
+public class GroupAdminServer {
   private static final Log LOG = LogFactory.getLog(GroupAdminServer.class);
 
     private MasterServices master;
@@ -77,13 +77,11 @@ public class GroupAdminServer implements GroupAdmin {
     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) {
@@ -95,7 +93,6 @@ public class GroupAdminServer implements GroupAdmin {
     return getGroupInfoManager().getGroup(groupName);
   }
 
-  @Override
   public void moveServers(Set<HostPort> servers, String targetGroupName)
       throws IOException {
     if (servers == null) {
@@ -232,7 +229,6 @@ public class GroupAdminServer implements GroupAdmin {
     }
   }
 
-  @Override
   public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
     if (tables == null) {
       throw new ConstraintException(
@@ -277,7 +273,6 @@ public class GroupAdminServer implements GroupAdmin {
     }
   }
 
-  @Override
   public void addGroup(String name) throws IOException {
     if (master.getCoprocessorHost() != null) {
       master.getCoprocessorHost().preAddGroup(name);
@@ -288,7 +283,6 @@ public class GroupAdminServer implements GroupAdmin {
     }
   }
 
-  @Override
   public void removeGroup(String name) throws IOException {
     GroupInfoManager manager = getGroupInfoManager();
     synchronized (manager) {
@@ -320,7 +314,6 @@ public class GroupAdminServer implements GroupAdmin {
     }
   }
 
-  @Override
   public boolean balanceGroup(String groupName) throws IOException {
     ServerManager serverManager = master.getServerManager();
     AssignmentManager assignmentManager = master.getAssignmentManager();
@@ -376,17 +369,14 @@ public class GroupAdminServer implements GroupAdmin {
     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;
   }
@@ -486,8 +476,4 @@ public class GroupAdminServer implements GroupAdmin {
       LOG.debug("Failed to perform group information cleanup for table: " + tableName, ex);
     }
   }
-
-  @Override
-  public void close() throws IOException {
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 5836d2d..907a4f7 100644
--- 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
@@ -40,11 +40,11 @@ public class MXBeanImpl implements MXBean {
 
   private static MXBeanImpl instance = null;
 
-  private GroupAdmin groupAdmin;
+  private GroupAdminServer groupAdminServer;
   private MasterServices master;
 
   public synchronized static MXBeanImpl init(
-      final GroupAdmin groupAdmin,
+      final GroupAdminServer groupAdmin,
       MasterServices master) {
     if (instance == null) {
       instance = new MXBeanImpl(groupAdmin, master);
@@ -52,9 +52,9 @@ public class MXBeanImpl implements MXBean {
     return instance;
   }
 
-  protected MXBeanImpl(final GroupAdmin groupAdmin,
+  protected MXBeanImpl(final GroupAdminServer groupAdmin,
       MasterServices master) {
-    this.groupAdmin = groupAdmin;
+    this.groupAdminServer = groupAdmin;
     this.master = master;
   }
 
@@ -63,7 +63,7 @@ public class MXBeanImpl implements MXBean {
     Map<String, List<HostPort>> data = new HashMap<String, List<HostPort>>();
     for (final ServerName entry :
       master.getServerManager().getOnlineServersList()) {
-      GroupInfo groupInfo = groupAdmin.getGroupOfServer(
+      GroupInfo groupInfo = groupAdminServer.getGroupOfServer(
           new HostPort(entry.getHostname(), entry.getPort()));
       if(!data.containsKey(groupInfo.getName())) {
         data.put(groupInfo.getName(), new LinkedList<HostPort>());
@@ -80,7 +80,7 @@ public class MXBeanImpl implements MXBean {
       onlineServers.add(new HostPort(entry.getHostname(), entry.getPort()));
     }
     List list = Lists.newArrayList();
-    for (GroupInfo group: groupAdmin.listGroups()) {
+    for (GroupInfo group: groupAdminServer.listGroups()) {
       List<HostPort> deadServers = Lists.newArrayList();
       for (HostPort server: group.getServers()) {
         if (!onlineServers.contains(server)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 50a303f..afbc8b8 100644
--- 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
@@ -103,9 +103,8 @@ public class TestGroups extends TestGroupsBase {
             ((GroupBasedLoadBalancer) master.getLoadBalancer()).isOnline();
       }
     });
-    admin.setBalancerRunning(false, true);
-    groupAdmin = new VerifyingGroupAdminClient(admin.getConnection().getGroupAdmin(),
-        TEST_UTIL.getConfiguration());
+    admin.setBalancerRunning(false,true);
+    groupAdmin = new VerifyingGroupAdminClient(TEST_UTIL.getConfiguration());
   }
 
   @AfterClass
@@ -167,7 +166,7 @@ public class TestGroups extends TestGroupsBase {
         GroupInfo groupInfo = groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP);
         List<ServerName> finalList = Lists.newArrayList();
         HostPort lastServer = groupInfo.getServers().last();
-        for (ServerName server: master.getServerManager().getOnlineServersList()) {
+        for (ServerName server : master.getServerManager().getOnlineServersList()) {
           if (!server.getHostPort().equals(lastServer)) {
             finalList.add(server);
           }
@@ -176,7 +175,8 @@ public class TestGroups extends TestGroupsBase {
         return finalList;
       }
     });
-    MXBean info = new MXBeanImpl(groupAdmin, mockMaster);
+
+    MXBean info = new MXBeanImpl(master.getGroupAdminServer(), mockMaster);
 
 
     GroupInfo defaultGroup = groupAdmin.getGroupInfo(GroupInfo.DEFAULT_GROUP);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 6db4578..aaac714 100644
--- 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
@@ -69,14 +69,14 @@ public abstract class TestGroupsBase {
   protected static HBaseTestingUtility TEST_UTIL;
   protected static HBaseAdmin admin;
   protected static HBaseCluster cluster;
-  protected static GroupAdmin groupAdmin;
+  protected static VerifyingGroupAdminClient 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,
+  protected GroupInfo addGroup(VerifyingGroupAdminClient gAdmin, String groupName,
                                int serverCount) throws IOException, InterruptedException {
     GroupInfo defaultInfo = gAdmin
         .getGroupInfo(GroupInfo.DEFAULT_GROUP);
@@ -97,7 +97,7 @@ public abstract class TestGroupsBase {
     return result;
   }
 
-  static void removeGroup(GroupAdminClient groupAdmin, String groupName) throws IOException {
+  static void removeGroup(VerifyingGroupAdminClient groupAdmin, String groupName) throws IOException {
     GroupInfo groupInfo = groupAdmin.getGroupInfo(groupName);
     groupAdmin.moveTables(groupInfo.getTables(), GroupInfo.DEFAULT_GROUP);
     groupAdmin.moveServers(groupInfo.getServers(), GroupInfo.DEFAULT_GROUP);
@@ -119,7 +119,7 @@ public abstract class TestGroupsBase {
   }
 
   protected void deleteGroups() throws IOException {
-    GroupAdminClient groupAdmin = new GroupAdminClient(TEST_UTIL.getConfiguration());
+    VerifyingGroupAdminClient groupAdmin = new VerifyingGroupAdminClient(TEST_UTIL.getConfiguration());
     for(GroupInfo group: groupAdmin.listGroups()) {
       if(!group.getName().equals(GroupInfo.DEFAULT_GROUP)) {
         groupAdmin.moveTables(group.getTables(), GroupInfo.DEFAULT_GROUP);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsOfflineMode.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsOfflineMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsOfflineMode.java
index d5da85d..0586c8a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsOfflineMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/group/TestGroupsOfflineMode.java
@@ -93,7 +93,7 @@ public class TestGroupsOfflineMode {
     final TableName failoverTable = TableName.valueOf("testOffline");
     TEST_UTIL.createTable(failoverTable, Bytes.toBytes("f"));
 
-    GroupAdminClient groupAdmin = new GroupAdminClient(TEST_UTIL.getConfiguration());
+    HBaseAdmin groupAdmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 
     final HRegionServer killRS = ((MiniHBaseCluster)cluster).getRegionServer(0);
     final HRegionServer groupRS = ((MiniHBaseCluster)cluster).getRegionServer(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 2b0c2df..712d130 100644
--- 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
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -43,65 +44,54 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Set;
 
-public class VerifyingGroupAdminClient implements GroupAdmin {
+public class VerifyingGroupAdminClient {
   private HTableInterface table;
   private ZooKeeperWatcher zkw;
-  private GroupSerDe serDe;
-  private GroupAdmin wrapped;
+  private HBaseAdmin wrapped;
 
-  public VerifyingGroupAdminClient(GroupAdmin groupAdmin, Configuration conf)
+  public VerifyingGroupAdminClient(Configuration conf)
       throws IOException {
-    wrapped = groupAdmin;
+    wrapped = new HBaseAdmin(conf);
     table = HConnectionManager.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);
   }
@@ -149,7 +139,7 @@ public class VerifyingGroupAdminClient implements GroupAdmin {
     }
   }
 
-  @Override
   public void close() throws IOException {
+    wrapped.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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 b829819..acc84ab 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
@@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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 b56b93f..70ab8fe 100644
--- a/hbase-shell/src/main/ruby/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase.rb
@@ -89,4 +89,3 @@ require 'hbase/table'
 require 'hbase/replication_admin'
 require 'hbase/security'
 require 'hbase/visibility_labels'
-require 'hbase/group_admin'

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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 2ae31c3..3d1a5ea 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -978,5 +978,98 @@ module Hbase
     def get_security_capabilities
       @admin.getSecurityCapabilities
     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.toString)
+        else
+          res += v.toString
+        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/b5c9a602/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
deleted file mode 100644
index bb4cefe..0000000
--- a/hbase-shell/src/main/ruby/hbase/group_admin.rb
+++ /dev/null
@@ -1,121 +0,0 @@
-#
-# Copyright The Apache Software Foundation
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-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.toString)
-        else
-          res += v.toString
-        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/b5c9a602/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 458176f..e75535e 100644
--- a/hbase-shell/src/main/ruby/hbase/hbase.rb
+++ b/hbase-shell/src/main/ruby/hbase/hbase.rb
@@ -44,10 +44,6 @@ module Hbase
       ::Hbase::Admin.new(configuration, formatter)
     end
 
-    def group_admin(formatter)
-      ::Hbase::GroupAdmin.new(configuration, formatter)
-    end
-
     # Create new one each time
     def table(table, shell)
       ::Hbase::Table.new(configuration, table, shell)

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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 c4500af..55e8b45 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -87,10 +87,6 @@ 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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 41538b9..1b079fb 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -54,10 +54,6 @@ 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/b5c9a602/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
index 7f91ee5..4f814e4 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_group.rb
@@ -32,7 +32,7 @@ EOF
       end
 
       def command(group_name)
-        group_admin.addGroup(group_name)
+        admin.addGroup(group_name)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 4c59f63..df6555f 100644
--- a/hbase-shell/src/main/ruby/shell/commands/balance_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/balance_group.rb
@@ -30,7 +30,7 @@ EOF
       end
 
       def command(group_name)
-        group_admin.balanceGroup(group_name)
+        admin.balanceGroup(group_name)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 5ed8226..7f501b9 100644
--- a/hbase-shell/src/main/ruby/shell/commands/get_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/get_group.rb
@@ -34,7 +34,7 @@ EOF
       def command(group_name)
         now = Time.now
         formatter.header([ "GROUP INFORMATION" ])
-        group_admin.getGroup(group_name) do |s|
+        admin.getGroup(group_name) do |s|
           formatter.row([ s ])
         end
         formatter.footer(now)

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index c78d4d2..0de6fc4 100644
--- a/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/get_server_group.rb
@@ -31,7 +31,7 @@ EOF
 
       def command(server)
         now = Time.now
-        groupName = group_admin.getGroupOfServer(server).getName
+        groupName = admin.getGroupOfServer(server).getName
         formatter.row([ groupName ])
         formatter.footer(now,1)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index dd8766d..eb8d121 100644
--- a/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/get_table_group.rb
@@ -32,7 +32,7 @@ EOF
       def command(table)
         now = Time.now
         groupName =
-            group_admin.getGroupOfTable(table).getName
+            admin.getGroupOfTable(table).getName
         formatter.row([ groupName ])
         formatter.footer(now,1)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 2e7dd08..2d028e8 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_groups.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_groups.rb
@@ -38,7 +38,7 @@ EOF
         formatter.header([ "GROUPS" ])
 
         regex = /#{regex}/ unless regex.is_a?(Regexp)
-        list = group_admin.listGroups.grep(regex)
+        list = admin.listGroups.grep(regex)
         list.each do |group|
           formatter.row([ group ])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 5e5c850..3e185f4 100644
--- a/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/move_group_servers.rb
@@ -30,7 +30,7 @@ EOF
       end
 
       def command(dest, *servers)
-        group_admin.moveServers(dest, *servers)
+        admin.moveServers(dest, *servers)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index f495f2c..48eee71 100644
--- a/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/move_group_tables.rb
@@ -30,7 +30,7 @@ EOF
       end
 
       def command(dest, *servers)
-        group_admin.moveTables(dest, *servers)
+        admin.moveTables(dest, *servers)
       end
     end
   end

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5c9a602/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
index 66863a4..15a988f 100644
--- a/hbase-shell/src/main/ruby/shell/commands/remove_group.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/remove_group.rb
@@ -30,7 +30,7 @@ EOF
       end
 
       def command(group_name)
-        group_admin.removeGroup(group_name)
+        admin.removeGroup(group_name)
       end
     end
   end