You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ss...@apache.org on 2015/06/04 16:04:51 UTC

[1/2] hbase git commit: HBASE-13764 Backport HBASE-7782 (HBaseTestingUtility.truncateTable() not acting like CLI) to branch-1.x

Repository: hbase
Updated Branches:
  refs/heads/branch-1.0 0214c125e -> 9046b37ee


HBASE-13764 Backport HBASE-7782 (HBaseTestingUtility.truncateTable() not acting like CLI) to branch-1.x

Signed-off-by: Srikanth Srungarapu <ss...@cloudera.com>


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

Branch: refs/heads/branch-1.0
Commit: e1d177cb8050fa2f3fce9319b4f6b8b04cd438ec
Parents: 0214c12
Author: Ashish Singhi <as...@huawei.com>
Authored: Wed May 27 15:30:00 2015 +0530
Committer: Srikanth Srungarapu <ss...@cloudera.com>
Committed: Tue Jun 2 23:24:40 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/HBaseTestingUtility.java       | 64 ++++++++++++++++++--
 .../hadoop/hbase/mapreduce/TestRowCounter.java  |  2 +-
 ...estReplicationChangingPeerRegionservers.java |  4 +-
 .../replication/TestReplicationSmallTests.java  |  4 +-
 .../regionserver/TestReplicationSink.java       |  4 +-
 5 files changed, 66 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d177cb/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index a95faf1..1e9f782 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1804,22 +1804,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   // ==========================================================================
 
   /**
-   * Provide an existing table name to truncate
+   * Provide an existing table name to truncate.
+   * Scans the table and issues a delete for each row read.
    * @param tableName existing table
    * @return HTable to that new table
    * @throws IOException
    */
