You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/05/31 07:02:05 UTC

[07/36] hbase git commit: HBASE-20653 Add missing observer hooks for region server group to MasterObserver

HBASE-20653 Add missing observer hooks for region server group to MasterObserver

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-19064
Commit: 40a73a5ca73c9e9e2ff9be1bf823056b108686af
Parents: c19fbf2
Author: Nihal Jain <ni...@gmail.com>
Authored: Wed May 30 23:37:48 2018 +0530
Committer: tedyu <yu...@gmail.com>
Committed: Wed May 30 21:29:07 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 24 ++++++
 .../hadoop/hbase/rsgroup/TestRSGroups.java      | 86 ++++++++++++++++++++
 .../hbase/rsgroup/TestRSGroupsWithACL.java      | 50 ++++++------
 .../hbase/coprocessor/MasterObserver.java       | 62 ++++++++++++++
 .../hbase/master/MasterCoprocessorHost.java     | 72 ++++++++++++++++
 5 files changed, 267 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index fa7537a..2efc3a4 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -161,11 +161,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, group="
               + groupName);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfo(groupName);
+        }
         checkPermission("getRSGroupInfo");
         RSGroupInfo rsGroupInfo = groupAdminServer.getRSGroupInfo(groupName);
         if (rsGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(rsGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfo(groupName);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -180,11 +186,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, table="
           + tableName);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfoOfTable(tableName);
+        }
         checkPermission("getRSGroupInfoOfTable");
         RSGroupInfo RSGroupInfo = groupAdminServer.getRSGroupInfoOfTable(tableName);
         if (RSGroupInfo != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfoOfTable(tableName);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -312,10 +324,16 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preListRSGroups();
+        }
         checkPermission("listRSGroup");
         for (RSGroupInfo RSGroupInfo : groupAdminServer.listRSGroups()) {
           builder.addRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postListRSGroups();
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -331,11 +349,17 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       LOG.info(master.getClientIdAuditPrefix() + " initiates rsgroup info retrieval, server="
           + hp);
       try {
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().preGetRSGroupInfoOfServer(hp);
+        }
         checkPermission("getRSGroupInfoOfServer");
         RSGroupInfo info = groupAdminServer.getRSGroupOfServer(hp);
         if (info != null) {
           builder.setRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(info));
         }
+        if (master.getMasterCoprocessorHost() != null) {
+          master.getMasterCoprocessorHost().postGetRSGroupInfoOfServer(hp);
+        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 3e74f81..c2fc0f1 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -292,6 +292,14 @@ public class TestRSGroups extends TestRSGroupsBase {
     boolean postRemoveServersCalled = false;
     boolean preMoveServersAndTables = false;
     boolean postMoveServersAndTables = false;
+    boolean preGetRSGroupInfoCalled = false;
+    boolean postGetRSGroupInfoCalled = false;
+    boolean preGetRSGroupInfoOfTableCalled = false;
+    boolean postGetRSGroupInfoOfTableCalled = false;
+    boolean preListRSGroupsCalled = false;
+    boolean postListRSGroupsCalled = false;
+    boolean preGetRSGroupInfoOfServerCalled = false;
+    boolean postGetRSGroupInfoOfServerCalled = false;
 
     @Override
     public Optional<MasterObserver> getMasterObserver() {
@@ -370,7 +378,85 @@ public class TestRSGroups extends TestRSGroupsBase {
         String groupName, boolean balancerRan) throws IOException {
       postBalanceRSGroupCalled = true;
     }
+
+    @Override
+    public void preGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final String groupName) throws IOException {
+      preGetRSGroupInfoCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final String groupName) throws IOException {
+      postGetRSGroupInfoCalled = true;
+    }
+
+    @Override
+    public void preGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final TableName tableName) throws IOException {
+      preGetRSGroupInfoOfTableCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final TableName tableName) throws IOException {
+      postGetRSGroupInfoOfTableCalled = true;
+    }
+
+    @Override
+    public void preListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      preListRSGroupsCalled = true;
+    }
+
+    @Override
+    public void postListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      postListRSGroupsCalled = true;
+    }
+
+    @Override
+    public void preGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final Address server) throws IOException {
+      preGetRSGroupInfoOfServerCalled = true;
+    }
+
+    @Override
+    public void postGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final Address server) throws IOException {
+      postGetRSGroupInfoOfServerCalled = true;
+    }
+  }
+
+  @Test
+  public void testGetRSGroupInfoCPHookCalled() throws Exception {
+    rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
+    assertTrue(observer.preGetRSGroupInfoCalled);
+    assertTrue(observer.postGetRSGroupInfoCalled);
+  }
+
+  @Test
+  public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception {
+    rsGroupAdmin.getRSGroupInfoOfTable(TableName.META_TABLE_NAME);
+    assertTrue(observer.preGetRSGroupInfoOfTableCalled);
+    assertTrue(observer.postGetRSGroupInfoOfTableCalled);
+  }
+
+  @Test
+  public void testListRSGroupsCPHookCalled() throws Exception {
+    rsGroupAdmin.listRSGroups();
+    assertTrue(observer.preListRSGroupsCalled);
+    assertTrue(observer.postListRSGroupsCalled);
   }
