You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 21:58:32 UTC

svn commit: r1445918 [27/29] - in /hbase/branches/hbase-7290: ./ bin/ conf/ dev-support/ hbase-client/ hbase-common/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ hbase-common/src/ma...

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java Wed Feb 13 20:58:23 2013
@@ -38,6 +38,8 @@ 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.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -46,6 +48,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.BlockingRpcChannel;
+
 @Category(LargeTests.class)
 public class TestAccessControlFilter {
   private static Log LOG = LogFactory.getLog(TestAccessControlFilter.class);
@@ -93,14 +97,14 @@ public class TestAccessControlFilter {
       public Object run() throws Exception {
         HTable aclmeta = new HTable(TEST_UTIL.getConfiguration(),
             AccessControlLists.ACL_TABLE_NAME);
-        AccessControllerProtocol acls = aclmeta.coprocessorProxy(
-            AccessControllerProtocol.class, Bytes.toBytes("testtable"));
-        UserPermission perm = new UserPermission(Bytes.toBytes(READER.getShortName()), 
-                                                 TABLE, null, Permission.Action.READ);
-        acls.grant(perm);
-        perm = new UserPermission(Bytes.toBytes(LIMITED.getShortName()), 
-                                  TABLE, FAMILY, PUBLIC_COL, Permission.Action.READ);
-        acls.grant(perm);
+        byte[] table = Bytes.toBytes("testtable");
+        BlockingRpcChannel service = aclmeta.coprocessorService(table);
+        AccessControlService.BlockingInterface protocol =
+          AccessControlService.newBlockingStub(service);
+        ProtobufUtil.grant(protocol, READER.getShortName(),
+          TABLE, null, null, Permission.Action.READ);
+        ProtobufUtil.grant(protocol, LIMITED.getShortName(),
+          TABLE, FAMILY, PUBLIC_COL, Permission.Action.READ);
         return null;
       }
     });

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Wed Feb 13 20:58:23 2013
@@ -23,19 +23,26 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+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.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.UnknownRowLockException;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -51,14 +58,19 @@ import org.apache.hadoop.hbase.coprocess
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -68,7 +80,6 @@ import org.junit.BeforeClass;
 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.ByteString;
 import com.google.protobuf.ServiceException;
@@ -146,34 +157,38 @@ public class TestAccessController {
     htd.setOwner(USER_OWNER);
     admin.createTable(htd);
 
-    // initilize access control
-    HTable meta = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-    BlockingRpcChannel service = meta.coprocessorService(TEST_TABLE);
-    AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-
     HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0);
     RegionCoprocessorHost rcpHost = region.getCoprocessorHost();
     RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
       Coprocessor.PRIORITY_HIGHEST, 1, conf);
 
