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/02/22 06:59:10 UTC

[21/50] [abbrv] hbase git commit: HBASE-19400 Add missing security checks in MasterRpcServices

HBASE-19400 Add missing security checks in MasterRpcServices

- Added ADMIN permission check for following rpc calls:
  normalize, setNormalizerRunning, runCatalogScan, enableCatalogJanitor, runCleanerChore,
  setCleanerChoreRunning, execMasterService, execProcedure, execProcedureWithRet
- Moved authorizationEnabled check to start of AccessChecker's functions. Currently, and IDK why,
  we call authManager.authorize() first and then discard its result if authorizationEnabled is false. Weird.


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

Branch: refs/heads/HBASE-19064
Commit: 6b8439366134b6749895b6019bc068d169865b48
Parents: 148356a
Author: Apekshit Sharma <ap...@apache.org>
Authored: Tue Feb 13 12:33:43 2018 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Tue Feb 20 13:37:20 2018 -0700

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     |   3 +-
 .../hadoop/hbase/master/MasterRpcServices.java  | 120 ++++-----
 .../hbase/regionserver/HRegionServer.java       |   2 +-
 .../hbase/regionserver/RSRpcServices.java       |  25 +-
 .../hbase/security/access/AccessChecker.java    | 102 ++++----
 .../hbase/security/access/AccessController.java |   5 +-
 .../security/access/TestAccessController.java   |  13 +-
 .../access/TestAdminOnlyOperations.java         | 244 +++++++++++++++++++
 8 files changed, 389 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/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 435d138..7fec32d 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