+
+  @Test
+  public void testGetRSGroupInfoOfServerCPHookCalled() throws Exception {
+    ServerName masterServerName = ((MiniHBaseCluster) cluster).getMaster().getServerName();
+    rsGroupAdmin.getRSGroupOfServer(masterServerName.getAddress());
+    assertTrue(observer.preGetRSGroupInfoOfServerCalled);
+    assertTrue(observer.postGetRSGroupInfoOfServerCalled);
+  }
+
   @Test
   public void testMoveServersAndTables() throws Exception {
     super.testMoveServersAndTables();

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
index afdff71..a63626d 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
@@ -223,9 +223,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -235,9 +233,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -247,9 +243,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -259,9 +253,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -271,9 +263,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -283,9 +273,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -295,9 +283,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -307,9 +293,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -319,9 +303,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
-    verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
-        USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
+    validateAdminPermissions(action);
   }
 
   @Test
@@ -331,6 +313,20 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
       return null;
     };
 
+    validateAdminPermissions(action);
+  }
+
+  @Test
+  public void testRemoveServers() throws Exception {
+    AccessTestAction action = () -> {
+      rsGroupAdminEndpoint.checkPermission("removeServers");
+      return null;
+    };
+
+    validateAdminPermissions(action);
+  }
+
+  private void validateAdminPermissions(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
         USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/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 f60a04d..3175af3 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
@@ -1213,6 +1213,68 @@ public interface MasterObserver {
       Set<Address> servers) throws IOException {}
 
   /**
+   * Called before getting region server group info of the passed groupName.
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName name of the group to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final String groupName) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed groupName.
+   * @param ctx the environment to interact with the framework and master
+   * @param groupName name of the group to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final String groupName) throws IOException {}
+
+  /**
+   * Called before getting region server group info of the passed tableName.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName name of the table to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final TableName tableName) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed tableName.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName name of the table to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final TableName tableName) throws IOException {}
+
+  /**
+   * Called before listing region server group information.
+   * @param ctx the environment to interact with the framework and master
+   */
+  default void preListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * Called after listing region server group information.
+   * @param ctx the environment to interact with the framework and master
+   */
+  default void postListRSGroups(final ObserverContext<MasterCoprocessorEnvironment> ctx)
+      throws IOException {}
+
+  /**
+   * Called before getting region server group info of the passed server.
+   * @param ctx the environment to interact with the framework and master
+   * @param server server to get RSGroupInfo for
+   */
+  default void preGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final Address server) throws IOException {}
+
+  /**
+   * Called after getting region server group info of the passed server.
+   * @param ctx the environment to interact with the framework and master
+   * @param server server to get RSGroupInfo for
+   */
+  default void postGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      final Address server) throws IOException {}
+
+  /**
    * Called before add a replication peer
    * @param ctx the environment to interact with the framework and master
    * @param peerId a short name that identifies the peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/40a73a5c/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 072ae8a..e563cd4 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
@@ -1404,6 +1404,78 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preGetRSGroupInfo(final String groupName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfo(this, groupName);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfo(final String groupName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfo(this, groupName);
+      }
+    });
+  }
+
+  public void preGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfoOfTable(this, tableName);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfoOfTable(final TableName tableName) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfoOfTable(this, tableName);
+      }
+    });
+  }
+
+  public void preListRSGroups() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preListRSGroups(this);
+      }
+    });
+  }
+
+  public void postListRSGroups() throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postListRSGroups(this);
+      }
+    });
+  }
+
+  public void preGetRSGroupInfoOfServer(final Address server) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetRSGroupInfoOfServer(this, server);
+      }
+    });
+  }
+
+  public void postGetRSGroupInfoOfServer(final Address server) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetRSGroupInfoOfServer(this, server);
+      }
+    });
+  }
+
   public void preAddReplicationPeer(final String peerId, final ReplicationPeerConfig peerConfig)
       throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {