You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/05/09 01:33:25 UTC

[hbase] branch branch-1.3 updated: HBASE-22374 Backport AccessChecker refactor to branch-1.3

This is an automated email from the ASF dual-hosted git repository.

apurtell pushed a commit to branch branch-1.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1.3 by this push:
     new 7db8f03  HBASE-22374 Backport AccessChecker refactor to branch-1.3
7db8f03 is described below

commit 7db8f03744f60c1bc51ab8a46d0ff4a6dfb35222
Author: Andrew Purtell <ap...@apache.org>
AuthorDate: Wed May 8 18:13:02 2019 -0700

    HBASE-22374 Backport AccessChecker refactor to branch-1.3
---
 .../hbase/security/access/AccessChecker.java       | 306 ++++++++++++++++++++
 .../hbase/security/access/AccessController.java    | 316 ++++++---------------
 .../hbase/security/access/TableAuthManager.java    |   2 +-
 .../hbase/security/access/SecureTestUtil.java      |  14 +-
 .../security/access/TestAccessController.java      |   4 +-
 .../security/access/TestAccessController2.java     |   3 +
 6 files changed, 409 insertions(+), 236 deletions(-)

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
new file mode 100644
index 0000000..f2b494e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -0,0 +1,306 @@
+/*
+ * 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 java.io.IOException;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public final class AccessChecker {
+  private static final Logger AUDITLOG =
+      LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
+  private TableAuthManager authManager;
+  /**
+   * if we are active, usually false, only true if "hbase.security.authorization"
+   * has been set to true in site configuration.see HBASE-19483.
+   */
+  private boolean authorizationEnabled;
+
+  public static boolean isAuthorizationSupported(Configuration conf) {
+    // branch-1.4 and branch-1 use a default of false but we must assume it is true
+    // for compatibility with branch-1.3 tests
+    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+  }
+
+  /**
+   * Constructor with existing configuration
+   *
+   * @param conf Existing configuration to use
+   * @param zkw reference to the {@link ZooKeeperWatcher}
+   */
+  public AccessChecker(final Configuration conf, final ZooKeeperWatcher 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);
+      } catch (IOException ioe) {
+        throw new RuntimeException("Error obtaining AccessChecker", ioe);
+      }
+    } else {
+      throw new NullPointerException("Error obtaining AccessChecker, zk found null.");
+    }
+    authorizationEnabled = isAuthorizationSupported(conf);
+  }
+
+  public TableAuthManager getAuthManager() {
+    return authManager;
+  }
+
+  public void logResult(AuthResult result) {
+    if (AUDITLOG.isTraceEnabled()) {
+      InetAddress remoteAddr = RpcServer.getRemoteAddress();
+      AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") +
+          " for user " + (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN") +
+          "; reason: " + result.getReason() +
+          "; remote address: " + (remoteAddr != null ? remoteAddr : "") +
+          "; request: " + result.getRequest() +
+          "; context: " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions for the
+   * given table, column family and column qualifier.
+   * @param tableName Table requested
+   * @param family Column family requested
+   * @param qualifier Column qualifier requested
+   * @throws IOException if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requirePermission(User user, String request, TableName tableName, byte[] family,
+      byte[] qualifier, Action... permissions) throws IOException {
+    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);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions", user,
+            permission, tableName, family, qualifier);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions for the
+   * given table, column family and column qualifier.
+   * @param tableName Table requested
+   * @param family Column family param
+   * @param qualifier Column qualifier param
+   * @throws IOException if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requireTablePermission(User user, String request, TableName tableName, byte[] family,
+      byte[] qualifier, Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, tableName, null, null, permission)) {
+        result = AuthResult.allow(request, "Table permission granted", user,
+            permission, tableName, null, null);
+        result.getParams().setFamily(family).setQualifier(qualifier);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions", user,
+            permission, tableName, family, qualifier);
+        result.getParams().setFamily(family).setQualifier(qualifier);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has any of the given permissions to access the table.
+   *
+   * @param tableName Table requested
+   * @param permissions Actions being requested
+   * @throws IOException if obtaining the current user fails
+   * @throws AccessDeniedException if user has no authorization
+   */
+  public void requireAccess(User user, String request, TableName tableName,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.hasAccess(user, tableName, permission)) {
+        result = AuthResult.allow(request, "Table permission granted", user,
+            permission, tableName, null, null);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions", user,
+            permission, tableName, null, null);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
+    }
+  }
+
+  /**
+   * Authorizes that the current user has global privileges for the given action.
+   * @param perm The action being requested
+   * @throws IOException if obtaining the current user fails
+   * @throws AccessDeniedException if authorization is denied
+   */
+  public void requirePermission(User user, String request, Action perm) throws IOException {
+    requireGlobalPermission(user, request, perm, null, null);
+  }
+
+  /**
+   * Checks that the user has the given global permission. The generated
+   * audit log message will contain context information for the operation
+   * being authorized, based on the given parameters.
+   * @param perm Action being requested
+   * @param tableName Affected table name.
+   * @param familyMap Affected column families.
+   */
+  public void requireGlobalPermission(User user, String request, Action perm, TableName tableName,
+      Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
+    AuthResult result = null;
+    if (authManager.authorize(user, perm)) {
+      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.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() + ")");
+      }
+    }
+  }
+
+  /**
+   * Checks that the user has the given global permission. The generated
+   * audit log message will contain context information for the operation
+   * being authorized, based on the given parameters.
+   * @param perm Action being requested
+   * @param namespace  The given namespace
+   */
+  public void requireGlobalPermission(User user, String request, Action perm,
+      String namespace) throws IOException {
+    AuthResult authResult = null;
+    if (authManager.authorize(user, perm)) {
+      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.getParams().setNamespace(namespace);
+      logResult(authResult);
+      if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions for user '" +
+            (user != null ? user.getShortName() : "null") +"' (global, action=" +
+            perm.toString() + ")");
+      }
+    }
+  }
+
+  /**
+   * Checks that the user has the given global or namespace permission.
+   * @param namespace  The given namespace
+   * @param permissions Actions being requested
+   */
+  public void requireNamespacePermission(User user, String request, String namespace,
+      Action... permissions) throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, namespace, permission)) {
+        result = AuthResult.allow(request, "Namespace permission granted",
+            user, permission, namespace);
+        break;
+      } else {
+        // rest of the world
+        result = AuthResult.deny(request, "Insufficient permissions", user,
+            permission, namespace);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions "
+          + result.toContextString());
+    }
+  }
+
+  /**
+   * Checks that the user has the given global or namespace permission.
+   * @param namespace   The given namespace
+   * @param permissions Actions being requested
+   */
+  public void requireNamespacePermission(User user, String request, String namespace,
+      TableName tableName,
+      Map<byte[], ? extends Collection<byte[]>> familyMap, Action... permissions)
+      throws IOException {
+    AuthResult result = null;
+
+    for (Action permission : permissions) {
+      if (authManager.authorize(user, namespace, permission)) {
+        result = 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.getParams().setTableName(tableName).setFamilies(familyMap);
+      }
+    }
+    logResult(result);
+    if (authorizationEnabled && !result.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions "
+          + result.toContextString());
+    }
+  }
+}
\ No newline at end of file
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 a6b1631..a82ac6e 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.security.access;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.HashMap;
@@ -175,10 +174,10 @@ public class AccessController extends BaseMasterAndRegionObserver
   private static final String TAG_CHECK_PASSED = "tag_check_passed";
   private static final byte[] TRUE = Bytes.toBytes(true);
 
-  TableAuthManager authManager = null;
+  private AccessChecker accessChecker;
 
   /** flags if we are running on a region of the _acl_ table */
-  boolean aclRegion = false;
+  private boolean aclRegion = false;
 
   /** defined only for Endpoint implementation, so it can have way to
    access region services */
@@ -193,19 +192,19 @@ public class AccessController extends BaseMasterAndRegionObserver
   /** Provider for mapping principal names to Users */
   private UserProvider userProvider;
 
-  /** if we are active, usually true, only not true if "hbase.security.authorization"
-   has been set to false in site configuration */
-  boolean authorizationEnabled;
+  /** if we are active, usually false, only true if "hbase.security.authorization"
+   has been set to true in site configuration */
+  private boolean authorizationEnabled;
 
   /** if we are able to support cell ACLs */
-  boolean cellFeaturesEnabled;
+  private boolean cellFeaturesEnabled;
 
   /** if we should check EXEC permissions */
-  boolean shouldCheckExecPermission;
+  private boolean shouldCheckExecPermission;
 
   /** if we should terminate access checks early as soon as table or CF grants
     allow access; pre-0.98 compatible behavior */
-  boolean compatibleEarlyTermination;
+  private boolean compatibleEarlyTermination;
 
   /** if we have been successfully initialized */
   private volatile boolean initialized = false;
