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 2013/11/27 00:33:31 UTC

svn commit: r1545882 [2/2] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ hbase-server/src/m...

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=1545882&r1=1545881&r2=1545882&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 Tue Nov 26 23:33:31 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -87,6 +88,8 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 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;
@@ -95,6 +98,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.BlockingRpcChannel;
 import com.google.protobuf.ServiceException;
 
@@ -103,9 +107,15 @@ import com.google.protobuf.ServiceExcept
  * levels of authorized users.
  */
 @Category(LargeTests.class)
-@SuppressWarnings("rawtypes")
 public class TestAccessController extends SecureTestUtil {
   private static final Log LOG = LogFactory.getLog(TestAccessController.class);
+
+  static {
+    Logger.getLogger(AccessController.class).setLevel(Level.TRACE);
+    Logger.getLogger(AccessControlFilter.class).setLevel(Level.TRACE);
+    Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
+  }
+
   @Rule public TestTableName TEST_TABLE = new TestTableName();
   private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration conf;
@@ -125,9 +135,10 @@ public class TestAccessController extend
   // user with no permissions
   private static User USER_NONE;
 
-  private static TableName TEST_TABLE2 =
-      TableName.valueOf("testtable2");
+  private static TableName TEST_TABLE2 = TableName.valueOf("testtable2");
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
+  private static byte[] TEST_QUALIFIER = Bytes.toBytes("q1");
+  private static byte[] TEST_ROW = Bytes.toBytes("r1");
 
   private static MasterCoprocessorEnvironment CP_ENV;
   private static AccessController ACCESS_CONTROLLER;
@@ -235,19 +246,50 @@ public class TestAccessController extend
     assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size());
   }
 
-  @Override
-  public void verifyAllowed(PrivilegedExceptionAction action, User... users) throws Exception {
+  /**
+   * An AccessTestAction performs an action that will be examined to confirm
+   * the results conform to expected access rights.
+   * <p>
+   * To indicate an action was allowed, return null or a non empty list of
+   * KeyValues.
+   * <p>
+   * To indicate the action was not allowed, either throw an AccessDeniedException
+   * or return an empty list of KeyValues.
+   */
+  static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+
+  public void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
+    for (AccessTestAction action : actions) {
+      try {
+        Object obj = user.runAs(action);
+        if (obj != null && obj instanceof List<?>) {
+          List<?> results = (List<?>) obj;
+          if (results != null && results.isEmpty()) {
+            fail("Empty non null results from action for user '" + user.getShortName() + "'");
+          }
+        }
+      } catch (AccessDeniedException ade) {
+        fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
+      }
+    }
+  }
+
+  public void verifyAllowed(AccessTestAction action, User... users) throws Exception {
     for (User user : users) {
       verifyAllowed(user, action);
     }
   }
 
-  @Override
-  public void verifyDenied(User user, PrivilegedExceptionAction... actions) throws Exception {
-    for (PrivilegedExceptionAction action : actions) {
+  public void verifyDenied(User user, AccessTestAction... actions) throws Exception {
+    for (AccessTestAction action : actions) {
       try {
-        user.runAs(action);
-        fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
+        Object obj = user.runAs(action);
+        if (obj != null && obj instanceof List<?>) {
+          List<?> results = (List<?>) obj;
+          if (results != null && !results.isEmpty()) {
+            fail("Expected no results for user '" + user.getShortName() + "'");
+          }
+        }
       } catch (IOException e) {
         boolean isAccessDeniedException = false;
         if(e instanceof RetriesExhaustedWithDetailsException) {
@@ -293,8 +335,7 @@ public class TestAccessController extend
     }
   }
 
-  @Override
-  public void verifyDenied(PrivilegedExceptionAction action, User... users) throws Exception {
+  public void verifyDenied(AccessTestAction action, User... users) throws Exception {
     for (User user : users) {
       verifyDenied(user, action);
     }
@@ -302,7 +343,7 @@ public class TestAccessController extend
 
   @Test
   public void testTableCreate() throws Exception {
-    PrivilegedExceptionAction createTable = new PrivilegedExceptionAction() {
+    AccessTestAction createTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testnewtable"));
@@ -321,7 +362,7 @@ public class TestAccessController extend
 
   @Test
   public void testTableModify() throws Exception {
-    PrivilegedExceptionAction modifyTable = new PrivilegedExceptionAction() {
+    AccessTestAction modifyTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
@@ -339,7 +380,7 @@ public class TestAccessController extend
 
   @Test
   public void testTableDelete() throws Exception {
-    PrivilegedExceptionAction deleteTable = new PrivilegedExceptionAction() {
+    AccessTestAction deleteTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER
@@ -355,7 +396,7 @@ public class TestAccessController extend
   @Test
   public void testAddColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE.getTableName(),
@@ -372,7 +413,7 @@ public class TestAccessController extend
   public void testModifyColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
     hcd.setMaxVersions(10);
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -387,7 +428,7 @@ public class TestAccessController extend
 
   @Test
   public void testDeleteColumn() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -402,7 +443,7 @@ public class TestAccessController extend
 
   @Test
   public void testTableDisable() throws Exception {
-    PrivilegedExceptionAction disableTable = new PrivilegedExceptionAction() {
+    AccessTestAction disableTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -411,7 +452,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction disableAclTable = new PrivilegedExceptionAction() {
+    AccessTestAction disableAclTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -429,7 +470,7 @@ public class TestAccessController extend
 
   @Test
   public void testTableEnable() throws Exception {
-    PrivilegedExceptionAction enableTable = new PrivilegedExceptionAction() {
+    AccessTestAction enableTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER
@@ -453,7 +494,7 @@ public class TestAccessController extend
     }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
     final ServerName server = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preMove(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -477,7 +518,7 @@ public class TestAccessController extend
     }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
 
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preAssign(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -501,7 +542,7 @@ public class TestAccessController extend
     }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
 
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preUnassign(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -525,7 +566,7 @@ public class TestAccessController extend
     }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
 
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preRegionOffline(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -540,7 +581,7 @@ public class TestAccessController extend
 
   @Test
   public void testBalance() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preBalance(ObserverContext.createAndPrepare(CP_ENV, null));
@@ -554,7 +595,7 @@ public class TestAccessController extend
 
   @Test
   public void testBalanceSwitch() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preBalanceSwitch(ObserverContext.createAndPrepare(CP_ENV, null), true);
@@ -568,7 +609,7 @@ public class TestAccessController extend
 
   @Test
   public void testShutdown() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preShutdown(ObserverContext.createAndPrepare(CP_ENV, null));
@@ -582,7 +623,7 @@ public class TestAccessController extend
 
   @Test
   public void testStopMaster() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preStopMaster(ObserverContext.createAndPrepare(CP_ENV, null));
@@ -594,14 +635,14 @@ public class TestAccessController extend
     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE);
   }
 
-  private void verifyWrite(PrivilegedExceptionAction action) throws Exception {
+  private void verifyWrite(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW);
     verifyDenied(action, USER_NONE, USER_RO);
   }
 
   @Test
   public void testSplit() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
@@ -615,12 +656,12 @@ public class TestAccessController extend
 
   @Test
   public void testSplitWithSplitRow() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preSplit(
             ObserverContext.createAndPrepare(RCP_ENV, null),
-            Bytes.toBytes("row2"));
+            TEST_ROW);
         return null;
       }
     };
@@ -632,7 +673,7 @@ public class TestAccessController extend
 
   @Test
   public void testFlush() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preFlush(ObserverContext.createAndPrepare(RCP_ENV, null));
@@ -646,7 +687,7 @@ public class TestAccessController extend
 
   @Test
   public void testCompact() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null,
@@ -661,7 +702,7 @@ public class TestAccessController extend
 
   @Test
   public void testPreCompactSelection() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCompactSelection(ObserverContext.createAndPrepare(RCP_ENV, null), null, null);
@@ -673,12 +714,12 @@ public class TestAccessController extend
     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE);
   }
 
-  private void verifyRead(PrivilegedExceptionAction action) throws Exception {
+  private void verifyRead(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW, USER_RO);
     verifyDenied(action, USER_NONE);
   }
 
-  private void verifyReadWrite(PrivilegedExceptionAction action) throws Exception {
+  private void verifyReadWrite(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW);
     verifyDenied(action, USER_NONE, USER_RO);
   }
@@ -686,10 +727,10 @@ public class TestAccessController extend
   @Test
   public void testRead() throws Exception {
     // get action
-    PrivilegedExceptionAction getAction = new PrivilegedExceptionAction() {
+    AccessTestAction getAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get g = new Get(Bytes.toBytes("random_row"));
+        Get g = new Get(TEST_ROW);
         g.addFamily(TEST_FAMILY);
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
@@ -703,7 +744,7 @@ public class TestAccessController extend
     verifyRead(getAction);
 
     // action for scanning
-    PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() {
+    AccessTestAction scanAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         Scan s = new Scan();
@@ -733,11 +774,11 @@ public class TestAccessController extend
   // test put, delete, increment
   public void testWrite() throws Exception {
     // put action
-    PrivilegedExceptionAction putAction = new PrivilegedExceptionAction() {
+    AccessTestAction putAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Put p = new Put(Bytes.toBytes("random_row"));
-        p.add(TEST_FAMILY, Bytes.toBytes("Qualifier"), Bytes.toBytes(1));
+        Put p = new Put(TEST_ROW);
+        p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
           t.put(p);
@@ -750,10 +791,10 @@ public class TestAccessController extend
     verifyWrite(putAction);
 
     // delete action
-    PrivilegedExceptionAction deleteAction = new PrivilegedExceptionAction() {
+    AccessTestAction deleteAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteFamily(TEST_FAMILY);
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
@@ -767,11 +808,11 @@ public class TestAccessController extend
     verifyWrite(deleteAction);
 
     // increment action
-    PrivilegedExceptionAction incrementAction = new PrivilegedExceptionAction() {
+    AccessTestAction incrementAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Increment inc = new Increment(Bytes.toBytes("random_row"));
-        inc.addColumn(TEST_FAMILY, Bytes.toBytes("Qualifier"), 1);
+        Increment inc = new Increment(TEST_ROW);
+        inc.addColumn(TEST_FAMILY, TEST_QUALIFIER, 1);
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
           t.increment(inc);
@@ -787,14 +828,14 @@ public class TestAccessController extend
   @Test
   public void testReadWrite() throws Exception {
     // action for checkAndDelete
-    PrivilegedExceptionAction checkAndDeleteAction = new PrivilegedExceptionAction() {
+    AccessTestAction checkAndDeleteAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteFamily(TEST_FAMILY);
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
-          t.checkAndDelete(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
+          t.checkAndDelete(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
             Bytes.toBytes("test_value"), d);
         } finally {
           t.close();
@@ -805,14 +846,14 @@ public class TestAccessController extend
     verifyReadWrite(checkAndDeleteAction);
 
     // action for checkAndPut()
-    PrivilegedExceptionAction checkAndPut = new PrivilegedExceptionAction() {
+    AccessTestAction checkAndPut = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Put p = new Put(Bytes.toBytes("random_row"));
-        p.add(TEST_FAMILY, Bytes.toBytes("Qualifier"), Bytes.toBytes(1));
+        Put p = new Put(TEST_ROW);
+        p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
-          t.checkAndPut(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
+          t.checkAndPut(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
            Bytes.toBytes("test_value"), p);
         } finally {
           t.close();
@@ -832,7 +873,7 @@ public class TestAccessController extend
     //so users creating HFiles have write permissions
     fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
 
-    PrivilegedExceptionAction bulkLoadAction = new PrivilegedExceptionAction() {
+    AccessTestAction bulkLoadAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         int numRows = 3;
@@ -842,7 +883,7 @@ public class TestAccessController extend
 
         Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
         new BulkLoadHelper(bulkLoadBasePath)
-            .bulkLoadHFile(TEST_TABLE.getTableName(), TEST_FAMILY, Bytes.toBytes("q"), hfileRanges, numRows);
+            .bulkLoadHFile(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_QUALIFIER, hfileRanges, numRows);
 
         return null;
       }
@@ -939,11 +980,11 @@ public class TestAccessController extend
   @Test
   public void testAppend() throws Exception {
 
-    PrivilegedExceptionAction appendAction = new PrivilegedExceptionAction() {
+    AccessTestAction appendAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        byte[] row = Bytes.toBytes("random_row");
-        byte[] qualifier = Bytes.toBytes("q");
+        byte[] row = TEST_ROW;
+        byte[] qualifier = TEST_QUALIFIER;
         Put put = new Put(row);
         put.add(TEST_FAMILY, qualifier, Bytes.toBytes(1));
         Append append = new Append(row);
@@ -964,9 +1005,263 @@ public class TestAccessController extend
   }
 
   @Test
-  public void testGrantRevoke() throws Exception {
+  public void testCellPermissions() throws Exception {
+    // table/column/qualifier level permissions
+    final byte[] TEST_ROW = Bytes.toBytes("cellpermtest");
+    final byte[] TEST_Q1 = Bytes.toBytes("q1");
+    final byte[] TEST_Q2 = Bytes.toBytes("q2");
+    final byte[] TEST_Q3 = Bytes.toBytes("q3");
+    final byte[] TEST_Q4 = Bytes.toBytes("q4");
+    // test value
+    final byte[] ZERO = Bytes.toBytes(0L);
+
+    /* ---- Setup ---- */
+
+    // additional test user
+    final User userOther = User.createUserForTesting(conf, "user_check_cell_perms_other",
+      new String[0]);
+
+    // store two sets of values, one store with a cell level ACL, and one without
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Put p;
+          // with ro ACL
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(userOther.getShortName(), new Permission(Permission.Action.READ));
+          t.put(p);
+          // with rw ACL
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(userOther.getShortName(), new Permission(Permission.Action.READ,
+            Permission.Action.WRITE));
+          t.put(p);
+          // no ACL
+          p = new Put(TEST_ROW)
+            .add(TEST_FAMILY, TEST_Q3, ZERO)
+            .add(TEST_FAMILY, TEST_Q4, ZERO);
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, USER_OWNER);
+
+    /* ---- Gets ---- */
+
+    AccessTestAction getQ1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(get).listCells();
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    AccessTestAction getQ2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(get).listCells();
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    AccessTestAction getQ3 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(get).listCells();
+        } finally {
+          t.close();
+        }
+      }
+    };
 
-    PrivilegedExceptionAction grantAction = new PrivilegedExceptionAction() {
+    AccessTestAction getQ4 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q4);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          return t.get(get).listCells();
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    // Confirm special read access set at cell level
+
+    verifyAllowed(getQ1, userOther);
+    verifyAllowed(getQ2, userOther);
+
+    // Confirm this access does not extend to other cells
+
+    verifyDenied(getQ3, userOther);
+    verifyDenied(getQ4, userOther);
+
+    /* ---- Scans ---- */
+
+    // check that a scan over the test data returns the expected number of KVs
+
+    final List<Cell> scanResults = Lists.newArrayList();
+
+    AccessTestAction scanAction = new AccessTestAction() {
+      @Override
+      public List<Cell> run() throws Exception {
+        Scan scan = new Scan();
+        scan.setStartRow(TEST_ROW);
+        scan.setStopRow(Bytes.add(TEST_ROW, new byte[]{ 0 } ));
+        scan.addFamily(TEST_FAMILY);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          ResultScanner scanner = t.getScanner(scan);
+          Result result = null;
+          do {
+            result = scanner.next();
+            if (result != null) {
+              scanResults.addAll(result.listCells());
+            }
+          } while (result != null);
+        } finally {
+          t.close();
+        }
+        return scanResults;
+      }
+    };
+
+    // owner will see all values
+    scanResults.clear();
+    verifyAllowed(scanAction, USER_OWNER);
+    assertEquals(4, scanResults.size());
+
+    // other user will see 2 values
+    scanResults.clear();
+    verifyAllowed(scanAction, userOther);
+    assertEquals(2, scanResults.size());
+
+    /* ---- Increments ---- */
+
+    AccessTestAction incrementQ1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, 1L);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.increment(i);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    AccessTestAction incrementQ2 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.increment(i);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    AccessTestAction incrementQ2newDenyACL = new AccessTestAction() {
+      @Override
+      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 userOther
+        i.setACL(userOther.getShortName(), new Permission(Permission.Action.READ));
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.increment(i);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    AccessTestAction incrementQ3 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3, 1L);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.increment(i);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    verifyDenied(incrementQ1, userOther);
+    verifyDenied(incrementQ3, userOther);
+
+    // We should be able to increment Q2 twice, the previous ACL will be
+    // carried forward
+    verifyAllowed(incrementQ2, userOther);
+    verifyAllowed(incrementQ2newDenyACL, userOther);
+    // But not again after we denied ourselves write permission with an ACL
+    // update
+    verifyDenied(incrementQ2, userOther);
+
+    /* ---- Deletes ---- */
+
+    AccessTestAction deleteFamily = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Delete delete = new Delete(TEST_ROW).deleteFamily(TEST_FAMILY);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.delete(delete);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    AccessTestAction deleteQ1 = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Delete delete = new Delete(TEST_ROW).deleteColumn(TEST_FAMILY, TEST_Q1);
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          t.delete(delete);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    };
+
+    verifyDenied(deleteFamily, userOther);
+    verifyDenied(deleteQ1, userOther);
+    verifyAllowed(deleteQ1, USER_OWNER);
+  }
+
+  @Test
+  public void testGrantRevoke() throws Exception {
+    AccessTestAction grantAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -983,7 +1278,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction revokeAction = new PrivilegedExceptionAction() {
+    AccessTestAction revokeAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -1000,7 +1295,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction getPermissionsAction = new PrivilegedExceptionAction() {
+    AccessTestAction getPermissionsAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -1052,7 +1347,7 @@ public class TestAccessController extend
         .createUserForTesting(TEST_UTIL.getConfiguration(), "gbluser", new String[0]);
 
     // prepare actions:
-    PrivilegedExceptionAction putActionAll = new PrivilegedExceptionAction() {
+    AccessTestAction putActionAll = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         Put p = new Put(Bytes.toBytes("a"));
@@ -1067,7 +1362,8 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction putAction1 = new PrivilegedExceptionAction() {
+
+    AccessTestAction putAction1 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         Put p = new Put(Bytes.toBytes("a"));
@@ -1081,7 +1377,8 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction putAction2 = new PrivilegedExceptionAction() {
+
+    AccessTestAction putAction2 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         Put p = new Put(Bytes.toBytes("a"));
@@ -1095,10 +1392,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction getActionAll = new PrivilegedExceptionAction() {
+
+    AccessTestAction getActionAll = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get g = new Get(Bytes.toBytes("random_row"));
+        Get g = new Get(TEST_ROW);
         g.addFamily(family1);
         g.addFamily(family2);
         HTable t = new HTable(conf, tableName);
@@ -1110,10 +1408,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction getAction1 = new PrivilegedExceptionAction() {
+
+    AccessTestAction getAction1 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get g = new Get(Bytes.toBytes("random_row"));
+        Get g = new Get(TEST_ROW);
         g.addFamily(family1);
         HTable t = new HTable(conf, tableName);
         try {
@@ -1124,10 +1423,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction getAction2 = new PrivilegedExceptionAction() {
+
+    AccessTestAction getAction2 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get g = new Get(Bytes.toBytes("random_row"));
+        Get g = new Get(TEST_ROW);
         g.addFamily(family2);
         HTable t = new HTable(conf, tableName);
         try {
@@ -1138,10 +1438,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction deleteActionAll = new PrivilegedExceptionAction() {
+
+    AccessTestAction deleteActionAll = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteFamily(family1);
         d.deleteFamily(family2);
         HTable t = new HTable(conf, tableName);
@@ -1153,10 +1454,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction deleteAction1 = new PrivilegedExceptionAction() {
+
+    AccessTestAction deleteAction1 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteFamily(family1);
         HTable t = new HTable(conf, tableName);
         try {
@@ -1167,10 +1469,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction deleteAction2 = new PrivilegedExceptionAction() {
+
+    AccessTestAction deleteAction2 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteFamily(family2);
         HTable t = new HTable(conf, tableName);
         try {
@@ -1372,10 +1675,10 @@ public class TestAccessController extend
     // create temp users
     User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
 
-    PrivilegedExceptionAction getQualifierAction = new PrivilegedExceptionAction() {
+    AccessTestAction getQualifierAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get g = new Get(Bytes.toBytes("random_row"));
+        Get g = new Get(TEST_ROW);
         g.addColumn(family1, qualifier);
         HTable t = new HTable(conf, tableName);
         try {
@@ -1386,10 +1689,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction putQualifierAction = new PrivilegedExceptionAction() {
+
+    AccessTestAction putQualifierAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Put p = new Put(Bytes.toBytes("random_row"));
+        Put p = new Put(TEST_ROW);
         p.add(family1, qualifier, Bytes.toBytes("v1"));
         HTable t = new HTable(conf, tableName);
         try {
@@ -1400,10 +1704,11 @@ public class TestAccessController extend
         return null;
       }
     };
-    PrivilegedExceptionAction deleteQualifierAction = new PrivilegedExceptionAction() {
+
+    AccessTestAction deleteQualifierAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete d = new Delete(Bytes.toBytes("random_row"));
+        Delete d = new Delete(TEST_ROW);
         d.deleteColumn(family1, qualifier);
         // d.deleteFamily(family1);
         HTable t = new HTable(conf, tableName);
@@ -1657,7 +1962,7 @@ public class TestAccessController extend
   }
 
   /** global operations */
-  private void verifyGlobal(PrivilegedExceptionAction<?> action) throws Exception {
+  private void verifyGlobal(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER);
 
     verifyDenied(action, USER_CREATE, USER_RW, USER_NONE, USER_RO);
@@ -1724,7 +2029,7 @@ public class TestAccessController extend
   public void testCheckPermissions() throws Exception {
     // --------------------------------------
     // test global permissions
-    PrivilegedExceptionAction<Void> globalAdmin = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction globalAdmin = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkGlobalPerms(Permission.Action.ADMIN);
@@ -1736,7 +2041,7 @@ public class TestAccessController extend
 
     // --------------------------------------
     // test multiple permissions
-    PrivilegedExceptionAction<Void> globalReadWrite = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction globalReadWrite = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkGlobalPerms(Permission.Action.READ, Permission.Action.WRITE);
@@ -1770,7 +2075,7 @@ public class TestAccessController extend
       acl.close();
     }
 
-    PrivilegedExceptionAction<Void> tableRead = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction tableRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
@@ -1778,7 +2083,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction<Void> columnRead = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction columnRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, null, Permission.Action.READ);
@@ -1786,7 +2091,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction<Void> qualifierRead = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction qualifierRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1, Permission.Action.READ);
@@ -1794,7 +2099,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction<Void> multiQualifierRead = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction multiQualifierRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), new Permission[] {
@@ -1804,7 +2109,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction<Void> globalAndTableRead = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction globalAndTableRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), new Permission[] { new Permission(Permission.Action.READ),
@@ -1813,7 +2118,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction<Void> noCheck = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction noCheck = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), new Permission[0]);
@@ -1839,7 +2144,7 @@ public class TestAccessController extend
 
     // --------------------------------------
     // test family level multiple permissions
-    PrivilegedExceptionAction<Void> familyReadWrite = new PrivilegedExceptionAction<Void>() {
+    AccessTestAction familyReadWrite = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
         checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, null, Permission.Action.READ,
@@ -1880,7 +2185,7 @@ public class TestAccessController extend
 
   @Test
   public void testStopRegionServer() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preStopRegionServer(ObserverContext.createAndPrepare(RSCP_ENV, null));
@@ -1894,7 +2199,7 @@ public class TestAccessController extend
 
   @Test
   public void testOpenRegion() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preOpen(ObserverContext.createAndPrepare(RCP_ENV, null));
@@ -1908,7 +2213,7 @@ public class TestAccessController extend
 
   @Test
   public void testCloseRegion() throws Exception {
-    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+    AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preClose(ObserverContext.createAndPrepare(RCP_ENV, null), false);
@@ -1922,7 +2227,7 @@ public class TestAccessController extend
 
   @Test
   public void testSnapshot() throws Exception {
-    PrivilegedExceptionAction snapshotAction = new PrivilegedExceptionAction() {
+    AccessTestAction snapshotAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -1931,7 +2236,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction deleteAction = new PrivilegedExceptionAction() {
+    AccessTestAction deleteAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDeleteSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -1940,7 +2245,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction restoreAction = new PrivilegedExceptionAction() {
+    AccessTestAction restoreAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preRestoreSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -1949,7 +2254,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction cloneAction = new PrivilegedExceptionAction() {
+    AccessTestAction cloneAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
@@ -2011,7 +2316,7 @@ public class TestAccessController extend
       final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet()
           .iterator().next();
 
-      PrivilegedExceptionAction moveAction = new PrivilegedExceptionAction() {
+      AccessTestAction moveAction = new AccessTestAction() {
         @Override
         public Object run() throws Exception {
           admin.move(firstRegion.getKey().getEncodedNameAsBytes(),
@@ -2037,7 +2342,7 @@ public class TestAccessController extend
       }
       // Verify write permission for user "admin2" who has the global
       // permissions.
-      PrivilegedExceptionAction putAction = new PrivilegedExceptionAction() {
+      AccessTestAction putAction = new AccessTestAction() {
         @Override
         public Object run() throws Exception {
           Put put = new Put(Bytes.toBytes("test"));
@@ -2068,7 +2373,7 @@ public class TestAccessController extend
       acl.close();
     }
 
-    PrivilegedExceptionAction listTablesAction = new PrivilegedExceptionAction() {
+    AccessTestAction listTablesAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@@ -2081,7 +2386,7 @@ public class TestAccessController extend
       }
     };
 
-    PrivilegedExceptionAction getTableDescAction = new PrivilegedExceptionAction() {
+    AccessTestAction getTableDescAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
@@ -2117,7 +2422,7 @@ public class TestAccessController extend
       acl.close();
     }
 
-    PrivilegedExceptionAction deleteTableAction = new PrivilegedExceptionAction() {
+    AccessTestAction deleteTableAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());