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/30 13:11:17 UTC

[11/43] hbase git commit: HBASE-20653 Add missing observer hooks for region server group to MasterObserver - revert due to pending discussion

HBASE-20653 Add missing observer hooks for region server group to MasterObserver - revert due to pending discussion


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

Branch: refs/heads/HBASE-19064
Commit: fe73fe8def88a513e53196461bfa845ae91ccf96
Parents: 8d19bbd
Author: tedyu <yu...@gmail.com>
Authored: Tue May 29 19:42:28 2018 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue May 29 19:42:28 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     | 49 +++++------
 .../hadoop/hbase/rsgroup/TestRSGroups.java      | 86 --------------------
 .../hbase/rsgroup/TestRSGroupsWithACL.java      |  8 +-
 .../hbase/coprocessor/MasterObserver.java       | 62 --------------
 .../hbase/master/MasterCoprocessorHost.java     | 72 ----------------
 .../hadoop/hbase/regionserver/HRegion.java      |  8 ++
 .../hbase/security/access/AccessController.java | 24 ------
 7 files changed, 37 insertions(+), 272 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fe73fe8d/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 51d978b..8546a40 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
+import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -77,8 +78,10 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGro
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveServersResponse;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -158,16 +161,11 @@ 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);
       }
@@ -182,16 +180,11 @@ 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);
       }
@@ -314,15 +307,10 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
       ListRSGroupInfosResponse.Builder builder = ListRSGroupInfosResponse.newBuilder();
       LOG.info(master.getClientIdAuditPrefix() + " list rsgroup");
       try {
-        if (master.getMasterCoprocessorHost() != null) {
-          master.getMasterCoprocessorHost().preListRSGroupInfos();
-        }
+        checkPermission("listRSGroup");
         for (RSGroupInfo RSGroupInfo : groupAdminServer.listRSGroups()) {
           builder.addRSGroupInfo(RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo));
         }
-        if (master.getMasterCoprocessorHost() != null) {
-          master.getMasterCoprocessorHost().postListRSGroupInfos();
-        }
       } catch (IOException e) {
         CoprocessorRpcUtils.setControllerException(controller, e);
       }
@@ -338,16 +326,11 @@ 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);
       }
@@ -527,4 +510,22 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
         collect(Collectors.toSet());
     groupAdminServer.removeServers(clearedServer);
   }
