You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/04/12 00:02:08 UTC

svn commit: r1586785 - in /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access: SecureTestUtil.java TestAccessController.java TestCellACLWithMultipleVersions.java TestNamespaceCommands.java

Author: apurtell
Date: Fri Apr 11 22:02:08 2014
New Revision: 1586785

URL: http://svn.apache.org/r1586785
Log:
HBASE-10963 Refactor cell ACL tests

Modified:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java?rev=1586785&r1=1586784&r2=1586785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java Fri Apr 11 22:02:08 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.security.access;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -57,6 +58,7 @@ import com.google.protobuf.ServiceExcept
  * Utility methods for testing security
  */
 public class SecureTestUtil {
+  
   private static final Log LOG = LogFactory.getLog(SecureTestUtil.class);
   private static final int WAIT_TIME = 10000;
 
@@ -85,6 +87,17 @@ public class SecureTestUtil {
     conf.setInt("hfile.format.version", 3);
   }
 
+  public static void verifyConfiguration(Configuration conf) {
+    if (!(conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY).contains(
+        AccessController.class.getName())
+        && conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY).contains(
+            AccessController.class.getName()) && conf.get(
+        CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY).contains(
+        AccessController.class.getName()))) {
+      throw new RuntimeException("AccessController is missing from a system coprocessor list");
+    }
+  }
+
   public void checkTablePerms(Configuration conf, byte[] table, byte[] family, byte[] column,
       Permission.Action... actions) throws IOException {
     Permission[] perms = new Permission[actions.length];
@@ -148,6 +161,21 @@ public class SecureTestUtil {
     }
   }
 