-  public HTable truncateTable(byte[] tableName) throws IOException {
-    return truncateTable(TableName.valueOf(tableName));
+  public HTable deleteTableData(byte[] tableName) throws IOException {
+    return deleteTableData(TableName.valueOf(tableName));
   }
 
   /**
-   * Provide an existing table name to truncate
+   * Provide an existing table name to truncate.
+   * Scans the table and issues a delete for each row read.
    * @param tableName existing table
    * @return HTable to that new table
    * @throws IOException
    */
-  public HTable truncateTable(TableName tableName) throws IOException {
+  public HTable deleteTableData(TableName tableName) throws IOException {
     HTable table = new HTable(getConfiguration(), tableName);
     Scan scan = new Scan();
     ResultScanner resScan = table.getScanner(scan);
@@ -1833,6 +1835,58 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
+   * Truncate a table using the admin command.
+   * Effectively disables, deletes, and recreates the table.
+   * @param tableName table which must exist.
+   * @param preserveRegions keep the existing split points
+   * @return HTable for the new table
+   */
+  public HTable truncateTable(final TableName tableName, final boolean preserveRegions)
+      throws IOException {
+    Admin admin = getHBaseAdmin();
+    admin.truncateTable(tableName, preserveRegions);
+    return new HTable(getConfiguration(), tableName);
+  }
+
+  /**
+   * Truncate a table using the admin command.
+   * Effectively disables, deletes, and recreates the table.
+   * For previous behavior of issuing row deletes, see
+   * deleteTableData.
+   * Expressly does not preserve regions of existing table.
+   * @param tableName table which must exist.
+   * @return HTable for the new table
+   */
+  public HTable truncateTable(final TableName tableName) throws IOException {
+    return truncateTable(tableName, false);
+  }
+
+  /**
+   * Truncate a table using the admin command.
+   * Effectively disables, deletes, and recreates the table.
+   * @param tableName table which must exist.
+   * @param preserveRegions keep the existing split points
+   * @return HTable for the new table
+   */
+  public HTable truncateTable(final byte[] tableName, final boolean preserveRegions)
+      throws IOException {
+    return truncateTable(TableName.valueOf(tableName), preserveRegions);
+  }
+
+  /**
+   * Truncate a table using the admin command.
+   * Effectively disables, deletes, and recreates the table.
+   * For previous behavior of issuing row deletes, see
+   * deleteTableData.
+   * Expressly does not preserve regions of existing table.
+   * @param tableName table which must exist.
+   * @return HTable for the new table
+   */
+  public HTable truncateTable(final byte[] tableName) throws IOException {
+    return truncateTable(tableName, false);
+  }
+
+  /**
    * Load table with rows from 'aaa' to 'zzz'.
    * @param t Table
    * @param f Family

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d177cb/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
index 53813d9..84c38ce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
@@ -161,7 +161,7 @@ public class TestRowCounter {
     long ts;
 
     // clean up content of TABLE_NAME
-    HTable table = TEST_UTIL.truncateTable(TableName.valueOf(TABLE_NAME));
+    HTable table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
     ts = System.currentTimeMillis();
     put1.add(family, col1, ts, Bytes.toBytes("val1"));
     table.put(put1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d177cb/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
index d858321..5397c6c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java
@@ -58,9 +58,9 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
                           utility1.getHBaseCluster().getRegionServerThreads()) {
       utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
-    utility1.truncateTable(tableName);
+    utility1.deleteTableData(tableName);
     // truncating the table will send one Delete per row to the slave cluster
-    // in an async fashion, which is why we cannot just call truncateTable on
+    // in an async fashion, which is why we cannot just call deleteTableData on
     // utility2 since late writes could make it to the slave in some way.
     // Instead, we truncate the first table and wait for all the Deletes to
     // make it to the slave.

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d177cb/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index d8d735f..06f78e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -79,9 +79,9 @@ public class TestReplicationSmallTests extends TestReplicationBase {
         utility1.getHBaseCluster().getRegionServerThreads()) {
       utility1.getHBaseAdmin().rollWALWriter(r.getRegionServer().getServerName());
     }
-    utility1.truncateTable(tableName);
+    utility1.deleteTableData(tableName);
     // truncating the table will send one Delete per row to the slave cluster
-    // in an async fashion, which is why we cannot just call truncateTable on
+    // in an async fashion, which is why we cannot just call deleteTableData on
     // utility2 since late writes could make it to the slave in some way.
     // Instead, we truncate the first table and wait for all the Deletes to
     // make it to the slave.

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1d177cb/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 7efb4e3..db58ccb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -118,8 +118,8 @@ public class TestReplicationSink {
    */
   @Before
   public void setUp() throws Exception {
-    table1 = TEST_UTIL.truncateTable(TABLE_NAME1);
-    table2 = TEST_UTIL.truncateTable(TABLE_NAME2);
+    table1 = TEST_UTIL.deleteTableData(TABLE_NAME1);
+    table2 = TEST_UTIL.deleteTableData(TABLE_NAME2);
   }
 
   /**


[2/2] hbase git commit: HBASE-13658 Improve the test run time for TestAccessController class

Posted by ss...@apache.org.
HBASE-13658 Improve the test run time for TestAccessController class

Signed-off-by: Srikanth Srungarapu <ss...@cloudera.com>


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

Branch: refs/heads/branch-1.0
Commit: 9046b37ee6dfc2a276be2ff0eaebc445ee7cdc7c
Parents: e1d177c
Author: Ashish Singhi <as...@huawei.com>
Authored: Wed Jun 3 15:39:15 2015 +0530
Committer: Srikanth Srungarapu <ss...@cloudera.com>
Committed: Thu Jun 4 07:02:52 2015 -0700

----------------------------------------------------------------------
 .../security/access/TestAccessController.java   | 1362 +++++++++---------
 1 file changed, 696 insertions(+), 666 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9046b37e/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 4986d97..ce4aa68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -103,14 +103,10 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.TestTableName;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -134,7 +130,7 @@ public class TestAccessController extends SecureTestUtil {
     Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
   }
 
-  @Rule public TestTableName TEST_TABLE = new TestTableName();
+  private static TableName TEST_TABLE = TableName.valueOf("testtable1");
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration conf;
 
@@ -172,7 +168,7 @@ public class TestAccessController extends SecureTestUtil {
   private static MasterCoprocessorEnvironment CP_ENV;
   private static AccessController ACCESS_CONTROLLER;
   private static RegionServerCoprocessorEnvironment RSCP_ENV;
-  private RegionCoprocessorEnvironment RCP_ENV;
+  private static RegionCoprocessorEnvironment RCP_ENV;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -220,26 +216,27 @@ public class TestAccessController extends SecureTestUtil {
     USER_ADMIN_CF = User.createUserForTesting(conf, "col_family_admin", new String[0]);
 
     systemUserConnection = TEST_UTIL.getConnection();
+    setUpTableAndUserPermissions();
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
+    cleanUp();
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Before
-  public void setUp() throws Exception {
+  private static void setUpTableAndUserPermissions() throws Exception {
     // Create the test table (owner added to the _acl_ table)
     Admin admin = TEST_UTIL.getHBaseAdmin();
-    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
     hcd.setMaxVersions(100);
     htd.addFamily(hcd);
     htd.setOwner(USER_OWNER);
     admin.createTable(htd, new byte[][] { Bytes.toBytes("s") });
-    TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE.getTableName());
+    TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
 
-    HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0);
+    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);
@@ -253,26 +250,26 @@ public class TestAccessController extends SecureTestUtil {
       Permission.Action.WRITE);
 
     grantOnTable(TEST_UTIL, USER_RW.getShortName(),
-      TEST_TABLE.getTableName(), TEST_FAMILY, null,
+      TEST_TABLE, TEST_FAMILY, null,
       Permission.Action.READ,
       Permission.Action.WRITE);
 
     // USER_CREATE is USER_RW plus CREATE permissions
     grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
-      TEST_TABLE.getTableName(), null, null,
+      TEST_TABLE, null, null,
       Permission.Action.CREATE,
       Permission.Action.READ,
       Permission.Action.WRITE);
 
     grantOnTable(TEST_UTIL, USER_RO.getShortName(),
-      TEST_TABLE.getTableName(), TEST_FAMILY, null,
+      TEST_TABLE, TEST_FAMILY, null,
       Permission.Action.READ);
 
     grantOnTable(TEST_UTIL, USER_ADMIN_CF.getShortName(),
-      TEST_TABLE.getTableName(), TEST_FAMILY,
+      TEST_TABLE, TEST_FAMILY,
       null, Permission.Action.ADMIN, Permission.Action.CREATE);
 
-    assertEquals(5, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size());
+    assertEquals(5, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
     try {
       assertEquals(5, AccessControlClient.getUserPermissions(systemUserConnection,
           TEST_TABLE.toString()).size());
@@ -281,21 +278,18 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
-  @After
-  public void tearDown() throws Exception {
+  private static void cleanUp() throws Exception {
     // Clean the _acl_ table
     try {
-      TEST_UTIL.deleteTable(TEST_TABLE.getTableName());
+      TEST_UTIL.deleteTable(TEST_TABLE);
     } catch (TableNotFoundException ex) {
       // Test deleted the table, no problem
-      LOG.info("Test deleted table " + TEST_TABLE.getTableName());
+      LOG.info("Test deleted table " + TEST_TABLE);
     }
     // Verify all table/namespace permissions are erased
-    assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size());
-    assertEquals(
-      0,
-      AccessControlLists.getNamespacePermissions(conf,
-        TEST_TABLE.getTableName().getNamespaceAsString()).size());
+    assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
+    assertEquals(0,
+      AccessControlLists.getNamespacePermissions(conf, TEST_TABLE.getNamespaceAsString()).size());
   }
 
   @Test
@@ -322,11 +316,11 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction modifyTable = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        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.getTableName(), htd);
+          TEST_TABLE, htd);
         return null;
       }
     };
@@ -341,7 +335,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER
-            .preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE.getTableName());
+            .preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
         return null;
       }
     };
@@ -357,7 +351,7 @@ public class TestAccessController extends SecureTestUtil {
       public Object run() throws Exception {
         ACCESS_CONTROLLER
             .preTruncateTable(ObserverContext.createAndPrepare(CP_ENV, null),
-              TEST_TABLE.getTableName());
+              TEST_TABLE);
         return null;
       }
     };
@@ -372,7 +366,7 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction action = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE.getTableName(),
+        ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE,
           hcd);
         return null;
       }
@@ -390,7 +384,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null),
-          TEST_TABLE.getTableName(), hcd);
+          TEST_TABLE, hcd);
         return null;
       }
     };
@@ -405,7 +399,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null),
-          TEST_TABLE.getTableName(), TEST_FAMILY);
+          TEST_TABLE, TEST_FAMILY);
         return null;
       }
     };
@@ -420,7 +414,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
-          TEST_TABLE.getTableName());
+          TEST_TABLE);
         return null;
       }
     };
@@ -447,7 +441,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER
-            .preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE.getTableName());
+            .preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
         return null;
       }
     };
@@ -459,7 +453,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testMove() throws Exception {
     List<HRegionLocation> regions;
-    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE.getTableName())) {
+    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
       regions = locator.getAllRegionLocations();
     }
     HRegionLocation location = regions.get(0);
@@ -481,7 +475,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testAssign() throws Exception {
     List<HRegionLocation> regions;
-    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE.getTableName())) {
+    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
       regions = locator.getAllRegionLocations();
     }
     HRegionLocation location = regions.get(0);
@@ -501,7 +495,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testUnassign() throws Exception {
     List<HRegionLocation> regions;
-    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE.getTableName())) {
+    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
       regions = locator.getAllRegionLocations();
     }
     HRegionLocation location = regions.get(0);
@@ -521,7 +515,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testRegionOffline() throws Exception {
     List<HRegionLocation> regions;
-    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE.getTableName())) {
+    try (RegionLocator locator = systemUserConnection.getRegionLocator(TEST_TABLE)) {
       regions = locator.getAllRegionLocations();
     }
     HRegionLocation location = regions.get(0);
@@ -631,21 +625,25 @@ public class TestAccessController extends SecureTestUtil {
 
   @Test
   public void testMergeRegions() throws Exception {
+    final TableName tname = TableName.valueOf("testMergeRegions");
+    createTestTable(tname);
+    try {
+      final List<HRegion> regions = TEST_UTIL.getHBaseCluster().findRegionsForTable(tname);
+      assertTrue("not enough regions: " + regions.size(), regions.size() >= 2);
+      AccessTestAction action = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null),
+            regions.get(0), regions.get(1));
+          return null;
+        }
+      };
 
-    final List<HRegion> regions = TEST_UTIL.getHBaseCluster().findRegionsForTable(TEST_TABLE.getTableName());
-
-    AccessTestAction action = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        ACCESS_CONTROLLER.preMerge(
-            ObserverContext.createAndPrepare(RSCP_ENV, null),
-            regions.get(0),regions.get(1));
-        return null;
-      }
-    };
-
-    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER);
-    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE);
+      verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER);
+      verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE);
+    } finally {
+      TEST_UTIL.deleteTable(tname);
+    }
   }
 
   @Test
@@ -696,7 +694,7 @@ public class TestAccessController extends SecureTestUtil {
         Get g = new Get(TEST_ROW);
         g.addFamily(TEST_FAMILY);
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName())) {
+            Table t = conn.getTable(TEST_TABLE)) {
           t.get(g);
         }
         return null;
@@ -711,7 +709,7 @@ public class TestAccessController extends SecureTestUtil {
         Scan s = new Scan();
         s.addFamily(TEST_FAMILY);
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table table = conn.getTable(TEST_TABLE.getTableName())) {
+            Table table = conn.getTable(TEST_TABLE)) {
           ResultScanner scanner = table.getScanner(s);
           try {
             for (Result r = scanner.next(); r != null; r = scanner.next()) {
@@ -738,7 +736,7 @@ public class TestAccessController extends SecureTestUtil {
         Put p = new Put(TEST_ROW);
         p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName())) {
+            Table t = conn.getTable(TEST_TABLE)) {
           t.put(p);
         }
         return null;
@@ -753,7 +751,7 @@ public class TestAccessController extends SecureTestUtil {
         Delete d = new Delete(TEST_ROW);
         d.deleteFamily(TEST_FAMILY);
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName())) {
+            Table t = conn.getTable(TEST_TABLE)) {
           t.delete(d);
         }
         return null;
@@ -768,7 +766,7 @@ public class TestAccessController extends SecureTestUtil {
         Increment inc = new Increment(TEST_ROW);
         inc.addColumn(TEST_FAMILY, TEST_QUALIFIER, 1);
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           t.increment(inc);
         }
         return null;
@@ -786,7 +784,7 @@ public class TestAccessController extends SecureTestUtil {
         Delete d = new Delete(TEST_ROW);
         d.deleteFamily(TEST_FAMILY);
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           t.checkAndDelete(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
               Bytes.toBytes("test_value"), d);
         }
@@ -802,7 +800,7 @@ public class TestAccessController extends SecureTestUtil {
         Put p = new Put(TEST_ROW);
         p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           t.checkAndPut(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
               Bytes.toBytes("test_value"), p);
         }
@@ -814,37 +812,39 @@ public class TestAccessController extends SecureTestUtil {
 
   @Test
   public void testBulkLoad() throws Exception {
-    FileSystem fs = TEST_UTIL.getTestFileSystem();
-    final Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoad");
-    fs.mkdirs(dir);
-    //need to make it globally writable
-    //so users creating HFiles have write permissions
-    fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
-
-    AccessTestAction bulkLoadAction = new AccessTestAction() {
-      @Override
-      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}}};
+    try {
+      FileSystem fs = TEST_UTIL.getTestFileSystem();
+      final Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoad");
+      fs.mkdirs(dir);
+      // need to make it globally writable
+      // so users creating HFiles have write permissions
+      fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
+
+      AccessTestAction bulkLoadAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          int numRows = 3;
 
-        Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
-        new BulkLoadHelper(bulkLoadBasePath)
-            .bulkLoadHFile(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_QUALIFIER, hfileRanges, numRows);
+          // Making the assumption that the test table won't split between the range
+          byte[][][] hfileRanges = { { { (byte) 0 }, { (byte) 9 } } };
 
-        return null;
-      }
-    };
+          Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
+          new BulkLoadHelper(bulkLoadBasePath).bulkLoadHFile(TEST_TABLE, TEST_FAMILY,
+            TEST_QUALIFIER, hfileRanges, numRows);
 
-    // User performing bulk loads must have privilege to read table metadata
-    // (ADMIN or CREATE)
-    verifyAllowed(bulkLoadAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE);
-    verifyDenied(bulkLoadAction, USER_RW, USER_NONE, USER_RO);
+          return null;
+        }
+      };
 
-    // Reinit after the bulk upload
-    TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE.getTableName());
-    TEST_UTIL.getHBaseAdmin().enableTable(TEST_TABLE.getTableName());
+      // User performing bulk loads must have privilege to read table metadata
+      // (ADMIN or CREATE)
+      verifyAllowed(bulkLoadAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE);
+      verifyDenied(bulkLoadAction, USER_RW, USER_NONE, USER_RO);
+    } finally {
+      // Reinit after the bulk upload
+      TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE);
+      TEST_UTIL.getHBaseAdmin().enableTable(TEST_TABLE);
+    }
   }
 
   public class BulkLoadHelper {
@@ -935,7 +935,7 @@ public class TestAccessController extends SecureTestUtil {
         Append append = new Append(row);
         append.add(TEST_FAMILY, qualifier, Bytes.toBytes(2));
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName())) {
+            Table t = conn.getTable(TEST_TABLE)) {
           t.put(put);
           t.append(append);
         }
@@ -954,11 +954,11 @@ public class TestAccessController extends SecureTestUtil {
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
             Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE.getTableName(),
-              TEST_FAMILY, null, Action.READ);
+          ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+            Action.READ);
         }
         return null;
       }
@@ -969,11 +969,11 @@ public class TestAccessController extends SecureTestUtil {
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
             Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE.getTableName(),
-              TEST_FAMILY, null, Action.READ);
+          ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+            Action.READ);
         }
         return null;
       }
@@ -984,10 +984,10 @@ public class TestAccessController extends SecureTestUtil {
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
             Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)){
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName());
+          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol, TEST_TABLE.getTableName());
+              AccessControlService.newBlockingStub(service);
+          ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
         }
         return null;
       }
@@ -1009,16 +1009,21 @@ public class TestAccessController extends SecureTestUtil {
 
     verifyAllowed(grantAction, SUPERUSER, USER_ADMIN, USER_OWNER);
     verifyDenied(grantAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
+    try {
+      verifyAllowed(revokeAction, SUPERUSER, USER_ADMIN, USER_OWNER);
+      verifyDenied(revokeAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
 
-    verifyAllowed(revokeAction, SUPERUSER, USER_ADMIN, USER_OWNER);
-    verifyDenied(revokeAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
-
-    verifyAllowed(getTablePermissionsAction, SUPERUSER, USER_ADMIN, USER_OWNER);
-    verifyDenied(getTablePermissionsAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
+      verifyAllowed(getTablePermissionsAction, SUPERUSER, USER_ADMIN, USER_OWNER);
+      verifyDenied(getTablePermissionsAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
 
-    verifyAllowed(getGlobalPermissionsAction, SUPERUSER, USER_ADMIN);
-    verifyDenied(getGlobalPermissionsAction, USER_CREATE,
-        USER_OWNER, USER_RW, USER_RO, USER_NONE);
+      verifyAllowed(getGlobalPermissionsAction, SUPERUSER, USER_ADMIN);
+      verifyDenied(getGlobalPermissionsAction, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
+        USER_NONE);
+    } finally {
+      // Cleanup, Grant the revoked permission back to the user
+      grantOnTable(TEST_UTIL, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+        Permission.Action.READ);
+    }
   }
 
   @Test
@@ -1040,238 +1045,232 @@ public class TestAccessController extends SecureTestUtil {
     htd.addFamily(new HColumnDescriptor(family2));
     admin.createTable(htd);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-
-    // create temp users
-    User tblUser = User
-        .createUserForTesting(TEST_UTIL.getConfiguration(), "tbluser", new String[0]);
-    User gblUser = User
-        .createUserForTesting(TEST_UTIL.getConfiguration(), "gbluser", new String[0]);
-
-    // prepare actions:
-    AccessTestAction putActionAll = new AccessTestAction() {
-      @Override
-      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"));
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName);) {
-          t.put(p);
+    try {
+      // create temp users
+      User tblUser =
+          User.createUserForTesting(TEST_UTIL.getConfiguration(), "tbluser", new String[0]);
+      User gblUser =
+          User.createUserForTesting(TEST_UTIL.getConfiguration(), "gbluser", new String[0]);
+
+      // prepare actions:
+      AccessTestAction putActionAll = new AccessTestAction() {
+        @Override
+        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"));
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName);) {
+            t.put(p);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction putAction1 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Put p = new Put(Bytes.toBytes("a"));
-        p.add(family1, qualifier, Bytes.toBytes("v1"));
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.put(p);
+      AccessTestAction putAction1 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Put p = new Put(Bytes.toBytes("a"));
+          p.add(family1, qualifier, Bytes.toBytes("v1"));
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.put(p);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction putAction2 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Put p = new Put(Bytes.toBytes("a"));
-        p.add(family2, qualifier, Bytes.toBytes("v2"));
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName);) {
-          t.put(p);
+      AccessTestAction putAction2 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Put p = new Put(Bytes.toBytes("a"));
+          p.add(family2, qualifier, Bytes.toBytes("v2"));
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName);) {
+            t.put(p);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction getActionAll = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Get g = new Get(TEST_ROW);
-        g.addFamily(family1);
-        g.addFamily(family2);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName);) {
-          t.get(g);
+      AccessTestAction getActionAll = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Get g = new Get(TEST_ROW);
+          g.addFamily(family1);
+          g.addFamily(family2);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName);) {
+            t.get(g);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction getAction1 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Get g = new Get(TEST_ROW);
-        g.addFamily(family1);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.get(g);
+      AccessTestAction getAction1 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Get g = new Get(TEST_ROW);
+          g.addFamily(family1);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.get(g);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction getAction2 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Get g = new Get(TEST_ROW);
-        g.addFamily(family2);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.get(g);
+      AccessTestAction getAction2 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Get g = new Get(TEST_ROW);
+          g.addFamily(family2);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.get(g);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction deleteActionAll = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Delete d = new Delete(TEST_ROW);
-        d.deleteFamily(family1);
-        d.deleteFamily(family2);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.delete(d);
+      AccessTestAction deleteActionAll = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Delete d = new Delete(TEST_ROW);
+          d.deleteFamily(family1);
+          d.deleteFamily(family2);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.delete(d);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction deleteAction1 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Delete d = new Delete(TEST_ROW);
-        d.deleteFamily(family1);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.delete(d);
+      AccessTestAction deleteAction1 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Delete d = new Delete(TEST_ROW);
+          d.deleteFamily(family1);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.delete(d);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction deleteAction2 = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Delete d = new Delete(TEST_ROW);
-        d.deleteFamily(family2);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.delete(d);
+      AccessTestAction deleteAction2 = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Delete d = new Delete(TEST_ROW);
+          d.deleteFamily(family2);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.delete(d);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    // initial check:
-    verifyDenied(tblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      // initial check:
+      verifyDenied(tblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    verifyDenied(gblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(gblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(gblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(gblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // grant table read permission
-    grantGlobal(TEST_UTIL, gblUser.getShortName(),
-      Permission.Action.READ);
-    grantOnTable(TEST_UTIL, tblUser.getShortName(),
-      tableName, null, null,
-      Permission.Action.READ);
+      // grant table read permission
+      grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.READ);
+      grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null,
+        Permission.Action.READ);
 
-    // check
-    verifyAllowed(tblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      // check
+      verifyAllowed(tblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(gblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(gblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // grant table write permission while revoking read permissions
-    grantGlobal(TEST_UTIL, gblUser.getShortName(),
-      Permission.Action.WRITE);
-    grantOnTable(TEST_UTIL, tblUser.getShortName(),
-      tableName, null, null,
-      Permission.Action.WRITE);
+      // grant table write permission while revoking read permissions
+      grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.WRITE);
+      grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null,
+        Permission.Action.WRITE);
 
-    verifyDenied(tblUser, getActionAll, getAction1, getAction2);
-    verifyAllowed(tblUser, putActionAll, putAction1, putAction2);
-    verifyAllowed(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(tblUser, getActionAll, getAction1, getAction2);
+      verifyAllowed(tblUser, putActionAll, putAction1, putAction2);
+      verifyAllowed(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    verifyDenied(gblUser, getActionAll, getAction1, getAction2);
-    verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
-    verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(gblUser, getActionAll, getAction1, getAction2);
+      verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
+      verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // revoke table permissions
-    revokeGlobal(TEST_UTIL, gblUser.getShortName());
-    revokeFromTable(TEST_UTIL, tblUser.getShortName(),
-      tableName, null, null);
+      // revoke table permissions
+      revokeGlobal(TEST_UTIL, gblUser.getShortName());
+      revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null);
 
-    verifyDenied(tblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(tblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    verifyDenied(gblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(gblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(gblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(gblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // grant column family read permission
-    grantGlobal(TEST_UTIL, gblUser.getShortName(),
-      Permission.Action.READ);
-    grantOnTable(TEST_UTIL, tblUser.getShortName(),
-      tableName, family1, null, Permission.Action.READ);
+      // grant column family read permission
+      grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.READ);
+      grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, family1, null,
+        Permission.Action.READ);
 
-    // Access should be denied for family2
-    verifyAllowed(tblUser, getActionAll, getAction1);
-    verifyDenied(tblUser, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      // Access should be denied for family2
+      verifyAllowed(tblUser, getActionAll, getAction1);
+      verifyDenied(tblUser, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(gblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(gblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // grant column family write permission
-    grantGlobal(TEST_UTIL, gblUser.getShortName(),
-      Permission.Action.WRITE);
-    grantOnTable(TEST_UTIL, tblUser.getShortName(),
-      tableName, family2, null, Permission.Action.WRITE);
+      // grant column family write permission
+      grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.WRITE);
+      grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null,
+        Permission.Action.WRITE);
 
-    // READ from family1, WRITE to family2 are allowed
-    verifyAllowed(tblUser, getActionAll, getAction1);
-    verifyAllowed(tblUser, putAction2, deleteAction2);
-    verifyDenied(tblUser, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1);
+      // READ from family1, WRITE to family2 are allowed
+      verifyAllowed(tblUser, getActionAll, getAction1);
+      verifyAllowed(tblUser, putAction2, deleteAction2);
+      verifyDenied(tblUser, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1);
 
-    verifyDenied(gblUser, getActionAll, getAction1, getAction2);
-    verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
-    verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      verifyDenied(gblUser, getActionAll, getAction1, getAction2);
+      verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
+      verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // revoke column family permission
-    revokeGlobal(TEST_UTIL, gblUser.getShortName());
-    revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null);
+      // revoke column family permission
+      revokeGlobal(TEST_UTIL, gblUser.getShortName());
+      revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null);
 
-    // Revoke on family2 should not have impact on family1 permissions
-    verifyAllowed(tblUser, getActionAll, getAction1);
-    verifyDenied(tblUser, getAction2);
-    verifyDenied(tblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
+      // Revoke on family2 should not have impact on family1 permissions
+      verifyAllowed(tblUser, getActionAll, getAction1);
+      verifyDenied(tblUser, getAction2);
+      verifyDenied(tblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // Should not have access as global permissions are completely revoked
-    verifyDenied(gblUser, getActionAll, getAction1, getAction2);
-    verifyDenied(gblUser, putActionAll, putAction1, putAction2);
-    verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
+      // Should not have access as global permissions are completely revoked
+      verifyDenied(gblUser, getActionAll, getAction1, getAction2);
+      verifyDenied(gblUser, putActionAll, putAction1, putAction2);
+      verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
 
-    // delete table
-    admin.disableTable(tableName);
-    admin.deleteTable(tableName);
+    } finally {
+      // delete table
+      TEST_UTIL.deleteTable(tableName);
+    }
   }
 
   private boolean hasFoundUserPermission(UserPermission userPermission, List<UserPermission> perms) {
@@ -1298,93 +1297,91 @@ public class TestAccessController extends SecureTestUtil {
     admin.createTable(htd);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
 
-    // create temp users
-    User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
+    try {
+      // create temp users
+      User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
 
-    AccessTestAction getQualifierAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Get g = new Get(TEST_ROW);
-        g.addColumn(family1, qualifier);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.get(g);
+      AccessTestAction getQualifierAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Get g = new Get(TEST_ROW);
+          g.addColumn(family1, qualifier);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.get(g);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction putQualifierAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Put p = new Put(TEST_ROW);
-        p.add(family1, qualifier, Bytes.toBytes("v1"));
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.put(p);
+      AccessTestAction putQualifierAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Put p = new Put(TEST_ROW);
+          p.add(family1, qualifier, Bytes.toBytes("v1"));
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.put(p);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    AccessTestAction deleteQualifierAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        Delete d = new Delete(TEST_ROW);
-        d.deleteColumn(family1, qualifier);
-        // d.deleteFamily(family1);
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(tableName)) {
-          t.delete(d);
+      AccessTestAction deleteQualifierAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          Delete d = new Delete(TEST_ROW);
+          d.deleteColumn(family1, qualifier);
+          // d.deleteFamily(family1);
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(tableName)) {
+            t.delete(d);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, null);
+      revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, null);
 
-    verifyDenied(user, getQualifierAction);
-    verifyDenied(user, putQualifierAction);
-    verifyDenied(user, deleteQualifierAction);
+      verifyDenied(user, getQualifierAction);
+      verifyDenied(user, putQualifierAction);
+      verifyDenied(user, deleteQualifierAction);
 
-    grantOnTable(TEST_UTIL, user.getShortName(),
-      tableName, family1, qualifier,
-      Permission.Action.READ);
+      grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.READ);
 
-    verifyAllowed(user, getQualifierAction);
-    verifyDenied(user, putQualifierAction);
-    verifyDenied(user, deleteQualifierAction);
+      verifyAllowed(user, getQualifierAction);
+      verifyDenied(user, putQualifierAction);
+      verifyDenied(user, deleteQualifierAction);
 
-    // only grant write permission
-    // TODO: comment this portion after HBASE-3583
-    grantOnTable(TEST_UTIL, user.getShortName(),
-      tableName, family1, qualifier,
-      Permission.Action.WRITE);
+      // only grant write permission
+      // TODO: comment this portion after HBASE-3583
+      grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.WRITE);
 
-    verifyDenied(user, getQualifierAction);
-    verifyAllowed(user, putQualifierAction);
-    verifyAllowed(user, deleteQualifierAction);
+      verifyDenied(user, getQualifierAction);
+      verifyAllowed(user, putQualifierAction);
+      verifyAllowed(user, deleteQualifierAction);
 
-    // grant both read and write permission
-    grantOnTable(TEST_UTIL, user.getShortName(),
-      tableName, family1, qualifier,
-      Permission.Action.READ, Permission.Action.WRITE);
+      // grant both read and write permission
+      grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.READ, Permission.Action.WRITE);
 
-    verifyAllowed(user, getQualifierAction);
-    verifyAllowed(user, putQualifierAction);
-    verifyAllowed(user, deleteQualifierAction);
+      verifyAllowed(user, getQualifierAction);
+      verifyAllowed(user, putQualifierAction);
+      verifyAllowed(user, deleteQualifierAction);
 
-    // revoke family level permission won't impact column level
-    revokeFromTable(TEST_UTIL, user.getShortName(),
-      tableName, family1, qualifier);
+      // revoke family level permission won't impact column level
+      revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier);
 
-    verifyDenied(user, getQualifierAction);
-    verifyDenied(user, putQualifierAction);
-    verifyDenied(user, deleteQualifierAction);
+      verifyDenied(user, getQualifierAction);
+      verifyDenied(user, putQualifierAction);
+      verifyDenied(user, deleteQualifierAction);
 
-    // delete table
-    admin.disableTable(tableName);
-    admin.deleteTable(tableName);
+    } finally {
+      // delete table
+      TEST_UTIL.deleteTable(tableName);
+    }
   }
 
   @Test
@@ -1407,116 +1404,117 @@ public class TestAccessController extends SecureTestUtil {
     htd.setOwner(USER_OWNER);
     admin.createTable(htd);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-
-    List<UserPermission> perms;
-    Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
     try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
-    } finally {
-      acl.close();
-    }
+      List<UserPermission> perms;
+      Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
+      try {
+        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
+        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());
-    assertTrue("Owner should have all permissions on table",
+      UserPermission ownerperm =
+          new UserPermission(Bytes.toBytes(USER_OWNER.getName()), tableName, null, Action.values());
+      assertTrue("Owner should have all permissions on table",
         hasFoundUserPermission(ownerperm, perms));
 
-    User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
-    byte[] userName = Bytes.toBytes(user.getShortName());
+      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(),
+      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
-    grantOnTable(TEST_UTIL, user.getShortName(),
-        tableName, family1, qualifier, Permission.Action.READ);
+      // grant read permission
+      grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.READ);
 
-    acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
-    } finally {
-      acl.close();
-    }
+      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
+      try {
+        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
+        AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+      } finally {
+        acl.close();
+      }
 
-    UserPermission upToVerify = new UserPermission(
-      userName, tableName, family1, qualifier, Permission.Action.READ);
-    assertTrue("User should be granted permission: " + upToVerify.toString(),
+      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(
-      userName, tableName, family1, qualifier, Permission.Action.WRITE);
-    assertFalse("User should not be granted permission: " + upToVerify.toString(),
+      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
-    grantOnTable(TEST_UTIL, user.getShortName(),
-      tableName, family1, qualifier,
-      Permission.Action.WRITE, Permission.Action.READ);
+      // grant read+write
+      grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+        Permission.Action.WRITE, Permission.Action.READ);
 
-    acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
-    } finally {
-      acl.close();
-    }
+      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
+      try {
+        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
+        AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+      } finally {
+        acl.close();
+      }
 
-    upToVerify = new UserPermission(userName, tableName, family1,
-      qualifier, Permission.Action.WRITE, Permission.Action.READ);
-    assertTrue("User should be granted permission: " + upToVerify.toString(),
+      upToVerify =
+          new UserPermission(userName, tableName, family1, qualifier, Permission.Action.WRITE,
+              Permission.Action.READ);
+      assertTrue("User should be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
-    // revoke
-    revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
+      // revoke
+      revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
         Permission.Action.WRITE, Permission.Action.READ);
 
-    acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
-    } finally {
-      acl.close();
-    }
+      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
+      try {
+        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
+        AccessControlService.BlockingInterface protocol =
+            AccessControlService.newBlockingStub(service);
+        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+      } finally {
+        acl.close();
+      }
 
-    assertFalse("User should not be granted permission: " + upToVerify.toString(),
+      assertFalse("User should not be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
-    // disable table before modification
-    admin.disableTable(tableName);
+      // disable table before modification
+      admin.disableTable(tableName);
 
-    User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
-    htd.setOwner(newOwner);
-    admin.modifyTable(tableName, htd);
+      User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
+      htd.setOwner(newOwner);
+      admin.modifyTable(tableName, htd);
 
-    acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol, tableName);
-    } finally {
-      acl.close();
-    }
+      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
+      try {
+        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
+        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",
+      UserPermission newOwnerperm =
+          new UserPermission(Bytes.toBytes(newOwner.getName()), tableName, null, Action.values());
+      assertTrue("New owner should have all permissions on table",
         hasFoundUserPermission(newOwnerperm, perms));
-
-    // delete table
-    admin.deleteTable(tableName);
+    } finally {
+      // delete table
+      TEST_UTIL.deleteTable(tableName);
+    }
   }
 
   @Test
@@ -1580,127 +1578,134 @@ public class TestAccessController extends SecureTestUtil {
     User userQualifier = User.createUserForTesting(conf, "user_check_perms_q", new String[0]);
 
     grantOnTable(TEST_UTIL, userTable.getShortName(),
-      TEST_TABLE.getTableName(), null, null,
+      TEST_TABLE, null, null,
       Permission.Action.READ);
     grantOnTable(TEST_UTIL, userColumn.getShortName(),
-      TEST_TABLE.getTableName(), TEST_FAMILY, null,
+      TEST_TABLE, TEST_FAMILY, null,
       Permission.Action.READ);
     grantOnTable(TEST_UTIL, userQualifier.getShortName(),
-      TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+      TEST_TABLE, TEST_FAMILY, TEST_Q1,
       Permission.Action.READ);
 
-    AccessTestAction tableRead = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
-          Permission.Action.READ);
-        return null;
-      }
-    };
-
-    AccessTestAction columnRead = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
-          Permission.Action.READ);
-        return null;
-      }
-    };
+    try {
+      AccessTestAction tableRead = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, null, null, Permission.Action.READ);
+          return null;
+        }
+      };
 
-    AccessTestAction qualifierRead = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
-          Permission.Action.READ);
-        return null;
-      }
-    };
+      AccessTestAction columnRead = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ);
+          return null;
+        }
+      };
 
-    AccessTestAction multiQualifierRead = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
-              Permission.Action.READ),
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
-              Permission.Action.READ), });
-        return null;
-      }
-    };
+      AccessTestAction qualifierRead = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ);
+          return null;
+        }
+      };
 
-    AccessTestAction globalAndTableRead = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(),
-          new Permission[] { new Permission(Permission.Action.READ),
-            new TablePermission(TEST_TABLE.getTableName(), null, (byte[]) null,
-            Permission.Action.READ), });
-        return null;
-      }
-    };
+      AccessTestAction multiQualifierRead = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, 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;
+        }
+      };
 
-    AccessTestAction noCheck = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[0]);
-        return null;
-      }
-    };
+      AccessTestAction globalAndTableRead = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[] {
+              new Permission(Permission.Action.READ),
+              new TablePermission(TEST_TABLE, null, (byte[]) null, Permission.Action.READ), });
+          return null;
+        }
+      };
 
-    verifyAllowed(tableRead, SUPERUSER, userTable);
-    verifyDenied(tableRead, userColumn, userQualifier);
+      AccessTestAction noCheck = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[0]);
+          return null;
+        }
+      };
 
-    verifyAllowed(columnRead, SUPERUSER, userTable, userColumn);
-    verifyDenied(columnRead, userQualifier);
+      verifyAllowed(tableRead, SUPERUSER, userTable);
+      verifyDenied(tableRead, userColumn, userQualifier);
 
-    verifyAllowed(qualifierRead, SUPERUSER, userTable, userColumn, userQualifier);
+      verifyAllowed(columnRead, SUPERUSER, userTable, userColumn);
+      verifyDenied(columnRead, userQualifier);
 
-    verifyAllowed(multiQualifierRead, SUPERUSER, userTable, userColumn);
-    verifyDenied(multiQualifierRead, userQualifier);
+      verifyAllowed(qualifierRead, SUPERUSER, userTable, userColumn, userQualifier);
 
-    verifyAllowed(globalAndTableRead, SUPERUSER);
-    verifyDenied(globalAndTableRead, userTable, userColumn, userQualifier);
+      verifyAllowed(multiQualifierRead, SUPERUSER, userTable, userColumn);
+      verifyDenied(multiQualifierRead, userQualifier);
 
-    verifyAllowed(noCheck, SUPERUSER, userTable, userColumn, userQualifier);
+      verifyAllowed(globalAndTableRead, SUPERUSER);
+      verifyDenied(globalAndTableRead, userTable, userColumn, userQualifier);
 
-    // --------------------------------------
-    // test family level multiple permissions
-    AccessTestAction familyReadWrite = new AccessTestAction() {
-      @Override
-      public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
-          Permission.Action.READ, Permission.Action.WRITE);
-        return null;
-      }
-    };
+      verifyAllowed(noCheck, SUPERUSER, userTable, userColumn, userQualifier);
 
-    verifyAllowed(familyReadWrite, SUPERUSER, USER_OWNER, USER_CREATE, USER_RW);
-    verifyDenied(familyReadWrite, USER_NONE, USER_RO);
+      // --------------------------------------
+      // test family level multiple permissions
+      AccessTestAction familyReadWrite = new AccessTestAction() {
+        @Override
+        public Void run() throws Exception {
+          checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ,
+            Permission.Action.WRITE);
+          return null;
+        }
+      };
 
-    // --------------------------------------
-    // check for wrong table region
-    CheckPermissionsRequest checkRequest = CheckPermissionsRequest.newBuilder()
-      .addPermission(AccessControlProtos.Permission.newBuilder()
-          .setType(AccessControlProtos.Permission.Type.Table)
-          .setTablePermission(
-              AccessControlProtos.TablePermission.newBuilder()
-                  .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE.getTableName()))
-                  .addAction(AccessControlProtos.Permission.Action.CREATE))
-      ).build();
-    Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(channel);
+      verifyAllowed(familyReadWrite, SUPERUSER, USER_OWNER, USER_CREATE, USER_RW);
+      verifyDenied(familyReadWrite, USER_NONE, USER_RO);
+
+      // --------------------------------------
+      // check for wrong table region
+      CheckPermissionsRequest checkRequest =
+          CheckPermissionsRequest
+              .newBuilder()
+              .addPermission(
+                AccessControlProtos.Permission
+                    .newBuilder()
+                    .setType(AccessControlProtos.Permission.Type.Table)
+                    .setTablePermission(
+                      AccessControlProtos.TablePermission.newBuilder()
+                          .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE))
+                          .addAction(AccessControlProtos.Permission.Action.CREATE))).build();
+      Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
       try {
-        // 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();
       }
+
     } finally {
-      acl.close();
+      revokeFromTable(TEST_UTIL, userTable.getShortName(), TEST_TABLE, null, null,
+        Permission.Action.READ);
+      revokeFromTable(TEST_UTIL, userColumn.getShortName(), TEST_TABLE, TEST_FAMILY, null,
+        Permission.Action.READ);
+      revokeFromTable(TEST_UTIL, userQualifier.getShortName(), TEST_TABLE, TEST_FAMILY, TEST_Q1,
+        Permission.Action.READ);
     }
   }
 
@@ -1816,15 +1821,6 @@ public class TestAccessController extends SecureTestUtil {
     LOG.debug("Test for global authorization for a new registered RegionServer.");
     MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
 
-    // Since each RegionServer running on different user, add global
-    // permissions for the new user.
-    String currentUser = User.getCurrent().getShortName();
-    String activeUserForNewRs = currentUser + ".hfs." +
-      hbaseCluster.getLiveRegionServerThreads().size();
-    grantGlobal(TEST_UTIL, activeUserForNewRs,
-      Permission.Action.ADMIN, Permission.Action.CREATE, Permission.Action.READ,
-        Permission.Action.WRITE);
-
     final Admin admin = TEST_UTIL.getHBaseAdmin();
     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE2);
     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
@@ -1889,35 +1885,40 @@ public class TestAccessController extends SecureTestUtil {
     User TABLE_ADMIN = User.createUserForTesting(conf, "UserA", new String[0]);
 
     // Grant TABLE ADMIN privs
-    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
-      TEST_TABLE.getTableName(), null, null,
+    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(), TEST_TABLE, null, null,
       Permission.Action.ADMIN);
-
-    AccessTestAction listTablesAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-             Admin admin = conn.getAdmin()) {
-          return Arrays.asList(admin.listTables());
+    try {
+      AccessTestAction listTablesAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+              Admin admin = conn.getAdmin()) {
+            return Arrays.asList(admin.listTables());
+          }
         }
-      }
-    };
+      };
 
-    AccessTestAction getTableDescAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-            Admin admin = conn.getAdmin();) {
-          return admin.getTableDescriptor(TEST_TABLE.getTableName());
+      AccessTestAction getTableDescAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+              Admin admin = conn.getAdmin();) {
+            return admin.getTableDescriptor(TEST_TABLE);
+          }
         }
-      }
-    };
+      };
 
-    verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, TABLE_ADMIN);
-    verifyIfEmptyList(listTablesAction, USER_RW, USER_RO, USER_NONE);
+      verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, TABLE_ADMIN);
+      verifyIfEmptyList(listTablesAction, USER_RW, USER_RO, USER_NONE);
 
-    verifyAllowed(getTableDescAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, TABLE_ADMIN);
-    verifyDenied(getTableDescAction, USER_RW, USER_RO, USER_NONE);
+      verifyAllowed(getTableDescAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER,
+        TABLE_ADMIN);
+      verifyDenied(getTableDescAction, USER_RW, USER_RO, USER_NONE);
+    } finally {
+      // Cleanup, revoke TABLE ADMIN privs
+      revokeFromTable(TEST_UTIL, TABLE_ADMIN.getShortName(), TEST_TABLE, null, null,
+        Permission.Action.ADMIN);
+    }
   }
 
   @Test
@@ -1945,20 +1946,22 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testTableDeletion() throws Exception {
     User TABLE_ADMIN = User.createUserForTesting(conf, "TestUser", new String[0]);
+    final TableName tname = TableName.valueOf("testTableDeletion");
+    createTestTable(tname);
 
     // Grant TABLE ADMIN privs
-    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
-      TEST_TABLE.getTableName(), null, null,
+    grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(), tname, null, null,
       Permission.Action.ADMIN);
 
     AccessTestAction deleteTableAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Connection unmanagedConnection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Connection unmanagedConnection =
+            ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
         Admin admin = unmanagedConnection.getAdmin();
         try {
-          admin.disableTable(TEST_TABLE.getTableName());
-          admin.deleteTable(TEST_TABLE.getTableName());
+          admin.disableTable(tname);
+          admin.deleteTable(tname);
         } finally {
           admin.close();
           unmanagedConnection.close();
@@ -1971,19 +1974,28 @@ public class TestAccessController extends SecureTestUtil {
     verifyAllowed(deleteTableAction, TABLE_ADMIN);
   }
 
+  private void createTestTable(TableName tname) throws Exception {
+    HTableDescriptor htd = new HTableDescriptor(tname);
+    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
+    hcd.setMaxVersions(100);
+    htd.addFamily(hcd);
+    htd.setOwner(USER_OWNER);
+    TEST_UTIL.createTable(htd, new byte[][] { Bytes.toBytes("s") });
+  }
+
   @Test
   public void testNamespaceUserGrant() throws Exception {
     AccessTestAction getAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           return t.get(new Get(TEST_ROW));
         }
       }
     };
 
-    String namespace = TEST_TABLE.getTableName().getNamespaceAsString();
+    String namespace = TEST_TABLE.getNamespaceAsString();
 
     // Grant namespace READ to USER_NONE, this should supersede any table permissions
     grantOnNamespace(TEST_UTIL, USER_NONE.getShortName(), namespace, Permission.Action.READ);
@@ -2003,7 +2015,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           return t.get(new Get(TEST_ROW));
         }
       }
@@ -2013,8 +2025,8 @@ public class TestAccessController extends SecureTestUtil {
 
     // Grant table READ permissions to testGrantRevoke.
     try {
-      grantOnTableUsingAccessControlClient(TEST_UTIL, systemUserConnection, testGrantRevoke.getShortName(),
-          TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
+      grantOnTableUsingAccessControlClient(TEST_UTIL, systemUserConnection,
+        testGrantRevoke.getShortName(), TEST_TABLE, null, null, Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.grant. ", e);
     }
@@ -2024,8 +2036,8 @@ public class TestAccessController extends SecureTestUtil {
 
     // Revoke table READ permission to testGrantRevoke.
     try {
-      revokeFromTableUsingAccessControlClient(TEST_UTIL, systemUserConnection, testGrantRevoke.getShortName(),
-          TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
+      revokeFromTableUsingAccessControlClient(TEST_UTIL, systemUserConnection,
+        testGrantRevoke.getShortName(), TEST_TABLE, null, null, Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.revoke ", e);
     }
@@ -2043,7 +2055,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName())) {
+            Table t = conn.getTable(TEST_TABLE)) {
           return t.get(new Get(TEST_ROW));
         }
       }
@@ -2052,26 +2064,30 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(getAction, testGlobalGrantRevoke);
 
     // Grant table READ permissions to testGlobalGrantRevoke.
+    String userName = testGlobalGrantRevoke.getShortName();
     try {
       grantGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection,
-          testGlobalGrantRevoke.getShortName(), Permission.Action.READ);
+          userName, Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.grant. ", e);
     }
+    try {
+      // Now testGlobalGrantRevoke should be able to read also
+      verifyAllowed(getAction, testGlobalGrantRevoke);
 
-    // Now testGlobalGrantRevoke should be able to read also
-    verifyAllowed(getAction, testGlobalGrantRevoke);
+      // Revoke table READ permission to testGlobalGrantRevoke.
+      try {
+        revokeGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection,
+          userName, Permission.Action.READ);
+      } catch (Throwable e) {
+        LOG.error("error during call of AccessControlClient.revoke ", e);
+      }
 
-    // Revoke table READ permission to testGlobalGrantRevoke.
-    try {
-      revokeGlobalUsingAccessControlClient(TEST_UTIL, systemUserConnection,
-          testGlobalGrantRevoke.getShortName(), Permission.Action.READ);
-    } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.revoke ", e);
+      // Now testGlobalGrantRevoke shouldn't be able read
+      verifyDenied(getAction, testGlobalGrantRevoke);
+    } finally {
+      revokeGlobal(TEST_UTIL, userName, Permission.Action.READ);
     }
-
-    // Now testGlobalGrantRevoke shouldn't be able read
-    verifyDenied(getAction, testGlobalGrantRevoke);
   }
 
   @Test
@@ -2082,7 +2098,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           return t.get(new Get(TEST_ROW));
         }
       }
@@ -2090,27 +2106,32 @@ public class TestAccessController extends SecureTestUtil {
 
     verifyDenied(getAction, testNS);
 
+    String userName = testNS.getShortName();
+    String namespace = TEST_TABLE.getNamespaceAsString();
     // Grant namespace READ to testNS, this should supersede any table permissions
     try {
-      grantOnNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, testNS.getShortName(),
-          TEST_TABLE.getTableName().getNamespaceAsString(), Permission.Action.READ);
+      grantOnNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
+        namespace, Permission.Action.READ);
     } catch (Throwable e) {
       LOG.error("error during call of AccessControlClient.grant. ", e);
     }
+    try {
+      // Now testNS should be able to read also
+      verifyAllowed(getAction, testNS);
 
-    // Now testNS should be able to read also
-    verifyAllowed(getAction, testNS);
+      // Revoke namespace READ to testNS, this should supersede any table permissions
+      try {
+        revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, userName,
+          namespace, Permission.Action.READ);
+      } catch (Throwable e) {
+        LOG.error("error during call of AccessControlClient.revoke ", e);
+      }
 
-    // Revoke namespace READ to testNS, this should supersede any table permissions
-    try {
-      revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, systemUserConnection, testNS.getShortName(),
-          TEST_TABLE.getTableName().getNamespaceAsString(), Permission.Action.READ);
-    } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.revoke ", e);
+      // Now testNS shouldn't be able read
+      verifyDenied(getAction, testNS);
+    } finally {
+      revokeFromNamespace(TEST_UTIL, userName, namespace, Permission.Action.READ);
     }
-
-    // Now testNS shouldn't be able read
-    verifyDenied(getAction, testNS);
   }
 
 
@@ -2165,7 +2186,7 @@ public class TestAccessController extends SecureTestUtil {
     for (JVMClusterUtil.RegionServerThread thread:
         TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
       HRegionServer rs = thread.getRegionServer();
-      for (HRegion region: rs.getOnlineRegions(TEST_TABLE.getTableName())) {
+      for (HRegion region: rs.getOnlineRegions(TEST_TABLE)) {
         region.getCoprocessorHost().load(PingCoprocessor.class,
           Coprocessor.PRIORITY_USER, conf);
       }
@@ -2177,32 +2198,37 @@ public class TestAccessController extends SecureTestUtil {
     User userB = User.createUserForTesting(conf, "UserB", new String[0]);
 
     grantOnTable(TEST_UTIL, userA.getShortName(),
-      TEST_TABLE.getTableName(), null, null,
+      TEST_TABLE, null, null,
       Permission.Action.EXEC);
-
-    // Create an action for invoking our test endpoint
-    AccessTestAction execEndpointAction = new AccessTestAction() {
-      @Override
-      public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
-          BlockingRpcChannel service = t.coprocessorService(HConstants.EMPTY_BYTE_ARRAY);
-          PingCoprocessor.newBlockingStub(service).noop(null, NoopRequest.newBuilder().build());
+    try {
+      // Create an action for invoking our test endpoint
+      AccessTestAction execEndpointAction = new AccessTestAction() {
+        @Override
+        public Object run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(conf);
+              Table t = conn.getTable(TEST_TABLE);) {
+            BlockingRpcChannel service = t.coprocessorService(HConstants.EMPTY_BYTE_ARRAY);
+            PingCoprocessor.newBlockingStub(service).noop(null, NoopRequest.newBuilder().build());
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
 
-    String namespace = TEST_TABLE.getTableName().getNamespaceAsString();
-    // Now grant EXEC to the entire namespace to user B
-    grantOnNamespace(TEST_UTIL, userB.getShortName(), namespace, Permission.Action.EXEC);
-    // User B should now be allowed also
-    verifyAllowed(execEndpointAction, userA, userB);
+      String namespace = TEST_TABLE.getNamespaceAsString();
+      // Now grant EXEC to the entire namespace to user B
+      grantOnNamespace(TEST_UTIL, userB.getShortName(), namespace, Permission.Action.EXEC);
+      // User B should now be allowed also
+      verifyAllowed(execEndpointAction, userA, userB);
 
-    revokeFromNamespace(TEST_UTIL, userB.getShortName(), namespace, Permission.Action.EXEC);
-    // Verify that EXEC permission is checked correctly
-    verifyDenied(execEndpointAction, userB);
-    verifyAllowed(execEndpointAction, userA);
+      revokeFromNamespace(TEST_UTIL, userB.getShortName(), namespace, Permission.Action.EXEC);
+      // Verify that EXEC permission is checked correctly
+      verifyDenied(execEndpointAction, userB);
+      verifyAllowed(execEndpointAction, userA);
+    } finally {
+      // Cleanup, revoke the userA privileges
+      revokeFromTable(TEST_UTIL, userA.getShortName(), TEST_TABLE, null, null,
+        Permission.Action.EXEC);
+    }
   }
 
   @Test
@@ -2211,7 +2237,7 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table t = conn.getTable(TEST_TABLE.getTableName());) {
+            Table t = conn.getTable(TEST_TABLE);) {
           KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
             HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
             new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE,
@@ -2251,15 +2277,15 @@ public class TestAccessController extends SecureTestUtil {
     try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       List<UserPermission> existingPerms =
           AccessControlClient.getUserPermissions(connection,
-            TEST_TABLE.getTableName().getNameAsString());
+            TEST_TABLE.getNameAsString());
       assertTrue(existingPerms != null);
       assertTrue(existingPerms.size() > 1);
       try (Admin admin = connection.getAdmin()) {
-        admin.disableTable(TEST_TABLE.getTableName());
-        admin.truncateTable(TEST_TABLE.getTableName(), true);
+        admin.disableTable(TEST_TABLE);
+        admin.truncateTable(TEST_TABLE, true);
       }
       List<UserPermission> perms = AccessControlClient.getUserPermissions(connection,
-        TEST_TABLE.getTableName().getNameAsString());
+        TEST_TABLE.getNameAsString());
       assertTrue(perms != null);
       assertEquals(existingPerms.size(), perms.size());
     } catch (Throwable thw) {
@@ -2283,20 +2309,22 @@ public class TestAccessController extends SecureTestUtil {
 
   @Test
   public void testAccessControlClientUserPerms() throws Exception {
-    // adding default prefix explicitly as it is not included in the table name.
-    assertEquals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR,
-                 TEST_TABLE.getTableName().getNamespaceAsString());
-    final String regex = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR +
-      TableName.NAMESPACE_DELIM + TEST_TABLE.getTableName().getNameAsString();
-    User testUserPerms = User.createUserForTesting(conf, "testUserPerms", new String[0]);
-    assertEquals(0, testUserPerms.runAs(getPrivilegedAction(regex)).size());
-    // Grant TABLE ADMIN privs to testUserPerms
-    grantOnTable(TEST_UTIL, testUserPerms.getShortName(), TEST_TABLE.getTableName(), null,
-      null, Action.ADMIN);
-    List<UserPermission> perms = testUserPerms.runAs(getPrivilegedAction(regex));
-    assertNotNull(perms);
-    // USER_ADMIN, USER_CREATE, USER_RW, USER_RO, testUserPerms, USER_ADMIN_CF has row each.
-    assertEquals(6, perms.size());
+    TableName tname = TableName.valueOf("testAccessControlClientUserPerms");
+    createTestTable(tname);
+    try {
+      final String regex = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR +
+          TableName.NAMESPACE_DELIM + tname.getNameAsString();
+      User testUserPerms = User.createUserForTesting(conf, "testUserPerms", new String[0]);
+      assertEquals(0, testUserPerms.runAs(getPrivilegedAction(regex)).size());
+      // Grant TABLE ADMIN privs to testUserPerms
+      grantOnTable(TEST_UTIL, testUserPerms.getShortName(), tname, null, null, Action.ADMIN);
+      List<UserPermission> perms = testUserPerms.runAs(getPrivilegedAction(regex));
+      assertNotNull(perms);
+      // Superuser, testUserPerms
+      assertEquals(2, perms.size());
+    } finally {
+      TEST_UTIL.deleteTable(tname);
+    }
   }
 
   @Test
@@ -2361,14 +2389,16 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction prepareBulkLoadAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        ACCESS_CONTROLLER.prePrepareBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null), null);
+        ACCESS_CONTROLLER.prePrepareBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null),
+          null);
         return null;
       }
     };
     AccessTestAction cleanupBulkLoadAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        ACCESS_CONTROLLER.preCleanupBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null), null);
+        ACCESS_CONTROLLER.preCleanupBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null),
+          null);
         return null;
       }
     };