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 2015/08/19 03:27:21 UTC

[4/6] hbase git commit: HBASE-14210 Create test for cell level ACLs involving user group

HBASE-14210 Create test for cell level ACLs involving user group

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.1
Commit: 840684613aad7525450436d91e251f51709d5ce4
Parents: 0acbff2
Author: Ashish Singhi <as...@huawei.com>
Authored: Tue Aug 18 16:54:44 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 18 16:55:01 2015 -0700

----------------------------------------------------------------------
 .../access/TestCellACLWithMultipleVersions.java | 356 +++++++++++--------
 .../hbase/security/access/TestCellACLs.java     | 110 +++---
 2 files changed, 283 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/84068461/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 289b0e5..3be7ef8 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.security.access;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hbase.client.Table;
 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.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.TestTableName;
@@ -82,10 +85,14 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
   private static Configuration conf;
 
+  private static final String GROUP = "group";
+  private static User GROUP_USER;
   private static User USER_OWNER;
   private static User USER_OTHER;
   private static User USER_OTHER2;
 
+  private static String[] usersAndGroups;
+
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     // setup configuration
@@ -116,6 +123,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
     USER_OTHER = User.createUserForTesting(conf, "other", new String[0]);
     USER_OTHER2 = User.createUserForTesting(conf, "other2", new String[0]);
+    GROUP_USER = User.createUserForTesting(conf, "group_user", new String[] { GROUP });
+
+    usersAndGroups = new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) };
   }
 
   @AfterClass
@@ -149,6 +159,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   public void testCellPermissionwithVersions() throws Exception {
     // store two sets of values, one store with a cell level ACL, and one
     // without
+    final Map<String, Permission> writePerms = prepareCellPermissions(usersAndGroups, Action.WRITE);
+    final Map<String, Permission> readPerms = prepareCellPermissions(usersAndGroups, Action.READ);
     verifyAllowed(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
@@ -157,20 +169,20 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
           Put p;
           // with ro ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(writePerms);
           t.put(p);
           // with ro ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+          p.setACL(readPerms);
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(writePerms);
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+          p.setACL(readPerms);
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(writePerms);
           t.put(p);
         }
         return null;
@@ -204,6 +216,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     };
     // Confirm special read access set at cell level
 
+    verifyAllowed(GROUP_USER, getQ1, 2);
     verifyAllowed(USER_OTHER, getQ1, 2);
 
     // store two sets of values, one store with a cell level ACL, and one
@@ -215,13 +228,13 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
             Table t = conn.getTable(TEST_TABLE.getTableName())) {
           Put p;
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(writePerms);
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+          p.setACL(readPerms);
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(writePerms);
           t.put(p);
         }
         return null;
@@ -230,6 +243,15 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     // Confirm special read access set at cell level
 
     verifyAllowed(USER_OTHER, get2, 1);
+    verifyAllowed(GROUP_USER, get2, 1);
+  }
+
+  private Map<String, Permission> prepareCellPermissions(String[] users, Action... action) {
+    Map<String, Permission> perms = new HashMap<String, Permission>(2);
+    for (String user : users) {
+      perms.put(user, new Permission(action));
+    }
+    return perms;
   }
 
   @Test
@@ -275,18 +297,16 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            // with rw ACL for "user1" and "user2"
+            // with rw ACL for "user1", "user2" and "@group"
             Put p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, ZERO);
             p.add(TEST_FAMILY1, TEST_Q2, ZERO);
-            Map<String, Permission> perms = new HashMap<String, Permission>();
-            perms.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            perms.put(user2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
+            Map<String, Permission> perms =
+                prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
+                    AuthUtil.toGroupEntry(GROUP) }, Action.READ, Action.WRITE);
             p.setACL(perms);
             t.put(p);
-            // with rw ACL for "user1" and "user2"
+            // with rw ACL for "user1", "user2" and "@group"
             p = new Put(TEST_ROW2);
             p.add(TEST_FAMILY1, TEST_Q1, ZERO);
             p.add(TEST_FAMILY1, TEST_Q2, ZERO);