+  public void verifyAllowed(User user, AccessTestAction action, int count) throws Exception {
+    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() + "'");
+        }
+        assertEquals(results.size(), count);
+      }
+    } catch (AccessDeniedException ade) {
+      fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
+    }
+  }
+
   public void verifyDenied(User user, AccessTestAction... actions) throws Exception {
     for (AccessTestAction action : actions) {
       try {

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=1586785&r1=1586784&r2=1586785&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 Fri Apr 11 22:02:08 2014
@@ -24,8 +24,6 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -37,7 +35,6 @@ 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.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -61,7 +58,6 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -110,7 +106,6 @@ 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.RpcCallback;
 import com.google.protobuf.RpcController;
@@ -160,17 +155,6 @@ public class TestAccessController extend
   private static RegionServerCoprocessorEnvironment RSCP_ENV;
   private RegionCoprocessorEnvironment RCP_ENV;
 
-  static void verifyConfiguration(Configuration conf) {
-    if (!(conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY)
-            .contains(AccessController.class.getName())
-          && conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY)
-            .contains(AccessController.class.getName())
-          && conf.get(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY)
-            .contains(AccessController.class.getName()))) {
-      throw new RuntimeException("AccessController is missing from a system coprocessor list");
-    }
-  }
-
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     // setup configuration
@@ -181,7 +165,7 @@ public class TestAccessController extend
     conf.set("hbase.master.logcleaner.plugins",
       "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
     // Enable security
-    SecureTestUtil.enableSecurity(conf);
+    enableSecurity(conf);
     // Verify enableSecurity sets up what we require
     verifyConfiguration(conf);
 
@@ -237,25 +221,25 @@ public class TestAccessController extend
 
     // Set up initial grants
 
-    SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(),
+    grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(),
       Permission.Action.ADMIN,
       Permission.Action.CREATE,
       Permission.Action.READ,
       Permission.Action.WRITE);
 
-    SecureTestUtil.grantOnTable(TEST_UTIL, USER_RW.getShortName(),
+    grantOnTable(TEST_UTIL, USER_RW.getShortName(),
       TEST_TABLE.getTableName(), TEST_FAMILY, null,
       Permission.Action.READ,
       Permission.Action.WRITE);
 
     // USER_CREATE is USER_RW plus CREATE permissions
-    SecureTestUtil.grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
+    grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
       TEST_TABLE.getTableName(), null, null,
       Permission.Action.CREATE,
       Permission.Action.READ,
       Permission.Action.WRITE);
 
-    SecureTestUtil.grantOnTable(TEST_UTIL, USER_RO.getShortName(),
+    grantOnTable(TEST_UTIL, USER_RO.getShortName(),
       TEST_TABLE.getTableName(), TEST_FAMILY, null,
       Permission.Action.READ);
 
@@ -956,383 +940,6 @@ public class TestAccessController extend
   }
 
   @Test
-  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();
-        }
-      }
-    };
-
-    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 testCellPermissionsWithDeleteMutipleVersions() throws Exception {
-    // table/column/qualifier level permissions
-    final byte[] TEST_ROW1 = Bytes.toBytes("r1");
-    final byte[] TEST_ROW2 = Bytes.toBytes("r2");
-    final byte[] TEST_Q1 = Bytes.toBytes("q1");
-    final byte[] TEST_Q2 = Bytes.toBytes("q2");
-    final byte[] ZERO = Bytes.toBytes(0L);
-
-    // additional test user
-    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 {
-        HTable t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
-          // with rw ACL for "user1"
-          Put p = new Put(TEST_ROW1);
-          p.add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.add(TEST_FAMILY, TEST_Q2, ZERO);
-          p.setACL(user1.getShortName(), new Permission(Permission.Action.READ,
-              Permission.Action.WRITE));
-          t.put(p);
-          // with rw ACL for "user1"
-          p = new Put(TEST_ROW2);
-          p.add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.add(TEST_FAMILY, TEST_Q2, ZERO);
-          p.setACL(user1.getShortName(), new Permission(Permission.Action.READ,
-              Permission.Action.WRITE));
-          t.put(p);
-        } finally {
-          t.close();
-        }
-        return null;
-      }
-    }, USER_OWNER);
-
-    verifyAllowed(new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        HTable t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
-          // with rw ACL for "user1" and "user2"
-          Put p = new Put(TEST_ROW1);
-          p.add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.add(TEST_FAMILY, 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));
-          p.setACL(perms);
-          t.put(p);
-          // with rw ACL for "user1" and "user2"
-          p = new Put(TEST_ROW2);
-          p.add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.add(TEST_FAMILY, TEST_Q2, ZERO);
-          p.setACL(perms);
-          t.put(p);
-        } finally {
-          t.close();
-        }
-        return null;
-      }
-    }, user1);
-
-    // user1 should be allowed to delete TEST_ROW1 as he is having write permission on both
-    // versions of the cells
-    user1.runAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        HTable t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
-          Delete d = new Delete(TEST_ROW1);
-          d.deleteColumns(TEST_FAMILY, TEST_Q1);
-          d.deleteColumns(TEST_FAMILY, TEST_Q2);
-          t.delete(d);
-        } finally {
-          t.close();
-        }
-        return null;
-      }
-    });
-    // 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>() {
-      @Override
-      public Void run() throws Exception {
-        HTable t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
-          Delete d = new Delete(TEST_ROW2);
-          d.deleteColumns(TEST_FAMILY, TEST_Q1);
-          d.deleteColumns(TEST_FAMILY, TEST_Q2);
-          t.delete(d);
-          fail("user2 should not be allowed to delete the row");
-        } catch (Exception e) {
-
-        } finally {
-          t.close();
-        }
-        return null;
-      }
-    });
-    // 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 {
-        HTable t = new HTable(conf, TEST_TABLE.getTableName());
-        try {
-          Delete d = new Delete(TEST_ROW2);
-          d.deleteFamily(TEST_FAMILY);
-          t.delete(d);
-        } finally {
-          t.close();
-        }
-        return null;
-      }
-    });
-  }
-
-  @Test
   public void testGrantRevoke() throws Exception {
     AccessTestAction grantAction = new AccessTestAction() {
       @Override
@@ -1568,9 +1175,9 @@ public class TestAccessController extend
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant table read permission
-    SecureTestUtil.grantGlobal(TEST_UTIL, gblUser.getShortName(),
+    grantGlobal(TEST_UTIL, gblUser.getShortName(),
       Permission.Action.READ);
-    SecureTestUtil.grantOnTable(TEST_UTIL, tblUser.getShortName(),
+    grantOnTable(TEST_UTIL, tblUser.getShortName(),
       tableName, null, null,
       Permission.Action.READ);
 
@@ -1584,9 +1191,9 @@ public class TestAccessController extend
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant table write permission while revoking read permissions
-    SecureTestUtil.grantGlobal(TEST_UTIL, gblUser.getShortName(),
+    grantGlobal(TEST_UTIL, gblUser.getShortName(),
       Permission.Action.WRITE);
-    SecureTestUtil.grantOnTable(TEST_UTIL, tblUser.getShortName(),
+    grantOnTable(TEST_UTIL, tblUser.getShortName(),
       tableName, null, null,
       Permission.Action.WRITE);
 
@@ -1599,8 +1206,8 @@ public class TestAccessController extend
     verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // revoke table permissions
-    SecureTestUtil.revokeGlobal(TEST_UTIL, gblUser.getShortName());
-    SecureTestUtil.revokeFromTable(TEST_UTIL, tblUser.getShortName(),
+    revokeGlobal(TEST_UTIL, gblUser.getShortName());
+    revokeFromTable(TEST_UTIL, tblUser.getShortName(),
       tableName, null, null);
 
     verifyDenied(tblUser, getActionAll, getAction1, getAction2);
@@ -1612,9 +1219,9 @@ public class TestAccessController extend
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant column family read permission
-    SecureTestUtil.grantGlobal(TEST_UTIL, gblUser.getShortName(),
+    grantGlobal(TEST_UTIL, gblUser.getShortName(),
       Permission.Action.READ);
-    SecureTestUtil.grantOnTable(TEST_UTIL, tblUser.getShortName(),
+    grantOnTable(TEST_UTIL, tblUser.getShortName(),
       tableName, family1, null, Permission.Action.READ);
 
     // Access should be denied for family2
@@ -1628,9 +1235,9 @@ public class TestAccessController extend
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant column family write permission
-    SecureTestUtil.grantGlobal(TEST_UTIL, gblUser.getShortName(),
+    grantGlobal(TEST_UTIL, gblUser.getShortName(),
       Permission.Action.WRITE);
-    SecureTestUtil.grantOnTable(TEST_UTIL, tblUser.getShortName(),
+    grantOnTable(TEST_UTIL, tblUser.getShortName(),
       tableName, family2, null, Permission.Action.WRITE);
 
     // READ from family1, WRITE to family2 are allowed
@@ -1645,8 +1252,8 @@ public class TestAccessController extend
     verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // revoke column family permission
-    SecureTestUtil.revokeGlobal(TEST_UTIL, gblUser.getShortName());
-    SecureTestUtil.revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null);
+    revokeGlobal(TEST_UTIL, gblUser.getShortName());
+    revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null);
 
     // Revoke on family2 should not have impact on family1 permissions
     verifyAllowed(tblUser, getActionAll, getAction1);
@@ -1736,13 +1343,13 @@ public class TestAccessController extend
       }
     };
 
-    SecureTestUtil.revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, null);
+    revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, null);
 
     verifyDenied(user, getQualifierAction);
     verifyDenied(user, putQualifierAction);
     verifyDenied(user, deleteQualifierAction);
 
-    SecureTestUtil.grantOnTable(TEST_UTIL, user.getShortName(),
+    grantOnTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier,
       Permission.Action.READ);
 
@@ -1752,7 +1359,7 @@ public class TestAccessController extend
 
     // only grant write permission
     // TODO: comment this portion after HBASE-3583
-    SecureTestUtil.grantOnTable(TEST_UTIL, user.getShortName(),
+    grantOnTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier,
       Permission.Action.WRITE);
 
@@ -1761,7 +1368,7 @@ public class TestAccessController extend
     verifyAllowed(user, deleteQualifierAction);
 
     // grant both read and write permission
-    SecureTestUtil.grantOnTable(TEST_UTIL, user.getShortName(),
+    grantOnTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier,
       Permission.Action.READ, Permission.Action.WRITE);
 
@@ -1770,7 +1377,7 @@ public class TestAccessController extend
     verifyAllowed(user, deleteQualifierAction);
 
     // revoke family level permission won't impact column level
-    SecureTestUtil.revokeFromTable(TEST_UTIL, user.getShortName(),
+    revokeFromTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier);
 
     verifyDenied(user, getQualifierAction);
@@ -1828,7 +1435,7 @@ public class TestAccessController extend
       hasFoundUserPermission(up, perms));
 
     // grant read permission
-    SecureTestUtil.grantOnTable(TEST_UTIL, user.getShortName(),
+    grantOnTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier, Permission.Action.READ);
 
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -1852,7 +1459,7 @@ public class TestAccessController extend
       hasFoundUserPermission(upToVerify, perms));
 
     // grant read+write
-    SecureTestUtil.grantOnTable(TEST_UTIL, user.getShortName(),
+    grantOnTable(TEST_UTIL, user.getShortName(),
       tableName, family1, qualifier,
       Permission.Action.WRITE, Permission.Action.READ);
 
@@ -1872,7 +1479,7 @@ public class TestAccessController extend
       hasFoundUserPermission(upToVerify, perms));
 
     // revoke
-    SecureTestUtil.revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+    revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
       Permission.Action.WRITE, Permission.Action.READ);
     
     acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
@@ -2031,13 +1638,13 @@ public class TestAccessController extend
     User userColumn = User.createUserForTesting(conf, "user_check_perms_family", new String[0]);
     User userQualifier = User.createUserForTesting(conf, "user_check_perms_q", new String[0]);
 
-    SecureTestUtil.grantOnTable(TEST_UTIL, userTable.getShortName(),
+    grantOnTable(TEST_UTIL, userTable.getShortName(),
       TEST_TABLE.getTableName(), null, null,
       Permission.Action.READ);
-    SecureTestUtil.grantOnTable(TEST_UTIL, userColumn.getShortName(),
+    grantOnTable(TEST_UTIL, userColumn.getShortName(),
       TEST_TABLE.getTableName(), TEST_FAMILY, null,
       Permission.Action.READ);
-    SecureTestUtil.grantOnTable(TEST_UTIL, userQualifier.getShortName(),
+    grantOnTable(TEST_UTIL, userQualifier.getShortName(),
       TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
       Permission.Action.READ);
 
@@ -2252,7 +1859,7 @@ public class TestAccessController extend
     String currentUser = User.getCurrent().getShortName();
     String activeUserForNewRs = currentUser + ".hfs." +
       hbaseCluster.getLiveRegionServerThreads().size();
-    SecureTestUtil.grantGlobal(TEST_UTIL, activeUserForNewRs,
+    grantGlobal(TEST_UTIL, activeUserForNewRs,
       Permission.Action.ADMIN, Permission.Action.CREATE, Permission.Action.READ,
         Permission.Action.WRITE);
 
@@ -2320,7 +1927,7 @@ public class TestAccessController extend
     User TABLE_ADMIN = User.createUserForTesting(conf, "UserA", new String[0]);
 
     // Grant TABLE ADMIN privs
-    SecureTestUtil.grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
+    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
       TEST_TABLE.getTableName(), null, null,
       Permission.Action.ADMIN);
 
@@ -2362,7 +1969,7 @@ public class TestAccessController extend
     User TABLE_ADMIN = User.createUserForTesting(conf, "TestUser", new String[0]);
 
     // Grant TABLE ADMIN privs
-    SecureTestUtil.grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
+    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
       TEST_TABLE.getTableName(), null, null,
       Permission.Action.ADMIN);
 
@@ -2401,7 +2008,7 @@ public class TestAccessController extend
     verifyDenied(getAction, USER_NONE);
 
     // Grant namespace READ to USER_NONE, this should supersede any table permissions
-    SecureTestUtil.grantOnNamespace(TEST_UTIL, USER_NONE.getShortName(),
+    grantOnNamespace(TEST_UTIL, USER_NONE.getShortName(),
       TEST_TABLE.getTableName().getNamespaceAsString(),
       Permission.Action.READ);
 
@@ -2471,7 +2078,7 @@ public class TestAccessController extend
     User userA = User.createUserForTesting(conf, "UserA", new String[0]);
     User userB = User.createUserForTesting(conf, "UserB", new String[0]);
 
-    SecureTestUtil.grantOnTable(TEST_UTIL, userA.getShortName(),
+    grantOnTable(TEST_UTIL, userA.getShortName(),
       TEST_TABLE.getTableName(), null, null,
       Permission.Action.EXEC);
 
@@ -2495,7 +2102,7 @@ public class TestAccessController extend
     verifyAllowed(execEndpointAction, userA);
 
     // Now grant EXEC to the entire namespace to user B
-    SecureTestUtil.grantOnNamespace(TEST_UTIL, userB.getShortName(),
+    grantOnNamespace(TEST_UTIL, userB.getShortName(),
       TEST_TABLE.getTableName().getNamespaceAsString(),
       Permission.Action.EXEC);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java?rev=1586785&r1=1586784&r2=1586785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java Fri Apr 11 22:02:08 2014
@@ -20,7 +20,9 @@ package org.apache.hadoop.hbase.security
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.util.List;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,19 +33,20 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
-import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.TestTableName;
+
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
+
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -51,6 +54,7 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+
 @Category(MediumTests.class)
 public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   private static final Log LOG = LogFactory.getLog(TestCellACLWithMultipleVersions.class);
@@ -63,25 +67,16 @@ public class TestCellACLWithMultipleVers
 
   @Rule
   public TestTableName TEST_TABLE = new TestTableName();
-  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
+  private static final byte[] TEST_ROW = Bytes.toBytes("cellpermtest");
+  private static final byte[] TEST_Q1 = Bytes.toBytes("q1");
+  private static final byte[] ZERO = Bytes.toBytes(0L);
+
   private static Configuration conf;
 
-  // user is table owner. will have all permissions on table
   private static User USER_OWNER;
-  private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
-
-  private static AccessController ACCESS_CONTROLLER;
-
-  static void verifyConfiguration(Configuration conf) {
-    if (!(conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY).contains(
-        AccessController.class.getName())
-        && conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY).contains(
-            AccessController.class.getName()) && conf.get(
-        CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY).contains(
-        AccessController.class.getName()))) {
-      throw new RuntimeException("AccessController is missing from a system coprocessor list");
-    }
-  }
+  private static User USER_OTHER;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -93,7 +88,7 @@ public class TestCellACLWithMultipleVers
     conf.set("hbase.master.logcleaner.plugins",
         "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
     // Enable security
-    SecureTestUtil.enableSecurity(conf);
+    enableSecurity(conf);
     // Verify enableSecurity sets up what we require
     verifyConfiguration(conf);
 
@@ -104,19 +99,19 @@ public class TestCellACLWithMultipleVers
     MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster().getMaster()
         .getMasterCoprocessorHost();
     cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
-    ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
-    cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
-        Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    AccessController ac = (AccessController)
+      cpHost.findCoprocessor(AccessController.class.getName());
+    cpHost.createEnvironment(AccessController.class, ac, Coprocessor.PRIORITY_HIGHEST, 1, conf);
     RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
         .getRegionServerCoprocessorHost();
-    rsHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
-        Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    rsHost.createEnvironment(AccessController.class, ac, Coprocessor.PRIORITY_HIGHEST, 1, conf);
 
     // Wait for the ACL table to become available
     TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
 
     // create a set of test users
     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
+    USER_OTHER = User.createUserForTesting(conf, "other", new String[0]);
   }
 
   @AfterClass
@@ -139,19 +134,6 @@ public class TestCellACLWithMultipleVers
 
   @Test
   public void testCellPermissionwithVersions() throws Exception {
-    // table/column/qualifier level permissions
-    final byte[] TEST_ROW = Bytes.toBytes("cellpermtest");
-    final byte[] TEST_ROW1 = Bytes.toBytes("cellpermtest1");
-    final byte[] TEST_Q1 = Bytes.toBytes("q1");
-    // 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() {
@@ -162,20 +144,20 @@ public class TestCellACLWithMultipleVers
           Put p;
           // with ro ACL
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
           t.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));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.READ));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
           t.put(p);
           p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.WRITE));
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
           t.put(p);
         } finally {
           t.close();
@@ -203,7 +185,7 @@ public class TestCellACLWithMultipleVers
     AccessTestAction get2 = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Get get = new Get(TEST_ROW1);
+        Get get = new Get(TEST_ROW);
         get.setMaxVersions(10);
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
@@ -215,7 +197,7 @@ public class TestCellACLWithMultipleVers
     };
     // Confirm special read access set at cell level
 