@@ -79,7 +79,6 @@ 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.security.access.TableAuthManager;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -127,7 +126,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver {
 
   @Override
   public void stop(CoprocessorEnvironment env) {
-    TableAuthManager.release(accessChecker.getAuthManager());
+    accessChecker.stop();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 377a9c6..b4f0faf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -349,6 +350,24 @@ public class MasterRpcServices extends RSRpcServices
     return new MasterAnnotationReadingPriorityFunction(this);
   }
 
+  /**
+   * Checks for the following pre-checks in order:
+   * <ol>
+   *   <li>Master is initialized</li>
+   *   <li>Rpc caller has admin permissions</li>
+   * </ol>
+   * @param requestName name of rpc request. Used in reporting failures to provide context.
+   * @throws ServiceException If any of the above listed pre-check fails.
+   */
+  private void rpcPreCheck(String requestName) throws ServiceException {
+    try {
+      master.checkInitialized();
+      requirePermission(requestName, Permission.Action.ADMIN);
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
   enum BalanceSwitchMode {
     SYNC,
     ASYNC
@@ -393,25 +412,6 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   /**
-   * Sets normalizer on/off flag in ZK.
-   */
-  public boolean normalizerSwitch(boolean on) {
-    boolean oldValue = master.getRegionNormalizerTracker().isNormalizerOn();
-    boolean newValue = on;
-    try {
-      try {
-        master.getRegionNormalizerTracker().setNormalizerOn(newValue);
-      } catch (KeeperException ke) {
-        throw new IOException(ke);
-      }
-      LOG.info(master.getClientIdAuditPrefix() + " set normalizerSwitch=" + newValue);
-    } catch (IOException ioe) {
-      LOG.warn("Error flipping normalizer switch", ioe);
-    }
-    return oldValue;
-  }
-
-  /**
    * @return list of blocking services and their security info classes that this server supports
    */
   @Override
@@ -696,24 +696,16 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
       EnableCatalogJanitorRequest req) throws ServiceException {
-    try {
-      master.checkInitialized();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+    rpcPreCheck("enableCatalogJanitor");
     return EnableCatalogJanitorResponse.newBuilder().setPrevValue(
       master.catalogJanitorChore.setEnabled(req.getEnable())).build();
   }
 
   @Override
-  public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c,
-                                                               SetCleanerChoreRunningRequest req)
-    throws ServiceException {
-    try {
-      master.checkInitialized();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+  public SetCleanerChoreRunningResponse setCleanerChoreRunning(
+    RpcController c, SetCleanerChoreRunningRequest req) throws ServiceException {
+    rpcPreCheck("setCleanerChoreRunning");
+
     boolean prevValue =
       master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled();
     master.getLogCleaner().setEnabled(req.getOn());
@@ -793,10 +785,9 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
       final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
+    rpcPreCheck("execMasterService");
     try {
-      master.checkInitialized();
       ServerRpcController execController = new ServerRpcController();
-
       ClientProtos.CoprocessorServiceCall call = request.getCall();
       String serviceName = call.getServiceName();
       String methodName = call.getMethodName();
@@ -816,14 +807,11 @@ public class MasterRpcServices extends RSRpcServices
       final com.google.protobuf.Message.Builder responseBuilder =
           service.getResponsePrototype(methodDesc).newBuilderForType();
       service.callMethod(methodDesc, execController, execRequest,
-          new com.google.protobuf.RpcCallback<com.google.protobuf.Message>() {
-        @Override
-        public void run(com.google.protobuf.Message message) {
+        (message) -> {
           if (message != null) {
             responseBuilder.mergeFrom(message);
           }
-        }
-      });
+        });
       com.google.protobuf.Message execResult = responseBuilder.build();
       if (execController.getFailedOn() != null) {
         throw execController.getFailedOn();
@@ -841,8 +829,8 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ExecProcedureResponse execProcedure(RpcController controller,
       ExecProcedureRequest request) throws ServiceException {
+    rpcPreCheck("execProcedure");
     try {
-      master.checkInitialized();
       ProcedureDescription desc = request.getProcedure();
       MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
         desc.getSignature());
@@ -850,12 +838,8 @@ public class MasterRpcServices extends RSRpcServices
         throw new ServiceException(new DoNotRetryIOException("The procedure is not registered: "
           + desc.getSignature()));
       }
-
-      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
-        + desc.getSignature());
-
+      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
       mpm.execProcedure(desc);
-
       // send back the max amount of time the client should wait for the procedure
       // to complete
       long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
@@ -876,21 +860,16 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public ExecProcedureResponse execProcedureWithRet(RpcController controller,
       ExecProcedureRequest request) throws ServiceException {
+    rpcPreCheck("execProcedureWithRet");
     try {
-      master.checkInitialized();
       ProcedureDescription desc = request.getProcedure();
-      MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
-        desc.getSignature());
+      MasterProcedureManager mpm =
+        master.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
       if (mpm == null) {
-        throw new ServiceException("The procedure is not registered: "
-          + desc.getSignature());
+        throw new ServiceException("The procedure is not registered: " + desc.getSignature());
       }
-
-      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
-        + desc.getSignature());
-
+      LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
       byte[] data = mpm.execProcedureWithRet(desc);
-
       ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
       // set return data if available
       if (data != null) {
@@ -1189,8 +1168,7 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public AbortProcedureResponse abortProcedure(
-      RpcController rpcController,
-      AbortProcedureRequest request) throws ServiceException {
+      RpcController rpcController, AbortProcedureRequest request) throws ServiceException {
     try {
       AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
       boolean abortResult =
@@ -1419,8 +1397,8 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public RunCatalogScanResponse runCatalogScan(RpcController c,
       RunCatalogScanRequest req) throws ServiceException {
+    rpcPreCheck("runCatalogScan");
     try {
-      master.checkInitialized();
       return ResponseConverter.buildRunCatalogScanResponse(master.catalogJanitorChore.scan());
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
@@ -1430,13 +1408,9 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
     throws ServiceException {
-    try {
-      master.checkInitialized();
-      boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
-      return ResponseConverter.buildRunCleanerChoreResponse(result);
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
-    }
+    rpcPreCheck("runCleanerChore");
+    boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
+    return ResponseConverter.buildRunCleanerChoreResponse(result);
   }
 
   @Override
@@ -1769,6 +1743,7 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public NormalizeResponse normalize(RpcController controller,
       NormalizeRequest request) throws ServiceException {
+    rpcPreCheck("normalize");
     try {
       return NormalizeResponse.newBuilder().setNormalizerRan(master.normalizeRegions()).build();
     } catch (IOException ex) {
@@ -1779,13 +1754,18 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
       SetNormalizerRunningRequest request) throws ServiceException {
+    rpcPreCheck("setNormalizerRunning");
+
+    // Sets normalizer on/off flag in ZK.
+    boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
+    boolean newValue = request.getOn();
     try {
-      master.checkInitialized();
-      boolean prevValue = normalizerSwitch(request.getOn());
-      return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
-    } catch (IOException ioe) {
-      throw new ServiceException(ioe);
+      master.getRegionNormalizerTracker().setNormalizerOn(newValue);
+    } catch (KeeperException ke) {
+      LOG.warn("Error flipping normalizer switch", ke);
     }
+    LOG.info("{} set normalizerSwitch={}", master.getClientIdAuditPrefix(), newValue);
+    return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e2aac03..49b7b80 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -630,7 +630,7 @@ public class HRegionServer extends HasThread implements
         masterAddressTracker = null;
         clusterStatusTracker = null;
       }
-      this.rpcServices.start();
+      this.rpcServices.start(zooKeeper);
       // This violates 'no starting stuff in Constructor' but Master depends on the below chore
       // and executor being created and takes a different startup route. Lots of overlap between HRS
       // and M (An M IS A HRS now). Need to refactor so less duplication between M and its super

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 5b4e3b8..05bbb47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -118,6 +118,8 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -128,6 +130,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -321,6 +324,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   final AtomicBoolean clearCompactionQueues = new AtomicBoolean(false);
 
+  // We want to vet all accesses at the point of entry itself; limiting scope of access checker
+  // instance to only this class to prevent its use from spreading deeper into implementation.
+  // Initialized in start() since AccessChecker needs ZKWatcher which is created by HRegionServer
+  // after RSRpcServices constructor and before start() is called.
+  // Initialized only if authorization is enabled, else remains null.
+  private AccessChecker accessChecker;
+
   /**
    * Services launched in RSRpcServices. By default they are on but you can use the below
    * booleans to selectively enable/disable either Admin or Client Service (Rare is the case
@@ -1240,6 +1250,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return new AnnotationReadingPriorityFunction(this);
   }
 
+  protected void requirePermission(String request, Permission.Action perm) throws IOException {
+    if (accessChecker != null) {
+      accessChecker.requirePermission(RpcServer.getRequestUser().orElse(null), request, perm);
+    }
+  }
+
+
   public static String getHostname(Configuration conf, boolean isMaster)
       throws UnknownHostException {
     String hostname = conf.get(isMaster? HRegionServer.MASTER_HOSTNAME_KEY :
@@ -1404,12 +1421,18 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return regionServer.getRegionServerSpaceQuotaManager();
   }
 
-  void start() {
+  void start(ZKWatcher zkWatcher) {
+    if (AccessChecker.isAuthorizationSupported(getConfiguration())) {
+      accessChecker = new AccessChecker(getConfiguration(), zkWatcher);
+    }
     this.scannerIdGenerator = new ScannerIdGenerator(this.regionServer.serverName);
     rpcServer.start();
   }
 
   void stop() {
+    if (accessChecker != null) {
+      accessChecker.stop();
+    }
     closeAllScanners();
     rpcServer.stop();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
index d88e522..7feeaa0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -40,6 +40,9 @@ import org.slf4j.LoggerFactory;
 public final class AccessChecker {
   private static final Logger AUDITLOG =
       LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
+  // TODO: we should move to a design where we don't even instantiate an AccessChecker if
+  // authorization is not enabled (like in RSRpcServices), instead of always instantiating one and
+  // calling requireXXX() only to do nothing (since authorizationEnabled will be false).
   private TableAuthManager authManager;
   /**
    * if we are active, usually false, only true if "hbase.security.authorization"
@@ -59,8 +62,6 @@ public final class AccessChecker {
    */
   public AccessChecker(final Configuration conf, final ZKWatcher zkw)
       throws RuntimeException {
-    // If zk is null or IOException while obtaining auth manager,
-    // throw RuntimeException so that the coprocessor is unloaded.
     if (zkw != null) {
       try {
         this.authManager = TableAuthManager.getOrCreate(zkw, conf);
@@ -73,6 +74,13 @@ public final class AccessChecker {
     authorizationEnabled = isAuthorizationSupported(conf);
   }
 
+  /**
+   * Releases {@link TableAuthManager}'s reference.
+   */
+  public void stop() {
+    TableAuthManager.release(authManager);
+  }
+
   public TableAuthManager getAuthManager() {
     return authManager;
   }
@@ -87,6 +95,9 @@ public final class AccessChecker {
    */
   public void requireAccess(User user, String request, TableName tableName,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
@@ -101,7 +112,7 @@ public final class AccessChecker {
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -130,22 +141,21 @@ public final class AccessChecker {
   public void requireGlobalPermission(User user, String request,
       Action perm, TableName tableName,
       Map<byte[], ? extends Collection<byte[]>> familyMap)throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result;
     if (authManager.authorize(user, perm)) {
-      result = AuthResult.allow(request, "Global check allowed",
-          user, perm, tableName, familyMap);
+      result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
       result.getParams().setTableName(tableName).setFamilies(familyMap);
       logResult(result);
     } else {
-      result = AuthResult.deny(request, "Global check failed",
-          user, perm, tableName, familyMap);
+      result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
       result.getParams().setTableName(tableName).setFamilies(familyMap);
       logResult(result);
-      if (authorizationEnabled) {
-        throw new AccessDeniedException(
-            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
-                + "' (global, action=" + perm.toString() + ")");
-      }
+      throw new AccessDeniedException(
+          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+              + "' (global, action=" + perm.toString() + ")");
     }
   }
 
@@ -159,22 +169,21 @@ public final class AccessChecker {
    */
   public void requireGlobalPermission(User user, String request, Action perm,
       String namespace) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult authResult;
     if (authManager.authorize(user, perm)) {
-      authResult = AuthResult.allow(request, "Global check allowed",
-          user, perm, null);
+      authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
     } else {
-      authResult = AuthResult.deny(request, "Global check failed",
-          user, perm, null);
+      authResult = AuthResult.deny(request, "Global check failed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
-      if (authorizationEnabled) {
-        throw new AccessDeniedException(
-            "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
-                + "' (global, action=" + perm.toString() + ")");
-      }
+      throw new AccessDeniedException(
+          "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+              + "' (global, action=" + perm.toString() + ")");
     }
   }
 
@@ -186,22 +195,23 @@ public final class AccessChecker {
    */
   public void requireNamespacePermission(User user, String request, String namespace,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, namespace, permission)) {
         result =
-            AuthResult.allow(request, "Namespace permission granted",
-                user, permission, namespace);
+            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         break;
       } else {
         // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions",
-            user, permission, namespace);
+        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -215,24 +225,25 @@ public final class AccessChecker {
   public void requireNamespacePermission(User user, String request, String namespace,
       TableName tableName, Map<byte[], ? extends Collection<byte[]>> familyMap,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, namespace, permission)) {
         result =
-            AuthResult.allow(request, "Namespace permission granted",
-                user, permission, namespace);
+            AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         result.getParams().setTableName(tableName).setFamilies(familyMap);
         break;
       } else {
         // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions",
-            user, permission, namespace);
+        result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
         result.getParams().setTableName(tableName).setFamilies(familyMap);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -249,23 +260,24 @@ public final class AccessChecker {
    */
   public void requirePermission(User user, String request, TableName tableName, byte[] family,
       byte[] qualifier, Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
       if (authManager.authorize(user, tableName, family, qualifier, permission)) {
         result = AuthResult.allow(request, "Table permission granted",
-            user, permission, tableName, family,
-                qualifier);
+            user, permission, tableName, family, qualifier);
         break;
       } else {
         // rest of the world
         result = AuthResult.deny(request, "Insufficient permissions",
-                user, permission, tableName, family,
-                qualifier);
+                user, permission, tableName, family, qualifier);
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -283,6 +295,9 @@ public final class AccessChecker {
   public void requireTablePermission(User user, String request,
       TableName tableName,byte[] family, byte[] qualifier,
       Action... permissions) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     AuthResult result = null;
 
     for (Action permission : permissions) {
@@ -299,7 +314,7 @@ public final class AccessChecker {
       }
     }
     logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
+    if (!result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -321,12 +336,13 @@ public final class AccessChecker {
 
   public static void logResult(AuthResult result) {
     if (AUDITLOG.isTraceEnabled()) {
-      AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") + " for user " + (
-          result.getUser() != null ?
-              result.getUser().getShortName() :
-              "UNKNOWN") + "; reason: " + result.getReason() + "; remote address: "
-          + RpcServer.getRemoteAddress().map(InetAddress::toString).orElse("")
-          + "; request: " + result.getRequest() + "; context: " + result.toContextString());
+      AUDITLOG.trace(
+        "Access {} for user {}; reason: {}; remote address: {}; request: {}; context: {}",
+        (result.isAllowed() ? "allowed" : "denied"),
+        (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN"),
+        result.getReason(),
+        RpcServer.getRemoteAddress().map(InetAddress::toString).orElse(""),
+        result.getRequest(), result.toContextString());
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/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 1fbf01d..5fef8ab 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
@@ -795,13 +795,14 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
     // set the user-provider.
     this.userProvider = UserProvider.instantiate(env.getConfiguration());
+    // Throws RuntimeException if fails to load TableAuthManager so that coprocessor is unloaded.
     accessChecker = new AccessChecker(env.getConfiguration(), zk);
     tableAcls = new MapMaker().weakValues().makeMap();
   }
 
   @Override
   public void stop(CoprocessorEnvironment env) {
-    TableAuthManager.release(getAuthManager());
+    accessChecker.stop();
   }
 
   /*********************************** Observer/Service Getters ***********************************/
@@ -2456,7 +2457,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       throws IOException {
     requirePermission(ctx, "replicateLogEntries", Action.WRITE);
   }
-  
+
   @Override
   public void  preClearCompactionQueues(ObserverContext<RegionServerCoprocessorEnvironment> ctx)
           throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index d5a0063..2e9be30 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -223,14 +223,15 @@ public class TestAccessController extends SecureTestUtil {
     conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
 
     TEST_UTIL.startMiniCluster();
-    MasterCoprocessorHost cpHost =
+    MasterCoprocessorHost masterCpHost =
       TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
-    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
-    ACCESS_CONTROLLER = cpHost.findCoprocessor(AccessController.class);
-    CP_ENV = cpHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
-    RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
+    masterCpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
+    ACCESS_CONTROLLER = masterCpHost.findCoprocessor(AccessController.class);
+    CP_ENV = masterCpHost.createEnvironment(
+        ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    RegionServerCoprocessorHost rsCpHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
       .getRegionServerCoprocessorHost();
-    RSCP_ENV = rsHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    RSCP_ENV = rsCpHost.createEnvironment(ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
new file mode 100644
index 0000000..d4b0650
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
@@ -0,0 +1,244 @@
+/**
+ * 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.security.access;
+
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.HashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This class tests operations in MasterRpcServices which require ADMIN access.
+ * It doesn't test all operations which require ADMIN access, only those which get vetted within
+ * MasterRpcServices at the point of entry itself (unlike old approach of using
+ * hooks in AccessController).
+ *
+ * Sidenote:
+ * There is one big difference between how security tests for AccessController hooks work, and how
+ * the tests in this class for security in MasterRpcServices work.
+ * The difference arises because of the way AC & MasterRpcServices get the user.
+ *
+ * In AccessController, it first checks if there is an active rpc user in ObserverContext. If not,
+ * it uses UserProvider for current user. This *might* make sense in the context of coprocessors,
+ * because they can be called outside the context of RPCs.
+ * But in the context of MasterRpcServices, only one way makes sense - RPCServer.getRequestUser().
+ *
+ * In AC tests, when we do FooUser.runAs on AccessController instance directly, it bypasses
+ * the rpc framework completely, but works because UserProvider provides the correct user, i.e.
+ * FooUser in this case.
+ *
+ * But this doesn't work for the tests here, so we go around by doing complete RPCs.
+ */
+@Category({SecurityTests.class, MediumTests.class})
+public class TestAdminOnlyOperations {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAdminOnlyOperations.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+
+  // user granted with all global permission
+  private static User USER_ADMIN;
+  // user without admin permissions
+  private static User USER_NON_ADMIN;
+
+  private static final String GROUP_ADMIN = "admin_group";
+  private static User USER_GROUP_ADMIN;
+
+  // Dummy service to test execService calls. Needs to be public so can be loaded as Coprocessor.
+  public static class DummyCpService implements MasterCoprocessor {
+    public DummyCpService() {}
+
+    @Override
+    public Iterable<Service> getServices() {
+      return Collections.singleton(mock(TestRpcServiceProtos.TestProtobufRpcProto.class));
+    }
+  }
+
+  private static void enableSecurity(Configuration conf) throws IOException {
+    conf.set("hadoop.security.authorization", "false");
+    conf.set("hadoop.security.authentication", "simple");
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+      "," + DummyCpService.class.getName());
+    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true");
+    SecureTestUtil.configureSuperuser(conf);
+  }
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+
+    // Enable security
+    enableSecurity(conf);
+    TEST_UTIL.startMiniCluster();
+
+    // Wait for the ACL table to become available
+    TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
+
+    // Create users
+    USER_ADMIN = User.createUserForTesting(conf, "admin", new String[0]);
+    USER_NON_ADMIN = User.createUserForTesting(conf, "non_admin", new String[0]);
+    USER_GROUP_ADMIN =
+        User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
+
+    // Assign permissions to users and groups
+    SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(), Permission.Action.ADMIN);
+    SecureTestUtil.grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+    // No permissions to USER_NON_ADMIN
+  }
+
+  interface Action {
+    void run(Admin admin) throws Exception;
+  }
+
+  private void verifyAllowed(User user, Action action) throws Exception {
+    user.runAs((PrivilegedExceptionAction<?>) () -> {
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (IOException e) {
+        fail(e.toString());
+      }
+      return null;
+    });
+  }
+
+  private void verifyDenied(User user, Action action) throws Exception {
+    user.runAs((PrivilegedExceptionAction<?>) () -> {
+      boolean accessDenied = false;
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (AccessDeniedException e) {
+        accessDenied = true;
+      }
+      assertTrue("Expected access to be denied", accessDenied);
+      return null;
+    });
+  }
+
+  private void verifyAdminCheckForAction(Action action) throws Exception {
+    verifyAllowed(USER_ADMIN, action);
+    verifyAllowed(USER_GROUP_ADMIN, action);
+    verifyDenied(USER_NON_ADMIN, action);
+  }
+
+  @Test
+  public void testEnableCatalogJanitor() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.enableCatalogJanitor(true));
+  }
+
+  @Test
+  public void testRunCatalogJanitor() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.runCatalogJanitor());
+  }
+
+  @Test
+  public void testCleanerChoreRunning() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.cleanerChoreSwitch(true));
+  }
+
+  @Test
+  public void testRunCleanerChore() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.runCleanerChore());
+  }
+
+  @Test
+  public void testExecProcedure() throws Exception {
+    verifyAdminCheckForAction((admin) -> {
+      // Using existing table instead of creating a new one.
+      admin.execProcedure("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
+          new HashMap<>());
+    });
+  }
+
+  @Test
+  public void testExecService() throws Exception {
+    Action action = (admin) -> {
+      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface service =
+          TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(admin.coprocessorService());
+      service.ping(null, TestProtos.EmptyRequestProto.getDefaultInstance());
+    };
+
+    verifyAllowed(USER_ADMIN, action);
+    verifyAllowed(USER_GROUP_ADMIN, action);
+    // This is same as above verifyAccessDenied
+    USER_NON_ADMIN.runAs((PrivilegedExceptionAction<?>) () -> {
+      boolean accessDenied = false;
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Admin admin = conn.getAdmin()) {
+        action.run(admin);
+      } catch (ServiceException e) {
+        // For MasterRpcServices.execService.
+        if (e.getCause() instanceof AccessDeniedException) {
+          accessDenied = true;
+        }
+      }
+      assertTrue("Expected access to be denied", accessDenied);
+      return null;
+    });
+  }
+
+  @Test
+  public void testExecProcedureWithRet() throws Exception {
+    verifyAdminCheckForAction((admin) -> {
+      // Using existing table instead of creating a new one.
+      admin.execProcedureWithReturn("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(),
+          new HashMap<>());
+    });
+  }
+
+  @Test
+  public void testNormalize() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.normalize());
+  }
+
+  @Test
+  public void testSetNormalizerRunning() throws Exception {
+    verifyAdminCheckForAction((admin) -> admin.normalizerSwitch(true));
+  }
+}