@@ -316,32 +336,42 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     });
     // user2 should not be allowed to delete TEST_ROW2 as he is having write permission only on one
     // version of the cells.
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+    verifyUserDeniedForDeleteMultipleVersions(user2, TEST_ROW2, TEST_Q1, TEST_Q2);
+
+    // GROUP_USER should not be allowed to delete TEST_ROW2 as he is having write permission only on
+    // one version of the cells.
+    verifyUserDeniedForDeleteMultipleVersions(GROUP_USER, TEST_ROW2, TEST_Q1, TEST_Q2);
+
+    // user1 should be allowed to delete the cf. (All data under cf for a row)
+    user1.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Delete d = new Delete(TEST_ROW2);
-            d.deleteColumns(TEST_FAMILY1, TEST_Q1);
-            d.deleteColumns(TEST_FAMILY1, TEST_Q2);
+            d.deleteFamily(TEST_FAMILY1);
             t.delete(d);
-            fail("user2 should not be allowed to delete the row");
-          } catch (Exception e) {
-
           }
         }
         return null;
       }
     });
-    // user1 should be allowed to delete the cf. (All data under cf for a row)
-    user1.runAs(new PrivilegedExceptionAction<Void>() {
+  }
+
+  private void verifyUserDeniedForDeleteMultipleVersions(final User user, final byte[] row,
+      final byte[] q1, final byte[] q2) throws IOException, InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(TEST_ROW2);
-            d.deleteFamily(TEST_FAMILY1);
+            Delete d = new Delete(row);
+            d.addColumns(TEST_FAMILY1, q1);
+            d.addColumns(TEST_FAMILY1, q2);
             t.delete(d);
+            fail(user.getShortName() + " should not be allowed to delete the row");
+          } catch (Exception e) {
+
           }
         }
         return null;
@@ -361,14 +391,19 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             // Store a read write ACL without a timestamp, server will use current time
             Put p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q2, ONE);
-            p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
+            Map<String, Permission> readAndWritePerms =
+                prepareCellPermissions(usersAndGroups, Action.READ, Action.WRITE);
+            p.setACL(readAndWritePerms);
+            t.put(p);
+            p = new Put(TEST_ROW).add(TEST_FAMILY2, TEST_Q2, ONE);
+            p.setACL(readAndWritePerms);
             t.put(p);
             LOG.info("Stored at current time");
             // Store read only ACL at a future time
             p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1,
                 EnvironmentEdgeManager.currentTime() + 1000000, ZERO);
-            p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
+            p.setACL(prepareCellPermissions(new String[]{ USER_OTHER.getShortName(),
+                AuthUtil.toGroupEntry(GROUP)}, Action.READ));
             t.put(p);
           }
         }
@@ -402,17 +437,32 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       }
     };
 
-    verifyAllowed(getQ1, USER_OWNER, USER_OTHER);
-    verifyAllowed(getQ2, USER_OWNER, USER_OTHER);
+    verifyAllowed(getQ1, USER_OWNER, USER_OTHER, GROUP_USER);
+    verifyAllowed(getQ2, USER_OWNER, USER_OTHER, GROUP_USER);
 
 
     // Issue a DELETE for the family, should succeed because the future ACL is
     // not considered
+    AccessTestAction deleteFamily1 = getDeleteFamilyAction(TEST_FAMILY1);
+    AccessTestAction deleteFamily2 = getDeleteFamilyAction(TEST_FAMILY2);
+
+    verifyAllowed(deleteFamily1, USER_OTHER);
+    verifyAllowed(deleteFamily2, GROUP_USER);
+
+    // The future put should still exist
+
+    verifyAllowed(getQ1, USER_OWNER, USER_OTHER,GROUP_USER);
+
+    // The other put should be covered by the tombstone
 
-    AccessTestAction deleteFamily = new AccessTestAction() {
+    verifyIfNull(getQ2, USER_OTHER, GROUP_USER);
+  }
+
+  private AccessTestAction getDeleteFamilyAction(final byte[] fam) {
+    AccessTestAction deleteFamilyAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Delete delete = new Delete(TEST_ROW).deleteFamily(TEST_FAMILY1);
+        Delete delete = new Delete(TEST_ROW).addFamily(fam);
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             t.delete(delete);
@@ -421,16 +471,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         return null;
       }
     };
-
-    verifyAllowed(deleteFamily, USER_OTHER);
-
-    // The future put should still exist
-    
-    verifyAllowed(getQ1, USER_OWNER, USER_OTHER);
-    
-    // The other put should be covered by the tombstone
-
-    verifyIfNull(getQ2, USER_OTHER);
+    return deleteFamilyAction;
   }
 
   @Test
@@ -444,32 +485,27 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
             Put p = new Put(TEST_ROW);
             p.add(TEST_FAMILY1, TEST_Q1, 123L, ZERO);
             p.add(TEST_FAMILY1, TEST_Q2, 123L, ZERO);
-            Map<String, Permission> perms = new HashMap<String, Permission>();
-            perms.put(USER_OTHER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            perms.put(USER_OTHER2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            p.setACL(perms);
+            p.setACL(prepareCellPermissions(
+              new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP),
+                  USER_OTHER2.getShortName() }, Permission.Action.READ, Permission.Action.WRITE));
             t.put(p);
 
             // This version (TS = 125) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
             p.add(TEST_FAMILY1, TEST_Q1, 125L, ONE);
             p.add(TEST_FAMILY1, TEST_Q2, 125L, ONE);
-            perms = new HashMap<String, Permission>();
-            perms.put(USER_OTHER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            p.setACL(perms);
+            p.setACL(prepareCellPermissions(
+              new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
+              Action.READ, Action.WRITE));
             t.put(p);
 
             // This version (TS = 127) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
             p.add(TEST_FAMILY1, TEST_Q1, 127L, TWO);
             p.add(TEST_FAMILY1, TEST_Q2, 127L, TWO);
-            perms = new HashMap<String, Permission>();
-            perms.put(USER_OTHER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            p.setACL(perms);
+            p.setACL(prepareCellPermissions(
+              new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
+              Action.READ, Action.WRITE));
             t.put(p);
 
             return null;
@@ -524,28 +560,26 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Map<String, Permission> permsU1andOwner = new HashMap<String, Permission>();
-            permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            Map<String, Permission> permsU2andOwner = new HashMap<String, Permission>();
-            permsU2andOwner.put(user2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU2andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
+            Map<String, Permission> permsU1andOwner =
+                prepareCellPermissions(
+                  new String[] { user1.getShortName(), USER_OWNER.getShortName() }, Action.READ,
+                  Action.WRITE);
+            Map<String, Permission> permsU2andGUandOwner =
+                prepareCellPermissions(
+                  new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
+                      USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
             Put p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 123, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY2, TEST_Q1, 123, ZERO);
             p.add(TEST_FAMILY2, TEST_Q2, 123, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
 
             p = new Put(TEST_ROW1);
@@ -556,7 +590,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 127, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 127, ZERO);
@@ -591,17 +625,23 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       }
     });
 
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+    verifyUserDeniedForDeleteExactVersion(user2, TEST_ROW1, TEST_Q1, TEST_Q2);
+    verifyUserDeniedForDeleteExactVersion(GROUP_USER, TEST_ROW1, TEST_Q1, TEST_Q2);
+  }
+
+  private void verifyUserDeniedForDeleteExactVersion(final User user, final byte[] row,
+      final byte[] q1, final byte[] q2) throws IOException, InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(TEST_ROW1, 127);
-            d.deleteColumns(TEST_FAMILY1, TEST_Q1);
-            d.deleteColumns(TEST_FAMILY1, TEST_Q2);
-            d.deleteFamily(TEST_FAMILY2, 129);
+            Delete d = new Delete(row, 127);
+            d.addColumns(TEST_FAMILY1, q1);
+            d.addColumns(TEST_FAMILY1, q2);
+            d.addFamily(TEST_FAMILY2, 129);
             t.delete(d);
-            fail("user2 can not do the delete");
+            fail(user.getShortName() + " can not do the delete");
           } catch (Exception e) {
 
           }
@@ -626,28 +666,26 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Map<String, Permission> permsU1andOwner = new HashMap<String, Permission>();
-            permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            Map<String, Permission> permsU2andOwner = new HashMap<String, Permission>();
-            permsU2andOwner.put(user2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU2andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
+            Map<String, Permission> permsU1andOwner =
+                prepareCellPermissions(
+                  new String[] { user1.getShortName(), USER_OWNER.getShortName() }, Action.READ,
+                  Action.WRITE);
+            Map<String, Permission> permsU2andGUandOwner =
+                prepareCellPermissions(
+                  new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
+                      USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
             Put p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 123, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 127, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 127, ZERO);
@@ -676,16 +714,22 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       }
     });
 
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+    verifyUserDeniedForIncrementMultipleVersions(user2, TEST_ROW1, TEST_Q2);
+    verifyUserDeniedForIncrementMultipleVersions(GROUP_USER, TEST_ROW1, TEST_Q2);
+  }
+
+  private void verifyUserDeniedForIncrementMultipleVersions(final User user, final byte[] row,
+      final byte[] q1) throws IOException, InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Increment inc = new Increment(TEST_ROW1);
+            Increment inc = new Increment(row);
             inc.setTimeRange(0, 127);
-            inc.addColumn(TEST_FAMILY1, TEST_Q2, 2L);
+            inc.addColumn(TEST_FAMILY1, q1, 2L);
             t.increment(inc);
-            fail();
+            fail(user.getShortName() + " cannot do the increment.");
           } catch (Exception e) {
 
           }
@@ -710,15 +754,17 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Map<String, Permission> permsU1andOwner = new HashMap<String, Permission>();
-            permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            Map<String, Permission> permsU2andOwner = new HashMap<String, Permission>();
-            permsU2andOwner.put(user2.getShortName(), new Permission(Permission.Action.READ,
+            Map<String, Permission> permsU1andOwner =
+                prepareCellPermissions(
+                  new String[] { user1.getShortName(), USER_OWNER.getShortName() }, Action.READ,
+                  Action.WRITE);
+            Map<String, Permission> permsU2andGUandOwner =
+                prepareCellPermissions(
+                  new String[] { user1.getShortName(), AuthUtil.toGroupEntry(GROUP),
+                      USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
+            permsU2andGUandOwner.put(user2.getShortName(), new Permission(Permission.Action.READ,
                 Permission.Action.WRITE));
-            permsU2andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
+            permsU2andGUandOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
                 Permission.Action.WRITE));
             Put p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 123, ZERO);
@@ -726,12 +772,12 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 123, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 127, ZERO);
-            p.setACL(permsU2andOwner);
+            p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 127, ZERO);
@@ -763,18 +809,24 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       }
     });
 
-    // Should be denied.
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+    verifyUserDeniedForPutMultipleVersions(user2, TEST_ROW1, TEST_Q1, TEST_Q2, ZERO);
+    verifyUserDeniedForPutMultipleVersions(GROUP_USER, TEST_ROW1, TEST_Q1, TEST_Q2, ZERO);
+  }
+
+  private void verifyUserDeniedForPutMultipleVersions(final User user, final byte[] row,
+      final byte[] q1, final byte[] q2, final byte[] value) throws IOException,
+      InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Put p = new Put(TEST_ROW1);
+            Put p = new Put(row);
             // column Q1 covers version at 123 fr which user2 do not have permission
-            p.add(TEST_FAMILY1, TEST_Q1, 124, ZERO);
-            p.add(TEST_FAMILY1, TEST_Q2, ZERO);
+            p.addColumn(TEST_FAMILY1, q1, 124, value);
+            p.addColumn(TEST_FAMILY1, q2, value);
             t.put(p);
-            fail();
+            fail(user.getShortName() + " cannot do the put.");
           } catch (Exception e) {
 
           }
@@ -787,55 +839,57 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   @Test
   public void testCellPermissionsForCheckAndDelete() throws Exception {
     final byte[] TEST_ROW1 = Bytes.toBytes("r1");
+    final byte[] TEST_Q3 = Bytes.toBytes("q3");
     final byte[] ZERO = Bytes.toBytes(0L);
 
     final User user1 = User.createUserForTesting(conf, "user1", new String[0]);
     final User user2 = User.createUserForTesting(conf, "user2", new String[0]);
-    
+
     verifyAllowed(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Map<String, Permission> permsU1andOwner = new HashMap<String, Permission>();
-            permsU1andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            Map<String, Permission> permsU1andU2andOwner = new HashMap<String, Permission>();
-            permsU1andU2andOwner.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andU2andOwner.put(user2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andU2andOwner.put(USER_OWNER.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            Map<String, Permission> permsU1andU2 = new HashMap<String, Permission>();
-            permsU1andU2.put(user1.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
-            permsU1andU2.put(user2.getShortName(), new Permission(Permission.Action.READ,
-                Permission.Action.WRITE));
+            Map<String, Permission> permsU1andOwner =
+                prepareCellPermissions(
+                  new String[] { user1.getShortName(), USER_OWNER.getShortName() }, Action.READ,
+                  Action.WRITE);
+            Map<String, Permission> permsU1andU2andGUandOwner =
+                prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
+                    AuthUtil.toGroupEntry(GROUP), USER_OWNER.getShortName() }, Action.READ,
+                  Action.WRITE);
+            Map<String, Permission> permsU1_U2andGU =
+                prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
+                    AuthUtil.toGroupEntry(GROUP) }, Action.READ, Action.WRITE);
 
             Put p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 120, ZERO);
             p.add(TEST_FAMILY1, TEST_Q2, 120, ZERO);
-            p.setACL(permsU1andU2andOwner);
+            p.add(TEST_FAMILY1, TEST_Q3, 120, ZERO);
+            p.setACL(permsU1andU2andGUandOwner);
             t.put(p);
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 123, ZERO);
             p.add(TEST_FAMILY1, TEST_Q2, 123, ZERO);
+            p.add(TEST_FAMILY1, TEST_Q3, 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q1, 127, ZERO);
-            p.setACL(permsU1andU2);
+            p.setACL(permsU1_U2andGU);
             t.put(p);
 
             p = new Put(TEST_ROW1);
             p.add(TEST_FAMILY1, TEST_Q2, 127, ZERO);
             p.setACL(user2.getShortName(), new Permission(Permission.Action.READ));
             t.put(p);
+
+            p = new Put(TEST_ROW1);
+            p.addColumn(TEST_FAMILY1, TEST_Q3, 127, ZERO);
+            p.setACL(AuthUtil.toGroupEntry(GROUP), new Permission(Permission.Action.READ));
+            t.put(p);
           }
         }
         return null;
@@ -859,32 +913,54 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     });
     // user2 shouldn't be allowed to do the checkAndDelete. user2 having RW permission on the latest
     // version cell but not on cell version TS=123
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+    verifyUserDeniedForCheckAndDelete(user2, TEST_ROW1, ZERO);
+
+    // GROUP_USER shouldn't be allowed to do the checkAndDelete. GROUP_USER having RW permission on
+    // the latest
+    // version cell but not on cell version TS=123
+    verifyUserDeniedForCheckAndDelete(GROUP_USER, TEST_ROW1, ZERO);
+
+    // user2 should be allowed to do the checkAndDelete when delete tries to delete the old version
+    // TS=120. user2 having R permission on the latest version(no W permission) cell
+    // and W permission on cell version TS=120.
+    verifyUserAllowedforCheckAndDelete(user2, TEST_ROW1, TEST_Q2, ZERO);
+
+    // GROUP_USER should be allowed to do the checkAndDelete when delete tries to delete the old
+    // version
+    // TS=120. user2 having R permission on the latest version(no W permission) cell
+    // and W permission on cell version TS=120.
+    verifyUserAllowedforCheckAndDelete(GROUP_USER, TEST_ROW1, TEST_Q3, ZERO);
+  }
+
+  private void verifyUserAllowedforCheckAndDelete(final User user, final byte[] row,
+      final byte[] q1, final byte[] value) throws IOException, InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(TEST_ROW1);
-            d.deleteColumns(TEST_FAMILY1, TEST_Q1);
-            t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q1, ZERO, d);
-            fail("user2 should not be allowed to do checkAndDelete");
-          } catch (Exception e) {
+            Delete d = new Delete(row);
+            d.addColumn(TEST_FAMILY1, q1, 120);
+            t.checkAndDelete(row, TEST_FAMILY1, q1, value, d);
           }
         }
         return null;
       }
     });