-    verifyAllowed(userOther, getQ1, 2);
+    verifyAllowed(USER_OTHER, getQ1, 2);
 
     // store two sets of values, one store with a cell level ACL, and one
     // without
@@ -225,14 +207,14 @@ public class TestCellACLWithMultipleVers
         HTable t = new HTable(conf, TEST_TABLE.getTableName());
         try {
           Put p;
-          p = new Put(TEST_ROW1).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.WRITE));
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
           t.put(p);
-          p = new Put(TEST_ROW1).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.READ));
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.READ));
           t.put(p);
-          p = new Put(TEST_ROW1).add(TEST_FAMILY, TEST_Q1, ZERO);
-          p.setACL(userOther.getShortName(), new Permission(Permission.Action.WRITE));
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE));
           t.put(p);
         } finally {
           t.close();
@@ -242,22 +224,129 @@ public class TestCellACLWithMultipleVers
     }, USER_OWNER);
     // Confirm special read access set at cell level
 
-    verifyAllowed(userOther, get2, 1);
+    verifyAllowed(USER_OTHER, get2, 1);
   }
 
-  public void verifyAllowed(User user, AccessTestAction action, int count) throws Exception {
-    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() + "'");
+  @Test
+  public void testCellPermissionsWithDeleteMutipleVersions() throws Exception {
+    // table/column/qualifier level permissions
+    final byte[] TEST_ROW1 = Bytes.toBytes("r1");
+    final byte[] TEST_ROW2 = Bytes.toBytes("r2");
+    final byte[] TEST_Q1 = Bytes.toBytes("q1");
+    final byte[] TEST_Q2 = Bytes.toBytes("q2");
+    final byte[] ZERO = Bytes.toBytes(0L);
+
+    // additional test user
+    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 {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          // with rw ACL for "user1"
+          Put p = new Put(TEST_ROW1);
+          p.add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(user1.getShortName(), new Permission(Permission.Action.READ,
+              Permission.Action.WRITE));
+          t.put(p);
+          // with rw ACL for "user1"
+          p = new Put(TEST_ROW2);
+          p.add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(user1.getShortName(), new Permission(Permission.Action.READ,
+              Permission.Action.WRITE));
+          t.put(p);
+        } finally {
+          t.close();
         }
-        assertEquals(results.size(), count);
+        return null;
       }