+
+  public void checkPermission(String request) throws IOException {
+    accessChecker.requirePermission(getActiveUser(), request, Action.ADMIN);
+  }
+
+  /**
+   * Returns the active user to which authorization checks should be applied.
+   * If we are in the context of an RPC call, the remote user is used,
+   * otherwise the currently logged in user is used.
+   */
+  private User getActiveUser() throws IOException {
+    // for non-rpc handling, fallback to system user
+    Optional<User> optionalUser = RpcServer.getRequestUser();
+    if (optionalUser.isPresent()) {
+      return optionalUser.get();
+    }
+    return userProvider.getCurrent();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe73fe8d/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 3bd76be..3e74f81 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,14 +292,6 @@ 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 preListRSGroupInfosCalled = false;
-    boolean postListRSGroupInfosCalled = false;
-    boolean preGetRSGroupInfoOfServerCalled = false;
-    boolean postGetRSGroupInfoOfServerCalled = false;
 
     @Override
     public Optional<MasterObserver> getMasterObserver() {
@@ -378,85 +370,7 @@ 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 preListRSGroupInfos(final ObserverContext<MasterCoprocessorEnvironment> ctx)
-        throws IOException {
-      preListRSGroupInfosCalled = true;
-    }
-
-    @Override
-    public void postListRSGroupInfos(final ObserverContext<MasterCoprocessorEnvironment> ctx)
-        throws IOException {
-      postListRSGroupInfosCalled = 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 testListRSGroupInfosCPHookCalled() throws Exception {
-    rsGroupAdmin.listRSGroups();
-    assertTrue(observer.preListRSGroupInfosCalled);
-    assertTrue(observer.postListRSGroupInfosCalled);
   }
-
-  @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/fe73fe8d/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 a4ee9df..83d76a4 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
@@ -235,7 +235,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
   @Test
   public void testGetRSGroupInfo() throws Exception {
     AccessTestAction action = () -> {
-      accessController.preGetRSGroupInfo(CTX, null);
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfo");
       return null;
     };
 
@@ -245,7 +245,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
   @Test
   public void testGetRSGroupInfoOfTable() throws Exception {
     AccessTestAction action = () -> {
-      accessController.preGetRSGroupInfoOfTable(CTX, null);
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfoOfTable");
       return null;
     };
 
@@ -305,7 +305,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
   @Test
   public void testListRSGroup() throws Exception {
     AccessTestAction action = () -> {
-      accessController.preListRSGroupInfos(CTX);
+      rsGroupAdminEndpoint.checkPermission("listRSGroup");
       return null;
     };
 
@@ -315,7 +315,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
   @Test
   public void testGetRSGroupInfoOfServer() throws Exception {
     AccessTestAction action = () -> {
-      accessController.preGetRSGroupInfoOfServer(CTX, null);
+      rsGroupAdminEndpoint.checkPermission("getRSGroupInfoOfServer");
       return null;
     };
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe73fe8d/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 92209f8..f60a04d 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,68 +1213,6 @@ 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 preListRSGroupInfos(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 postListRSGroupInfos(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/fe73fe8d/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 da4cb26..072ae8a 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,78 +1404,6 @@ 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 preListRSGroupInfos() throws IOException {
-    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
-      @Override
-      public void call(MasterObserver observer) throws IOException {
-        observer.preListRSGroupInfos(this);
-      }
-    });
-  }
-
-  public void postListRSGroupInfos() throws IOException {
-    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
-      @Override
-      public void call(MasterObserver observer) throws IOException {
-        observer.postListRSGroupInfos(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() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe73fe8d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index c0203a4..404eb1a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -920,6 +920,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       Collection<HStore> stores = this.stores.values();
       try {
         // update the stores that we are replaying
+        LOG.debug("replaying wal for " + this.getRegionInfo().getEncodedName());
         stores.forEach(HStore::startReplayingFromWAL);
         // Recover any edits if available.
         maxSeqId = Math.max(maxSeqId,
@@ -927,6 +928,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // Make sure mvcc is up to max.
         this.mvcc.advanceTo(maxSeqId);
       } finally {
+        LOG.debug("stopping wal replay for " + this.getRegionInfo().getEncodedName());
         // update the stores that we are done replaying
         stores.forEach(HStore::stopReplayingFromWAL);
       }
@@ -938,6 +940,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     this.writestate.compacting.set(0);
 
     if (this.writestate.writesEnabled) {
+      LOG.debug("Cleaning up temporary data for " + this.getRegionInfo().getEncodedName());
       // Remove temporary data left over from old regions
       status.setStatus("Cleaning up temporary data from old regions");
       fs.cleanupTempDir();
@@ -948,6 +951,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // Get rid of any splits or merges that were lost in-progress.  Clean out
       // these directories here on open.  We may be opening a region that was
       // being split but we crashed in the middle of it all.
+      LOG.debug("Cleaning up detritus for " + this.getRegionInfo().getEncodedName());
       fs.cleanupAnySplitDetritus();
       fs.cleanupMergesDir();
     }
@@ -969,6 +973,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       WALSplitter.getMaxRegionSequenceId(fs.getFileSystem(), fs.getRegionDir());
     long nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
     if (writestate.writesEnabled) {
+      LOG.debug("writing seq id for " + this.getRegionInfo().getEncodedName());
       WALSplitter.writeRegionSequenceIdFile(fs.getFileSystem(), fs.getRegionDir(), nextSeqId - 1);
     }
 
@@ -979,6 +984,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     this.closed.set(false);
 
     if (coprocessorHost != null) {
+      LOG.debug("Running coprocessor post-open hooks for " + this.getRegionInfo().getEncodedName());
       status.setStatus("Running coprocessor post-open hooks");
       coprocessorHost.postOpen();
     }
@@ -7125,10 +7131,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   throws IOException {
     // Refuse to open the region if we are missing local compression support
     checkCompressionCodecs();
+    LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName());
     // Refuse to open the region if encryption configuration is incorrect or
     // codec support is missing
     checkEncryption();
     // Refuse to open the region if a required class cannot be loaded
+    LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName());
     checkClassLoading();
     this.openSeqNum = initialize(reporter);
     this.mvcc.advanceTo(openSeqNum);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe73fe8d/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 1ac98f5..7e824e2 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
@@ -1307,18 +1307,6 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
-  public void preGetRSGroupInfo(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      String groupName) throws IOException {
-    requirePermission(ctx, "getRSGroupInfo", Action.ADMIN);
-  }
-
-  @Override
-  public void preGetRSGroupInfoOfTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName) throws IOException {
-    requirePermission(ctx, "getRSGroupInfoOfTable", Action.ADMIN);
-  }
-
-  @Override
   public void preMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {
     requirePermission(ctx, "moveServersAndTables", Action.ADMIN);
@@ -1355,18 +1343,6 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
-  public void preListRSGroupInfos(final ObserverContext<MasterCoprocessorEnvironment> ctx)
-      throws IOException {
-    requirePermission(ctx, "listRSGroup", Action.ADMIN);
-  }
-
-  @Override
-  public void preGetRSGroupInfoOfServer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      Address server) throws IOException {
-    requirePermission(ctx, "getRSGroupInfoOfServer", Action.ADMIN);
-  }
-
-  @Override
   public void preRemoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       Set<Address> servers) throws IOException {
     requirePermission(ctx, "removeServers", Action.ADMIN);