-    // user2 should be allowed to do the checkAndDelete when delete tries to delete the old version
-    // TS=120. user2 having R permission on the latest version(no W permission) cell
-    // and W permission on cell version TS=120.
-    user2.runAs(new PrivilegedExceptionAction<Void>() {
+  }
+
+  private void verifyUserDeniedForCheckAndDelete(final User user, final byte[] row,
+      final byte[] value) throws IOException, InterruptedException {
+    user.runAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(TEST_ROW1);
-            d.deleteColumn(TEST_FAMILY1, TEST_Q2, 120);
-            t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q2, ZERO, d);
+            Delete d = new Delete(row);
+            d.addColumns(TEST_FAMILY1, TEST_Q1);
+            t.checkAndDelete(row, TEST_FAMILY1, TEST_Q1, value, d);
+            fail(user.getShortName() + " should not be allowed to do checkAndDelete");
+          } catch (Exception e) {
           }
         }
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/84068461/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java
index bccf17c..393bb36 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java
@@ -19,11 +19,14 @@ package org.apache.hadoop.hbase.security.access;
 
 import static org.junit.Assert.assertEquals;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -32,6 +35,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
@@ -84,8 +89,11 @@ public class TestCellACLs extends SecureTestUtil {
 
   private static Configuration conf;
 
+  private static final String GROUP = "group";
+  private static User GROUP_USER;
   private static User USER_OWNER;
   private static User USER_OTHER;
+  private static String[] usersAndGroups;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -116,6 +124,9 @@ public class TestCellACLs extends SecureTestUtil {
     // create a set of test users
     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
     USER_OTHER = User.createUserForTesting(conf, "other", new String[0]);
+    GROUP_USER = User.createUserForTesting(conf, "group_user", new String[] { GROUP });
+
+    usersAndGroups = new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) };
   }
 
   @AfterClass