-    } catch (AccessDeniedException ade) {
-      fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
-    }
+    }, USER_OWNER);
+
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          // with rw ACL for "user1" and "user2"
+          Put p = new Put(TEST_ROW1);
+          p.add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.add(TEST_FAMILY, 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));
+          p.setACL(perms);
+          t.put(p);
+          // with rw ACL for "user1" and "user2"
+          p = new Put(TEST_ROW2);
+          p.add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(perms);
+          t.put(p);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    }, user1);
+
+    // user1 should be allowed to delete TEST_ROW1 as he is having write permission on both
+    // versions of the cells
+    user1.runAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Delete d = new Delete(TEST_ROW1);
+          d.deleteColumns(TEST_FAMILY, TEST_Q1);
+          d.deleteColumns(TEST_FAMILY, TEST_Q2);
+          t.delete(d);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    });
+    // 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>() {
+      @Override
+      public Void run() throws Exception {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Delete d = new Delete(TEST_ROW2);
+          d.deleteColumns(TEST_FAMILY, TEST_Q1);
+          d.deleteColumns(TEST_FAMILY, TEST_Q2);
+          t.delete(d);
+          fail("user2 should not be allowed to delete the row");
+        } catch (Exception e) {
+
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    });
+    // 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 {
+        HTable t = new HTable(conf, TEST_TABLE.getTableName());
+        try {
+          Delete d = new Delete(TEST_ROW2);
+          d.deleteFamily(TEST_FAMILY);
+          t.delete(d);
+        } finally {
+          t.close();
+        }
+        return null;
+      }
+    });
   }
 
   @After

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java?rev=1586785&r1=1586784&r2=1586785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java Fri Apr 11 22:02:08 2014
@@ -66,7 +66,7 @@ public class TestNamespaceCommands exten
   @BeforeClass
   public static void beforeClass() throws Exception {
     conf = UTIL.getConfiguration();
-    SecureTestUtil.enableSecurity(conf);
+    enableSecurity(conf);
 
     SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
     USER_RW = User.createUserForTesting(conf, "rw_user", new String[0]);
@@ -83,7 +83,7 @@ public class TestNamespaceCommands exten
 
     UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(TestNamespace).build());
 
-    SecureTestUtil.grantOnNamespace(UTIL, USER_NSP_WRITE.getShortName(),
+    grantOnNamespace(UTIL, USER_NSP_WRITE.getShortName(),
       TestNamespace, Permission.Action.WRITE);
   }
   
@@ -99,7 +99,7 @@ public class TestNamespaceCommands exten
     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
       // Grant and check state in ACL table
-      SecureTestUtil.grantOnNamespace(UTIL, userTestNamespace, TestNamespace,
+      grantOnNamespace(UTIL, userTestNamespace, TestNamespace,
         Permission.Action.WRITE);
 
       Result result = acl.get(new Get(Bytes.toBytes(userTestNamespace)));
@@ -118,7 +118,7 @@ public class TestNamespaceCommands exten
       assertEquals(Permission.Action.WRITE, namespacePerms.get(0).getActions()[0]);
 
       // Revoke and check state in ACL table
-      SecureTestUtil.revokeFromNamespace(UTIL, userTestNamespace, TestNamespace,
+      revokeFromNamespace(UTIL, userTestNamespace, TestNamespace,
         Permission.Action.WRITE);
 
       perms = AccessControlLists.getNamespacePermissions(conf, TestNamespace);