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 2014/05/01 03:04:36 UTC

svn commit: r1591524 [1/2] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/

Author: apurtell
Date: Thu May  1 01:04:36 2014
New Revision: 1591524

URL: http://svn.apache.org/r1591524
Log:
HBASE-11077 [AccessController] Restore compatible early-out access denial

Added:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java
Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthResult.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlConstants.java Thu May  1 01:04:36 2014
@@ -25,6 +25,22 @@ import org.apache.hadoop.classification.
 @InterfaceStability.Evolving
 public interface AccessControlConstants {
 
+  /**
+   * Configuration option that toggles whether EXEC permission checking is
+   * performed during coprocessor endpoint invocations.
+   */
+  public static final String EXEC_PERMISSION_CHECKS_KEY = "hbase.security.exec.permission.checks";
+  /** Default setting for hbase.security.exec.permission.checks; false */
+  public static final boolean DEFAULT_EXEC_PERMISSION_CHECKS = false;
+
+  /**
+   * Configuration or CF schema option for early termination of access checks
+   * if table or CF permissions grant access. Pre-0.98 compatible behavior
+   */
+  public static final String CF_ATTRIBUTE_EARLY_OUT = "hbase.security.access.early_out";
+  /** Default setting for hbase.security.access.early_out */
+  public static final boolean DEFAULT_ATTRIBUTE_EARLY_OUT = true;
+
   // Operation attributes for cell level security
 
   /** Cell level ACL */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java Thu May  1 01:04:36 2014
@@ -22,10 +22,10 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -48,11 +48,20 @@ import org.apache.hadoop.hbase.util.Simp
  */
 class AccessControlFilter extends FilterBase {
 
+  public static enum Strategy {
+    /** Filter only by checking the table or CF permissions */
+    CHECK_TABLE_AND_CF_ONLY,
+    /** Cell permissions can override table or CF permissions */
+    CHECK_CELL_DEFAULT,
+    /** Cell permissions must authorize */
+    CHECK_CELL_FIRST,
+  };
+
   private TableAuthManager authManager;
   private TableName table;
   private User user;
   private boolean isSystemTable;
-  private boolean cellFirstStrategy;
+  private Strategy strategy;
   private Map<ByteRange, Integer> cfVsMaxVersions;
   private int familyMaxVersions;
   private int currentVersions;
@@ -66,12 +75,12 @@ class AccessControlFilter extends Filter
   }
 
   AccessControlFilter(TableAuthManager mgr, User ugi, TableName tableName,
-      boolean cellFirstStrategy, Map<ByteRange, Integer> cfVsMaxVersions) {
+      Strategy strategy, Map<ByteRange, Integer> cfVsMaxVersions) {
     authManager = mgr;
     table = tableName;
     user = ugi;
     isSystemTable = tableName.isSystemTable();
-    this.cellFirstStrategy = cellFirstStrategy;
+    this.strategy = strategy;
     this.cfVsMaxVersions = cfVsMaxVersions;
     this.prevFam = new SimpleByteRange();
     this.prevQual = new SimpleByteRange();
@@ -103,12 +112,37 @@ class AccessControlFilter extends Filter
     if (currentVersions > familyMaxVersions) {
       return ReturnCode.SKIP;
     }
-    if (authManager.authorize(user, table, cell, cellFirstStrategy, Permission.Action.READ)) {
-      return ReturnCode.INCLUDE;
+    // XXX: Compare in place, don't clone
+    byte[] family = CellUtil.cloneFamily(cell);
+    byte[] qualifier = CellUtil.cloneQualifier(cell);
+    switch (strategy) {
+      // Filter only by checking the table or CF permissions
+      case CHECK_TABLE_AND_CF_ONLY: {
+        if (authManager.authorize(user, table, family, qualifier, Permission.Action.READ)) {
+          return ReturnCode.INCLUDE;
+        }
+      }
+      break;
+      // Cell permissions can override table or CF permissions
+      case CHECK_CELL_DEFAULT: {
+        if (authManager.authorize(user, table, family, qualifier, Permission.Action.READ) ||
+            authManager.authorize(user, table, cell, Permission.Action.READ)) {
+          return ReturnCode.INCLUDE;
+        }
+      }
+      break;
+      // Cell permissions must authorize
+      case CHECK_CELL_FIRST: {
+        if (authManager.authorize(user, table, cell, Permission.Action.READ) &&
+            authManager.authorize(user, table, family, qualifier, Permission.Action.READ)) {
+          return ReturnCode.INCLUDE;
+        }
+      }
+      break;
+      default:
+        throw new RuntimeException("Unhandled strategy " + strategy);
     }
-    // Before per cell ACLs we used to return the NEXT_COL hint, but we can
-    // no longer do that since, given the possibility of per cell ACLs
-    // anywhere, we now need to examine all KVs with this filter.
+
     return ReturnCode.SKIP;
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Thu May  1 01:04:36 2014
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -103,6 +104,7 @@ import com.google.common.collect.Immutab
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.MapMaker;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
@@ -149,9 +151,6 @@ public class AccessController extends Ba
   private static final Log AUDITLOG =
     LogFactory.getLog("SecurityLogger."+AccessController.class.getName());
 
-  static final String EXEC_PERMISSION_CHECKS_KEY = "hbase.security.exec.permission.checks";
-  static final boolean DEFAULT_EXEC_PERMISSION_CHECKS = false;
-
   TableAuthManager authManager = null;
 
   // flags if we are running on a region of the _acl_ table
@@ -167,11 +166,15 @@ public class AccessController extends Ba
 
   private UserProvider userProvider;
 
-  // flags if we are able to support cell ACLs
-  boolean canPersistCellACLs;
+  // if we are able to support cell ACLs
+  boolean cellFeaturesEnabled;
+
+  // if we should check EXEC permissions
+  boolean shouldCheckExecPermission;
 
-  // flags if we should check EXEC permissions
-  boolean shouldCheckExecPermissions;
+  // if we should terminate access checks early as soon as table or CF grants
+  // allow access; pre-0.98 compatible behavior
+  boolean compatibleEarlyTermination;
 
   private volatile boolean initialized = false;
 
@@ -185,6 +188,7 @@ public class AccessController extends Ba
 
   void initialize(RegionCoprocessorEnvironment e) throws IOException {
     final HRegion region = e.getRegion();
+    Configuration conf = e.getConfiguration();
     Map<byte[], ListMultimap<String,TablePermission>> tables =
         AccessControlLists.loadAll(region);
     // For each table, write out the table's permissions to the respective
@@ -193,11 +197,9 @@ public class AccessController extends Ba
       tables.entrySet()) {
       byte[] entry = t.getKey();
       ListMultimap<String,TablePermission> perms = t.getValue();
-      byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, e.getConfiguration());
+      byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
       this.authManager.getZKPermissionWatcher().writeToZookeeper(entry, serialized);
     }
-    shouldCheckExecPermissions = e.getConfiguration().getBoolean(EXEC_PERMISSION_CHECKS_KEY,
-      DEFAULT_EXEC_PERMISSION_CHECKS);
     initialized = true;
   }
 
@@ -250,7 +252,7 @@ public class AccessController extends Ba
    * the request
    * @return an authorization result
    */
-  AuthResult permissionGranted(String request, User user, Permission.Action permRequest,
+  AuthResult permissionGranted(String request, User user, Action permRequest,
       RegionCoprocessorEnvironment e,
       Map<byte [], ? extends Collection<?>> families) {
     HRegionInfo hri = e.getRegion().getRegionInfo();
@@ -259,7 +261,7 @@ public class AccessController extends Ba
     // 1. All users need read access to hbase:meta table.
     // this is a very common operation, so deal with it quickly.
     if (hri.isMetaRegion()) {
-      if (permRequest == Permission.Action.READ) {
+      if (permRequest == Action.READ) {
         return AuthResult.allow(request, "All users allowed", user,
           permRequest, tableName, families);
       }
@@ -275,11 +277,11 @@ public class AccessController extends Ba
     // so the user need to be allowed to write on it.
     // e.g. When a table is removed an entry is removed from hbase:meta and _acl_
     // and the user need to be allowed to write on both tables.
-    if (permRequest == Permission.Action.WRITE &&
+    if (permRequest == Action.WRITE &&
        (hri.isMetaRegion() ||
         Bytes.equals(tableName.getName(), AccessControlLists.ACL_GLOBAL_NAME)) &&
-       (authManager.authorize(user, Permission.Action.CREATE) ||
-        authManager.authorize(user, Permission.Action.ADMIN)))
+       (authManager.authorize(user, Action.CREATE) ||
+        authManager.authorize(user, Action.ADMIN)))
     {
        return AuthResult.allow(request, "Table permission granted", user,
         permRequest, tableName, families);
@@ -340,6 +342,29 @@ public class AccessController extends Ba
         user, permRequest, tableName, families);
   }
 
+  /**
+   * Check the current user for authorization to perform a specific action
+   * against the given set of row data.
+   * @param opType the operation type
+   * @param user the user
+   * @param e the coprocessor environment
+   * @param families the map of column families to qualifiers present in
+   * the request
+   * @param actions the desired actions
+   * @return an authorization result
+   */
+  AuthResult permissionGranted(OpType opType, User user, RegionCoprocessorEnvironment e,
+      Map<byte [], ? extends Collection<?>> families, Action... actions) {
+    AuthResult result = null;
+    for (Action action: actions) {
+      result = permissionGranted(opType.toString(), user, action, e, families);
+      if (!result.isAllowed()) {
+        return result;
+      }
+    }
+    return result;
+  }
+
   private void logResult(AuthResult result) {
     if (AUDITLOG.isTraceEnabled()) {
       RequestContext ctx = RequestContext.get();
@@ -407,7 +432,7 @@ public class AccessController extends Ba
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if authorization is denied
    */
-  private void requirePermission(String request, Permission.Action perm) throws IOException {
+  private void requirePermission(String request, Action perm) throws IOException {
     requireGlobalPermission(request, perm, null, null);
   }
 
@@ -419,7 +444,7 @@ public class AccessController extends Ba
    * @param families The map of column families-qualifiers.
    * @throws AccessDeniedException if the authorization check failed
    */
-  private void requirePermission(String request, Permission.Action perm,
+  private void requirePermission(String request, Action perm,
         RegionCoprocessorEnvironment env,
         Map<byte[], ? extends Collection<?>> families)
       throws IOException {
@@ -444,7 +469,7 @@ public class AccessController extends Ba
    * @param tableName Affected table name.
    * @param familyMap Affected column families.
    */
-  private void requireGlobalPermission(String request, Permission.Action perm, TableName tableName,
+  private void requireGlobalPermission(String request, Action perm, TableName tableName,
       Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
     User user = getActiveUser();
     if (authManager.authorize(user, perm)) {
@@ -464,7 +489,7 @@ public class AccessController extends Ba
    * @param perm Action being requested
    * @param namespace
    */
-  private void requireGlobalPermission(String request, Permission.Action perm,
+  private void requireGlobalPermission(String request, Action perm,
                                        String namespace) throws IOException {
     User user = getActiveUser();
     if (authManager.authorize(user, perm)) {
@@ -477,8 +502,52 @@ public class AccessController extends Ba
     }
   }
 
+  /**
+   * Returns <code>true</code> if the current user is allowed the given action
+   * over at least one of the column qualifiers in the given column families.
+   */
+  private boolean hasFamilyQualifierPermission(User user,
+      Action perm,
+      RegionCoprocessorEnvironment env,
+      Map<byte[], ? extends Collection<byte[]>> familyMap)
+    throws IOException {
+    HRegionInfo hri = env.getRegion().getRegionInfo();
+    TableName tableName = hri.getTable();
+
+    if (user == null) {
+      return false;
+    }
+
+    if (familyMap != null && familyMap.size() > 0) {
+      // at least one family must be allowed
+      for (Map.Entry<byte[], ? extends Collection<byte[]>> family :
+          familyMap.entrySet()) {
+        if (family.getValue() != null && !family.getValue().isEmpty()) {
+          for (byte[] qualifier : family.getValue()) {
+            if (authManager.matchPermission(user, tableName,
+                family.getKey(), qualifier, perm)) {
+              return true;
+            }
+          }
+        } else {
+          if (authManager.matchPermission(user, tableName, family.getKey(),
+              perm)) {
+            return true;
+          }
+        }
+      }
+    } else if (LOG.isDebugEnabled()) {
+      LOG.debug("Empty family map passed for permission check");
+    }
+
+    return false;
+  }
+
   private enum OpType {
     GET_CLOSEST_ROW_BEFORE("getClosestRowBefore"),
+    GET("get"),
+    EXISTS("exists"),
+    SCAN("scan"),
     PUT("put"),
     DELETE("delete"),
     CHECK_AND_PUT("checkAndPut"),
@@ -499,220 +568,175 @@ public class AccessController extends Ba
     }
   }
 
-  private void requireCoveringPermission(OpType request, RegionCoprocessorEnvironment e,
+  /**
+   * Determine if cell ACLs covered by the operation grant access. This is expensive.
+   * @return false if cell ACLs failed to grant access, true otherwise
+   * @throws IOException
+   */
+  private boolean checkCoveringPermission(OpType request, RegionCoprocessorEnvironment e,
       byte[] row, Map<byte[], ? extends Collection<?>> familyMap, long opTs, Action... actions)
       throws IOException {
-    User user = getActiveUser();
-
-    // First check table or CF level permissions, if they grant access we can
-    // early out before needing to enumerate over per KV perms.
-
-    List<Action> cellCheckActions = Lists.newArrayList();
-    // TODO: permissionGranted should support checking multiple actions or
-    // we should convert actions into a bitmap and pass that around. See
-    // HBASE-7123.
-    AuthResult results[] = new AuthResult[actions.length];
-    for (int i = 0; i < actions.length; i++) {
-      results[i] = permissionGranted(request.type, user, actions[i], e, familyMap);
-      if (!results[i].isAllowed()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Got " + results[i] + ", added to cellCheckActions");
-        }
-        cellCheckActions.add(actions[i]);
-      }
-    }
-    // If all permissions checks passed, we can early out
-    if (cellCheckActions.isEmpty()) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("All permissions checks passed, we can early out");
-      }
-      for (int i = 0; i < results.length; i++) {
-        logResult(results[i]);
-      }
-      return;
+    if (!cellFeaturesEnabled) {
+      return false;
     }
-
-    // Table or CF permissions do not allow, enumerate the covered KVs. We
-    // can stop at the first which does not grant access.
-    int cellsChecked = 0;
+    long cellGrants = 0;
+    User user = getActiveUser();
     long latestCellTs = 0;
-    if (canPersistCellACLs) {
-      Get get = new Get(row);
-      // Only in case of Put/Delete op, consider TS within cell (if set for individual cells).
-      // When every cell, within a Mutation, can be linked with diff TS we can not rely on only one
-      // version. We have to get every cell version and check its TS against the TS asked for in
-      // Mutation and skip those Cells which is outside this Mutation TS.In case of Put, we have to
-      // consider only one such passing cell. In case of Delete we have to consider all the cell
-      // versions under this passing version. When Delete Mutation contains columns which are a
-      // version delete just consider only one version for those column cells.
-      boolean considerCellTs  = (request == OpType.PUT || request == OpType.DELETE);
-      if (considerCellTs) {
-        get.setMaxVersions();
-      } else {
-        get.setMaxVersions(1);
-      }
-      boolean diffCellTsFromOpTs = false;
-      for (Map.Entry<byte[], ? extends Collection<?>> entry: familyMap.entrySet()) {
-        byte[] col = entry.getKey();
-        // TODO: HBASE-7114 could possibly unify the collection type in family
-        // maps so we would not need to do this
-        if (entry.getValue() instanceof Set) {
-          Set<byte[]> set = (Set<byte[]>)entry.getValue();
-          if (set == null || set.isEmpty()) {
-            get.addFamily(col);
-          } else {
-            for (byte[] qual: set) {
-              get.addColumn(col, qual);
-            }
+    Get get = new Get(row);
+    // Only in case of Put/Delete op, consider TS within cell (if set for individual cells).
+    // When every cell, within a Mutation, can be linked with diff TS we can not rely on only one
+    // version. We have to get every cell version and check its TS against the TS asked for in
+    // Mutation and skip those Cells which is outside this Mutation TS.In case of Put, we have to
+    // consider only one such passing cell. In case of Delete we have to consider all the cell
+    // versions under this passing version. When Delete Mutation contains columns which are a
+    // version delete just consider only one version for those column cells.
+    boolean considerCellTs  = (request == OpType.PUT || request == OpType.DELETE);
+    if (considerCellTs) {
+      get.setMaxVersions();
+    } else {
+      get.setMaxVersions(1);
+    }
+    boolean diffCellTsFromOpTs = false;
+    for (Map.Entry<byte[], ? extends Collection<?>> entry: familyMap.entrySet()) {
+      byte[] col = entry.getKey();
+      // TODO: HBASE-7114 could possibly unify the collection type in family
+      // maps so we would not need to do this
+      if (entry.getValue() instanceof Set) {
+        Set<byte[]> set = (Set<byte[]>)entry.getValue();
+        if (set == null || set.isEmpty()) {
+          get.addFamily(col);
+        } else {
+          for (byte[] qual: set) {
+            get.addColumn(col, qual);
           }
-        } else if (entry.getValue() instanceof List) {
-          List<Cell> list = (List<Cell>)entry.getValue();
-          if (list == null || list.isEmpty()) {
-            get.addFamily(col);
-          } else {
-            // In case of family delete, a Cell will be added into the list with Qualifier as null.
-            for (Cell cell : list) {
-              if (cell.getQualifierLength() == 0
-                  && (cell.getTypeByte() == Type.DeleteFamily.getCode() 
-                  || cell.getTypeByte() == Type.DeleteFamilyVersion.getCode())) {
-                get.addFamily(col);
-              } else {
-                get.addColumn(col, CellUtil.cloneQualifier(cell));
-              }
-              if (considerCellTs) {
-                long cellTs = cell.getTimestamp();
-                latestCellTs = Math.max(latestCellTs, cellTs);
-                diffCellTsFromOpTs = diffCellTsFromOpTs || (opTs != cellTs);
-              }
+        }
+      } else if (entry.getValue() instanceof List) {
+        List<Cell> list = (List<Cell>)entry.getValue();
+        if (list == null || list.isEmpty()) {
+          get.addFamily(col);
+        } else {
+          // In case of family delete, a Cell will be added into the list with Qualifier as null.
+          for (Cell cell : list) {
+            if (cell.getQualifierLength() == 0
+                && (cell.getTypeByte() == Type.DeleteFamily.getCode() 
+                || cell.getTypeByte() == Type.DeleteFamilyVersion.getCode())) {
+              get.addFamily(col);
+            } else {
+              get.addColumn(col, CellUtil.cloneQualifier(cell));
+            }
+            if (considerCellTs) {
+              long cellTs = cell.getTimestamp();
+              latestCellTs = Math.max(latestCellTs, cellTs);
+              diffCellTsFromOpTs = diffCellTsFromOpTs || (opTs != cellTs);
             }
           }
-        } else {
-          throw new RuntimeException("Unhandled collection type " +
-            entry.getValue().getClass().getName());
         }
+      } else {
+        throw new RuntimeException("Unhandled collection type " +
+          entry.getValue().getClass().getName());
       }
-      // We want to avoid looking into the future. So, if the cells of the
-      // operation specify a timestamp, or the operation itself specifies a
-      // timestamp, then we use the maximum ts found. Otherwise, we bound
-      // the Get to the current server time. We add 1 to the timerange since
-      // the upper bound of a timerange is exclusive yet we need to examine
-      // any cells found there inclusively.
-      long latestTs = Math.max(opTs, latestCellTs);
-      if (latestTs == 0 || latestTs == HConstants.LATEST_TIMESTAMP) {
-        latestTs = EnvironmentEdgeManager.currentTimeMillis();
-      }
-      get.setTimeRange(0, latestTs + 1);
-      // In case of Put operation we set to read all versions. This was done to consider the case
-      // where columns are added with TS other than the Mutation TS. But normally this wont be the
-      // case with Put. There no need to get all versions but get latest version only.
-      if (!diffCellTsFromOpTs && request == OpType.PUT) {
-        get.setMaxVersions(1);
-      }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Scanning for cells with " + get);
-      }
-      // This Map is identical to familyMap. The key is a BR rather than byte[].
-      // It will be easy to do gets over this new Map as we can create get keys over the Cell cf by
-      // new SimpleByteRange(cell.familyArray, cell.familyOffset, cell.familyLen)
-      Map<ByteRange, List<Cell>> familyMap1 = new HashMap<ByteRange, List<Cell>>();
-      for (Entry<byte[], ? extends Collection<?>> entry : familyMap.entrySet()) {
-        if (entry.getValue() instanceof List) {
-          familyMap1.put(new SimpleByteRange(entry.getKey()), (List<Cell>) entry.getValue());
-        }
-      }
-      RegionScanner scanner = getRegion(e).getScanner(new Scan(get));
-      List<Cell> cells = Lists.newArrayList();
-      Cell prevCell = null;
-      ByteRange curFam = new SimpleByteRange();
-      boolean curColAllVersions = (request == OpType.DELETE);
-      long curColCheckTs = opTs;
-      boolean foundColumn = false;
-      try {
-        boolean more = false;
-        do {
-          cells.clear();
-          // scan with limit as 1 to hold down memory use on wide rows
-          more = scanner.next(cells, 1);
-          for (Cell cell: cells) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Found cell " + cell);
-            }
-            boolean colChange = prevCell == null || !CellUtil.matchingColumn(prevCell, cell);
-            if (colChange) foundColumn = false;
-            prevCell = cell;
-            if (!curColAllVersions && foundColumn) {
-              continue;
-            }
-            if (colChange && considerCellTs) {
-              curFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
-              List<Cell> cols = familyMap1.get(curFam);
-              for (Cell col : cols) {
-                // null/empty qualifier is used to denote a Family delete. The TS and delete type
-                // associated with this is applicable for all columns within the family. That is
-                // why the below (col.getQualifierLength() == 0) check.
-                if ((col.getQualifierLength() == 0 && request == OpType.DELETE)
-                    || CellUtil.matchingQualifier(cell, col)) {
-                  byte type = col.getTypeByte();
-                  if (considerCellTs)
-                    curColCheckTs = col.getTimestamp();
-                  // For a Delete op we pass allVersions as true. When a Delete Mutation contains
-                  // a version delete for a column no need to check all the covering cells within
-                  // that column. Check all versions when Type is DeleteColumn or DeleteFamily
-                  // One version delete types are Delete/DeleteFamilyVersion
-                  curColAllVersions = (KeyValue.Type.DeleteColumn.getCode() == type)
-                      || (KeyValue.Type.DeleteFamily.getCode() == type);
-                  break;
+    }
+    // We want to avoid looking into the future. So, if the cells of the
+    // operation specify a timestamp, or the operation itself specifies a
+    // timestamp, then we use the maximum ts found. Otherwise, we bound
+    // the Get to the current server time. We add 1 to the timerange since
+    // the upper bound of a timerange is exclusive yet we need to examine
+    // any cells found there inclusively.
+    long latestTs = Math.max(opTs, latestCellTs);
+    if (latestTs == 0 || latestTs == HConstants.LATEST_TIMESTAMP) {
+      latestTs = EnvironmentEdgeManager.currentTimeMillis();
+    }
+    get.setTimeRange(0, latestTs + 1);
+    // In case of Put operation we set to read all versions. This was done to consider the case
+    // where columns are added with TS other than the Mutation TS. But normally this wont be the
+    // case with Put. There no need to get all versions but get latest version only.
+    if (!diffCellTsFromOpTs && request == OpType.PUT) {
+      get.setMaxVersions(1);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Scanning for cells with " + get);
+    }
+    // This Map is identical to familyMap. The key is a BR rather than byte[].
+    // It will be easy to do gets over this new Map as we can create get keys over the Cell cf by
+    // new SimpleByteRange(cell.familyArray, cell.familyOffset, cell.familyLen)
+    Map<ByteRange, List<Cell>> familyMap1 = new HashMap<ByteRange, List<Cell>>();
+    for (Entry<byte[], ? extends Collection<?>> entry : familyMap.entrySet()) {
+      if (entry.getValue() instanceof List) {
+        familyMap1.put(new SimpleByteRange(entry.getKey()), (List<Cell>) entry.getValue());
+      }
+    }
+    RegionScanner scanner = getRegion(e).getScanner(new Scan(get));
+    List<Cell> cells = Lists.newArrayList();
+    Cell prevCell = null;
+    ByteRange curFam = new SimpleByteRange();
+    boolean curColAllVersions = (request == OpType.DELETE);
+    long curColCheckTs = opTs;
+    boolean foundColumn = false;
+    try {
+      boolean more = false;
+      do {
+        cells.clear();
+        // scan with limit as 1 to hold down memory use on wide rows
+        more = scanner.next(cells, 1);
+        for (Cell cell: cells) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Found cell " + cell);
+          }
+          boolean colChange = prevCell == null || !CellUtil.matchingColumn(prevCell, cell);
+          if (colChange) foundColumn = false;
+          prevCell = cell;
+          if (!curColAllVersions && foundColumn) {
+            continue;
+          }
+          if (colChange && considerCellTs) {
+            curFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
+            List<Cell> cols = familyMap1.get(curFam);
+            for (Cell col : cols) {
+              // null/empty qualifier is used to denote a Family delete. The TS and delete type
+              // associated with this is applicable for all columns within the family. That is
+              // why the below (col.getQualifierLength() == 0) check.
+              if ((col.getQualifierLength() == 0 && request == OpType.DELETE)
+                  || CellUtil.matchingQualifier(cell, col)) {
+                byte type = col.getTypeByte();
+                if (considerCellTs) {
+                  curColCheckTs = col.getTimestamp();
                 }
+                // For a Delete op we pass allVersions as true. When a Delete Mutation contains
+                // a version delete for a column no need to check all the covering cells within
+                // that column. Check all versions when Type is DeleteColumn or DeleteFamily
+                // One version delete types are Delete/DeleteFamilyVersion
+                curColAllVersions = (KeyValue.Type.DeleteColumn.getCode() == type)
+                    || (KeyValue.Type.DeleteFamily.getCode() == type);
+                break;
               }
             }
-            if (cell.getTimestamp() > curColCheckTs) {
-              // Just ignore this cell. This is not a covering cell.
-              continue;
-            }
-            foundColumn = true;
-            for (Action action: cellCheckActions) {
-              // Are there permissions for this user for the cell?
-              if (!authManager.authorize(user, getTableName(e), cell, false, action)) {
-                AuthResult authResult = AuthResult.deny(request.type, "Insufficient permissions",
-                  user, action, getTableName(e), CellUtil.cloneFamily(cell),
-                  CellUtil.cloneQualifier(cell));
-                logResult(authResult);
-                throw new AccessDeniedException("Insufficient permissions " +
-                  authResult.toContextString());
-              }
+          }
+          if (cell.getTimestamp() > curColCheckTs) {
+            // Just ignore this cell. This is not a covering cell.
+            continue;
+          }
+          foundColumn = true;
+          for (Action action: actions) {
+            // Are there permissions for this user for the cell?
+            if (!authManager.authorize(user, getTableName(e), cell, action)) {
+              // We can stop if the cell ACL denies access
+              return false;
             }
-            cellsChecked++;
           }
-        } while (more);
-      } catch (AccessDeniedException ex) {
-        throw ex;
-      } catch (IOException ex) {
-        LOG.error("Exception while getting cells to calculate covering permission", ex);
-      } finally {
-        scanner.close();
-      }
-    }
-
-    // If there were no cells to check, throw the ADE
-    if (cellsChecked < 1) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("No cells found with scan");
-      }
-      AuthResult authResult = AuthResult.deny(request.type, "Insufficient permissions",
-        user, cellCheckActions.get(0), getTableName(e), familyMap);
-      logResult(authResult);
-      throw new AccessDeniedException("Insufficient permissions " +
-        authResult.toContextString());
-    }
-
-    // Log that authentication succeeded. We need to trade off logging maybe
-    // thousands of fine grained decisions with providing detail.
-    for (byte[] family: familyMap.keySet()) {
-      for (Action action: actions) {
-        logResult(AuthResult.allow(request.type, "Permission granted", user, action,
-          getTableName(e), family, null));
-      }
-    }
+          cellGrants++;
+        }
+      } while (more);
+    } catch (AccessDeniedException ex) {
+      throw ex;
+    } catch (IOException ex) {
+      LOG.error("Exception while getting cells to calculate covering permission", ex);
+    } finally {
+      scanner.close();
+    }
+    // We should not authorize unless we have found one or more cell ACLs that
+    // grant access. This code is used to check for additional permissions
+    // after no table or CF grants are found.
+    return cellGrants > 0;
   }
 
   private void addCellPermissions(final byte[] perms, Map<byte[], List<Cell>> familyMap) {
@@ -743,54 +767,37 @@ public class AccessController extends Ba
     }
   }
 
-  private void internalPreRead(final ObserverContext<RegionCoprocessorEnvironment> c,
-      final Query query) throws IOException {
-    TableName tableName = getTableName(c.getEnvironment());
-    User activeUser = getActiveUser();
-    Filter filter = query.getFilter();
-    boolean cellFirstStrategy = query.getACLStrategy();
-    // Don't wrap an AccessControlFilter
-    if (filter != null && filter instanceof AccessControlFilter) {
-      return;
-    }
-    Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<ByteRange, Integer>();
-    HRegion region = c.getEnvironment().getRegion();
-    for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {
-      cfVsMaxVersions.put(new SimpleByteRange(hcd.getName()), hcd.getMaxVersions());
-    }
-    Filter newFilter = (filter != null)
-      ? new FilterList(FilterList.Operator.MUST_PASS_ALL,
-          Lists.newArrayList(
-              new AccessControlFilter(authManager, activeUser, tableName,
-            cellFirstStrategy, cfVsMaxVersions),
-            filter))
-      : new AccessControlFilter(authManager, activeUser, tableName,
-          cellFirstStrategy, cfVsMaxVersions);
-    query.setFilter(newFilter);
-  }
-
   /* ---- MasterObserver implementation ---- */
 
   public void start(CoprocessorEnvironment env) throws IOException {
-    canPersistCellACLs = HFile.getFormatVersion(env.getConfiguration()) >=
-      HFile.MIN_FORMAT_VERSION_WITH_TAGS;
-    if (!canPersistCellACLs) {
+    CompoundConfiguration conf = new CompoundConfiguration();
+    conf.add(env.getConfiguration());
+
+    shouldCheckExecPermission = conf.getBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY,
+      AccessControlConstants.DEFAULT_EXEC_PERMISSION_CHECKS);
+
+    cellFeaturesEnabled = HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS;
+    if (!cellFeaturesEnabled) {
       LOG.info("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
           + " is required to persist cell ACLs. Consider setting " + HFile.FORMAT_VERSION_KEY
           + " accordingly.");
     }
+
     ZooKeeperWatcher zk = null;
     if (env instanceof MasterCoprocessorEnvironment) {
       // if running on HMaster
       MasterCoprocessorEnvironment mEnv = (MasterCoprocessorEnvironment) env;
-      zk = mEnv.getMasterServices().getZooKeeper();      
+      zk = mEnv.getMasterServices().getZooKeeper();
     } else if (env instanceof RegionServerCoprocessorEnvironment) {      
       RegionServerCoprocessorEnvironment rsEnv = (RegionServerCoprocessorEnvironment) env;
       zk = rsEnv.getRegionServerServices().getZooKeeper();      
     } else if (env instanceof RegionCoprocessorEnvironment) {
       // if running at region
       regionEnv = (RegionCoprocessorEnvironment) env;
+      conf.addStringMap(regionEnv.getRegion().getTableDesc().getConfiguration());
       zk = regionEnv.getRegionServerServices().getZooKeeper();
+      compatibleEarlyTermination = conf.getBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT,
+        AccessControlConstants.DEFAULT_ATTRIBUTE_EARLY_OUT);
     }
 
     // set the user-provider.
@@ -821,7 +828,7 @@ public class AccessController extends Ba
     for (byte[] family: families) {
       familyMap.put(family, null);
     }
-    requireGlobalPermission("createTable", Permission.Action.CREATE, desc.getTableName(), familyMap);
+    requireGlobalPermission("createTable", Action.CREATE, desc.getTableName(), familyMap);
   }
 
   @Override
@@ -940,8 +947,7 @@ public class AccessController extends Ba
   @Override
   public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
       TableName tableName, byte[] col) throws IOException {
-    AccessControlLists.removeTablePermissions(c.getEnvironment().getConfiguration(),
-                                              tableName, col);
+    AccessControlLists.removeTablePermissions(c.getEnvironment().getConfiguration(), tableName, col);
   }
 
   @Override
@@ -1033,8 +1039,9 @@ public class AccessController extends Ba
   @Override
   public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
       throws IOException {
-    requirePermission("balance", Permission.Action.ADMIN);
+    requirePermission("balance", Action.ADMIN);
   }
+
   @Override
   public void postBalance(ObserverContext<MasterCoprocessorEnvironment> c, List<RegionPlan> plans)
       throws IOException {}
@@ -1042,7 +1049,7 @@ public class AccessController extends Ba
   @Override
   public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
       boolean newValue) throws IOException {
-    requirePermission("balanceSwitch", Permission.Action.ADMIN);
+    requirePermission("balanceSwitch", Action.ADMIN);
     return newValue;
   }
 
@@ -1053,13 +1060,13 @@ public class AccessController extends Ba
   @Override
   public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
       throws IOException {
-    requirePermission("shutdown", Permission.Action.ADMIN);
+    requirePermission("shutdown", Action.ADMIN);
   }
 
   @Override
   public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
       throws IOException {
-    requirePermission("stopMaster", Permission.Action.ADMIN);
+    requirePermission("stopMaster", Action.ADMIN);
   }
 
   @Override
@@ -1078,7 +1085,7 @@ public class AccessController extends Ba
   public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("snapshot", Permission.Action.ADMIN);
+    requirePermission("snapshot", Action.ADMIN);
   }
 
   @Override
@@ -1091,7 +1098,7 @@ public class AccessController extends Ba
   public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("clone", Permission.Action.ADMIN);
+    requirePermission("clone", Action.ADMIN);
   }
 
   @Override
@@ -1104,7 +1111,7 @@ public class AccessController extends Ba
   public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
       throws IOException {
-    requirePermission("restore", Permission.Action.ADMIN);
+    requirePermission("restore", Action.ADMIN);
   }
 
   @Override
@@ -1116,7 +1123,7 @@ public class AccessController extends Ba
   @Override
   public void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot) throws IOException {
-    requirePermission("deleteSnapshot", Permission.Action.ADMIN);
+    requirePermission("deleteSnapshot", Action.ADMIN);
   }
 
   @Override
@@ -1254,20 +1261,131 @@ public class AccessController extends Ba
       final byte [] row, final byte [] family, final Result result)
       throws IOException {
     assert family != null;
-    requireCoveringPermission(OpType.GET_CLOSEST_ROW_BEFORE, c.getEnvironment(), row,
-        makeFamilyMap(family, null), HConstants.LATEST_TIMESTAMP, Permission.Action.READ);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, null);
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.GET_CLOSEST_ROW_BEFORE, user, env, families,
+      Action.READ);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.GET_CLOSEST_ROW_BEFORE, env, row,
+        families, HConstants.LATEST_TIMESTAMP, Action.READ));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
+  }
+
+  private void internalPreRead(final ObserverContext<RegionCoprocessorEnvironment> c,
+      final Query query, OpType opType) throws IOException {
+    Filter filter = query.getFilter();
+    // Don't wrap an AccessControlFilter
+    if (filter != null && filter instanceof AccessControlFilter) {
+      return;
+    }
+    User user = getActiveUser();
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<byte[]>> families = null;
+    switch (opType) {
+    case GET:
+    case EXISTS:
+      families = ((Get)query).getFamilyMap();
+      break;
+    case SCAN:
+      families = ((Scan)query).getFamilyMap();
+      break;
+    default:
+      throw new RuntimeException("Unhandled operation " + opType);
+    }
+    AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ);
+    HRegion region = getRegion(env);
+    TableName table = getTableName(region);
+    Map<ByteRange, Integer> cfVsMaxVersions = Maps.newHashMap();
+    for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {
+      cfVsMaxVersions.put(new SimpleByteRange(hcd.getName()), hcd.getMaxVersions());
+    }
+    if (!authResult.isAllowed()) {
+      if (!cellFeaturesEnabled || compatibleEarlyTermination) {
+        // Old behavior: Scan with only qualifier checks if we have partial
+        // permission. Backwards compatible behavior is to throw an
+        // AccessDeniedException immediately if there are no grants for table
+        // or CF or CF+qual. Only proceed with an injected filter if there are
+        // grants for qualifiers. Otherwise we will fall through below and log
+        // the result and throw an ADE. We may end up checking qualifier
+        // grants three times (permissionGranted above, here, and in the
+        // filter) but that's the price of backwards compatibility. 
+        if (hasFamilyQualifierPermission(user, Action.READ, env, families)) {
+          Filter ourFilter = new AccessControlFilter(authManager, user, table,
+            query.getACLStrategy() ? AccessControlFilter.Strategy.CHECK_CELL_FIRST :
+              AccessControlFilter.Strategy.CHECK_TABLE_AND_CF_ONLY,
+            cfVsMaxVersions);
+          // wrap any existing filter
+          if (filter != null) {
+            ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
+              Lists.newArrayList(ourFilter, filter));
+          }
+          authResult.setAllowed(true);;
+          authResult.setReason("Access allowed with filter");
+          switch (opType) {
+          case GET:
+          case EXISTS:
+            ((Get)query).setFilter(ourFilter);
+            break;
+          case SCAN:
+            ((Scan)query).setFilter(ourFilter);
+            break;
+          default:
+            throw new RuntimeException("Unhandled operation " + opType);
+          }
+        }
+      } else {
+        // New behavior: Any access we might be granted is more fine-grained
+        // than whole table or CF. Simply inject a filter and return what is
+        // allowed. We will not throw an AccessDeniedException. This is a
+        // behavioral change since 0.96. 
+        Filter ourFilter = new AccessControlFilter(authManager, user, table,
+          query.getACLStrategy() ? AccessControlFilter.Strategy.CHECK_CELL_FIRST :
+            AccessControlFilter.Strategy.CHECK_CELL_DEFAULT,
+          cfVsMaxVersions);
+        // wrap any existing filter
+        if (filter != null) {
+          ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
+            Lists.newArrayList(ourFilter, filter));
+        }
+        authResult.setAllowed(true);;
+        authResult.setReason("Access allowed with filter");
+        switch (opType) {
+        case GET:
+        case EXISTS:
+          ((Get)query).setFilter(ourFilter);
+          break;
+        case SCAN:
+          ((Scan)query).setFilter(ourFilter);
+          break;
+        default:
+          throw new RuntimeException("Unhandled operation " + opType);
+        }
+      }
+    }
+
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions (table=" + table +
+        ", action=READ)");
+    }
   }
 
   @Override
   public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Get get, final List<Cell> result) throws IOException {
-    internalPreRead(c, get);
+    internalPreRead(c, get, OpType.GET);
   }
 
   @Override
   public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Get get, final boolean exists) throws IOException {
-    internalPreRead(c, get);
+    internalPreRead(c, get, OpType.EXISTS);
     return exists;
   }
 
