You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/05/12 00:06:59 UTC

svn commit: r1337396 [5/5] - in /hbase/trunk: ./ security/src/main/java/org/apache/hadoop/hbase/security/ security/src/main/java/org/apache/hadoop/hbase/security/access/ security/src/main/java/org/apache/hadoop/hbase/security/token/ security/src/test/ ...

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1337396&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Fri May 11 22:06:57 2012
@@ -0,0 +1,1199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+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.Coprocessor;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.ServerName;
+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.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Performs authorization checks for common operations, according to different
+ * levels of authorized users.
+ */
+@Category(LargeTests.class)
+public class TestAccessController {
+  private static Log LOG = LogFactory.getLog(TestAccessController.class);
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+
+  // user with all permissions
+  private static User SUPERUSER;
+  // table owner user
+  private static User USER_OWNER;
+  // user with rw permissions
+  private static User USER_RW;
+  // user with read-only permissions
+  private static User USER_RO;
+  // user with no permissions
+  private static User USER_NONE;
+
+  private static byte[] TEST_TABLE = Bytes.toBytes("testtable");
+  private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
+
+  private static MasterCoprocessorEnvironment CP_ENV;
+  private static AccessController ACCESS_CONTROLLER;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // setup configuration
+    conf = TEST_UTIL.getConfiguration();
+    SecureTestUtil.enableSecurity(conf);
+
+    TEST_UTIL.startMiniCluster();
+    MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster()
+        .getMaster().getCoprocessorHost();
+    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
+    ACCESS_CONTROLLER = (AccessController)cpHost.findCoprocessor(
+        AccessController.class.getName());
+    CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+        Coprocessor.PRIORITY_HIGHEST, 1, conf);
+
+    // create a set of test users
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
+    USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
+    USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
+    USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
+    USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
+
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
+    htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+    htd.setOwnerString(USER_OWNER.getShortName());
+    admin.createTable(htd);
+
+    // initilize access control
+    HTable meta = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControllerProtocol protocol =
+        meta.coprocessorProxy(AccessControllerProtocol.class, TEST_TABLE);
+    protocol.grant(Bytes.toBytes(USER_RW.getShortName()),
+        new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ,
+            Permission.Action.WRITE));
+
+    protocol.grant(Bytes.toBytes(USER_RO.getShortName()),
+        new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  public void verifyAllowed(User user, PrivilegedExceptionAction action)
+    throws Exception {
+    try {
+      user.runAs(action);
+    } catch (AccessDeniedException ade) {
+      fail("Expected action to pass for user '" + user.getShortName() +
+          "' but was denied");
+    }
+  }
+
+  public void verifyAllowed(PrivilegedExceptionAction action, User... users)
+    throws Exception {
+    for (User user : users) {
+      verifyAllowed(user, action);
+    }
+  }
+
+  public void verifyDenied(User user, PrivilegedExceptionAction action)
+    throws Exception {
+    try {
+      user.runAs(action);
+      fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
+    } catch (RetriesExhaustedWithDetailsException e) {
+      // in case of batch operations, and put, the client assembles a
+      // RetriesExhaustedWithDetailsException instead of throwing an
+      // AccessDeniedException
+      boolean isAccessDeniedException = false;
+      for ( Throwable ex : e.getCauses()) {
+        if (ex instanceof AccessDeniedException) {
+          isAccessDeniedException = true;
+          break;
+        }
+      }
+      if (!isAccessDeniedException ) {
+        fail("Not receiving AccessDeniedException for user '" +
+            user.getShortName() + "'");
+      }
+    } catch (AccessDeniedException ade) {
+      // expected result
+    }
+  }
+
+  public void verifyDenied(PrivilegedExceptionAction action, User... users)
+      throws Exception {
+      for (User user : users) {
+        verifyDenied(user, action);
+      }
+    }
+
+  @Test
+  public void testTableCreate() throws Exception {
+    PrivilegedExceptionAction createTable = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        HTableDescriptor htd = new HTableDescriptor("testnewtable");
+        htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        ACCESS_CONTROLLER.preCreateTable(
+            ObserverContext.createAndPrepare(CP_ENV, null), htd, null);
+        return null;
+      }
+    };
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, createTable);
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, createTable);
+    verifyDenied(USER_RW, createTable);
+    verifyDenied(USER_RO, createTable);
+    verifyDenied(USER_NONE, createTable);
+  }
+
+  @Test
+  public void testTableModify() throws Exception {
+    PrivilegedExceptionAction modifyTable = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
+        htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        htd.addFamily(new HColumnDescriptor("fam_"+User.getCurrent().getShortName()));
+        ACCESS_CONTROLLER.preModifyTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, htd);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, modifyTable);
+    verifyDenied(USER_RW, modifyTable);
+    verifyDenied(USER_RO, modifyTable);
+    verifyDenied(USER_NONE, modifyTable);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, modifyTable);
+  }
+
+  @Test
+  public void testTableDelete() throws Exception {
+    PrivilegedExceptionAction deleteTable = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, deleteTable);
+    verifyDenied(USER_RW, deleteTable);
+    verifyDenied(USER_RO, deleteTable);
+    verifyDenied(USER_NONE, deleteTable);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, deleteTable);
+  }
+
+  @Test
+  public void testAddColumn() throws Exception {
+    final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, hcd);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testModifyColumn() throws Exception {
+    final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
+    hcd.setMaxVersions(10);
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, hcd);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testDeleteColumn() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, TEST_FAMILY);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testTableDisable() throws Exception {
+    PrivilegedExceptionAction disableTable = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, disableTable);
+    verifyDenied(USER_RW, disableTable);
+    verifyDenied(USER_RO, disableTable);
+    verifyDenied(USER_NONE, disableTable);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, disableTable);
+  }
+
+  @Test
+  public void testTableEnable() throws Exception {
+    PrivilegedExceptionAction enableTable = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, enableTable);
+    verifyDenied(USER_RW, enableTable);
+    verifyDenied(USER_RO, enableTable);
+    verifyDenied(USER_NONE, enableTable);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, enableTable);
+  }
+
+  @Test
+  public void testMove() throws Exception {
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
+    Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
+    final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
+        regions.entrySet().iterator().next();
+    final ServerName server = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preMove(ObserverContext.createAndPrepare(CP_ENV, null),
+            firstRegion.getKey(), server, server);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testAssign() throws Exception {
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
+    Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
+    final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
+        regions.entrySet().iterator().next();
+
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preAssign(ObserverContext.createAndPrepare(CP_ENV, null),
+            firstRegion.getKey());
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testUnassign() throws Exception {
+    HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
+    Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
+    final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
+        regions.entrySet().iterator().next();
+
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preUnassign(ObserverContext.createAndPrepare(CP_ENV, null),
+            firstRegion.getKey(), false);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testBalance() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBalance(ObserverContext.createAndPrepare(CP_ENV, null));
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testBalanceSwitch() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBalanceSwitch(ObserverContext.createAndPrepare(CP_ENV, null), true);
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testShutdown() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preShutdown(ObserverContext.createAndPrepare(CP_ENV, null));
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  @Test
+  public void testStopMaster() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preStopMaster(ObserverContext.createAndPrepare(CP_ENV, null));
+        return null;
+      }
+    };
+
+    // all others should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_RO, action);
+    verifyDenied(USER_NONE, action);
+
+    // verify that superuser can create tables
+    verifyAllowed(SUPERUSER, action);
+  }
+
+  private void verifyWrite(PrivilegedExceptionAction action) throws Exception {
+    // should be denied
+    verifyDenied(USER_NONE, action);
+    verifyDenied(USER_RO, action);
+
+    // should be allowed
+    verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_OWNER, action);
+    verifyAllowed(USER_RW, action);
+  }
+
+  private void verifyRead(PrivilegedExceptionAction action) throws Exception {
+    // should be denied
+    verifyDenied(USER_NONE, action);
+
+    // should be allowed
+    verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_OWNER, action);
+    verifyAllowed(USER_RW, action);
+    verifyAllowed(USER_RO, action);
+  }
+
+  @Test
+  public void testRead() throws Exception {
+    // get action
+    PrivilegedExceptionAction getAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Get g = new Get(Bytes.toBytes("random_row"));
+        g.addFamily(TEST_FAMILY);
+        HTable t = new HTable(conf, TEST_TABLE);
+        t.get(g);
+        return null;
+      }
+    };
+    verifyRead(getAction);
+
+    // action for scanning
+    PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Scan s = new Scan();
+        s.addFamily(TEST_FAMILY);
+
+        HTable table = new HTable(conf, TEST_TABLE);
+        ResultScanner scanner = table.getScanner(s);
+        try {
+          for (Result r = scanner.next(); r != null; r = scanner.next()) {
+            // do nothing
+          }
+        } catch (IOException e) {
+        } finally {
+          scanner.close();
+        }
+        return null;
+      }
+    };
+    verifyRead(scanAction);
+  }
+
+  @Test
+  // test put, delete, increment
+  public void testWrite() throws Exception {
+    // put action
+    PrivilegedExceptionAction putAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Put p = new Put(Bytes.toBytes("random_row"));
+        p.add(TEST_FAMILY, Bytes.toBytes("Qualifier"), Bytes.toBytes(1));
+        HTable t = new HTable(conf, TEST_TABLE);
+        t.put(p);
+        return null;
+      }
+    };
+    verifyWrite(putAction);
+
+    // delete action
+    PrivilegedExceptionAction deleteAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Delete d = new Delete(Bytes.toBytes("random_row"));
+        d.deleteFamily(TEST_FAMILY);
+        HTable t = new HTable(conf, TEST_TABLE);
+        t.delete(d);
+        return null;
+      }
+    };
+    verifyWrite(deleteAction);
+
+    // increment action
+    PrivilegedExceptionAction incrementAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Increment inc = new Increment(Bytes.toBytes("random_row"));
+        inc.addColumn(TEST_FAMILY, Bytes.toBytes("Qualifier"), 1);
+        HTable t = new HTable(conf, TEST_TABLE);
+        t.increment(inc);
+        return null;
+      }
+    };
+    verifyWrite(incrementAction);
+  }
+
+  @Test
+  public void testGrantRevoke() throws Exception {
+    final byte[] tableName = Bytes.toBytes("TempTable");
+    final byte[] family1 = Bytes.toBytes("f1");
+    final byte[] family2 = Bytes.toBytes("f2");
+    final byte[] qualifier = Bytes.toBytes("q");
+
+    // create table
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    if (admin.tableExists(tableName)) {
+      admin.disableTable(tableName);
+      admin.deleteTable(tableName);
+    }
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(family1));
+    htd.addFamily(new HColumnDescriptor(family2));
+    htd.setOwnerString(USER_OWNER.getShortName());
+    admin.createTable(htd);
+
+    // create temp users
+    User user = User.createUserForTesting(TEST_UTIL.getConfiguration(),
+        "user", new String[0]);
+
+    // perms only stored against the first region
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class,
+            tableName);
+
+    // prepare actions:
+    PrivilegedExceptionAction putActionAll = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Put p = new Put(Bytes.toBytes("a"));
+        p.add(family1, qualifier, Bytes.toBytes("v1"));
+        p.add(family2, qualifier, Bytes.toBytes("v2"));
+        HTable t = new HTable(conf, tableName);
+        t.put(p);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction putAction1 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Put p = new Put(Bytes.toBytes("a"));
+        p.add(family1, qualifier, Bytes.toBytes("v1"));
+        HTable t = new HTable(conf, tableName);
+        t.put(p);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction putAction2 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Put p = new Put(Bytes.toBytes("a"));
+        p.add(family2, qualifier, Bytes.toBytes("v2"));
+        HTable t = new HTable(conf, tableName);
+        t.put(p);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction getActionAll = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Get g = new Get(Bytes.toBytes("random_row"));
+        g.addFamily(family1);
+        g.addFamily(family2);
+        HTable t = new HTable(conf, tableName);
+        t.get(g);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction getAction1 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Get g = new Get(Bytes.toBytes("random_row"));
+        g.addFamily(family1);
+        HTable t = new HTable(conf, tableName);
+        t.get(g);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction getAction2 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Get g = new Get(Bytes.toBytes("random_row"));
+        g.addFamily(family2);
+        HTable t = new HTable(conf, tableName);
+        t.get(g);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction deleteActionAll = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Delete d = new Delete(Bytes.toBytes("random_row"));
+        d.deleteFamily(family1);
+        d.deleteFamily(family2);
+        HTable t = new HTable(conf, tableName);
+        t.delete(d);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction deleteAction1 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Delete d = new Delete(Bytes.toBytes("random_row"));
+        d.deleteFamily(family1);
+        HTable t = new HTable(conf, tableName);
+        t.delete(d);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction deleteAction2 = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Delete d = new Delete(Bytes.toBytes("random_row"));
+        d.deleteFamily(family2);
+        HTable t = new HTable(conf, tableName);
+        t.delete(d);
+        return null;
+      }
+    };
+
+    // initial check:
+    verifyDenied(user, getActionAll);
+    verifyDenied(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyDenied(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyDenied(user, deleteAction2);
+
+    // grant table read permission
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, null, Permission.Action.READ));
+    Thread.sleep(100);
+    // check
+    verifyAllowed(user, getActionAll);
+    verifyAllowed(user, getAction1);
+    verifyAllowed(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyDenied(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyDenied(user, deleteAction2);
+
+    // grant table write permission
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, null, Permission.Action.WRITE));
+    Thread.sleep(100);
+    verifyDenied(user, getActionAll);
+    verifyDenied(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyAllowed(user, putActionAll);
+    verifyAllowed(user, putAction1);
+    verifyAllowed(user, putAction2);
+
+    verifyAllowed(user, deleteActionAll);
+    verifyAllowed(user, deleteAction1);
+    verifyAllowed(user, deleteAction2);
+
+    // revoke table permission
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, null, Permission.Action.READ,
+        Permission.Action.WRITE));
+
+    protocol.revoke(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, null));
+    Thread.sleep(100);
+    verifyDenied(user, getActionAll);
+    verifyDenied(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyDenied(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyDenied(user, deleteAction2);
+
+    // grant column family read permission
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, family1, Permission.Action.READ));
+    Thread.sleep(100);
+
+    verifyAllowed(user, getActionAll);
+    verifyAllowed(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyDenied(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyDenied(user, deleteAction2);
+
+    // grant column family write permission
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, family2, Permission.Action.WRITE));
+    Thread.sleep(100);
+
+    verifyAllowed(user, getActionAll);
+    verifyAllowed(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyAllowed(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyAllowed(user, deleteAction2);
+
+    // revoke column family permission
+    protocol.revoke(Bytes.toBytes(user.getShortName()),
+      new TablePermission(tableName, family2));
+    Thread.sleep(100);
+
+    verifyAllowed(user, getActionAll);
+    verifyAllowed(user, getAction1);
+    verifyDenied(user, getAction2);
+
+    verifyDenied(user, putActionAll);
+    verifyDenied(user, putAction1);
+    verifyDenied(user, putAction2);
+
+    verifyDenied(user, deleteActionAll);
+    verifyDenied(user, deleteAction1);
+    verifyDenied(user, deleteAction2);
+
+    // delete table
+    admin.disableTable(tableName);
+    admin.deleteTable(tableName);
+  }
+
+  private boolean hasFoundUserPermission(UserPermission userPermission,
+                                         List<UserPermission> perms) {
+    return perms.contains(userPermission);
+  }
+
+  @Test
+  public void testGrantRevokeAtQualifierLevel() throws Exception {
+    final byte[] tableName = Bytes.toBytes("testGrantRevokeAtQualifierLevel");
+    final byte[] family1 = Bytes.toBytes("f1");
+    final byte[] family2 = Bytes.toBytes("f2");
+    final byte[] qualifier = Bytes.toBytes("q");
+
+    // create table
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+
+    if (admin.tableExists(tableName)) {
+      admin.disableTable(tableName);
+      admin.deleteTable(tableName);
+    }
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(family1));
+    htd.addFamily(new HColumnDescriptor(family2));
+    htd.setOwnerString(USER_OWNER.getShortName());
+    admin.createTable(htd);
+
+    // create temp users
+    User user = User.createUserForTesting(TEST_UTIL.getConfiguration(),
+        "user", new String[0]);
+
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class, tableName);
+
+    PrivilegedExceptionAction getQualifierAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Get g = new Get(Bytes.toBytes("random_row"));
+        g.addColumn(family1, qualifier);
+        HTable t = new HTable(conf, tableName);
+        t.get(g);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction putQualifierAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Put p = new Put(Bytes.toBytes("random_row"));
+        p.add(family1, qualifier, Bytes.toBytes("v1"));
+        HTable t = new HTable(conf, tableName);
+        t.put(p);
+        return null;
+      }
+    };
+    PrivilegedExceptionAction deleteQualifierAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        Delete d = new Delete(Bytes.toBytes("random_row"));
+        d.deleteColumn(family1, qualifier);
+        //d.deleteFamily(family1);
+        HTable t = new HTable(conf, tableName);
+        t.delete(d);
+        return null;
+      }
+    };
+
+    protocol.revoke(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, family1));
+    verifyDenied(user, getQualifierAction);
+    verifyDenied(user, putQualifierAction);
+    verifyDenied(user, deleteQualifierAction);
+
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, family1, qualifier,
+            Permission.Action.READ));
+    Thread.sleep(100);
+
+    verifyAllowed(user, getQualifierAction);
+    verifyDenied(user, putQualifierAction);
+    verifyDenied(user, deleteQualifierAction);
+
+    // only grant write permission
+    // TODO: comment this portion after HBASE-3583
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, family1, qualifier,
+            Permission.Action.WRITE));
+    Thread.sleep(100);
+
+    verifyDenied(user, getQualifierAction);
+    verifyAllowed(user, putQualifierAction);
+    verifyAllowed(user, deleteQualifierAction);
+
+    // grant both read and write permission.
+    protocol.grant(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, family1, qualifier,
+            Permission.Action.READ, Permission.Action.WRITE));
+    Thread.sleep(100);
+
+    verifyAllowed(user, getQualifierAction);
+    verifyAllowed(user, putQualifierAction);
+    verifyAllowed(user, deleteQualifierAction);
+
+    // revoke family level permission won't impact column level.
+    protocol.revoke(Bytes.toBytes(user.getShortName()),
+        new TablePermission(tableName, family1, qualifier));
+    Thread.sleep(100);
+
+    verifyDenied(user, getQualifierAction);
+    verifyDenied(user, putQualifierAction);
+    verifyDenied(user, deleteQualifierAction);
+
+    // delete table
+    admin.disableTable(tableName);
+    admin.deleteTable(tableName);
+  }
+
+  @Test
+  public void testPermissionList() throws Exception {
+    final byte[] tableName = Bytes.toBytes("testPermissionList");
+    final byte[] family1 = Bytes.toBytes("f1");
+    final byte[] family2 = Bytes.toBytes("f2");
+    final byte[] qualifier = Bytes.toBytes("q");
+    final byte[] user = Bytes.toBytes("user");
+
+    // create table
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    if (admin.tableExists(tableName)) {
+      admin.disableTable(tableName);
+      admin.deleteTable(tableName);
+    }
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(family1));
+    htd.addFamily(new HColumnDescriptor(family2));
+    htd.setOwnerString(USER_OWNER.getShortName());
+    admin.createTable(htd);
+
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class, tableName);
+
+    List<UserPermission> perms = protocol.getUserPermissions(tableName);
+
+    UserPermission up = new UserPermission(user,
+        tableName, family1, qualifier, Permission.Action.READ);
+    assertFalse("User should not be granted permission: " + up.toString(),
+        hasFoundUserPermission(up, perms));
+
+    // grant read permission
+    UserPermission upToSet = new UserPermission(user,
+        tableName, family1, qualifier, Permission.Action.READ);
+    protocol.grant(user, upToSet);
+    perms = protocol.getUserPermissions(tableName);
+
+    UserPermission upToVerify = new UserPermission(user,
+        tableName, family1, qualifier, Permission.Action.READ);
+    assertTrue("User should be granted permission: " + upToVerify.toString(),
+        hasFoundUserPermission(upToVerify, perms));
+
+    upToVerify = new UserPermission(user, tableName, family1, qualifier,
+        Permission.Action.WRITE);
+    assertFalse("User should not be granted permission: " + upToVerify.toString(),
+        hasFoundUserPermission(upToVerify, perms));
+
+    // grant read+write
+    upToSet = new UserPermission(user, tableName, family1, qualifier,
+        Permission.Action.WRITE, Permission.Action.READ);
+    protocol.grant(user, upToSet);
+    perms = protocol.getUserPermissions(tableName);
+
+    upToVerify = new UserPermission(user, tableName, family1, qualifier,
+        Permission.Action.WRITE, Permission.Action.READ);
+    assertTrue("User should be granted permission: " + upToVerify.toString(),
+            hasFoundUserPermission(upToVerify, perms));
+
+    protocol.revoke(user, upToSet);
+    perms = protocol.getUserPermissions(tableName);
+    assertFalse("User should not be granted permission: " + upToVerify.toString(),
+      hasFoundUserPermission(upToVerify, perms));
+
+    // delete table
+    admin.disableTable(tableName);
+    admin.deleteTable(tableName);
+  }
+
+  /** global operations*/
+  private void verifyGlobal(PrivilegedExceptionAction<?> action) throws Exception {
+    // should be allowed
+    verifyAllowed(SUPERUSER, action);
+
+    // should be denied
+    verifyDenied(USER_OWNER, action);
+    verifyDenied(USER_RW, action);
+    verifyDenied(USER_NONE, action);
+    verifyDenied(USER_RO, action);
+  }
+
+  public void checkGlobalPerms(Permission.Action... actions) throws IOException {
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class, new byte[0]);
+
+    Permission[] perms = new Permission[actions.length];
+    for (int i=0; i < actions.length; i++) {
+      perms[i] = new Permission(actions[i]);
+    }
+
+    protocol.checkPermissions(perms);
+  }
+
+  public void checkTablePerms(byte[] table, byte[] family, byte[] column,
+      Permission.Action... actions) throws IOException {
+    Permission[] perms = new Permission[actions.length];
+    for (int i=0; i < actions.length; i++) {
+      perms[i] = new TablePermission(table, family, column, actions[i]);
+    }
+
+    checkTablePerms(table, perms);
+  }
+
+  public void checkTablePerms(byte[] table, Permission...perms) throws IOException {
+    HTable acl = new HTable(conf, table);
+    AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class, new byte[0]);
+
+    protocol.checkPermissions(perms);
+  }
+
+  public void grant(AccessControllerProtocol protocol, User user, byte[] t, byte[] f,
+      byte[] q, Permission.Action... actions) throws IOException {
+    protocol.grant(Bytes.toBytes(user.getShortName()), new TablePermission(t, f, q, actions));
+  }
+
+  @Test
+  public void testCheckPermissions() throws Exception {
+    final HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    final AccessControllerProtocol protocol =
+        acl.coprocessorProxy(AccessControllerProtocol.class, TEST_TABLE);
+
+    //--------------------------------------
+    //test global permissions
+    PrivilegedExceptionAction<Void> globalAdmin = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkGlobalPerms(Permission.Action.ADMIN);
+        return null;
+      }
+    };
+    //verify that only superuser can admin
+    verifyGlobal(globalAdmin);
+
+    //--------------------------------------
+    //test multiple permissions
+    PrivilegedExceptionAction<Void> globalReadWrite = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkGlobalPerms(Permission.Action.READ, Permission.Action.WRITE);
+        return null;
+      }
+    };
+
+    verifyGlobal(globalReadWrite);
+
+    //--------------------------------------
+    //table/column/qualifier level permissions
+    final byte[] TEST_Q1 = Bytes.toBytes("q1");
+    final byte[] TEST_Q2 = Bytes.toBytes("q2");
+
+    User userTable = User.createUserForTesting(conf, "user_check_perms_table", new String[0]);
+    User userColumn = User.createUserForTesting(conf, "user_check_perms_family", new String[0]);
+    User userQualifier = User.createUserForTesting(conf, "user_check_perms_q", new String[0]);
+
+    grant(protocol, userTable, TEST_TABLE, null, null, Permission.Action.READ);
+    grant(protocol, userColumn, TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ);
+    grant(protocol, userQualifier, TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ);
+
+    PrivilegedExceptionAction<Void> tableRead = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, null, null, Permission.Action.READ);
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction<Void> columnRead = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ);
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction<Void> qualifierRead = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ);
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction<Void> multiQualifierRead = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, new Permission[] {
+          new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ),
+          new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q2, Permission.Action.READ),
+        });
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction<Void> globalAndTableRead = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, new Permission[] {
+          new Permission(Permission.Action.READ),
+          new TablePermission(TEST_TABLE, null, (byte[])null, Permission.Action.READ),
+        });
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction<Void> noCheck = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, new Permission[0]);
+        return null;
+      }
+    };
+
+    verifyAllowed(tableRead, SUPERUSER, userTable);
+    verifyDenied(tableRead, userColumn, userQualifier);
+
+    verifyAllowed(columnRead, SUPERUSER, userTable, userColumn);
+    verifyDenied(columnRead, userQualifier);
+
+    verifyAllowed(qualifierRead, SUPERUSER, userTable, userColumn, userQualifier);
+
+    verifyAllowed(multiQualifierRead, SUPERUSER, userTable, userColumn);
+    verifyDenied(multiQualifierRead, userQualifier);
+
+    verifyAllowed(globalAndTableRead, SUPERUSER);
+    verifyDenied(globalAndTableRead, userTable, userColumn, userQualifier);
+
+    verifyAllowed(noCheck, SUPERUSER, userTable, userColumn, userQualifier);
+
+    //--------------------------------------
+    //test family level multiple permissions
+    PrivilegedExceptionAction<Void> familyReadWrite = new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ,
+            Permission.Action.WRITE);
+        return null;
+      }
+    };
+    // should be allowed
+    verifyAllowed(familyReadWrite, SUPERUSER, USER_OWNER, USER_RW);
+    // should be denied
+    verifyDenied(familyReadWrite, USER_NONE, USER_RO);
+
+    //--------------------------------------
+    //check for wrong table region
+    try {
+      //but ask for TablePermissions for TEST_TABLE
+      protocol.checkPermissions(new Permission[] {(Permission) new TablePermission(
+          TEST_TABLE, null, (byte[])null, Permission.Action.CREATE)});
+      fail("this should have thrown CoprocessorException");
+    } catch(CoprocessorException ex) {
+      //expected
+    }
+
+  }
+}

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java?rev=1337396&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java Fri May 11 22:06:57 2012
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.LargeTests;
+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.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+
+/**
+ * Test the reading and writing of access permissions on {@code _acl_} table.
+ */
+@Category(LargeTests.class)
+public class TestTablePermissions {
+  private static final Log LOG = LogFactory.getLog(TestTablePermissions.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static ZooKeeperWatcher ZKW;
+  private final static Abortable ABORTABLE = new Abortable() {
+    private final AtomicBoolean abort = new AtomicBoolean(false);
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.info(why, e);
+      abort.set(true);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return abort.get();
+    }
+  };
+
+  private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
+  private static byte[] TEST_TABLE2 = Bytes.toBytes("perms_test2");
+  private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
+  private static byte[] TEST_QUALIFIER = Bytes.toBytes("col1");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // setup configuration
+    Configuration conf = UTIL.getConfiguration();
+    SecureTestUtil.enableSecurity(conf);
+
+    UTIL.startMiniCluster();
+    ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
+      "TestTablePermissions", ABORTABLE);
+
+    UTIL.createTable(TEST_TABLE, TEST_FAMILY);
+    UTIL.createTable(TEST_TABLE2, TEST_FAMILY);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testBasicWrite() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    // add some permissions
+    AccessControlLists.addTablePermission(conf, TEST_TABLE,
+        "george", new TablePermission(TEST_TABLE, null,
+            TablePermission.Action.READ, TablePermission.Action.WRITE));
+    AccessControlLists.addTablePermission(conf, TEST_TABLE,
+        "hubert", new TablePermission(TEST_TABLE, null,
+            TablePermission.Action.READ));
+    AccessControlLists.addTablePermission(conf, TEST_TABLE,
+        "humphrey", new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+            TablePermission.Action.READ));
+
+    // retrieve the same
+    ListMultimap<String,TablePermission> perms =
+        AccessControlLists.getTablePermissions(conf, TEST_TABLE);
+    List<TablePermission> userPerms = perms.get("george");
+    assertNotNull("Should have permissions for george", userPerms);
+    assertEquals("Should have 1 permission for george", 1, userPerms.size());
+    TablePermission permission = userPerms.get(0);
+    assertTrue("Permission should be for " + TEST_TABLE,
+        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertNull("Column family should be empty", permission.getFamily());
+
+    // check actions
+    assertNotNull(permission.getActions());
+    assertEquals(2, permission.getActions().length);
+    List<TablePermission.Action> actions = Arrays.asList(permission.getActions());
+    assertTrue(actions.contains(TablePermission.Action.READ));
+    assertTrue(actions.contains(TablePermission.Action.WRITE));
+
+    userPerms = perms.get("hubert");
+    assertNotNull("Should have permissions for hubert", userPerms);
+    assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
+    permission = userPerms.get(0);
+    assertTrue("Permission should be for " + TEST_TABLE,
+        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertNull("Column family should be empty", permission.getFamily());
+
+    // check actions
+    assertNotNull(permission.getActions());
+    assertEquals(1, permission.getActions().length);
+    actions = Arrays.asList(permission.getActions());
+    assertTrue(actions.contains(TablePermission.Action.READ));
+    assertFalse(actions.contains(TablePermission.Action.WRITE));
+
+    userPerms = perms.get("humphrey");
+    assertNotNull("Should have permissions for humphrey", userPerms);
+    assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
+    permission = userPerms.get(0);
+    assertTrue("Permission should be for " + TEST_TABLE,
+        Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertTrue("Permission should be for family " + TEST_FAMILY,
+        Bytes.equals(TEST_FAMILY, permission.getFamily()));
+    assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
+        Bytes.equals(TEST_QUALIFIER, permission.getQualifier()));
+
+    // check actions
+    assertNotNull(permission.getActions());
+    assertEquals(1, permission.getActions().length);
+    actions = Arrays.asList(permission.getActions());
+    assertTrue(actions.contains(TablePermission.Action.READ));
+    assertFalse(actions.contains(TablePermission.Action.WRITE));
+
+    // table 2 permissions
+    AccessControlLists.addTablePermission(conf, TEST_TABLE2, "hubert",
+        new TablePermission(TEST_TABLE2, null,
+            TablePermission.Action.READ, TablePermission.Action.WRITE));
+
+    // check full load
+    Map<byte[],ListMultimap<String,TablePermission>> allPerms =
+        AccessControlLists.loadAll(conf);
+    assertEquals("Full permission map should have entries for both test tables",
+        2, allPerms.size());
+
+    userPerms = allPerms.get(TEST_TABLE).get("hubert");
+    assertNotNull(userPerms);
+    assertEquals(1, userPerms.size());
+    permission = userPerms.get(0);
+    assertTrue(Bytes.equals(TEST_TABLE, permission.getTable()));
+    assertEquals(1, permission.getActions().length);
+    assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
+
+    userPerms = allPerms.get(TEST_TABLE2).get("hubert");
+    assertNotNull(userPerms);
+    assertEquals(1, userPerms.size());
+    permission = userPerms.get(0);
+    assertTrue(Bytes.equals(TEST_TABLE2, permission.getTable()));
+    assertEquals(2, permission.getActions().length);
+    actions = Arrays.asList(permission.getActions());
+    assertTrue(actions.contains(TablePermission.Action.READ));
+    assertTrue(actions.contains(TablePermission.Action.WRITE));
+  }
+
+  @Test
+  public void testPersistence() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    AccessControlLists.addTablePermission(conf, TEST_TABLE, "albert",
+        new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
+    AccessControlLists.addTablePermission(conf, TEST_TABLE, "betty",
+        new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
+            TablePermission.Action.WRITE));
+    AccessControlLists.addTablePermission(conf, TEST_TABLE, "clark",
+        new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ));
+    AccessControlLists.addTablePermission(conf, TEST_TABLE, "dwight",
+        new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+            TablePermission.Action.WRITE));
+
+    // verify permissions survive changes in table metadata
+    ListMultimap<String,TablePermission> preperms =
+        AccessControlLists.getTablePermissions(conf, TEST_TABLE);
+
+    HTable table = new HTable(conf, TEST_TABLE);
+    table.put(new Put(Bytes.toBytes("row1"))
+        .add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v1")));
+    table.put(new Put(Bytes.toBytes("row2"))
+        .add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v2")));
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    admin.split(TEST_TABLE);
+
+    // wait for split
+    Thread.sleep(10000);
+
+    ListMultimap<String,TablePermission> postperms =
+        AccessControlLists.getTablePermissions(conf, TEST_TABLE);
+
+    checkMultimapEqual(preperms, postperms);
+  }
+
+  @Test
+  public void testSerialization() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    ListMultimap<String,TablePermission> permissions = ArrayListMultimap.create();
+    permissions.put("george", new TablePermission(TEST_TABLE, null,
+        TablePermission.Action.READ));
+    permissions.put("george", new TablePermission(TEST_TABLE, TEST_FAMILY,
+        TablePermission.Action.WRITE));
+    permissions.put("george", new TablePermission(TEST_TABLE2, null,
+        TablePermission.Action.READ));
+    permissions.put("hubert", new TablePermission(TEST_TABLE2, null,
+        TablePermission.Action.READ, TablePermission.Action.WRITE));
+
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    AccessControlLists.writePermissions(new DataOutputStream(bos),
+        permissions, conf);
+
+    ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+    ListMultimap<String,TablePermission> copy =
+        AccessControlLists.readPermissions(new DataInputStream(bis), conf);
+
+    checkMultimapEqual(permissions, copy);
+  }
+
+  public void checkMultimapEqual(ListMultimap<String,TablePermission> first,
+      ListMultimap<String,TablePermission> second) {
+    assertEquals(first.size(), second.size());
+    for (String key : first.keySet()) {
+      List<TablePermission> firstPerms = first.get(key);
+      List<TablePermission> secondPerms = second.get(key);
+      assertNotNull(secondPerms);
+      assertEquals(firstPerms.size(), secondPerms.size());
+      LOG.info("First permissions: "+firstPerms.toString());
+      LOG.info("Second permissions: "+secondPerms.toString());
+      for (TablePermission p : firstPerms) {
+        assertTrue("Permission "+p.toString()+" not found", secondPerms.contains(p));
+      }
+    }
+  }
+
+  @Test
+  public void testEquals() throws Exception {
+    TablePermission p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
+    TablePermission p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
+    assertTrue(p1.equals(p2));
+    assertTrue(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ, TablePermission.Action.WRITE);
+    p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    assertTrue(p1.equals(p2));
+    assertTrue(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ, TablePermission.Action.WRITE);
+    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    assertTrue(p1.equals(p2));
+    assertTrue(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.READ, TablePermission.Action.WRITE);
+    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    assertTrue(p1.equals(p2));
+    assertTrue(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
+    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ);
+    assertFalse(p1.equals(p2));
+    assertFalse(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
+    p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.WRITE);
+    assertFalse(p1.equals(p2));
+    assertFalse(p2.equals(p1));
+    p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ, TablePermission.Action.WRITE);
+    assertFalse(p1.equals(p2));
+    assertFalse(p2.equals(p1));
+
+    p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
+    p2 = new TablePermission(TEST_TABLE2, null, TablePermission.Action.READ);
+    assertFalse(p1.equals(p2));
+    assertFalse(p2.equals(p1));
+
+    p2 = new TablePermission(TEST_TABLE, null);
+    assertFalse(p1.equals(p2));
+    assertFalse(p2.equals(p1));
+  }
+}

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java?rev=1337396&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java Fri May 11 22:06:57 2012
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the reading and writing of access permissions to and from zookeeper.
+ */
+@Category(LargeTests.class)
+public class TestZKPermissionsWatcher {
+  private static final Log LOG = LogFactory.getLog(TestZKPermissionsWatcher.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableAuthManager AUTH_A;
+  private static TableAuthManager AUTH_B;
+  private final static Abortable ABORTABLE = new Abortable() {
+    private final AtomicBoolean abort = new AtomicBoolean(false);
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.info(why, e);
+      abort.set(true);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return abort.get();
+    }
+  };
+
+  private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // setup configuration
+    Configuration conf = UTIL.getConfiguration();
+    SecureTestUtil.enableSecurity(conf);
+
+    // start minicluster
+    UTIL.startMiniCluster();
+    AUTH_A = TableAuthManager.get(new ZooKeeperWatcher(conf,
+      "TestZKPermissionsWatcher_1", ABORTABLE), conf);
+    AUTH_B = TableAuthManager.get(new ZooKeeperWatcher(conf,
+      "TestZKPermissionsWatcher_2", ABORTABLE), conf);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testPermissionsWatcher() throws Exception {
+    assertFalse(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    assertFalse(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    // update ACL: george RW
+    List<TablePermission> acl = new ArrayList<TablePermission>();
+    acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
+      TablePermission.Action.WRITE));
+    AUTH_A.setUserPermissions("george", TEST_TABLE, acl);
+    Thread.sleep(100);
+
+    // check it
+    assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    // update ACL: hubert R
+    acl = new ArrayList<TablePermission>();
+    acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
+    AUTH_B.setUserPermissions("hubert", TEST_TABLE, acl);
+    Thread.sleep(100);
+
+    // check it
+    assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+  }
+}

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java?rev=1337396&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Fri May 11 22:06:57 2012
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.token;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.security.PrivilegedExceptionAction;
+import java.util.UUID;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.coprocessor.BaseEndpointCoprocessor;
+import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
+import org.apache.hadoop.hbase.ipc.HBaseRPC;
+import org.apache.hadoop.hbase.ipc.HBaseServer;
+import org.apache.hadoop.hbase.ipc.RequestContext;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests for authentication token creation and usage
+ */
+@Category(LargeTests.class)
+public class TestTokenAuthentication {
+  public static interface IdentityProtocol extends CoprocessorProtocol {
+    public String whoami();
+    public String getAuthMethod();
+  }
+
+  public static class IdentityCoprocessor extends BaseEndpointCoprocessor
+      implements IdentityProtocol {
+    public String whoami() {
+      return RequestContext.getRequestUserName();
+    }
+
+    public String getAuthMethod() {
+      UserGroupInformation ugi = null;
+      User user = RequestContext.getRequestUser();
+      if (user != null) {
+        ugi = user.getUGI();
+      }
+      if (ugi != null) {
+        return ugi.getAuthenticationMethod().toString();
+      }
+      return null;
+    }
+  }
+
+  private static HBaseTestingUtility TEST_UTIL;
+  private static AuthenticationTokenSecretManager secretManager;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.set("hbase.coprocessor.region.classes",
+        IdentityCoprocessor.class.getName());
+    TEST_UTIL.startMiniCluster();
+    HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    secretManager = new AuthenticationTokenSecretManager(conf, rs.getZooKeeper(),
+        rs.getServerName().toString(), 
+        conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000), 
+        conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000));
+    secretManager.start(); 
+    while(secretManager.getCurrentKey() == null) {
+      Thread.sleep(1);
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testTokenCreation() throws Exception {
+    Token<AuthenticationTokenIdentifier> token =
+        secretManager.generateToken("testuser");
+
+    AuthenticationTokenIdentifier ident = new AuthenticationTokenIdentifier();
+    Writables.getWritable(token.getIdentifier(), ident);
+    assertEquals("Token username should match", "testuser",
+        ident.getUsername());
+    byte[] passwd = secretManager.retrievePassword(ident);
+    assertTrue("Token password and password from secret manager should match",
+        Bytes.equals(token.getPassword(), passwd));
+  }
+
+  // @Test - Disable due to kerberos requirement
+  public void testTokenAuthentication() throws Exception {
+    UserGroupInformation testuser =
+        UserGroupInformation.createUserForTesting("testuser", new String[]{"testgroup"});
+
+    testuser.setAuthenticationMethod(
+        UserGroupInformation.AuthenticationMethod.TOKEN);
+    final Configuration conf = TEST_UTIL.getConfiguration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    conf.set("randomkey", UUID.randomUUID().toString());
+    testuser.setConfiguration(conf);
+    Token<AuthenticationTokenIdentifier> token =
+        secretManager.generateToken("testuser");
+    testuser.addToken(token);
+
+    // verify the server authenticates us as this token user
+    testuser.doAs(new PrivilegedExceptionAction<Object>() {
+      public Object run() throws Exception {
+        HTable table = new HTable(conf, ".META.");
+        IdentityProtocol prot = table.coprocessorProxy(
+            IdentityProtocol.class, HConstants.EMPTY_START_ROW);
+        String myname = prot.whoami();
+        assertEquals("testuser", myname);
+        String authMethod = prot.getAuthMethod();
+        assertEquals("TOKEN", authMethod);
+        return null;
+      }
+    });
+  }
+}

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java?rev=1337396&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcher.java Fri May 11 22:06:57 2012
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.token;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the synchronization of token authentication master keys through
+ * ZKSecretWatcher
+ */
+@Category(LargeTests.class)
+public class TestZKSecretWatcher {
+  private static Log LOG = LogFactory.getLog(TestZKSecretWatcher.class);
+  private static HBaseTestingUtility TEST_UTIL;
+  private static AuthenticationTokenSecretManager KEY_MASTER;
+  private static AuthenticationTokenSecretManager KEY_SLAVE;
+  private static AuthenticationTokenSecretManager KEY_SLAVE2;
+  private static AuthenticationTokenSecretManager KEY_SLAVE3;
+
+  private static class MockAbortable implements Abortable {
+    private boolean abort;
+    public void abort(String reason, Throwable e) {
+      LOG.info("Aborting: "+reason, e);
+      abort = true;
+    }
+
+    public boolean isAborted() {
+      return abort;
+    }
+  }
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.startMiniZKCluster();
+    Configuration conf = TEST_UTIL.getConfiguration();
+
+    ZooKeeperWatcher zk = newZK(conf, "server1", new MockAbortable());
+    AuthenticationTokenSecretManager[] tmp = new AuthenticationTokenSecretManager[2];
+    tmp[0] = new AuthenticationTokenSecretManager(
+        conf, zk, "server1", 60*60*1000, 60*1000);
+    tmp[0].start();
+
+    zk = newZK(conf, "server2", new MockAbortable());
+    tmp[1] = new AuthenticationTokenSecretManager(
+        conf, zk, "server2", 60*60*1000, 60*1000);
+    tmp[1].start();
+
+    while (KEY_MASTER == null) {
+      for (int i=0; i<2; i++) {
+        if (tmp[i].isMaster()) {
+          KEY_MASTER = tmp[i];
+          KEY_SLAVE = tmp[ i+1 % 2 ];
+          break;
+        }
+      }
+      Thread.sleep(500);
+    }
+    LOG.info("Master is "+KEY_MASTER.getName()+
+        ", slave is "+KEY_SLAVE.getName());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testKeyUpdate() throws Exception {
+    // sanity check
+    assertTrue(KEY_MASTER.isMaster());
+    assertFalse(KEY_SLAVE.isMaster());
+    int maxKeyId = 0;
+
+    KEY_MASTER.rollCurrentKey();
+    AuthenticationKey key1 = KEY_MASTER.getCurrentKey();
+    assertNotNull(key1);
+    LOG.debug("Master current key: "+key1.getKeyId());
+
+    // wait for slave to update
+    Thread.sleep(1000);
+    AuthenticationKey slaveCurrent = KEY_SLAVE.getCurrentKey();
+    assertNotNull(slaveCurrent);
+    assertEquals(key1, slaveCurrent);
+    LOG.debug("Slave current key: "+slaveCurrent.getKeyId());
+
+    // generate two more keys then expire the original
+    KEY_MASTER.rollCurrentKey();
+    AuthenticationKey key2 = KEY_MASTER.getCurrentKey();
+    LOG.debug("Master new current key: "+key2.getKeyId());
+    KEY_MASTER.rollCurrentKey();
+    AuthenticationKey key3 = KEY_MASTER.getCurrentKey();
+    LOG.debug("Master new current key: "+key3.getKeyId());
+
+    // force expire the original key
+    key1.setExpiration(EnvironmentEdgeManager.currentTimeMillis() - 1000);
+    KEY_MASTER.removeExpiredKeys();
+    // verify removed from master
+    assertNull(KEY_MASTER.getKey(key1.getKeyId()));
+
+    // wait for slave to catch up
+    Thread.sleep(1000);
+    // make sure the slave has both new keys
+    AuthenticationKey slave2 = KEY_SLAVE.getKey(key2.getKeyId());
+    assertNotNull(slave2);
+    assertEquals(key2, slave2);
+    AuthenticationKey slave3 = KEY_SLAVE.getKey(key3.getKeyId());
+    assertNotNull(slave3);
+    assertEquals(key3, slave3);
+    slaveCurrent = KEY_SLAVE.getCurrentKey();
+    assertEquals(key3, slaveCurrent);
+    LOG.debug("Slave current key: "+slaveCurrent.getKeyId());
+
+    // verify that the expired key has been removed
+    assertNull(KEY_SLAVE.getKey(key1.getKeyId()));
+
+    // bring up a new slave
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ZooKeeperWatcher zk = newZK(conf, "server3", new MockAbortable());
+    KEY_SLAVE2 = new AuthenticationTokenSecretManager(
+        conf, zk, "server3", 60*60*1000, 60*1000);
+    KEY_SLAVE2.start();
+
+    Thread.sleep(1000);
+    // verify the new slave has current keys (and not expired)
+    slave2 = KEY_SLAVE2.getKey(key2.getKeyId());
+    assertNotNull(slave2);
+    assertEquals(key2, slave2);
+    slave3 = KEY_SLAVE2.getKey(key3.getKeyId());
+    assertNotNull(slave3);
+    assertEquals(key3, slave3);
+    slaveCurrent = KEY_SLAVE2.getCurrentKey();
+    assertEquals(key3, slaveCurrent);
+    assertNull(KEY_SLAVE2.getKey(key1.getKeyId()));
+
+    // test leader failover
+    KEY_MASTER.stop();
+
+    // wait for master to stop
+    Thread.sleep(1000);
+    assertFalse(KEY_MASTER.isMaster());
+
+    // check for a new master
+    AuthenticationTokenSecretManager[] mgrs =
+        new AuthenticationTokenSecretManager[]{ KEY_SLAVE, KEY_SLAVE2 };
+    AuthenticationTokenSecretManager newMaster = null;
+    int tries = 0;
+    while (newMaster == null && tries++ < 5) {
+      for (AuthenticationTokenSecretManager mgr : mgrs) {
+        if (mgr.isMaster()) {
+          newMaster = mgr;
+          break;
+        }
+      }
+      if (newMaster == null) {
+        Thread.sleep(500);
+      }
+    }
+    assertNotNull(newMaster);
+
+    AuthenticationKey current = newMaster.getCurrentKey();
+    // new master will immediately roll the current key, so it's current may be greater
+    assertTrue(current.getKeyId() >= slaveCurrent.getKeyId());
+    LOG.debug("New master, current key: "+current.getKeyId());
+
+    // roll the current key again on new master and verify the key ID increments
+    newMaster.rollCurrentKey();
+    AuthenticationKey newCurrent = newMaster.getCurrentKey();
+    LOG.debug("New master, rolled new current key: "+newCurrent.getKeyId());
+    assertTrue(newCurrent.getKeyId() > current.getKeyId());
+
+    // add another slave
+    ZooKeeperWatcher zk3 = newZK(conf, "server4", new MockAbortable());
+    KEY_SLAVE3 = new AuthenticationTokenSecretManager(
+        conf, zk3, "server4", 60*60*1000, 60*1000);
+    KEY_SLAVE3.start();
+    Thread.sleep(5000);
+
+    // check master failover again
+    newMaster.stop();
+
+    // wait for master to stop
+    Thread.sleep(5000);
+    assertFalse(newMaster.isMaster());
+
+    // check for a new master
+    mgrs = new AuthenticationTokenSecretManager[]{ KEY_SLAVE, KEY_SLAVE2, KEY_SLAVE3 };
+    newMaster = null;
+    tries = 0;
+    while (newMaster == null && tries++ < 5) {
+      for (AuthenticationTokenSecretManager mgr : mgrs) {
+        if (mgr.isMaster()) {
+          newMaster = mgr;
+          break;
+        }
+      }
+      if (newMaster == null) {
+        Thread.sleep(500);
+      }
+    }
+    assertNotNull(newMaster);
+
+    AuthenticationKey current2 = newMaster.getCurrentKey();
+    // new master will immediately roll the current key, so it's current may be greater
+    assertTrue(current2.getKeyId() >= newCurrent.getKeyId());
+    LOG.debug("New master 2, current key: "+current2.getKeyId());
+
+    // roll the current key again on new master and verify the key ID increments
+    newMaster.rollCurrentKey();
+    AuthenticationKey newCurrent2 = newMaster.getCurrentKey();
+    LOG.debug("New master 2, rolled new current key: "+newCurrent2.getKeyId());
+    assertTrue(newCurrent2.getKeyId() > current2.getKeyId());
+  }
+
+  private static ZooKeeperWatcher newZK(Configuration conf, String name,
+      Abortable abort) throws Exception {
+    Configuration copy = HBaseConfiguration.create(conf);
+    ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+    return zk;
+  }
+}

Modified: hbase/trunk/src/test/resources/hbase-site.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/resources/hbase-site.xml?rev=1337396&r1=1337395&r2=1337396&view=diff
==============================================================================
--- hbase/trunk/src/test/resources/hbase-site.xml (original)
+++ hbase/trunk/src/test/resources/hbase-site.xml Fri May 11 22:06:57 2012
@@ -123,6 +123,10 @@
     </description>
   </property>
   <property>
+    <name>hadoop.log.dir</name>
+    <value>${user.dir}/../logs</value>
+  </property>
+  <property>
     <name>hbase.zookeeper.property.clientPort</name>
     <value>21818</value>
     <description>Property from ZooKeeper's config zoo.cfg.