@@ -149,11 +160,11 @@ public class TestCellACLs extends SecureTestUtil {
           Put p;
           // with ro ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(USER_OTHER.getShortName(), new Permission(Action.READ));
+          p.setACL(prepareCellPermissions(usersAndGroups, 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(Action.READ, Action.WRITE));
+          p.setACL(prepareCellPermissions(usersAndGroups, Action.READ, Action.WRITE));
           t.put(p);
           // no ACL
           p = new Put(TEST_ROW)
@@ -223,13 +234,13 @@ public class TestCellACLs extends SecureTestUtil {
 
     // Confirm special read access set at cell level
 
-    verifyAllowed(getQ1, USER_OTHER);
-    verifyAllowed(getQ2, USER_OTHER);
+    verifyAllowed(getQ1, USER_OTHER, GROUP_USER);
+    verifyAllowed(getQ2, USER_OTHER, GROUP_USER);
 
     // Confirm this access does not extend to other cells
 
-    verifyIfNull(getQ3, USER_OTHER);
-    verifyIfNull(getQ4, USER_OTHER);
+    verifyIfNull(getQ3, USER_OTHER, GROUP_USER);
+    verifyIfNull(getQ4, USER_OTHER, GROUP_USER);
 
     /* ---- Scans ---- */
 
@@ -271,6 +282,10 @@ public class TestCellACLs extends SecureTestUtil {
     verifyAllowed(scanAction, USER_OTHER);
     assertEquals(2, scanResults.size());
 
+    scanResults.clear();
+    verifyAllowed(scanAction, GROUP_USER);
+    assertEquals(2, scanResults.size());
+
     /* ---- Increments ---- */
 
     AccessTestAction incrementQ1 = new AccessTestAction() {
@@ -331,16 +346,16 @@ public class TestCellACLs extends SecureTestUtil {
       }
     };
 
-    verifyDenied(incrementQ1, USER_OTHER);
-    verifyDenied(incrementQ3, USER_OTHER);
+    verifyDenied(incrementQ1, USER_OTHER, GROUP_USER);
+    verifyDenied(incrementQ3, USER_OTHER, GROUP_USER);
 
-    // We should be able to increment Q2 twice, the previous ACL will be
-    // carried forward
-    verifyAllowed(incrementQ2, USER_OTHER);
+    // We should be able to increment until the permissions are revoked (including the action in
+    // which permissions are revoked, the previous ACL will be carried forward)
+    verifyAllowed(incrementQ2, USER_OTHER, GROUP_USER);
     verifyAllowed(incrementQ2newDenyACL, USER_OTHER);
     // But not again after we denied ourselves write permission with an ACL
     // update
-    verifyDenied(incrementQ2, USER_OTHER);
+    verifyDenied(incrementQ2, USER_OTHER, GROUP_USER);
 
     /* ---- Deletes ---- */
 
@@ -372,8 +387,8 @@ public class TestCellACLs extends SecureTestUtil {
       }
     };
 
-    verifyDenied(deleteFamily, USER_OTHER);
-    verifyDenied(deleteQ1, USER_OTHER);
+    verifyDenied(deleteFamily, USER_OTHER, GROUP_USER);
+    verifyDenied(deleteQ1, USER_OTHER, GROUP_USER);
     verifyAllowed(deleteQ1, USER_OWNER);
   }
 
@@ -384,26 +399,18 @@ public class TestCellACLs extends SecureTestUtil {
   @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);
+    grantOnTable(TEST_UTIL, USER_OTHER.getShortName(), TEST_TABLE.getTableName(), TEST_FAMILY,
+      null, Action.READ);
+    // Grant read access to GROUP
+    grantOnTable(TEST_UTIL, AuthUtil.toGroupEntry(GROUP), TEST_TABLE.getTableName(), TEST_FAMILY,
+      null, Action.READ);
 
-    // A write by USER_OTHER should be denied.
+ // 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 {
-        Table 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);
+    verfifyUserDeniedForWrite(USER_OTHER, ZERO);
+    // A write by GROUP_USER from group GROUP should be denied.
+    verfifyUserDeniedForWrite(GROUP_USER, ZERO);
 
     // Add the cell
     verifyAllowed(new AccessTestAction() {
@@ -422,35 +429,52 @@ public class TestCellACLs extends SecureTestUtil {
     }, USER_OWNER);
 
     // A write by USER_OTHER should still be denied, just to make sure
+    verfifyUserDeniedForWrite(USER_OTHER, ONE);
+    // A write by GROUP_USER from group GROUP should still be denied
+    verfifyUserDeniedForWrite(GROUP_USER, ONE);
+
+    // A read by USER_OTHER should be allowed, just to make sure
+    verifyUserAllowedForRead(USER_OTHER);
+    // A read by GROUP_USER from group GROUP should be allowed
+    verifyUserAllowedForRead(GROUP_USER);
+  }
+
+  private void verfifyUserDeniedForWrite(final User user, final byte[] value) throws Exception {
     verifyDenied(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Table t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
+        try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table t = connection.getTable(TEST_TABLE.getTableName())) {
           Put p;
-          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ONE);
+          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, value);
           t.put(p);
-        } finally {
-          t.close();
         }
         return null;
       }
-    }, USER_OTHER);
+    }, user);
+  }
 
-    // A read by USER_OTHER should be allowed, just to make sure
+  private void verifyUserAllowedForRead(final User user) throws Exception {
     verifyAllowed(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Table t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
+        try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table t = connection.getTable(TEST_TABLE.getTableName())) {
           return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1));
-        } finally {
-          t.close();
         }
       }
-    }, USER_OTHER);
+    }, user);
   }
 
+  private Map<String, Permission> prepareCellPermissions(String[] users, Action... action) {
+    Map<String, Permission> perms = new HashMap<String, Permission>(2);
+    for (String user : users) {
+      perms.put(user, new Permission(action));
+    }
+    return perms;
+  }
+
+
   @After
   public void tearDown() throws Exception {
     // Clean the _acl_ table