You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2014/06/17 06:43:30 UTC

git commit: HBASE-11194 [AccessController] issue with covering permission check in case of concurrent op on same row (Anoop)

Repository: hbase
Updated Branches:
  refs/heads/master 0e647de3e -> 463fc9fbd


HBASE-11194 [AccessController] issue with covering permission check in case of concurrent op on same row (Anoop)


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

Branch: refs/heads/master
Commit: 463fc9fbd8194cef46e0d18cdb9dcec7aa3db559
Parents: 0e647de
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Jun 17 10:12:47 2014 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Jun 17 10:12:47 2014 +0530

----------------------------------------------------------------------
 .../hbase/security/access/AccessController.java | 202 +++++++++++++++----
 .../access/TestCellACLWithMultipleVersions.java |   2 +-
 2 files changed, 167 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/463fc9fb/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index e2eadc1..977a403 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
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessCont
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
@@ -151,6 +152,8 @@ public class AccessController extends BaseRegionObserver
 
   private static final Log AUDITLOG =
     LogFactory.getLog("SecurityLogger."+AccessController.class.getName());
+  private static final String CHECK_COVERING_PERM = "check_covering_perm";
+  private static final byte[] TRUE = Bytes.toBytes(true);
 
   TableAuthManager authManager = null;
 
@@ -1463,14 +1466,13 @@ public class AccessController extends BaseRegionObserver
     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());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        put.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
     }
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
@@ -1507,14 +1509,43 @@ public class AccessController extends BaseRegionObserver
     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());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        delete.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
+    }
+  }
+
+  @Override
+  public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+      MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+    if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+      for (int i = 0; i < miniBatchOp.size(); i++) {
+        Mutation m = miniBatchOp.getOperation(i);
+        if (m.getAttribute(CHECK_COVERING_PERM) != null) {
+          // We have a failure with table, cf and q perm checks and now giving a chance for cell
+          // perm check
+          OpType opType = (m instanceof Put) ? OpType.PUT : OpType.DELETE;
+          AuthResult authResult = null;
+          if (checkCoveringPermission(opType, c.getEnvironment(), m.getRow(), m.getFamilyCellMap(),
+              m.getTimeStamp(), Action.WRITE)) {
+            authResult = AuthResult.allow(opType.toString(), "Covering cell set", getActiveUser(),
+                Action.WRITE, table, m.getFamilyCellMap());
+          } else {
+            authResult = AuthResult.deny(opType.toString(), "Covering cell set", getActiveUser(),
+                Action.WRITE, table, m.getFamilyCellMap());
+          }
+          logResult(authResult);
+          if (!authResult.isAllowed()) {
+            throw new AccessDeniedException("Insufficient permissions "
+                + authResult.toContextString());
+          }
+        }
+      }
     }
   }
 
@@ -1539,14 +1570,13 @@ public class AccessController extends BaseRegionObserver
     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));
-      authResult.setReason("Covering cell set");
-    }
     logResult(authResult);
     if (!authResult.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        put.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
     }
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
@@ -1560,6 +1590,33 @@ public class AccessController extends BaseRegionObserver
   }
 
   @Override
+  public boolean preCheckAndPutAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
+      final byte[] row, final byte[] family, final byte[] qualifier,
+      final CompareFilter.CompareOp compareOp, final ByteArrayComparable comparator, final Put put,
+      final boolean result) throws IOException {
+    if (put.getAttribute(CHECK_COVERING_PERM) != null) {
+      // We had failure with table, cf and q perm checks and now giving a chance for cell
+      // perm check
+      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+      Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
+      AuthResult authResult = null;
+      if (checkCoveringPermission(OpType.CHECK_AND_PUT, c.getEnvironment(), row, families,
+          HConstants.LATEST_TIMESTAMP, Action.READ)) {
+        authResult = AuthResult.allow(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
+            getActiveUser(), Action.READ, table, families);
+      } else {
+        authResult = AuthResult.deny(OpType.CHECK_AND_PUT.toString(), "Covering cell set",
+            getActiveUser(), Action.READ, table, families);
+      }
+      logResult(authResult);
+      if (!authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
+    }
+    return result;
+  }
+
+  @Override
   public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final byte [] qualifier,
       final CompareFilter.CompareOp compareOp,
@@ -1577,14 +1634,41 @@ public class AccessController extends BaseRegionObserver
     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));