@@ -214,11 +213,11 @@ public class AccessController extends BaseMasterAndRegionObserver
   private volatile boolean aclTabAvailable = false;
 
   public static boolean isAuthorizationSupported(Configuration conf) {
-    return conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+    return AccessChecker.isAuthorizationSupported(conf);
   }
 
   public static boolean isCellAuthorizationSupported(Configuration conf) {
-    return isAuthorizationSupported(conf) &&
+    return AccessChecker.isAuthorizationSupported(conf) &&
         (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS);
   }
 
@@ -227,10 +226,10 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   public TableAuthManager getAuthManager() {
-    return authManager;
+    return accessChecker.getAuthManager();
   }
 
-  void initialize(RegionCoprocessorEnvironment e) throws IOException {
+  private void initialize(RegionCoprocessorEnvironment e) throws IOException {
     final Region region = e.getRegion();
     Configuration conf = e.getConfiguration();
     Map<byte[], ListMultimap<String,TablePermission>> tables =
@@ -242,7 +241,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       byte[] entry = t.getKey();
       ListMultimap<String,TablePermission> perms = t.getValue();
       byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
-      this.authManager.getZKPermissionWatcher().writeToZookeeper(entry, serialized);
+      getAuthManager().getZKPermissionWatcher().writeToZookeeper(entry, serialized);
     }
     initialized = true;
   }
@@ -252,7 +251,7 @@ public class AccessController extends BaseMasterAndRegionObserver
    * znodes.  This is called to synchronize ACL changes following {@code _acl_}
    * table updates.
    */
-  void updateACL(RegionCoprocessorEnvironment e,
+  private void updateACL(RegionCoprocessorEnvironment e,
       final Map<byte[], List<Cell>> familyMap) {
     Set<byte[]> entries =
         new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
@@ -266,7 +265,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         }
       }
     }