@@ -1281,11 +1399,22 @@ public class AccessController extends Ba
     // HBase value. A new ACL in a new Put applies to that Put. It doesn't
     // change the ACL of any previous Put. This allows simple evolution of
     // security policy over time without requiring expensive updates.
-    requireCoveringPermission(OpType.PUT, c.getEnvironment(), put.getRow(),
-      put.getFamilyCellMap(), put.getTimeStamp(), Permission.Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<Cell>> families = put.getFamilyCellMap();
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.PUT, user, env, families, Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.PUT, env, put.getRow(), families,
+        put.getTimeStamp(), Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
-      if (canPersistCellACLs) {
+      if (cellFeaturesEnabled) {
         addCellPermissions(bytes, put.getFamilyCellMap());
       } else {
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
@@ -1314,8 +1443,19 @@ public class AccessController extends Ba
     // compaction could remove them. If the user doesn't have permission to
     // overwrite any of the visible versions ('visible' defined as not covered
     // by a tombstone already) then we have to disallow this operation.
-    requireCoveringPermission(OpType.DELETE, c.getEnvironment(), delete.getRow(),
-      delete.getFamilyCellMap(), delete.getTimeStamp(), Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<Cell>> families = delete.getFamilyCellMap();
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.DELETE, user, env, families, Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.DELETE, env, delete.getRow(), families,
+        delete.getTimeStamp(), Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
   }
 
   @Override
@@ -1334,11 +1474,23 @@ public class AccessController extends Ba
       final ByteArrayComparable comparator, final Put put,
       final boolean result) throws IOException {
     // Require READ and WRITE permissions on the table, CF, and KV to update
-    requireCoveringPermission(OpType.CHECK_AND_PUT, c.getEnvironment(), row,
-        makeFamilyMap(family, qualifier), HConstants.LATEST_TIMESTAMP, Action.READ, Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.CHECK_AND_PUT, user, env, families,
+      Action.READ, Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.CHECK_AND_PUT, env, row, families,
+        HConstants.LATEST_TIMESTAMP, Action.READ, Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
-      if (canPersistCellACLs) {
+      if (cellFeaturesEnabled) {
         addCellPermissions(bytes, put.getFamilyCellMap());
       } else {
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
@@ -1360,8 +1512,20 @@ public class AccessController extends Ba
     }
     // Require READ and WRITE permissions on the table, CF, and the KV covered
     // by the delete
-    requireCoveringPermission(OpType.CHECK_AND_DELETE, c.getEnvironment(), row,
-        makeFamilyMap(family, qualifier), HConstants.LATEST_TIMESTAMP, Action.READ, Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.CHECK_AND_DELETE, user, env, families,
+      Action.READ, Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.CHECK_AND_DELETE, env, row, families,
+        HConstants.LATEST_TIMESTAMP, Action.READ, Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     return result;
   }
 
@@ -1372,8 +1536,20 @@ public class AccessController extends Ba
       throws IOException {
     // Require WRITE permission to the table, CF, and the KV to be replaced by the
     // incremented value
-    requireCoveringPermission(OpType.INCREMENT_COLUMN_VALUE, c.getEnvironment(), row,
-        makeFamilyMap(family, qualifier), HConstants.LATEST_TIMESTAMP, Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.INCREMENT_COLUMN_VALUE, user, env, families,
+      Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.INCREMENT_COLUMN_VALUE, env, row,
+        families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     return -1;
   }
 
@@ -1381,11 +1557,22 @@ public class AccessController extends Ba
   public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
       throws IOException {
     // Require WRITE permission to the table, CF, and the KV to be appended
-    requireCoveringPermission(OpType.APPEND, c.getEnvironment(), append.getRow(),
-        append.getFamilyCellMap(), HConstants.LATEST_TIMESTAMP, Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<Cell>> families = append.getFamilyCellMap();
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.APPEND, user, env, families, Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.APPEND, env, append.getRow(),
+        families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     byte[] bytes = append.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
-      if (canPersistCellACLs) {
+      if (cellFeaturesEnabled) {
         addCellPermissions(bytes, append.getFamilyCellMap());
       } else {
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
@@ -1400,11 +1587,23 @@ public class AccessController extends Ba
       throws IOException {
     // Require WRITE permission to the table, CF, and the KV to be replaced by
     // the incremented value
-    requireCoveringPermission(OpType.INCREMENT, c.getEnvironment(), increment.getRow(),
-        increment.getFamilyCellMap(), increment.getTimeRange().getMax(), Action.WRITE);
+    RegionCoprocessorEnvironment env = c.getEnvironment();
+    Map<byte[],? extends Collection<Cell>> families = increment.getFamilyCellMap();
+    User user = getActiveUser();
+    AuthResult authResult = permissionGranted(OpType.INCREMENT, user, env, families,
+      Action.WRITE);
+    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
+      authResult.setAllowed(checkCoveringPermission(OpType.APPEND, env, increment.getRow(),
+        families, increment.getTimeRange().getMax(), Action.WRITE));
+      authResult.setReason("Covering cell set");
+    }
+    logResult(authResult);
+    if (!authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    }
     byte[] bytes = increment.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
-      if (canPersistCellACLs) {
+      if (cellFeaturesEnabled) {
         addCellPermissions(bytes, increment.getFamilyCellMap());
       } else {
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
@@ -1418,7 +1617,7 @@ public class AccessController extends Ba
       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
     // If the HFile version is insufficient to persist tags, we won't have any
     // work to do here
-    if (!canPersistCellACLs) {
+    if (!cellFeaturesEnabled) {
       return newCell;
     }
 
@@ -1486,7 +1685,7 @@ public class AccessController extends Ba
   @Override
   public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Scan scan, final RegionScanner s) throws IOException {
-    internalPreRead(c, scan);
+    internalPreRead(c, scan, OpType.SCAN);
     return s;
   }
 
@@ -1563,7 +1762,7 @@ public class AccessController extends Ba
     if (!authResult.isAllowed()) {
       for(UserPermission userPerm:
           AccessControlLists.getUserTablePermissions(regionEnv.getConfiguration(), tableName)) {
-        for(Permission.Action userAction: userPerm.getActions()) {
+        for(Action userAction: userPerm.getActions()) {
           if(userAction.equals(action)) {
             return AuthResult.allow(method, "Access allowed", requestUser,
                 action, tableName, null, null);
@@ -1613,7 +1812,7 @@ public class AccessController extends Ba
       Service service, String methodName, Message request) throws IOException {
     // Don't intercept calls to our own AccessControlService, we check for
     // appropriate permissions in the service handlers
-    if (shouldCheckExecPermissions && !(service instanceof AccessControlService)) {
+    if (shouldCheckExecPermission && !(service instanceof AccessControlService)) {
       requirePermission("invoke(" + service.getDescriptorForType().getName() + "." +
         methodName + ")",
         getTableName(ctx.getEnvironment()), null, null,
@@ -1769,7 +1968,7 @@ public class AccessController extends Ba
       for (Permission permission : permissions) {
         if (permission instanceof TablePermission) {
           TablePermission tperm = (TablePermission) permission;
-          for (Permission.Action action : permission.getActions()) {
+          for (Action action : permission.getActions()) {
             if (!tperm.getTableName().equals(tableName)) {
               throw new CoprocessorException(AccessController.class, String.format("This method "
                   + "can only execute at the table specified in TablePermission. " +
@@ -1792,7 +1991,7 @@ public class AccessController extends Ba
           }
 
         } else {
-          for (Permission.Action action : permission.getActions()) {
+          for (Action action : permission.getActions()) {
             requirePermission("checkPermissions", action);
           }
         }
@@ -1815,17 +2014,19 @@ public class AccessController extends Ba
 
   private TableName getTableName(RegionCoprocessorEnvironment e) {
     HRegion region = e.getRegion();
-    TableName tableName = null;
-
     if (region != null) {
-      HRegionInfo regionInfo = region.getRegionInfo();
-      if (regionInfo != null) {
-        tableName = regionInfo.getTable();
-      }
+      return getTableName(region);
     }
-    return tableName;
+    return null;
   }
 
+  private TableName getTableName(HRegion region) {
+    HRegionInfo regionInfo = region.getRegionInfo();
+    if (regionInfo != null) {
+      return regionInfo.getTable();
+    }
+    return null;
+  }
 
   @Override
   public void preClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested)
@@ -1855,7 +2056,7 @@ public class AccessController extends Ba
   public void preStopRegionServer(
       ObserverContext<RegionServerCoprocessorEnvironment> env)
       throws IOException {
-    requirePermission("preStopRegionServer", Permission.Action.ADMIN);
+    requirePermission("preStopRegionServer", Action.ADMIN);
   }
 
   private Map<byte[], ? extends Collection<byte[]>> makeFamilyMap(byte[] family,
@@ -1876,7 +2077,7 @@ public class AccessController extends Ba
     // If the list is empty, this is a request for all table descriptors and requires GLOBAL
     // ADMIN privs.
     if (tableNamesList == null || tableNamesList.isEmpty()) {
-      requireGlobalPermission("getTableDescriptors", Permission.Action.ADMIN, null, null);
+      requireGlobalPermission("getTableDescriptors", Action.ADMIN, null, null);
     }
     // Otherwise, if the requestor has ADMIN or CREATE privs for all listed tables, the
     // request can be granted.
@@ -1893,7 +2094,7 @@ public class AccessController extends Ba
           // We don't care about this
         }
         requirePermission("getTableDescriptors", tableName, null, null,
-          Permission.Action.ADMIN, Permission.Action.CREATE);
+          Action.ADMIN, Action.CREATE);
       }
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthResult.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthResult.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthResult.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthResult.java Thu May  1 01:04:36 2014
@@ -33,12 +33,12 @@ import org.apache.hadoop.hbase.util.Byte
  */
 @InterfaceAudience.Private
 public class AuthResult {
-  private final boolean allowed;
+  private boolean allowed;
   private final String namespace;
   private final TableName table;
   private final Permission.Action action;
   private final String request;
-  private final String reason;
+  private String reason;
   private final User user;
 
   // "family" and "qualifier" should only be used if "families" is null.
@@ -121,6 +121,14 @@ public class AuthResult {
     return request;
   }
 
+  public void setAllowed(boolean allowed) {
+    this.allowed = allowed;
+  }
+
+  public void setReason(String reason) {
+    this.reason = reason;
+  }
+
   String toFamilyString() {
     StringBuilder sb = new StringBuilder();
     if (families != null) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java Thu May  1 01:04:36 2014
@@ -28,7 +28,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.security.User;
@@ -348,12 +347,14 @@ public class TableAuthManager {
     return false;
   }
 
-  private boolean checkCellPermissions(User user, Cell cell, Permission.Action action) {
+  /**
+   * Authorize a user for a given KV. This is called from AccessControlFilter.
+   */
+  public boolean authorize(User user, TableName table, Cell cell, Permission.Action action) {
     try {
       List<Permission> perms = AccessControlLists.getCellPermissionsForUser(user, cell);
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Perms for user " + user.getShortName() + " in cell " +
-          cell + ": " + perms);
+        LOG.trace("Perms for user " + user.getShortName() + " in cell " + cell + ": " + perms);
       }
       for (Permission p: perms) {
         if (p.implies(action)) {
@@ -369,46 +370,6 @@ public class TableAuthManager {
     return false;
   }
 
-  private boolean checkTableColumnPermissions(User user, TableName table, Cell cell,
-      Permission.Action action) {
-    // TODO: Do not clone here
-    byte[] family = CellUtil.cloneFamily(cell);
-    byte[] qualifier = CellUtil.cloneQualifier(cell);
-    // User is authorized at table or CF level
-    if (authorizeUser(user, table, family, qualifier, action)) {
-      return true;
-    }
-    String groupNames[] = user.getGroupNames();
-    if (groupNames != null) {
-      for (String group: groupNames) {
-        // TODO: authorizeGroup should check qualifier too?
-        // Group is authorized at table or CF level
-        if (authorizeGroup(group, table, family, action)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Authorize a user for a given KV. This is called from AccessControlFilter.
-   */
-  public boolean authorize(User user, TableName table, Cell cell, boolean cellFirstStrategy,
-      Permission.Action action) {
-    if (cellFirstStrategy) {
-      if (checkCellPermissions(user, cell, action)) {
-        return true;
-      }
-      return checkTableColumnPermissions(user, table, cell, action);
-    } else {
-      if (checkTableColumnPermissions(user, table, cell, action)) {
-        return true;
-      }
-      return checkCellPermissions(user, cell, action);
-    }
-  }
-
   public boolean authorize(User user, String namespace, Permission.Action action) {
     // Global authorizations supercede namespace level
     if (authorizeUser(user, action)) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java Thu May  1 01:04:36 2014
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.Waiter.Pr
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
@@ -84,7 +85,7 @@ public class SecureTestUtil {
     }
     conf.set("hbase.superuser", sb.toString());
     // Need HFile V3 for tags for security features
-    conf.setInt("hfile.format.version", 3);
+    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
   }
 
   public static void verifyConfiguration(Configuration conf) {
@@ -96,9 +97,12 @@ public class SecureTestUtil {
         AccessController.class.getName()))) {
       throw new RuntimeException("AccessController is missing from a system coprocessor list");
     }
+    if (conf.getInt(HFile.FORMAT_VERSION_KEY, 2) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
+      throw new RuntimeException("Post 0.96 security features require HFile version >= 3");
+    }
   }
 
-  public void checkTablePerms(Configuration conf, byte[] table, byte[] family, byte[] column,
+  public static void checkTablePerms(Configuration conf, byte[] table, byte[] family, byte[] column,
       Permission.Action... actions) throws IOException {
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {
@@ -108,7 +112,7 @@ public class SecureTestUtil {
     checkTablePerms(conf, table, perms);
   }
 
-  public void checkTablePerms(Configuration conf, byte[] table, Permission... perms) throws IOException {
+  public static void checkTablePerms(Configuration conf, byte[] table, Permission... perms) throws IOException {
     CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
     for (Permission p : perms) {
       request.addPermission(ProtobufUtil.toPermission(p));
@@ -139,7 +143,7 @@ public class SecureTestUtil {
    */
   static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
 
-  public void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
+  public static void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
     for (AccessTestAction action : actions) {
       try {
         Object obj = user.runAs(action);
@@ -155,13 +159,13 @@ public class SecureTestUtil {
     }
   }
 
-  public void verifyAllowed(AccessTestAction action, User... users) throws Exception {
+  public static void verifyAllowed(AccessTestAction action, User... users) throws Exception {
     for (User user : users) {
       verifyAllowed(user, action);
     }
   }
 
-  public void verifyAllowed(User user, AccessTestAction action, int count) throws Exception {
+  public static void verifyAllowed(User user, AccessTestAction action, int count) throws Exception {
     try {
       Object obj = user.runAs(action);
       if (obj != null && obj instanceof List<?>) {
@@ -176,14 +180,34 @@ public class SecureTestUtil {
     }
   }
 
-  public void verifyDenied(User user, AccessTestAction... actions) throws Exception {
+  public static void verifyDeniedWithException(User user, AccessTestAction... actions)
+      throws Exception {
+    verifyDenied(user, true, actions);
+  }
+
+  public static void verifyDeniedWithException(AccessTestAction action, User... users)
+      throws Exception {
+    for (User user : users) {
+      verifyDenied(user, true, action);
+    }
+  }
+
+  public static void verifyDenied(User user, AccessTestAction... actions) throws Exception {
+    verifyDenied(user, false, actions);
+  }
+
+  public static void verifyDenied(User user, boolean requireException,
+      AccessTestAction... actions) throws Exception {
     for (AccessTestAction action : actions) {
       try {
         Object obj = user.runAs(action);
+        if (requireException) {
+          fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
+        }
         if (obj != null && obj instanceof List<?>) {
           List<?> results = (List<?>) obj;
           if (results != null && !results.isEmpty()) {
-            fail("Expected no results for user '" + user.getShortName() + "'");
+            fail("Unexpected results for user '" + user.getShortName() + "'");
           }
         }
       } catch (IOException e) {
@@ -211,7 +235,7 @@ public class SecureTestUtil {
           } while((ex = ex.getCause()) != null);
         }
         if (!isAccessDeniedException) {
-          fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
+          fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
         }
       } catch (UndeclaredThrowableException ute) {
         // TODO why we get a PrivilegedActionException, which is unexpected?
@@ -226,12 +250,12 @@ public class SecureTestUtil {
             return;
           }
         }
-        fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
+        fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
       }
     }
   }
 
-  public void verifyDenied(AccessTestAction action, User... users) throws Exception {
+  public static void verifyDenied(AccessTestAction action, User... users) throws Exception {
     for (User user : users) {
       verifyDenied(user, action);
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java Thu May  1 01:04:36 2014
@@ -47,7 +47,7 @@ import org.junit.experimental.categories
 import org.junit.rules.TestName;
 
 @Category(LargeTests.class)
-public class TestAccessControlFilter {
+public class TestAccessControlFilter extends SecureTestUtil {
   @Rule public TestName name = new TestName();
   private static HBaseTestingUtility TEST_UTIL;
 
@@ -69,12 +69,14 @@ public class TestAccessControlFilter {
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
     Configuration conf = TEST_UTIL.getConfiguration();
-    SecureTestUtil.enableSecurity(conf);
-    String baseuser = User.getCurrent().getShortName();
-    conf.set("hbase.superuser", conf.get("hbase.superuser", "") +
-        String.format(",%s.hfs.0,%s.hfs.1,%s.hfs.2", baseuser, baseuser, baseuser));
+    enableSecurity(conf);
+    verifyConfiguration(conf);
+
+    // We expect 0.98 scanning semantics
+    conf.setBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, false);
+
     TEST_UTIL.startMiniCluster();
-    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
+    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName(), 50000);
 
     READER = User.createUserForTesting(conf, "reader", new String[0]);
     LIMITED = User.createUserForTesting(conf, "limited", new String[0]);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Thu May  1 01:04:36 2014
@@ -170,7 +170,7 @@ public class TestAccessController extend
     verifyConfiguration(conf);
 
     // Enable EXEC permission checking
-    conf.setBoolean(AccessController.EXEC_PERMISSION_CHECKS_KEY, true);
+    conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
 
     TEST_UTIL.startMiniCluster();
     MasterCoprocessorHost cpHost =

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java Thu May  1 01:04:36 2014
@@ -45,10 +45,8 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.TestTableName;
-
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -89,18 +87,13 @@ public class TestCellACLWithMultipleVers
   public static void setupBeforeClass() throws Exception {
     // setup configuration
     conf = TEST_UTIL.getConfiguration();
-    conf.set("hbase.master.hfilecleaner.plugins",
-        "org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner,"
-            + "org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner");
-    conf.set("hbase.master.logcleaner.plugins",
-        "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
     // Enable security
     enableSecurity(conf);
     // Verify enableSecurity sets up what we require
     verifyConfiguration(conf);
 
-    // Enable EXEC permission checking
-    conf.setBoolean(AccessController.EXEC_PERMISSION_CHECKS_KEY, true);
+    // We expect 0.98 cell ACL semantics
+    conf.setBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, false);
 
     TEST_UTIL.startMiniCluster();
     MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster().getMaster()

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java?rev=1591524&r1=1591523&r2=1591524&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java Thu May  1 01:04:36 2014
@@ -43,12 +43,11 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.TestTableName;
-
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -79,6 +78,7 @@ public class TestCellACLs extends Secure
   private static final byte[] TEST_Q3 = Bytes.toBytes("q3");
   private static final byte[] TEST_Q4 = Bytes.toBytes("q4");
   private static final byte[] ZERO = Bytes.toBytes(0L);
+  private static final byte[] ONE = Bytes.toBytes(1L);
 
   private static Configuration conf;
 
@@ -89,18 +89,13 @@ public class TestCellACLs extends Secure
   public static void setupBeforeClass() throws Exception {
     // setup configuration
     conf = TEST_UTIL.getConfiguration();
-    conf.set("hbase.master.hfilecleaner.plugins",
-        "org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner,"
-            + "org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner");
-    conf.set("hbase.master.logcleaner.plugins",
-        "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
     // Enable security
     enableSecurity(conf);
     // Verify enableSecurity sets up what we require
     verifyConfiguration(conf);
 
-    // Enable EXEC permission checking
-    conf.setBoolean(AccessController.EXEC_PERMISSION_CHECKS_KEY, true);
+    // We expect 0.98 cell ACL semantics
+    conf.setBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, false);
 
     TEST_UTIL.startMiniCluster();
     MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster().getMaster()
@@ -150,12 +145,11 @@ public class TestCellACLs extends Secure
           Put p;
           // with ro ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Action.READ));
           t.put(p);
           // with rw ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q2, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ,
-            Permission.Action.WRITE));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Action.READ, Action.WRITE));
           t.put(p);
           // no ACL
           p = new Put(TEST_ROW)
@@ -308,7 +302,7 @@ public class TestCellACLs extends Secure
       public Object run() throws Exception {
         Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L);
         // Tag this increment with an ACL that denies write permissions to USER_OTHER
-        i.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+        i.setACL(USER_OTHER.getShortName(), new Permission(Action.READ));
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
           t.increment(i);
@@ -379,6 +373,164 @@ public class TestCellACLs extends Secure
     verifyAllowed(deleteQ1, USER_OWNER);
   }
 
+  /**
+   * Insure we are not granting access in the absence of any cells found
+   * when scanning for covered cells.
+   */
+  @Test
+  public void testCoveringCheck() throws Exception {
+    // Grant read access to USER_OTHER
+    grantOnTable(TEST_UTIL, USER_OTHER.getShortName(), TEST_TABLE.getTableName(),
+      TEST_FAMILY, null, Action.READ);
+
+    // A write by USER_OTHER should be denied.
+    // This is where we could have a big problem if there is an error in the
+    // covering check logic.
+    verifyDenied(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Put p;
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, USER_OTHER);
+
+    // Add the cell
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Put p;
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, USER_OWNER);
+
+    // A write by USER_OTHER should still be denied, just to make sure
+    verifyDenied(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Put p;
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ONE);
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, USER_OTHER);
+
+    // A read by USER_OTHER should be allowed, just to make sure
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1));
+        } finally {
+          t.close();
+        }
+      }
+    }, USER_OTHER);
+  }
+
+  @Test
+  public void testCellStrategy() throws Exception {
+    // Set up our test actions
+    AccessTestAction readQ1Default = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1));
+        } finally {
+          t.close();
+        }
+      }
+    };
+    AccessTestAction readQ2Default = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2));
+        } finally {
+          t.close();
+        }
+      }
+    };
+    AccessTestAction readQ1CellFirst = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1);
+          get.setACLStrategy(true);
+          return t.get(get);
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    // Add test data
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Put p;
+          // The empty permission set on Q1
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(USER_OTHER.getShortName(), new Permission());
+          t.put(p);
+          // Read permissions on Q2
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(USER_OTHER.getShortName(), new Permission(Action.READ));
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, USER_OWNER);
+
+    // A read by USER_OTHER will be denied with the default cell strategy as
+    // there is no visibility without a grant and a cell ACL giving
+    // explicit permission
+    verifyDenied(readQ1Default, USER_OTHER);
+
+    // A read will be allowed by the default cell strategy if there is a cell
+    // ACL giving explicit permission.
+    verifyAllowed(readQ2Default, USER_OTHER);
+
+    // Grant read access to USER_OTHER
+    grantOnTable(TEST_UTIL, USER_OTHER.getShortName(), TEST_TABLE.getTableName(),
+      TEST_FAMILY, null, Action.READ);
+
+    // A read by USER_OTHER will now be allowed with the default cell strategy
+    // because we have a CF level grant and we take the union of permissions.
+    verifyAllowed(readQ1Default, USER_OTHER);
+
+    // A read by USER_OTHER will be denied with the cell first strategy
+    // because the empty perm set for USER_OTHER in the cell ACL there
+    // revokes access.
+    verifyDenied(readQ1CellFirst, USER_OTHER);
+  }
+
   @After
   public void tearDown() throws Exception {
     // Clean the _acl_ table