-      authResult.setReason("Covering cell set");
-    }
     logResult(authResult);
     if (!authResult.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        delete.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public boolean preCheckAndDeleteAfterRowLock(
+      final ObserverContext<RegionCoprocessorEnvironment> c, final byte[] row, final byte[] family,
+      final byte[] qualifier, final CompareFilter.CompareOp compareOp,
+      final ByteArrayComparable comparator, final Delete delete, final boolean result)
+      throws IOException {
+    if (delete.getAttribute(CHECK_COVERING_PERM) != null) {
+      // We had failure with table, cf and q perm checks and now giving a chance for cell
+      // perm check
+      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+      Map<byte[], ? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
+      AuthResult authResult = null;
+      if (checkCoveringPermission(OpType.CHECK_AND_DELETE, c.getEnvironment(), row, families,
+          HConstants.LATEST_TIMESTAMP, Action.READ)) {
+        authResult = AuthResult.allow(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
+            getActiveUser(), Action.READ, table, families);
+      } else {
+        authResult = AuthResult.deny(OpType.CHECK_AND_DELETE.toString(), "Covering cell set",
+            getActiveUser(), Action.READ, table, families);
+      }
+      logResult(authResult);
+      if (!authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
     }
     return result;
   }
@@ -1621,14 +1705,13 @@ public class AccessController extends BaseRegionObserver
     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());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        append.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
     }
     byte[] bytes = append.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
@@ -1642,6 +1725,30 @@ public class AccessController extends BaseRegionObserver
   }
 
   @Override
+  public Result preAppendAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
+      final Append append) throws IOException {
+    if (append.getAttribute(CHECK_COVERING_PERM) != null) {
+      // We had failure with table, cf and q perm checks and now giving a chance for cell
+      // perm check
+      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+      AuthResult authResult = null;
+      if (checkCoveringPermission(OpType.APPEND, c.getEnvironment(), append.getRow(),
+          append.getFamilyCellMap(), HConstants.LATEST_TIMESTAMP, Action.WRITE)) {
+        authResult = AuthResult.allow(OpType.APPEND.toString(), "Covering cell set",
+            getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
+      } else {
+        authResult = AuthResult.deny(OpType.APPEND.toString(), "Covering cell set",
+            getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
+      }
+      logResult(authResult);
+      if (!authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
+    }
+    return null;
+  }
+
+  @Override
   public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Increment increment)
       throws IOException {
@@ -1652,14 +1759,13 @@ public class AccessController extends BaseRegionObserver
     User user = getActiveUser();
     AuthResult authResult = permissionGranted(OpType.INCREMENT, user, env, families,
       Action.WRITE);
-    if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
-      authResult.setAllowed(checkCoveringPermission(OpType.INCREMENT, 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());
+      if (cellFeaturesEnabled && !compatibleEarlyTermination) {
+        increment.setAttribute(CHECK_COVERING_PERM, TRUE);
+      } else {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
     }
     byte[] bytes = increment.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
@@ -1673,6 +1779,30 @@ public class AccessController extends BaseRegionObserver
   }
 
   @Override
+  public Result preIncrementAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> c,
+      final Increment increment) throws IOException {
+    if (increment.getAttribute(CHECK_COVERING_PERM) != null) {
+      // We had failure with table, cf and q perm checks and now giving a chance for cell
+      // perm check
+      TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+      AuthResult authResult = null;
+      if (checkCoveringPermission(OpType.INCREMENT, c.getEnvironment(), increment.getRow(),
+          increment.getFamilyCellMap(), increment.getTimeRange().getMax(), Action.WRITE)) {
+        authResult = AuthResult.allow(OpType.INCREMENT.toString(), "Covering cell set",
+            getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
+      } else {
+        authResult = AuthResult.deny(OpType.INCREMENT.toString(), "Covering cell set",
+            getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
+      }
+      logResult(authResult);
+      if (!authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      }
+    }
+    return null;
+  }
+
+  @Override
   public Cell postMutationBeforeWAL(ObserverContext<RegionCoprocessorEnvironment> ctx,
       MutationType opType, Mutation mutation, Cell oldCell, Cell newCell) throws IOException {
     // If the HFile version is insufficient to persist tags, we won't have any

http://git-wip-us.apache.org/repos/asf/hbase/blob/463fc9fb/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
index 060c6c9..f01c1b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
@@ -872,7 +872,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
           Delete d = new Delete(TEST_ROW1);
-          d.deleteColumns(TEST_FAMILY1, TEST_Q1);
+          d.deleteColumns(TEST_FAMILY1, TEST_Q1, 120);
           t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q1, ZERO, d);
         } finally {
           t.close();