-    ZKPermissionWatcher zkw = this.authManager.getZKPermissionWatcher();
+    ZKPermissionWatcher zkw = getAuthManager().getZKPermissionWatcher();
     Configuration conf = regionEnv.getConfiguration();
     for (byte[] entry: entries) {
       try {
@@ -318,7 +317,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     }
 
     // 2. check for the table-level, if successful we can short-circuit
-    if (authManager.authorize(user, tableName, (byte[])null, permRequest)) {
+    if (getAuthManager().authorize(user, tableName, (byte[])null, permRequest)) {
       return AuthResult.allow(request, "Table permission granted", user,
         permRequest, tableName, families);
     }
@@ -328,7 +327,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       // all families must pass
       for (Map.Entry<byte [], ? extends Collection<?>> family : families.entrySet()) {
         // a) check for family level access
-        if (authManager.authorize(user, tableName, family.getKey(),
+        if (getAuthManager().authorize(user, tableName, family.getKey(),
             permRequest)) {
           continue;  // family-level permission overrides per-qualifier
         }
@@ -339,7 +338,7 @@ public class AccessController extends BaseMasterAndRegionObserver
             // for each qualifier of the family
             Set<byte[]> familySet = (Set<byte[]>)family.getValue();
             for (byte[] qualifier : familySet) {
-              if (!authManager.authorize(user, tableName, family.getKey(),
+              if (!getAuthManager().authorize(user, tableName, family.getKey(),
                                          qualifier, permRequest)) {
                 return AuthResult.deny(request, "Failed qualifier check", user,
                     permRequest, tableName, makeFamilyMap(family.getKey(), qualifier));
@@ -348,7 +347,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           } else if (family.getValue() instanceof List) { // List<KeyValue>
             List<KeyValue> kvList = (List<KeyValue>)family.getValue();
             for (KeyValue kv : kvList) {
-              if (!authManager.authorize(user, tableName, family.getKey(),
+              if (!getAuthManager().authorize(user, tableName, family.getKey(),
                       kv.getQualifier(), permRequest)) {
                 return AuthResult.deny(request, "Failed qualifier check", user,
                     permRequest, tableName, makeFamilyMap(family.getKey(), kv.getQualifier()));
@@ -395,18 +394,6 @@ public class AccessController extends BaseMasterAndRegionObserver
     return result;
   }
 
-  private void logResult(AuthResult result) {
-    if (AUDITLOG.isTraceEnabled()) {
-      InetAddress remoteAddr = RpcServer.getRemoteAddress();
-      AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") +
-          " for user " + (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN") +
-          "; reason: " + result.getReason() +
-          "; remote address: " + (remoteAddr != null ? remoteAddr : "") +
-          "; request: " + result.getRequest() +
-          "; context: " + result.toContextString());
-    }
-  }
-
   /**
    * 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,
@@ -430,26 +417,10 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if user has no authorization
    */
-  private void requirePermission(String request, TableName tableName, byte[] family,
+  public void requirePermission(String request, TableName tableName, byte[] family,
       byte[] qualifier, Action... permissions) throws IOException {
-    User user = getActiveUser();
-    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);
-        break;
-      } else {
-        // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions", user,
-                                 permission, tableName, family, qualifier);
-      }
-    }
-    logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
-    }
+    accessChecker.requirePermission(getActiveUser(), request,
+        tableName, family, qualifier, permissions);
   }
 
   /**
@@ -461,28 +432,10 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if user has no authorization
    */
-  private void requireTablePermission(String request, TableName tableName, byte[] family,
+  public void requireTablePermission(String request, TableName tableName, byte[] family,
       byte[] qualifier, Action... permissions) throws IOException {
-    User user = getActiveUser();
-    AuthResult result = null;
-
-    for (Action permission : permissions) {
-      if (authManager.authorize(user, tableName, null, null, permission)) {
-        result = AuthResult.allow(request, "Table permission granted", user,
-            permission, tableName, null, null);
-        result.getParams().setFamily(family).setQualifier(qualifier);
-        break;
-      } else {
-        // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions", user,
-            permission, tableName, family, qualifier);
-        result.getParams().setFamily(family).setQualifier(qualifier);
-      }
-    }
-    logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
-    }
+    accessChecker.requireTablePermission(getActiveUser(), request,
+        tableName, family, qualifier, permissions);
   }
 
   /**
@@ -493,26 +446,9 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if user has no authorization
    */
-  private void requireAccess(String request, TableName tableName,
+  public void requireAccess(String request, TableName tableName,
       Action... permissions) throws IOException {
-    User user = getActiveUser();
-    AuthResult result = null;
-
-    for (Action permission : permissions) {
-      if (authManager.hasAccess(user, tableName, permission)) {
-        result = AuthResult.allow(request, "Table permission granted", user,
-                                  permission, tableName, null, null);
-        break;
-      } else {
-        // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions", user,
-                                 permission, tableName, null, null);
-      }
-    }
-    logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
-    }
+    accessChecker.requireAccess(getActiveUser(), request, tableName, permissions);
   }
 
   /**
@@ -521,8 +457,8 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if authorization is denied
    */
-  private void requirePermission(String request, Action perm) throws IOException {
-    requireGlobalPermission(request, perm, null, null);
+  public void requirePermission(String request, Action perm) throws IOException {
+    accessChecker.requirePermission(getActiveUser(), request, perm);
   }
 
   /**
@@ -533,24 +469,9 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @param tableName Affected table name.
    * @param familyMap Affected column families.
    */
-  private void requireGlobalPermission(String request, Action perm, TableName tableName,
+  public void requireGlobalPermission(String request, Action perm, TableName tableName,
       Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
-    User user = getActiveUser();
-    AuthResult result = null;
-    if (authManager.authorize(user, perm)) {
-      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.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() + ")");
-      }
-    }
+    accessChecker.requireGlobalPermission(getActiveUser(), request, perm, tableName, familyMap);
   }
 
   /**
@@ -558,85 +479,33 @@ public class AccessController extends BaseMasterAndRegionObserver
    * audit log message will contain context information for the operation
    * being authorized, based on the given parameters.
    * @param perm Action being requested
-   * @param namespace
+   * @param namespace  The given namespace
    */
-  private void requireGlobalPermission(String request, Action perm,
-                                       String namespace) throws IOException {
-    User user = getActiveUser();
-    AuthResult authResult = null;
-    if (authManager.authorize(user, perm)) {
-      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.getParams().setNamespace(namespace);
-      logResult(authResult);
-      if (authorizationEnabled) {
-        throw new AccessDeniedException("Insufficient permissions for user '" +
-          (user != null ? user.getShortName() : "null") +"' (global, action=" +
-          perm.toString() + ")");
-      }
-    }
+  public void requireGlobalPermission(String request, Action perm,
+      String namespace) throws IOException {
+    accessChecker.requireGlobalPermission(getActiveUser(), request, perm, namespace);
   }
 
   /**
    * Checks that the user has the given global or namespace permission.
-   * @param namespace
+   * @param namespace  The given namespace
    * @param permissions Actions being requested
    */
   public void requireNamespacePermission(String request, String namespace,
       Action... permissions) throws IOException {
-    User user = getActiveUser();
-    AuthResult result = null;
-
-    for (Action permission : permissions) {
-      if (authManager.authorize(user, namespace, permission)) {
-        result = AuthResult.allow(request, "Namespace permission granted",
-            user, permission, namespace);
-        break;
-      } else {
-        // rest of the world
-        result = AuthResult.deny(request, "Insufficient permissions", user,
-            permission, namespace);
-      }
-    }
-    logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions "
-          + result.toContextString());
-    }
+    accessChecker.requireNamespacePermission(getActiveUser(), request, namespace, permissions);
   }
 
   /**
    * Checks that the user has the given global or namespace permission.
-   * @param namespace
+   * @param namespace   The given namespace
    * @param permissions Actions being requested
    */
   public void requireNamespacePermission(String request, String namespace, TableName tableName,
       Map<byte[], ? extends Collection<byte[]>> familyMap, Action... permissions)
       throws IOException {
-    User user = getActiveUser();
-    AuthResult result = null;
-
-    for (Action permission : permissions) {
-      if (authManager.authorize(user, namespace, permission)) {
-        result = 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.getParams().setTableName(tableName).setFamilies(familyMap);
-      }
-    }
-    logResult(result);
-    if (authorizationEnabled && !result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions "
-          + result.toContextString());
-    }
+    accessChecker.requireNamespacePermission(getActiveUser(), request, namespace,
+        tableName, familyMap, permissions);
   }
 
   /**
@@ -661,13 +530,13 @@ public class AccessController extends BaseMasterAndRegionObserver
           familyMap.entrySet()) {
         if (family.getValue() != null && !family.getValue().isEmpty()) {
           for (byte[] qualifier : family.getValue()) {
-            if (authManager.matchPermission(user, tableName,
+            if (getAuthManager().matchPermission(user, tableName,
                 family.getKey(), qualifier, perm)) {
               return true;
             }
           }
         } else {
-          if (authManager.matchPermission(user, tableName, family.getKey(),
+          if (getAuthManager().matchPermission(user, tableName, family.getKey(),
               perm)) {
             return true;
           }
@@ -769,7 +638,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           }
         }
       } else if (entry.getValue() == null) {
-        get.addFamily(col);        
+        get.addFamily(col);
       } else {
         throw new RuntimeException("Unhandled collection type " +
           entry.getValue().getClass().getName());
@@ -859,7 +728,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           foundColumn = true;
           for (Action action: actions) {
             // Are there permissions for this user for the cell?
-            if (!authManager.authorize(user, getTableName(e), cell, action)) {
+            if (!getAuthManager().authorize(user, getTableName(e), cell, action)) {
               // We can stop if the cell ACL denies access
               return false;
             }
@@ -974,26 +843,14 @@ public class AccessController extends BaseMasterAndRegionObserver
 
     // set the user-provider.
     this.userProvider = UserProvider.instantiate(env.getConfiguration());
-
-    // If zk is null or IOException while obtaining auth manager,
-    // throw RuntimeException so that the coprocessor is unloaded.
-    if (zk != null) {
-      try {
-        this.authManager = TableAuthManager.getOrCreate(zk, env.getConfiguration());
-      } catch (IOException ioe) {
-        throw new RuntimeException("Error obtaining TableAuthManager", ioe);
-      }
-    } else {
-      throw new RuntimeException("Error obtaining TableAuthManager, zk found null.");
-    }
-
+    accessChecker = new AccessChecker(env.getConfiguration(), zk);
     tableAcls = new MapMaker().weakValues().makeMap();
   }
 
   @Override
   public void stop(CoprocessorEnvironment env) {
-    if (this.authManager != null) {
-      TableAuthManager.release(authManager);
+    if (getAuthManager()!= null) {
+      TableAuthManager.release(getAuthManager());
     }
   }
 
@@ -1072,7 +929,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         return null;
       }
     });
-    this.authManager.getZKPermissionWatcher().deleteTableACLNode(tableName);
+    getAuthManager().getZKPermissionWatcher().deleteTableACLNode(tableName);
   }
 
   @Override
@@ -1320,25 +1177,29 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("snapshot", hTableDescriptor.getTableName(), null, null,
+    requirePermission("snapshot " + snapshot.getName(), hTableDescriptor.getTableName(), null, null,
       Permission.Action.ADMIN);
   }
 
   @Override
   public void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot) throws IOException {
-    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
+    User user = getActiveUser();
+    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
       // list it, if user is the owner of snapshot
+      AuthResult result = AuthResult.allow("listSnapshot " + snapshot.getName(),
+          "Snapshot owner check allowed", user, null, null, null);
+      accessChecker.logResult(result);
     } else {
-      requirePermission("listSnapshot", Action.ADMIN);
+      requirePermission("listSnapshot " + snapshot.getName(), Action.ADMIN);
     }
   }
-  
+
   @Override
   public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("clone", Action.ADMIN);
+    requirePermission("cloneSnapshot " + snapshot.getName(), Action.ADMIN);
   }
 
   @Override
@@ -1346,21 +1207,24 @@ public class AccessController extends BaseMasterAndRegionObserver
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
     if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
-      requirePermission("restoreSnapshot", hTableDescriptor.getTableName(), null, null,
-        Permission.Action.ADMIN);
+      requirePermission("restoreSnapshot " + snapshot.getName(), hTableDescriptor.getTableName(),
+        null, null, Permission.Action.ADMIN);
     } else {
-      requirePermission("restore", Action.ADMIN);
+      requirePermission("restoreSnapshot " + snapshot.getName(), Action.ADMIN);
     }
   }
 
   @Override
   public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot) throws IOException {
-    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
+    User user = getActiveUser();
+    if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
       // Snapshot owner is allowed to delete the snapshot
-      // TODO: We are not logging this for audit
+      AuthResult result = AuthResult.allow("deleteSnapshot " + snapshot.getName(),
+          "Snapshot owner check allowed", user, null, null, null);
+      accessChecker.logResult(result);
     } else {
-      requirePermission("deleteSnapshot", Action.ADMIN);
+      requirePermission("deleteSnapshot " + snapshot.getName(), Action.ADMIN);
     }
   }
 
@@ -1388,14 +1252,14 @@ public class AccessController extends BaseMasterAndRegionObserver
         return null;
       }
     });
-    this.authManager.getZKPermissionWatcher().deleteNamespaceACLNode(namespace);
+    getAuthManager().getZKPermissionWatcher().deleteNamespaceACLNode(namespace);
     LOG.info(namespace + " entry deleted in " + AccessControlLists.ACL_TABLE_NAME + " table.");
   }
 
   @Override
   public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
       NamespaceDescriptor ns) throws IOException {
-    // We require only global permission so that 
+    // We require only global permission so that
     // a user with NS admin cannot altering namespace configurations. i.e. namespace quota
     requireGlobalPermission("modifyNamespace", Action.ADMIN, ns.getName());
   }
@@ -1526,7 +1390,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         families, HConstants.LATEST_TIMESTAMP, Action.READ));
       authResult.setReason("Covering cell set");
     }
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (authorizationEnabled && !authResult.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " +
         authResult.toContextString());
@@ -1576,7 +1440,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           authResult.setReason("Access allowed with filter");
           // Only wrap the filter if we are enforcing authorizations
           if (authorizationEnabled) {
-            Filter ourFilter = new AccessControlFilter(authManager, user, table,
+            Filter ourFilter = new AccessControlFilter(getAuthManager(), user, table,
               AccessControlFilter.Strategy.CHECK_TABLE_AND_CF_ONLY,
               cfVsMaxVersions);
             // wrap any existing filter
@@ -1606,7 +1470,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         authResult.setReason("Access allowed with filter");
         // Only wrap the filter if we are enforcing authorizations
         if (authorizationEnabled) {
-          Filter ourFilter = new AccessControlFilter(authManager, user, table,
+          Filter ourFilter = new AccessControlFilter(getAuthManager(), user, table,
             AccessControlFilter.Strategy.CHECK_CELL_DEFAULT, cfVsMaxVersions);
           // wrap any existing filter
           if (filter != null) {
@@ -1628,7 +1492,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
 
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (authorizationEnabled && !authResult.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions for user '"
           + (user != null ? user.getShortName() : "null")
@@ -1665,7 +1529,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<Cell>> families = put.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.PUT, user, env, families, Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         put.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -1710,7 +1574,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     Map<byte[],? extends Collection<Cell>> families = delete.getFamilyCellMap();
     User user = getActiveUser();
     AuthResult authResult = permissionGranted(OpType.DELETE, user, env, families, Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         delete.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -1747,7 +1611,7 @@ public class AccessController extends BaseMasterAndRegionObserver
             authResult = AuthResult.deny(opType.toString(), "Covering cell set",
               getActiveUser(), Action.WRITE, table, m.getFamilyCellMap());
           }
-          logResult(authResult);
+          accessChecker.logResult(authResult);
           if (authorizationEnabled && !authResult.isAllowed()) {
             throw new AccessDeniedException("Insufficient permissions "
               + authResult.toContextString());
@@ -1780,7 +1644,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
     AuthResult authResult = permissionGranted(OpType.CHECK_AND_PUT, user, env, families,
       Action.READ, Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         put.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -1820,7 +1684,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         authResult = AuthResult.deny(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
             getActiveUser(), Action.READ, table, families);
       }
-      logResult(authResult);
+      accessChecker.logResult(authResult);
       if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
       }
@@ -1846,7 +1710,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     User user = getActiveUser();
     AuthResult authResult = permissionGranted(OpType.CHECK_AND_DELETE, user, env, families,
         Action.READ, Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         delete.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -1878,7 +1742,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         authResult = AuthResult.deny(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
             getActiveUser(), Action.READ, table, families);
       }
-      logResult(authResult);
+      accessChecker.logResult(authResult);
       if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
       }
@@ -1903,7 +1767,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
       authResult.setReason("Covering cell set");
     }
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (authorizationEnabled && !authResult.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
     }
@@ -1920,7 +1784,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<Cell>> families = append.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.APPEND, user, env, families, Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         append.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -1958,7 +1822,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         authResult = AuthResult.deny(OpType.APPEND.toString(), "Covering cell set",
             getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
       }
-      logResult(authResult);
+      accessChecker.logResult(authResult);
       if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " +
           authResult.toContextString());
@@ -1980,7 +1844,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     Map<byte[],? extends Collection<Cell>> families = increment.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.INCREMENT, user, env, families,
       Action.WRITE);
-    logResult(authResult);
+    accessChecker.logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         increment.setAttribute(CHECK_COVERING_PERM, TRUE);
@@ -2018,7 +1882,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         authResult = AuthResult.deny(OpType.INCREMENT.toString(), "Covering cell set",
             getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
       }
-      logResult(authResult);
+      accessChecker.logResult(authResult);
       if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " +
           authResult.toContextString());
@@ -2221,7 +2085,7 @@ public class AccessController extends BaseMasterAndRegionObserver
 
   @Override
   public void grant(RpcController controller,
-                    AccessControlProtos.GrantRequest request,
+                    final AccessControlProtos.GrantRequest request,
                     RpcCallback<AccessControlProtos.GrantResponse> done) {
     final UserPermission perm = ProtobufUtil.toUserPermission(request.getUserPermission());
     AccessControlProtos.GrantResponse response = null;
@@ -2250,7 +2114,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           @Override
           public Void run() throws Exception {
             AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm,
-                regionEnv.getTable(AccessControlLists.ACL_TABLE_NAME));
+              regionEnv.getTable(AccessControlLists.ACL_TABLE_NAME));
             return null;
           }
         });
@@ -2421,7 +2285,7 @@ public class AccessController extends BaseMasterAndRegionObserver
 
             AuthResult result = permissionGranted("checkPermissions", user, action, regionEnv,
               familyMap);
-            logResult(result);
+            accessChecker.logResult(result);
             if (!result.isAllowed()) {
               // Even if passive we need to throw an exception here, we support checking
               // effective permissions, so throw unconditionally
@@ -2436,14 +2300,14 @@ public class AccessController extends BaseMasterAndRegionObserver
 
           for (Action action : permission.getActions()) {
             AuthResult result;
-            if (authManager.authorize(user, action)) {
+            if (getAuthManager().authorize(user, action)) {
               result = AuthResult.allow("checkPermissions", "Global action allowed", user,
                 action, null, null);
             } else {
               result = AuthResult.deny("checkPermissions", "Global action denied", user, action,
                 null, null);
             }
-            logResult(result);
+            accessChecker.logResult(result);
             if (!result.isAllowed()) {
               // Even if passive we need to throw an exception here, we support checking
               // effective permissions, so throw unconditionally
@@ -2640,7 +2504,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void postReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
       List<WALEntry> entries, CellScanner cells) throws IOException {
   }
-  
+
   @Override
   public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final String userName, final Quotas quotas) throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
index dd1059c..a12757d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
@@ -758,7 +758,7 @@ public class TableAuthManager implements Closeable {
   }
 
   @VisibleForTesting
-  static int getTotalRefCount() {
+  public static int getTotalRefCount() {
     int total = 0;
     for (int count : refCount.values()) {
       total += count;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 22a9748..cbfae0e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -168,7 +168,7 @@ public class SecureTestUtil {
    * To indicate the action was not allowed, either throw an AccessDeniedException
    * or return an empty list of KeyValues.
    */
-  static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+  public static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
 
   /** This fails only in case of ADE or empty list for any of the actions. */
   public static void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
@@ -440,7 +440,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.grant(connection, namespace, user, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("grant failed: ", t);
         }
         return null;
       }
@@ -461,7 +461,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.revoke(connection, namespace, user, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("revoke failed: ", t);
         }
         return null;
       }
@@ -529,7 +529,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.grant(connection, table, user, family, qualifier, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("grant failed: ", t);
         }
         return null;
       }
@@ -550,7 +550,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.grant(connection, user, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("grant failed: ", t);
         }
         return null;
       }
@@ -595,7 +595,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.revoke(connection, table, user, family, qualifier, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("revoke failed: ", t);
         }
         return null;
       }
@@ -616,7 +616,7 @@ public class SecureTestUtil {
         try {
           AccessControlClient.revoke(connection, user, actions);
         } catch (Throwable t) {
-          t.printStackTrace();
+          LOG.error("revoke failed: ", t);
         }
         return null;
       }
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 da2b42c..dee669b 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
@@ -2052,7 +2052,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
-          null, null);
+          snapshot, null);
         return null;
       }
     };
@@ -2123,7 +2123,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
-          null, null);
+          snapshot, null);
         return null;
       }
     };
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index d400fa6..ec8bc95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -381,6 +381,7 @@ public class TestAccessController2 extends SecureTestUtil {
           Scan s1 = new Scan();
           s1.addFamily(TEST_FAMILY_2);
           try (ResultScanner scanner1 = table.getScanner(s1);) {
+            scanner1.next();
           }
         }
         return null;
@@ -411,6 +412,7 @@ public class TestAccessController2 extends SecureTestUtil {
           Scan s1 = new Scan();
           s1.addFamily(TEST_FAMILY_2);
           try (ResultScanner scanner1 = table.getScanner(s1);) {
+            scanner1.next();
           }
         }
         return null;
@@ -425,6 +427,7 @@ public class TestAccessController2 extends SecureTestUtil {
           Scan s1 = new Scan();
           s1.addColumn(TEST_FAMILY, Q2);
           try (ResultScanner scanner1 = table.getScanner(s1);) {
+            scanner1.next();
           }
         }
         return null;