-    protocol.grant(null, newGrantRequest(USER_ADMIN.getShortName(),
+    // initilize access control
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+
+      protocol.grant(null, RequestConverter.buildGrantRequest(USER_ADMIN.getShortName(),
         null, null, null,
         AccessControlProtos.Permission.Action.ADMIN,
         AccessControlProtos.Permission.Action.CREATE,
         AccessControlProtos.Permission.Action.READ,
         AccessControlProtos.Permission.Action.WRITE));
 
-    protocol.grant(null, newGrantRequest(USER_RW.getShortName(),
+      protocol.grant(null, RequestConverter.buildGrantRequest(USER_RW.getShortName(),
         TEST_TABLE, TEST_FAMILY, null,
         AccessControlProtos.Permission.Action.READ,
         AccessControlProtos.Permission.Action.WRITE));
 
-    protocol.grant(null, newGrantRequest(USER_RO.getShortName(), TEST_TABLE,
+      protocol.grant(null, RequestConverter.buildGrantRequest(USER_RO.getShortName(), TEST_TABLE,
         TEST_FAMILY, null, AccessControlProtos.Permission.Action.READ));
 
-    protocol.grant(null, newGrantRequest(USER_CREATE.getShortName(),
+      protocol.grant(null, RequestConverter.buildGrantRequest(USER_CREATE.getShortName(),
         TEST_TABLE, null, null, AccessControlProtos.Permission.Action.CREATE));
+    } finally {
+      acl.close();
+    }
   }
 
   @AfterClass
@@ -181,32 +196,6 @@ public class TestAccessController {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private static AccessControlProtos.GrantRequest newGrantRequest(
-      String username, byte[] table, byte[] family, byte[] qualifier,
-      AccessControlProtos.Permission.Action... actions) {
-    AccessControlProtos.Permission.Builder permissionBuilder =
-        AccessControlProtos.Permission.newBuilder();
-    for (AccessControlProtos.Permission.Action a : actions) {
-      permissionBuilder.addAction(a);
-    }
-    if (table != null) {
-      permissionBuilder.setTable(ByteString.copyFrom(table));
-    }
-    if (family != null) {
-      permissionBuilder.setFamily(ByteString.copyFrom(family));
-    }
-    if (qualifier != null) {
-      permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
-    }
-
-    return AccessControlProtos.GrantRequest.newBuilder()
-        .setPermission(
-            AccessControlProtos.UserPermission.newBuilder()
-                .setUser(ByteString.copyFromUtf8(username))
-                .setPermission(permissionBuilder.build())
-        ).build();
-  }
-
   public void verifyAllowed(User user, PrivilegedExceptionAction... actions) throws Exception {
     for (PrivilegedExceptionAction action : actions) {
       try {
@@ -228,28 +217,47 @@ public class TestAccessController {
       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
+      } catch (IOException e) {
         boolean isAccessDeniedException = false;
-        for (Throwable ex : e.getCauses()) {
-          if (ex instanceof ServiceException) {
-            ServiceException se = (ServiceException)ex;
-            if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) {
+        if(e instanceof RetriesExhaustedWithDetailsException) {
+          // in case of batch operations, and put, the client assembles a
+          // RetriesExhaustedWithDetailsException instead of throwing an
+          // AccessDeniedException
+          for(Throwable ex : ((RetriesExhaustedWithDetailsException) e).getCauses()) {
+            if (ex instanceof AccessDeniedException) {
               isAccessDeniedException = true;
               break;
             }
-          } else if (ex instanceof AccessDeniedException) {
-            isAccessDeniedException = true;
-            break;
           }
         }
+        else {
+          // For doBulkLoad calls AccessDeniedException
+          // is buried in the stack trace
+          Throwable ex = e;
+          do {
+            if (ex instanceof AccessDeniedException) {
+              isAccessDeniedException = true;
+              break;
+            }
+          } while((ex = ex.getCause()) != null);
+        }
         if (!isAccessDeniedException) {
           fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
         }
-      } catch (AccessDeniedException ade) {
-        // expected result
+      } catch (UndeclaredThrowableException ute) {
+        // TODO why we get a PrivilegedActionException, which is unexpected?
+        Throwable ex = ute.getUndeclaredThrowable();
+        if (ex instanceof PrivilegedActionException) {
+          ex = ((PrivilegedActionException) ex).getException();
+        }
+        if (ex instanceof ServiceException) {
+          ServiceException se = (ServiceException)ex;
+          if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) {
+            // expected result
+            return;
+          }
+        }
+        fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
       }
     }
   }
@@ -395,8 +403,13 @@ public class TestAccessController {
 
   @Test
   public void testMove() throws Exception {
+    Map<HRegionInfo, ServerName> regions;
     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
-    Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
+    try {
+      regions = table.getRegionLocations();
+    } finally {
+      table.close();
+    }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
     final ServerName server = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
     PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
@@ -413,8 +426,13 @@ public class TestAccessController {
 
   @Test
   public void testAssign() throws Exception {
+    Map<HRegionInfo, ServerName> regions;
     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
-    Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
+    try {
+      regions = table.getRegionLocations();
+    } finally {
+      table.close();
+    }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
 
     PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
@@ -431,8 +449,13 @@ public class TestAccessController {
 
   @Test
   public void testUnassign() throws Exception {
+    Map<HRegionInfo, ServerName> regions;
     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
-    Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
+    try {
+      regions = table.getRegionLocations();
+    } finally {
+      table.close();
+    }
     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
 
     PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
@@ -550,7 +573,8 @@ public class TestAccessController {
   public void testCompact() throws Exception {
     PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
-        ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null);
+        ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null,
+          ScanType.MINOR_COMPACT);
         return null;
       }
     };
@@ -590,7 +614,11 @@ public class TestAccessController {
         Get g = new Get(Bytes.toBytes("random_row"));
         g.addFamily(TEST_FAMILY);
         HTable t = new HTable(conf, TEST_TABLE);
-        t.get(g);
+        try {
+          t.get(g);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -603,14 +631,18 @@ public class TestAccessController {
         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
+          ResultScanner scanner = table.getScanner(s);
+          try {
+            for (Result r = scanner.next(); r != null; r = scanner.next()) {
+              // do nothing
+            }
+          } catch (IOException e) {
+          } finally {
+            scanner.close();
           }
-        } catch (IOException e) {
         } finally {
-          scanner.close();
+          table.close();
         }
         return null;
       }
@@ -627,7 +659,11 @@ public class TestAccessController {
         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);
+        try {
+          t.put(p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -639,7 +675,11 @@ public class TestAccessController {
         Delete d = new Delete(Bytes.toBytes("random_row"));
         d.deleteFamily(TEST_FAMILY);
         HTable t = new HTable(conf, TEST_TABLE);
-        t.delete(d);
+        try {
+          t.delete(d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -651,7 +691,11 @@ public class TestAccessController {
         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);
+        try {
+          t.increment(inc);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -665,10 +709,13 @@ public class TestAccessController {
       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.checkAndDelete(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
-          Bytes.toBytes("test_value"), d);
+        try {
+          t.checkAndDelete(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
+            Bytes.toBytes("test_value"), d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -679,10 +726,13 @@ public class TestAccessController {
       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.checkAndPut(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
-          Bytes.toBytes("test_value"), p);
+        try {
+          t.checkAndPut(Bytes.toBytes("random_row"), TEST_FAMILY, Bytes.toBytes("q"),
+           Bytes.toBytes("test_value"), p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -690,13 +740,129 @@ public class TestAccessController {
   }
 
   @Test
+  public void testBulkLoad() throws Exception {
+    FileSystem fs = TEST_UTIL.getTestFileSystem();
+    final Path dir = TEST_UTIL.getDataTestDir("testBulkLoad");
+    fs.mkdirs(dir);
+    //need to make it globally writable
+    //so users creating HFiles have write permissions
+    fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
+
+    PrivilegedExceptionAction bulkLoadAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        int numRows = 3;
+
+        //Making the assumption that the test table won't split between the range
+        byte[][][] hfileRanges = {{{(byte)0}, {(byte)9}}};
+
+        Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
+        new BulkLoadHelper(bulkLoadBasePath)
+            .bulkLoadHFile(TEST_TABLE, TEST_FAMILY, Bytes.toBytes("q"), hfileRanges, numRows);
+
+        return null;
+      }
+    };
+    verifyWrite(bulkLoadAction);
+
+    // Reinit after the bulk upload
+    TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE);
+    TEST_UTIL.getHBaseAdmin().enableTable(TEST_TABLE);
+  }
+
+  public class BulkLoadHelper {
+    private final FileSystem fs;
+    private final Path loadPath;
+    private final Configuration conf;
+
+    public BulkLoadHelper(Path loadPath) throws IOException {
+      fs = TEST_UTIL.getTestFileSystem();
+      conf = TEST_UTIL.getConfiguration();
+      loadPath = loadPath.makeQualified(fs);
+      this.loadPath = loadPath;
+    }
+
+    private void createHFile(Path path,
+        byte[] family, byte[] qualifier,
+        byte[] startKey, byte[] endKey, int numRows) throws IOException {
+
+      HFile.Writer writer = null;
+      long now = System.currentTimeMillis();
+      try {
+        writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
+            .withPath(fs, path)
+            .withComparator(KeyValue.KEY_COMPARATOR)
+            .create();
+        // subtract 2 since numRows doesn't include boundary keys
+        for (byte[] key : Bytes.iterateOnSplits(startKey, endKey, true, numRows-2)) {
+          KeyValue kv = new KeyValue(key, family, qualifier, now, key);
+          writer.append(kv);
+        }
+      } finally {
+        if(writer != null)
+          writer.close();
+      }
+    }
+
+    private void bulkLoadHFile(
+        byte[] tableName,
+        byte[] family,
+        byte[] qualifier,
+        byte[][][] hfileRanges,
+        int numRowsPerRange) throws Exception {
+
+      Path familyDir = new Path(loadPath, Bytes.toString(family));
+      fs.mkdirs(familyDir);
+      int hfileIdx = 0;
+      for (byte[][] range : hfileRanges) {
+        byte[] from = range[0];
+        byte[] to = range[1];
+        createHFile(new Path(familyDir, "hfile_"+(hfileIdx++)),
+            family, qualifier, from, to, numRowsPerRange);
+      }
+      //set global read so RegionServer can move it
+      setPermission(loadPath, FsPermission.valueOf("-rwxrwxrwx"));
+
+      HTable table = new HTable(conf, tableName);
+      try {
+        TEST_UTIL.waitTableAvailable(tableName, 30000);
+        LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
+        loader.doBulkLoad(loadPath, table);
+      } finally {
+        table.close();
+      }
+    }
+
+    public void setPermission(Path dir, FsPermission perm) throws IOException {
+      if(!fs.getFileStatus(dir).isDir()) {
+        fs.setPermission(dir,perm);
+      }
+      else {
+        for(FileStatus el : fs.listStatus(dir)) {
+          fs.setPermission(el.getPath(), perm);
+          setPermission(el.getPath() , perm);
+        }
+      }
+    }
+  }
+
+  @Test
   public void testAppend() throws Exception {
 
     PrivilegedExceptionAction appendAction = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
-        Append append = new Append(TEST_TABLE);
-        append.add(TEST_FAMILY, Bytes.toBytes("qualifier"), Bytes.toBytes("value"));
-        ACCESS_CONTROLLER.preAppend(ObserverContext.createAndPrepare(RCP_ENV, null), append);
+        byte[] row = Bytes.toBytes("random_row");
+        byte[] qualifier = Bytes.toBytes("q");
+        Put put = new Put(row);
+        put.add(TEST_FAMILY, qualifier, Bytes.toBytes(1));
+        Append append = new Append(row);
+        append.add(TEST_FAMILY, qualifier, Bytes.toBytes(2));
+        HTable t = new HTable(conf, TEST_TABLE);
+        try {
+          t.put(put);
+          t.append(append);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -711,10 +877,15 @@ public class TestAccessController {
     PrivilegedExceptionAction grantAction = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-        AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class,
-          TEST_TABLE);
-        protocol.grant(new UserPermission(Bytes.toBytes(USER_RO.getShortName()), TEST_TABLE,
-            TEST_FAMILY, (byte[]) null, Action.READ));
+        try {
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE);
+          AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+          ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE,
+            TEST_FAMILY, null, Action.READ);
+        } finally {
+          acl.close();
+        }
         return null;
       }
     };
@@ -722,10 +893,15 @@ public class TestAccessController {
     PrivilegedExceptionAction revokeAction = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-        AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class,
-          TEST_TABLE);
-        protocol.revoke(new UserPermission(Bytes.toBytes(USER_RO.getShortName()), TEST_TABLE,
-            TEST_FAMILY, (byte[]) null, Action.READ));
+        try {
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE);
+          AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+          ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE,
+            TEST_FAMILY, null, Action.READ);
+        } finally {
+          acl.close();
+        }
         return null;
       }
     };
@@ -733,9 +909,14 @@ public class TestAccessController {
     PrivilegedExceptionAction getPermissionsAction = new PrivilegedExceptionAction() {
       public Object run() throws Exception {
         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-        AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class,
-          TEST_TABLE);
-        protocol.getUserPermissions(TEST_TABLE);
+        try {
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE);
+          AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+          ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
+        } finally {
+          acl.close();
+        }
         return null;
       }
     };
@@ -774,11 +955,6 @@ public class TestAccessController {
     User gblUser = User
         .createUserForTesting(TEST_UTIL.getConfiguration(), "gbluser", 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 {
@@ -786,7 +962,11 @@ public class TestAccessController {
         p.add(family1, qualifier, Bytes.toBytes("v1"));
         p.add(family2, qualifier, Bytes.toBytes("v2"));
         HTable t = new HTable(conf, tableName);
-        t.put(p);
+        try {
+          t.put(p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -795,7 +975,11 @@ public class TestAccessController {
         Put p = new Put(Bytes.toBytes("a"));
         p.add(family1, qualifier, Bytes.toBytes("v1"));
         HTable t = new HTable(conf, tableName);
-        t.put(p);
+        try {
+          t.put(p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -804,7 +988,11 @@ public class TestAccessController {
         Put p = new Put(Bytes.toBytes("a"));
         p.add(family2, qualifier, Bytes.toBytes("v2"));
         HTable t = new HTable(conf, tableName);
-        t.put(p);
+        try {
+          t.put(p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -814,7 +1002,11 @@ public class TestAccessController {
         g.addFamily(family1);
         g.addFamily(family2);
         HTable t = new HTable(conf, tableName);
-        t.get(g);
+        try {
+          t.get(g);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -823,7 +1015,11 @@ public class TestAccessController {
         Get g = new Get(Bytes.toBytes("random_row"));
         g.addFamily(family1);
         HTable t = new HTable(conf, tableName);
-        t.get(g);
+        try {
+          t.get(g);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -832,7 +1028,11 @@ public class TestAccessController {
         Get g = new Get(Bytes.toBytes("random_row"));
         g.addFamily(family2);
         HTable t = new HTable(conf, tableName);
-        t.get(g);
+        try {
+          t.get(g);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -842,7 +1042,11 @@ public class TestAccessController {
         d.deleteFamily(family1);
         d.deleteFamily(family2);
         HTable t = new HTable(conf, tableName);
-        t.delete(d);
+        try {
+          t.delete(d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -851,7 +1055,11 @@ public class TestAccessController {
         Delete d = new Delete(Bytes.toBytes("random_row"));
         d.deleteFamily(family1);
         HTable t = new HTable(conf, tableName);
-        t.delete(d);
+        try {
+          t.delete(d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -860,7 +1068,11 @@ public class TestAccessController {
         Delete d = new Delete(Bytes.toBytes("random_row"));
         d.deleteFamily(family2);
         HTable t = new HTable(conf, tableName);
-        t.delete(d);
+        try {
+          t.delete(d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -875,10 +1087,18 @@ public class TestAccessController {
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant table read permission
-    protocol.grant(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, null,
-        Permission.Action.READ));
-    protocol
-        .grant(new UserPermission(Bytes.toBytes(gblUser.getShortName()), Permission.Action.READ));
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, tblUser.getShortName(),
+        tableName, null, null, Permission.Action.READ);
+      ProtobufUtil.grant(protocol, gblUser.getShortName(),
+        null, null, null, Permission.Action.READ);
+    } finally {
+      acl.close();
+    }
 
     Thread.sleep(100);
     // check
@@ -891,10 +1111,19 @@ public class TestAccessController {
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant table write permission
-    protocol.grant(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, null,
-        Permission.Action.WRITE));
-    protocol.grant(new UserPermission(Bytes.toBytes(gblUser.getShortName()),
-        Permission.Action.WRITE));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, tblUser.getShortName(),
+        tableName, null, null, Permission.Action.WRITE);
+      ProtobufUtil.grant(protocol, gblUser.getShortName(),
+        null, null, null, Permission.Action.WRITE);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyDenied(tblUser, getActionAll, getAction1, getAction2);
@@ -906,10 +1135,19 @@ public class TestAccessController {
     verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // revoke table permission
-    protocol.grant(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, null,
-        Permission.Action.READ, Permission.Action.WRITE));
-    protocol.revoke(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, null));
-    protocol.revoke(new UserPermission(Bytes.toBytes(gblUser.getShortName())));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, tblUser.getShortName(), tableName, null, null,
+        Permission.Action.READ, Permission.Action.WRITE);
+      ProtobufUtil.revoke(protocol, tblUser.getShortName(), tableName, null, null);
+      ProtobufUtil.revoke(protocol, gblUser.getShortName(), null, null, null);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyDenied(tblUser, getActionAll, getAction1, getAction2);
@@ -921,10 +1159,18 @@ public class TestAccessController {
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant column family read permission
-    protocol.grant(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, family1,
-        Permission.Action.READ));
-    protocol
-        .grant(new UserPermission(Bytes.toBytes(gblUser.getShortName()), Permission.Action.READ));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, tblUser.getShortName(),
+        tableName, family1, null, Permission.Action.READ);
+      ProtobufUtil.grant(protocol, gblUser.getShortName(),
+        null, null, null, Permission.Action.READ);
+    } finally {
+      acl.close();
+    }
 
     Thread.sleep(100);
 
@@ -939,10 +1185,19 @@ public class TestAccessController {
     verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // grant column family write permission
-    protocol.grant(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, family2,
-        Permission.Action.WRITE));
-    protocol.grant(new UserPermission(Bytes.toBytes(gblUser.getShortName()),
-        Permission.Action.WRITE));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, tblUser.getShortName(),
+        tableName, family2, null, Permission.Action.WRITE);
+      ProtobufUtil.grant(protocol, gblUser.getShortName(),
+        null, null, null, Permission.Action.WRITE);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     // READ from family1, WRITE to family2 are allowed
@@ -957,8 +1212,16 @@ public class TestAccessController {
     verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
     // revoke column family permission
-    protocol.revoke(new UserPermission(Bytes.toBytes(tblUser.getShortName()), tableName, family2));
-    protocol.revoke(new UserPermission(Bytes.toBytes(gblUser.getShortName())));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.revoke(protocol, tblUser.getShortName(), tableName, family2, null);
+      ProtobufUtil.revoke(protocol, gblUser.getShortName(), null, null, null);
+    } finally {
+      acl.close();
+    }
 
     Thread.sleep(100);
 
@@ -991,7 +1254,6 @@ public class TestAccessController {
 
     // create table
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -1004,16 +1266,16 @@ public class TestAccessController {
     // 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);
+        try {
+          t.get(g);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -1022,7 +1284,11 @@ public class TestAccessController {
         Put p = new Put(Bytes.toBytes("random_row"));
         p.add(family1, qualifier, Bytes.toBytes("v1"));
         HTable t = new HTable(conf, tableName);
-        t.put(p);
+        try {
+          t.put(p);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
@@ -1032,18 +1298,42 @@ public class TestAccessController {
         d.deleteColumn(family1, qualifier);
         // d.deleteFamily(family1);
         HTable t = new HTable(conf, tableName);
-        t.delete(d);
+        try {
+          t.delete(d);
+        } finally {
+          t.close();
+        }
         return null;
       }
     };
 
-    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()), tableName, family1));
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.revoke(protocol, user.getShortName(), tableName, family1, null);
+    } finally {
+      acl.close();
+    }
+
+    Thread.sleep(100);
+
     verifyDenied(user, getQualifierAction);
     verifyDenied(user, putQualifierAction);
     verifyDenied(user, deleteQualifierAction);
 
-    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()), tableName, family1,
-        qualifier, Permission.Action.READ));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, user.getShortName(),
+        tableName, family1, qualifier, Permission.Action.READ);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyAllowed(user, getQualifierAction);
@@ -1052,8 +1342,17 @@ public class TestAccessController {
 
     // only grant write permission
     // TODO: comment this portion after HBASE-3583
-    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()), tableName, family1,
-        qualifier, Permission.Action.WRITE));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, user.getShortName(),
+        tableName, family1, qualifier, Permission.Action.WRITE);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyDenied(user, getQualifierAction);
@@ -1061,8 +1360,18 @@ public class TestAccessController {
     verifyAllowed(user, deleteQualifierAction);
 
     // grant both read and write permission.
-    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()), tableName, family1,
-        qualifier, Permission.Action.READ, Permission.Action.WRITE));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, user.getShortName(),
+        tableName, family1, qualifier,
+          Permission.Action.READ, Permission.Action.WRITE);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyAllowed(user, getQualifierAction);
@@ -1070,8 +1379,17 @@ public class TestAccessController {
     verifyAllowed(user, deleteQualifierAction);
 
     // revoke family level permission won't impact column level.
-    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()), tableName, family1,
-        qualifier));
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.revoke(protocol, user.getShortName(),
+        tableName, family1, qualifier);
+    } finally {
+      acl.close();
+    }
+
     Thread.sleep(100);
 
     verifyDenied(user, getQualifierAction);
@@ -1089,7 +1407,6 @@ public class TestAccessController {
     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();
@@ -1103,50 +1420,85 @@ public class TestAccessController {
     htd.setOwner(USER_OWNER);
     admin.createTable(htd);
 
-    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-    AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class,
-      tableName);
+    List<UserPermission> perms;
 
-    List<UserPermission> perms = protocol.getUserPermissions(tableName);
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+    } finally {
+      acl.close();
+    }
 
-    UserPermission ownerperm = new UserPermission(Bytes.toBytes(USER_OWNER.getName()), tableName,
-        null, Action.values());
+    UserPermission ownerperm = new UserPermission(
+      Bytes.toBytes(USER_OWNER.getName()), tableName, null, Action.values());
     assertTrue("Owner should have all permissions on table",
       hasFoundUserPermission(ownerperm, perms));
 
-    UserPermission up = new UserPermission(user, tableName, family1, qualifier,
-        Permission.Action.READ);
+    User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
+    byte[] userName = Bytes.toBytes(user.getShortName());
+
+    UserPermission up = new UserPermission(userName,
+      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(upToSet);
-    perms = protocol.getUserPermissions(tableName);
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, user.getShortName(),
+        tableName, family1, qualifier, Permission.Action.READ);
+      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+    } finally {
+      acl.close();
+    }
 
-    UserPermission upToVerify = new UserPermission(user, tableName, family1, qualifier,
-        Permission.Action.READ);
+    UserPermission upToVerify = new UserPermission(
+      userName, 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);
+    upToVerify = new UserPermission(
+      userName, 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(upToSet);
-    perms = protocol.getUserPermissions(tableName);
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.grant(protocol, user.getShortName(),
+        tableName, family1, qualifier,
+          Permission.Action.WRITE, Permission.Action.READ);
+      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+    } finally {
+      acl.close();
+    }
 
-    upToVerify = new UserPermission(user, tableName, family1, qualifier, Permission.Action.WRITE,
-        Permission.Action.READ);
+    upToVerify = new UserPermission(userName, tableName, family1,
+      qualifier, Permission.Action.WRITE, Permission.Action.READ);
     assertTrue("User should be granted permission: " + upToVerify.toString(),
       hasFoundUserPermission(upToVerify, perms));
 
-    protocol.revoke(upToSet);
-    perms = protocol.getUserPermissions(tableName);
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      ProtobufUtil.revoke(protocol, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.WRITE, Permission.Action.READ);
+      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+    } finally {
+      acl.close();
+    }
+
     assertFalse("User should not be granted permission: " + upToVerify.toString(),
       hasFoundUserPermission(upToVerify, perms));
 
@@ -1156,9 +1508,19 @@ public class TestAccessController {
     User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
     htd.setOwner(newOwner);
     admin.modifyTable(tableName, htd);
-    perms = protocol.getUserPermissions(tableName);
-    UserPermission newOwnerperm = new UserPermission(Bytes.toBytes(newOwner.getName()), tableName,
-        null, Action.values());
+
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(tableName);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+    } finally {
+      acl.close();
+    }
+
+    UserPermission newOwnerperm = new UserPermission(
+      Bytes.toBytes(newOwner.getName()), tableName, null, Action.values());
     assertTrue("New owner should have all permissions on table",
       hasFoundUserPermission(newOwnerperm, perms));
 
@@ -1166,6 +1528,24 @@ public class TestAccessController {
     admin.deleteTable(tableName);
   }
 
+  @Test
+  public void testGlobalPermissionList() throws Exception {
+    List<UserPermission> perms;
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(service);
+      perms = ProtobufUtil.getUserPermissions(protocol, null);
+    } finally {
+      acl.close();
+    }
+    UserPermission adminPerm = new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
+      AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW"));
+    assertTrue("Only user admin has permission on table _acl_ per setup",
+      perms.size() == 1 && hasFoundUserPermission(adminPerm, perms));
+  }
+
   /** global operations */
   private void verifyGlobal(PrivilegedExceptionAction<?> action) throws Exception {
     verifyAllowed(action, SUPERUSER);
@@ -1174,25 +1554,27 @@ public class TestAccessController {
   }
 
   public void checkGlobalPerms(Permission.Action... actions) throws IOException {
-    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-    BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
-    AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(channel);
-
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {
       perms[i] = new Permission(actions[i]);
     }
-
     CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
     for (Action a : actions) {
       request.addPermission(AccessControlProtos.Permission.newBuilder()
           .addAction(ProtobufUtil.toPermissionAction(a)).build());
     }
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      protocol.checkPermissions(null, request.build());
-    } catch (ServiceException se) {
-      ProtobufUtil.toIOException(se);
+      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(channel);
+      try {
+        protocol.checkPermissions(null, request.build());
+      } catch (ServiceException se) {
+        ProtobufUtil.toIOException(se);
+      }
+    } finally {
+      acl.close();
     }
   }
 
@@ -1207,41 +1589,26 @@ public class TestAccessController {
   }
 
   public void checkTablePerms(byte[] table, Permission... perms) throws IOException {
-    HTable acl = new HTable(conf, table);
-    AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0]));
     CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
     for (Permission p : perms) {
       request.addPermission(ProtobufUtil.toPermission(p));
     }
+    HTable acl = new HTable(conf, table);
     try {
-      protocol.checkPermissions(null, request.build());
-    } catch (ServiceException se) {
-      ProtobufUtil.toIOException(se);
-    }
-  }
-
-  public void grant(AccessControlService.BlockingInterface protocol, User user,
-      byte[] t, byte[] f, byte[] q, Permission.Action... actions)
-      throws ServiceException {
-    List<AccessControlProtos.Permission.Action> permActions =
-        Lists.newArrayListWithCapacity(actions.length);
-    for (Action a : actions) {
-      permActions.add(ProtobufUtil.toPermissionAction(a));
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0]));
+      try {
+        protocol.checkPermissions(null, request.build());
+      } catch (ServiceException se) {
+        ProtobufUtil.toIOException(se);
+      }
+    } finally {
+      acl.close();
     }
-    AccessControlProtos.GrantRequest request =
-        newGrantRequest(user.getShortName(), t, f, q, permActions.toArray(
-            new AccessControlProtos.Permission.Action[actions.length]));
-    protocol.grant(null, request);
   }
 
   @Test
   public void testCheckPermissions() throws Exception {
-    final HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
-    BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
-    AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(channel);
-
     // --------------------------------------
     // test global permissions
     PrivilegedExceptionAction<Void> globalAdmin = new PrivilegedExceptionAction<Void>() {
@@ -1275,9 +1642,20 @@ public class TestAccessController {
     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);
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(channel);
+      ProtobufUtil.grant(protocol, userTable.getShortName(),
+        TEST_TABLE, null, null, Permission.Action.READ);
+      ProtobufUtil.grant(protocol, userColumn.getShortName(),
+        TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ);
+      ProtobufUtil.grant(protocol, userQualifier.getShortName(),
+        TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ);
+    } finally {
+      acl.close();
+    }
 
     PrivilegedExceptionAction<Void> tableRead = new PrivilegedExceptionAction<Void>() {
       @Override
@@ -1362,19 +1740,26 @@ public class TestAccessController {
 
     // --------------------------------------
     // check for wrong table region
+    CheckPermissionsRequest checkRequest = CheckPermissionsRequest.newBuilder()
+      .addPermission(AccessControlProtos.Permission.newBuilder()
+        .setTable(ByteString.copyFrom(TEST_TABLE))
+        .addAction(AccessControlProtos.Permission.Action.CREATE)
+      ).build();
+    acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     try {
-      CheckPermissionsRequest checkRequest =
-          CheckPermissionsRequest.newBuilder().addPermission(
-              AccessControlProtos.Permission.newBuilder()
-                  .setTable(ByteString.copyFrom(TEST_TABLE)).addAction(AccessControlProtos.Permission.Action.CREATE)
-          ).build();
-      // but ask for TablePermissions for TEST_TABLE
-      protocol.checkPermissions(null, checkRequest);
-      fail("this should have thrown CoprocessorException");
-    } catch (ServiceException ex) {
-      // expected
+      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(channel);
+      try {
+        // but ask for TablePermissions for TEST_TABLE
+        protocol.checkPermissions(null, checkRequest);
+        fail("this should have thrown CoprocessorException");
+      } catch (ServiceException ex) {
+        // expected
+      }
+    } finally {
+      acl.close();
     }
-
   }
 
   @Test

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java Wed Feb 13 20:58:23 2013
@@ -24,19 +24,21 @@ import static org.junit.Assert.assertNot
 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.DataOutput;
 import java.io.DataOutputStream;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 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.DeserializationException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.io.Text;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -104,6 +107,56 @@ public class TestTablePermissions {
     UTIL.shutdownMiniCluster();
   }
 
+
+  /**
+   * Test we can read permissions serialized with Writables.
+   * @throws DeserializationException
+   */
+  @Test
+  public void testMigration() throws DeserializationException {
+    Configuration conf = UTIL.getConfiguration();
+    ListMultimap<String,TablePermission> permissions = createPermissions();
+    byte [] bytes = writePermissionsAsBytes(permissions, conf);
+    AccessControlLists.readPermissions(bytes, conf);
+  }
+
+  /**
+   * Writes a set of permissions as {@link org.apache.hadoop.io.Writable} instances                                                                                                                     
+   * and returns the resulting byte array.  Used to verify we can read stuff written
+   * with Writable.
+   */
+  public static byte[] writePermissionsAsBytes(ListMultimap<String,? extends Permission> perms,
+      Configuration conf) {
+    try {
+       ByteArrayOutputStream bos = new ByteArrayOutputStream();
+       writePermissions(new DataOutputStream(bos), perms, conf);
+       return bos.toByteArray();
+    } catch (IOException ioe) {
+      // shouldn't happen here
+      throw new RuntimeException("Error serializing permissions", ioe);
+    }
+  }
+
+  /**
+   * Writes a set of permissions as {@link org.apache.hadoop.io.Writable} instances
+   * to the given output stream.
+   * @param out
+   * @param perms
+   * @param conf
+   * @throws IOException
+  */
+  public static void writePermissions(DataOutput out,                                                                                                                                                   
+      ListMultimap<String,? extends Permission> perms, Configuration conf)
+  throws IOException {
+    Set<String> keys = perms.keySet();
+    out.writeInt(keys.size());
+    for (String key : keys) {
+      Text.writeString(out, key);
+      HbaseObjectWritableFor96Migration.writeObject(out, perms.get(key), List.class, conf);
+    }
+  }
+
+
   @Test
   public void testBasicWrite() throws Exception {
     Configuration conf = UTIL.getConfiguration();
@@ -243,6 +296,16 @@ public class TestTablePermissions {
   @Test
   public void testSerialization() throws Exception {
     Configuration conf = UTIL.getConfiguration();
+    ListMultimap<String,TablePermission> permissions = createPermissions();
+    byte[] permsData = AccessControlLists.writePermissionsAsBytes(permissions, conf);
+
+    ListMultimap<String,TablePermission> copy =
+        AccessControlLists.readPermissions(permsData, conf);
+
+    checkMultimapEqual(permissions, copy);
+  }
+
+  private ListMultimap<String,TablePermission> createPermissions() {
     ListMultimap<String,TablePermission> permissions = ArrayListMultimap.create();
     permissions.put("george", new TablePermission(TEST_TABLE, null,
         TablePermission.Action.READ));
@@ -252,13 +315,7 @@ public class TestTablePermissions {
         TablePermission.Action.READ));
     permissions.put("hubert", new TablePermission(TEST_TABLE2, null,
         TablePermission.Action.READ, TablePermission.Action.WRITE));
-
-    byte[] permsData = AccessControlLists.writePermissionsAsBytes(permissions, conf);
-
-    ListMultimap<String,TablePermission> copy =
-        AccessControlLists.readPermissions(permsData, conf);
-
-    checkMultimapEqual(permissions, copy);
+    return permissions;
   }
 
   public void checkMultimapEqual(ListMultimap<String,TablePermission> first,

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Wed Feb 13 20:58:23 2013
@@ -21,27 +21,55 @@ package org.apache.hadoop.hbase.security
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
-import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
 
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.IpcProtocol;
+import org.apache.hadoop.hbase.Coprocessor;
 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.MediumTests;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
 import org.apache.hadoop.hbase.ipc.HBaseServer;
+import org.apache.hadoop.hbase.ipc.HBaseServerRPC;
+import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine;
 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.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.security.KerberosInfo;
+import org.apache.hadoop.hbase.security.TokenInfo;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.MockRegionServerServices;
+import org.apache.hadoop.hbase.util.Sleeper;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -50,48 +78,247 @@ import org.junit.experimental.categories
 /**
  * Tests for authentication token creation and usage
  */
-@Category(LargeTests.class)
+@Category(MediumTests.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();
+  private static Log LOG = LogFactory.getLog(TestTokenAuthentication.class);
+
+  @KerberosInfo(
+      serverPrincipal = "hbase.test.kerberos.principal")
+  @TokenInfo("HBASE_AUTH_TOKEN")
+  private static interface BlockingAuthenticationService
+  extends AuthenticationProtos.AuthenticationService.BlockingInterface, IpcProtocol {
+  }
+
+  /**
+   * Basic server process for RPC authentication testing
+   */
+  private static class TokenServer extends TokenProvider
+      implements BlockingAuthenticationService, Runnable, Server {
+
+    private static Log LOG = LogFactory.getLog(TokenServer.class);
+
+    private Configuration conf;
+    private RpcServer rpcServer;
+    private InetSocketAddress isa;
+    private ZooKeeperWatcher zookeeper;
+    private Sleeper sleeper;
+    private boolean started = false;
+    private boolean aborted = false;
+    private boolean stopped = false;
+    private long startcode;
+    private AuthenticationProtos.AuthenticationService.BlockingInterface blockingService;
+
+    public TokenServer(Configuration conf) throws IOException {
+      this.conf = conf;
+      this.startcode = EnvironmentEdgeManager.currentTimeMillis();
+
+      // Server to handle client requests.
+      String hostname = Strings.domainNamePointerToHostName(
+          DNS.getDefaultHost("default", "default"));
+      int port = 0;
+      // Creation of an ISA will force a resolve.
+      InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
+      if (initialIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initialIsa);
       }
+
+      this.rpcServer = HBaseServerRPC.getServer(TokenServer.class, this,
+          new Class<?>[]{AuthenticationProtos.AuthenticationService.Interface.class},
+          initialIsa.getHostName(), // BindAddress is IP we got for this server.
+          initialIsa.getPort(),
+          3, // handlers
+          1, // meta handlers (not used)
+          true,
+          this.conf, HConstants.QOS_THRESHOLD);
+      // Set our address.
+      this.isa = this.rpcServer.getListenerAddress();
+      this.sleeper = new Sleeper(1000, this);
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      return conf;
+    }
+
+    @Override
+    public CatalogTracker getCatalogTracker() {
       return null;
     }
+
+    @Override
+    public ZooKeeperWatcher getZooKeeper() {
+      return zookeeper;
+    }
+
+    @Override
+    public boolean isAborted() {
+      return aborted;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return new ServerName(isa.getHostName(), isa.getPort(), startcode);
+    }
+
+    @Override
+    public void abort(String reason, Throwable error) {
+      LOG.fatal("Aborting on: "+reason, error);
+      this.aborted = true;
+      this.stopped = true;
+      sleeper.skipSleepCycle();
+    }
+
+    private void initialize() throws IOException {
+      // ZK configuration must _not_ have hbase.security.authentication or it will require SASL auth
+      Configuration zkConf = new Configuration(conf);
+      zkConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+      this.zookeeper = new ZooKeeperWatcher(zkConf, TokenServer.class.getSimpleName(),
+          this, true);
+      this.rpcServer.start();
+
+      // mock RegionServerServices to provide to coprocessor environment
+      final RegionServerServices mockServices = new MockRegionServerServices() {
+        @Override
+        public RpcServer getRpcServer() { return rpcServer; }
+      };
+
+      // mock up coprocessor environment
+      super.start(new RegionCoprocessorEnvironment() {
+        @Override
+        public HRegion getRegion() { return null; }
+
+        @Override
+        public RegionServerServices getRegionServerServices() {
+          return mockServices;
+        }
+
+        @Override
+        public ConcurrentMap<String, Object> getSharedData() { return null; }
+
+        @Override
+        public int getVersion() { return 0; }
+
+        @Override
+        public String getHBaseVersion() { return null; }
+
+        @Override
+        public Coprocessor getInstance() { return null; }
+
+        @Override
+        public int getPriority() { return 0; }
+
+        @Override
+        public int getLoadSequence() { return 0; }
+
+        @Override
+        public Configuration getConfiguration() { return conf; }
+
+        @Override
+        public HTableInterface getTable(byte[] tableName) throws IOException { return null; }
+      });
+
+      started = true;
+    }
+
+    public void run() {
+      try {
+        initialize();
+        while (!stopped) {
+          this.sleeper.sleep();
+        }
+      } catch (Exception e) {
+        abort(e.getMessage(), e);
+      }
+      this.rpcServer.stop();
+    }
+
+    public boolean isStarted() {
+      return started;
+    }
+
+    @Override
+    public void stop(String reason) {
+      LOG.info("Stopping due to: "+reason);
+      this.stopped = true;
+      sleeper.skipSleepCycle();
+    }
+
+    @Override
+    public boolean isStopped() {
+      return stopped;
+    }
+
+    public InetSocketAddress getAddress() {
+      return isa;
+    }
+
+    public SecretManager<? extends TokenIdentifier> getSecretManager() {
+      return ((HBaseServer)rpcServer).getSecretManager();
+    }
+
+    @Override
+    public AuthenticationProtos.TokenResponse getAuthenticationToken(
+        RpcController controller, AuthenticationProtos.TokenRequest request)
+      throws ServiceException {
+      LOG.debug("Authentication token request from "+RequestContext.getRequestUserName());
+      // ignore passed in controller -- it's always null
+      ServerRpcController serverController = new ServerRpcController();
+      BlockingRpcCallback<AuthenticationProtos.TokenResponse> callback =
+          new BlockingRpcCallback<AuthenticationProtos.TokenResponse>();
+      getAuthenticationToken(serverController, request, callback);
+      try {
+        serverController.checkFailed();
+        return callback.get();
+      } catch (IOException ioe) {
+        throw new ServiceException(ioe);
+      }
+    }
+
+    @Override
+    public AuthenticationProtos.WhoAmIResponse whoami(
+        RpcController controller, AuthenticationProtos.WhoAmIRequest request)
+      throws ServiceException {
+      LOG.debug("whoami() request from "+RequestContext.getRequestUserName());
+      // ignore passed in controller -- it's always null
+      ServerRpcController serverController = new ServerRpcController();
+      BlockingRpcCallback<AuthenticationProtos.WhoAmIResponse> callback =
+          new BlockingRpcCallback<AuthenticationProtos.WhoAmIResponse>();
+      whoami(serverController, request, callback);
+      try {
+        serverController.checkFailed();
+        return callback.get();
+      } catch (IOException ioe) {
+        throw new ServiceException(ioe);
+      }
+    }
   }
 
+
   private static HBaseTestingUtility TEST_UTIL;
+  private static TokenServer server;
+  private static Thread serverThread;
   private static AuthenticationTokenSecretManager secretManager;
+  private static ClusterId clusterId = new ClusterId();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.startMiniZKCluster();
+    // security settings only added after startup so that ZK does not require SASL
     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(); 
+    conf.set("hadoop.security.authentication", "kerberos");
+    conf.set("hbase.security.authentication", "kerberos");
+    server = new TokenServer(conf);
+    serverThread = new Thread(server);
+    Threads.setDaemonThreadRunning(serverThread,
+        "TokenServer:"+server.getServerName().toString());
+    // wait for startup
+    while (!server.isStarted() && !server.isStopped()) {
+      Thread.sleep(10);
+    }
+
+    ZKClusterId.setClusterId(server.getZooKeeper(), clusterId);
+    secretManager = (AuthenticationTokenSecretManager)server.getSecretManager();
     while(secretManager.getCurrentKey() == null) {
       Thread.sleep(1);
     }
@@ -99,7 +326,9 @@ public class TestTokenAuthentication {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    server.stop("Test complete");
+    Threads.shutdown(serverThread);
+    TEST_UTIL.shutdownMiniZKCluster();
   }
 
   @Test
@@ -116,7 +345,7 @@ public class TestTokenAuthentication {
         Bytes.equals(token.getPassword(), passwd));
   }
 
-  // @Test - Disable due to kerberos requirement
+  @Test
   public void testTokenAuthentication() throws Exception {
     UserGroupInformation testuser =
         UserGroupInformation.createUserForTesting("testuser", new String[]{"testgroup"});
@@ -124,23 +353,36 @@ public class TestTokenAuthentication {
     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);
+    UserGroupInformation.setConfiguration(conf);
     Token<AuthenticationTokenIdentifier> token =
         secretManager.generateToken("testuser");
+    LOG.debug("Got token: " + token.toString());
     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);
+        Configuration c = server.getConfiguration();
+        c.set(HConstants.CLUSTER_ID, clusterId.toString());
+        ProtobufRpcClientEngine rpcClient =
+            new ProtobufRpcClientEngine(c);
+        try {
+          AuthenticationProtos.AuthenticationService.BlockingInterface proxy =
+              HBaseClientRPC.waitForProxy(rpcClient, BlockingAuthenticationService.class,
+                  server.getAddress(), c,
+                  HConstants.DEFAULT_HBASE_CLIENT_RPC_MAXATTEMPTS,
+                  HConstants.DEFAULT_HBASE_RPC_TIMEOUT,
+                  HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+
+          AuthenticationProtos.WhoAmIResponse response =
+              proxy.whoami(null, AuthenticationProtos.WhoAmIRequest.getDefaultInstance());
+          String myname = response.getUsername();
+          assertEquals("testuser", myname);
+          String authMethod = response.getAuthMethod();
+          assertEquals("TOKEN", authMethod);
+        } finally {
+          rpcClient.close();
+        }
         return null;
       }
     });

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java Wed Feb 13 20:58:23 2013
@@ -32,6 +32,9 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
+import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -115,11 +118,15 @@ public class TestThriftServerCmdLine {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniCluster();
+    //ensure that server time increments every time we do an operation, otherwise
+    //successive puts having the same timestamp will override each other
+    EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
+    EnvironmentEdgeManager.reset();
   }
 
   private void startCmdLineThread(final String[] args) {
@@ -142,7 +149,7 @@ public class TestThriftServerCmdLine {
     cmdLineThread.start();
   }
 
-  @Test(timeout=60 * 1000)
+  @Test(timeout=120 * 1000)
   public void testRunThriftServer() throws Exception {
     List<String> args = new ArrayList<String>();
     if (implType != null) {

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java Wed Feb 13 20:58:23 2013
@@ -31,7 +31,9 @@ import org.apache.hadoop.hbase.PleaseHol
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
 
 /**
  * A command-line utility that reads, writes, and verifies data. Unlike
@@ -67,7 +69,7 @@ public class LoadTestTool extends Abstra
       "<verify_percent>[:<#threads=" + DEFAULT_NUM_THREADS + ">]";
 
   private static final String OPT_USAGE_BLOOM = "Bloom filter type, one of " +
-      Arrays.toString(StoreFile.BloomType.values());
+      Arrays.toString(BloomType.values());
 
   private static final String OPT_USAGE_COMPRESSION = "Compression type, " +
       "one of " + Arrays.toString(Compression.Algorithm.values());
@@ -115,11 +117,11 @@ public class LoadTestTool extends Abstra
   private DataBlockEncoding dataBlockEncodingAlgo;
   private boolean encodeInCacheOnly;
   private Compression.Algorithm compressAlgo;
-  private StoreFile.BloomType bloomType;
+  private BloomType bloomType;
 
   // Writer options
   private int numWriterThreads = DEFAULT_NUM_THREADS;
-  private long minColsPerKey, maxColsPerKey;
+  private int minColsPerKey, maxColsPerKey;
   private int minColDataSize, maxColDataSize;
   private boolean isMultiPut;
 
@@ -260,7 +262,7 @@ public class LoadTestTool extends Abstra
 
       int colIndex = 0;
       minColsPerKey = 1;
-      maxColsPerKey = 2 * Long.parseLong(writeOpts[colIndex++]);
+      maxColsPerKey = 2 * Integer.parseInt(writeOpts[colIndex++]);
       int avgColDataSize =
           parseInt(writeOpts[colIndex++], 1, Integer.MAX_VALUE);
       minColDataSize = avgColDataSize / 2;
@@ -317,7 +319,7 @@ public class LoadTestTool extends Abstra
 
     String bloomStr = cmd.getOptionValue(OPT_BLOOM);
     bloomType = bloomStr == null ? null :
-        StoreFile.BloomType.valueOf(bloomStr);
+        BloomType.valueOf(bloomStr);
   }
 
   public void initTestTable() throws IOException {
@@ -342,16 +344,16 @@ public class LoadTestTool extends Abstra
       initTestTable();
     }
 
+    LoadTestDataGenerator dataGen = new MultiThreadedAction.DefaultDataGenerator(
+        minColDataSize, maxColDataSize, minColsPerKey, maxColsPerKey, COLUMN_FAMILY);
+
     if (isWrite) {
-      writerThreads = new MultiThreadedWriter(conf, tableName, COLUMN_FAMILY);
+      writerThreads = new MultiThreadedWriter(dataGen, conf, tableName);
       writerThreads.setMultiPut(isMultiPut);
-      writerThreads.setColumnsPerKey(minColsPerKey, maxColsPerKey);
-      writerThreads.setDataSize(minColDataSize, maxColDataSize);
     }
 
     if (isRead) {
-      readerThreads = new MultiThreadedReader(conf, tableName, COLUMN_FAMILY,
-          verifyPercent);
+      readerThreads = new MultiThreadedReader(dataGen, conf, tableName, verifyPercent);
       readerThreads.setMaxErrors(maxReadErrors);
       readerThreads.setKeyWindow(keyWindow);
     }

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Wed Feb 13 20:58:23 2013
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentSk
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -36,7 +37,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
@@ -49,10 +49,19 @@ public class MockRegionServerServices im
   private final ConcurrentSkipListMap<byte[], Boolean> rit = 
     new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
   private HFileSystem hfs = null;
+  private ZooKeeperWatcher zkw = null;
+
+  public MockRegionServerServices(ZooKeeperWatcher zkw){
+    this.zkw = zkw;
+  }
+
+  public MockRegionServerServices(){
+    this(null);
+  }
 
   @Override
-  public boolean removeFromOnlineRegions(String encodedRegionName, ServerName destination) {
-    return this.regions.remove(encodedRegionName) != null;
+  public boolean removeFromOnlineRegions(HRegion r, ServerName destination) {
+    return this.regions.remove(r.getRegionInfo().getEncodedName()) != null;
   }
 
   @Override
@@ -81,11 +90,6 @@ public class MockRegionServerServices im
   }
 
   @Override
-  public HLog getWAL() {
-    return null;
-  }
-
-  @Override
   public RpcServer getRpcServer() {
     return null;
   }
@@ -112,7 +116,7 @@ public class MockRegionServerServices im
 
   @Override
   public ZooKeeperWatcher getZooKeeper() {
-    return null;
+    return zkw;
   }
   
   public RegionServerAccounting getRegionServerAccounting() {
@@ -162,4 +166,10 @@ public class MockRegionServerServices im
   public Leases getLeases() {
     return null;
   }
+
+  @Override
+  public HLog getWAL(HRegionInfo regionInfo) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }