You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/01/19 21:21:44 UTC

hbase git commit: HBASE-16773 AccessController should access local region if possible

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 b75be7467 -> 342169cc2


HBASE-16773 AccessController should access local region if possible


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

Branch: refs/heads/branch-1.3
Commit: 342169cc223c0055750168f7872b8707b67ff5cc
Parents: b75be74
Author: tedyu <yu...@gmail.com>
Authored: Thu Jan 19 13:21:58 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Jan 19 13:21:58 2017 -0800

----------------------------------------------------------------------
 .../security/access/AccessControlLists.java     | 148 +++++++++++--------
 .../hbase/security/access/AccessController.java |  43 +++---
 .../security/access/TestTablePermissions.java   | 118 ++++++++-------
 3 files changed, 176 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/342169cc/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 4f992b4..50d575e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -143,9 +143,10 @@ public class AccessControlLists {
    * Stores a new user permission grant in the access control lists table.
    * @param conf the configuration
    * @param userPerm the details of the permission to be granted
+   * @param t acl table instance. It is closed upon method return
    * @throws IOException in the case of an error accessing the metadata table
    */
-  static void addUserPermission(Configuration conf, UserPermission userPerm)
+  static void addUserPermission(Configuration conf, UserPermission userPerm, Table t)
       throws IOException {
     Permission.Action[] actions = userPerm.getActions();
     byte[] rowKey = userPermissionRowKey(userPerm);
@@ -169,11 +170,10 @@ public class AccessControlLists {
           Bytes.toString(key)+": "+Bytes.toStringBinary(value)
       );
     }
-    // TODO: Pass in a Connection rather than create one each time.
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        table.put(p);
-      }
+    try {
+      t.put(p);
+    } finally {
+      t.close();
     }
   }
 
@@ -188,9 +188,10 @@ public class AccessControlLists {
    *
    * @param conf the configuration
    * @param userPerm the details of the permission to be revoked
+   * @param t acl table
    * @throws IOException if there is an error accessing the metadata table
    */
-  static void removeUserPermission(Configuration conf, UserPermission userPerm)
+  static void removeUserPermission(Configuration conf, UserPermission userPerm, Table t)
       throws IOException {
     Delete d = new Delete(userPermissionRowKey(userPerm));
     byte[] key = userPermissionKey(userPerm);
@@ -199,10 +200,17 @@ public class AccessControlLists {
       LOG.debug("Removing permission "+ userPerm.toString());
     }
     d.addColumns(ACL_LIST_FAMILY, key);
-    // TODO: Pass in a Connection rather than create one each time.
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        table.delete(d);
+    if (t == null) {
+      try (Connection connection = ConnectionFactory.createConnection(conf)) {
+        try (Table table = connection.getTable(ACL_TABLE_NAME)) {
+          table.delete(d);
+        }
+      }
+    } else {
+      try {
+        t.delete(d);
+      } finally {
+        t.close();
       }
     }
   }
@@ -210,25 +218,24 @@ public class AccessControlLists {
   /**
    * Remove specified table from the _acl_ table.
    */
-  static void removeTablePermissions(Configuration conf, TableName tableName)
+  static void removeTablePermissions(Configuration conf, TableName tableName, Table t)
       throws IOException{
     Delete d = new Delete(tableName.getName());
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removing permissions of removed table "+ tableName);
     }
-    // TODO: Pass in a Connection rather than create one each time.
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        table.delete(d);
-      }
+    try {
+      t.delete(d);
+    } finally {
+      t.close();
     }
   }
 
   /**
    * Remove specified namespace from the acl table.
    */
-  static void removeNamespacePermissions(Configuration conf, String namespace)
+  static void removeNamespacePermissions(Configuration conf, String namespace, Table t)
       throws IOException{
     Delete d = new Delete(Bytes.toBytes(toNamespaceEntry(namespace)));
 
@@ -236,58 +243,64 @@ public class AccessControlLists {
       LOG.debug("Removing permissions of removed namespace "+ namespace);
     }
 
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        table.delete(d);
-      }
+    try {
+      t.delete(d);
+    } finally {
+      t.close();
     }
   }
 
   /**
    * Remove specified table column from the acl table.
    */
-  static void removeTablePermissions(Configuration conf, TableName tableName, byte[] column)
-      throws IOException{
+  static void removeTablePermissions(TableName tableName, byte[] column, Table table,
+      boolean closeTable) throws IOException{
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removing permissions of removed column " + Bytes.toString(column) +
                 " from table "+ tableName);
     }
-    // TODO: Pass in a Connection rather than create one each time.
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        Scan scan = new Scan();
-        scan.addFamily(ACL_LIST_FAMILY);
+    Scan scan = new Scan();
+    scan.addFamily(ACL_LIST_FAMILY);
 
-        String columnName = Bytes.toString(column);
-        scan.setFilter(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(
-            String.format("(%s%s%s)|(%s%s)$",
-                ACL_KEY_DELIMITER, columnName, ACL_KEY_DELIMITER,
-                ACL_KEY_DELIMITER, columnName))));
+    String columnName = Bytes.toString(column);
+    scan.setFilter(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(
+        String.format("(%s%s%s)|(%s%s)$",
+            ACL_KEY_DELIMITER, columnName, ACL_KEY_DELIMITER,
+            ACL_KEY_DELIMITER, columnName))));
 
-        Set<byte[]> qualifierSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
-        ResultScanner scanner = table.getScanner(scan);
-        try {
-          for (Result res : scanner) {
-            for (byte[] q : res.getFamilyMap(ACL_LIST_FAMILY).navigableKeySet()) {
-              qualifierSet.add(q);
-            }
-          }
-        } finally {
-          scanner.close();
+    Set<byte[]> qualifierSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+    ResultScanner scanner = null;
+    try {
+      scanner = table.getScanner(scan);
+      for (Result res : scanner) {
+        for (byte[] q : res.getFamilyMap(ACL_LIST_FAMILY).navigableKeySet()) {
+          qualifierSet.add(q);
         }
+      }
 
-        if (qualifierSet.size() > 0) {
-          Delete d = new Delete(tableName.getName());
-          for (byte[] qualifier : qualifierSet) {
-            d.addColumns(ACL_LIST_FAMILY, qualifier);
-          }
-          table.delete(d);
+      if (qualifierSet.size() > 0) {
+        Delete d = new Delete(tableName.getName());
+        for (byte[] qualifier : qualifierSet) {
+          d.addColumns(ACL_LIST_FAMILY, qualifier);
         }
+        table.delete(d);
       }
+    }  finally {
+      if (scanner != null) scanner.close();
+      if (closeTable) table.close();
     }
   }
 
+  static void removeTablePermissions(Configuration conf, TableName tableName, byte[] column,
+      Table t) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing permissions of removed column " + Bytes.toString(column) +
+          " from table "+ tableName);
+    }
+    removeTablePermissions(tableName, column, t, true);
+  }
+
   static byte[] userPermissionRowKey(UserPermission userPerm) {
     byte[] row;
     if(userPerm.hasNamespace()) {
@@ -432,12 +445,12 @@ public class AccessControlLists {
 
   static ListMultimap<String, TablePermission> getTablePermissions(Configuration conf,
         TableName tableName) throws IOException {
-    return getPermissions(conf, tableName != null ? tableName.getName() : null);
+    return getPermissions(conf, tableName != null ? tableName.getName() : null, null);
   }
 
   static ListMultimap<String, TablePermission> getNamespacePermissions(Configuration conf,
         String namespace) throws IOException {
-    return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)));
+    return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null);
   }
 
   /**
@@ -450,24 +463,29 @@ public class AccessControlLists {
    * </p>
    */
   static ListMultimap<String, TablePermission> getPermissions(Configuration conf,
-      byte[] entryName) throws IOException {
+      byte[] entryName, Table t) throws IOException {
     if (entryName == null) entryName = ACL_GLOBAL_NAME;
 
     // for normal user tables, we just read the table row from _acl_
     ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
-    // TODO: Pass in a Connection rather than create one each time.
-    try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-        Get get = new Get(entryName);
-        get.addFamily(ACL_LIST_FAMILY);
-        Result row = table.get(get);
-        if (!row.isEmpty()) {
-          perms = parsePermissions(entryName, row);
-        } else {
-          LOG.info("No permissions found in " + ACL_TABLE_NAME + " for acl entry "
-              + Bytes.toString(entryName));
+    Get get = new Get(entryName);
+    get.addFamily(ACL_LIST_FAMILY);
+    Result row = null;
+    if (t == null) {
+      try (Connection connection = ConnectionFactory.createConnection(conf)) {
+        try (Table table = connection.getTable(ACL_TABLE_NAME)) {
+          row = table.get(get);
         }
       }
+    } else {
+      row = t.get(get);
+    }
+
+    if (!row.isEmpty()) {
+      perms = parsePermissions(entryName, row);
+    } else {
+      LOG.info("No permissions found in " + ACL_TABLE_NAME + " for acl entry "
+          + Bytes.toString(entryName));
     }
 
     return perms;
@@ -491,7 +509,7 @@ public class AccessControlLists {
       Configuration conf, byte[] entryName)
   throws IOException {
     ListMultimap<String,TablePermission> allPerms = getPermissions(
-      conf, entryName);
+      conf, entryName, null);
 
     List<UserPermission> perms = new ArrayList<UserPermission>();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/342169cc/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index a147b12..a6b1631 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Query;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.BaseMasterAndRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
@@ -269,10 +270,12 @@ public class AccessController extends BaseMasterAndRegionObserver
     Configuration conf = regionEnv.getConfiguration();
     for (byte[] entry: entries) {
       try {
-        ListMultimap<String,TablePermission> perms =
-          AccessControlLists.getPermissions(conf, entry);
-        byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
-        zkw.writeToZookeeper(entry, serialized);
+        try (Table t = regionEnv.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+          ListMultimap<String,TablePermission> perms =
+              AccessControlLists.getPermissions(conf, entry, t);
+          byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
+          zkw.writeToZookeeper(entry, serialized);
+        }
       } catch (IOException ex) {
         LOG.error("Failed updating permissions mirror for '" + Bytes.toString(entry) + "'",
             ex);
@@ -1043,7 +1046,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           @Override
           public Void run() throws Exception {
             AccessControlLists.addUserPermission(c.getEnvironment().getConfiguration(),
-                userperm);
+                userperm, c.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
             return null;
           }
         });
@@ -1058,13 +1061,14 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
-  public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
+  public void postDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> c,
       final TableName tableName) throws IOException {
     final Configuration conf = c.getEnvironment().getConfiguration();
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        AccessControlLists.removeTablePermissions(conf, tableName);
+        AccessControlLists.removeTablePermissions(conf, tableName,
+            c.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
         return null;
       }
     });
@@ -1090,7 +1094,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
-  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+  public void postTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName) throws IOException {
     final Configuration conf = ctx.getEnvironment().getConfiguration();
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
@@ -1099,7 +1103,8 @@ public class AccessController extends BaseMasterAndRegionObserver
         List<UserPermission> perms = tableAcls.get(tableName);
         if (perms != null) {
           for (UserPermission perm : perms) {
-            AccessControlLists.addUserPermission(conf, perm);
+            AccessControlLists.addUserPermission(conf, perm,
+                ctx.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
           }
         }
         tableAcls.remove(tableName);
@@ -1115,7 +1120,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
-  public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
+  public void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> c,
       TableName tableName, final HTableDescriptor htd) throws IOException {
     final Configuration conf = c.getEnvironment().getConfiguration();
     // default the table owner to current user, if not specified.
@@ -1126,7 +1131,8 @@ public class AccessController extends BaseMasterAndRegionObserver
       public Void run() throws Exception {
         UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
             htd.getTableName(), null, Action.values());
-        AccessControlLists.addUserPermission(conf, userperm);
+        AccessControlLists.addUserPermission(conf, userperm,
+            c.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
         return null;
       }
     });
@@ -1153,13 +1159,14 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
-  public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
+  public void postDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> c,
       final TableName tableName, final byte[] col) throws IOException {
     final Configuration conf = c.getEnvironment().getConfiguration();
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        AccessControlLists.removeTablePermissions(conf, tableName, col);
+        AccessControlLists.removeTablePermissions(conf, tableName, col,
+            c.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
         return null;
       }
     });
@@ -1370,13 +1377,14 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   @Override
-  public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
+  public void postDeleteNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final String namespace) throws IOException {
     final Configuration conf = ctx.getEnvironment().getConfiguration();
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        AccessControlLists.removeNamespacePermissions(conf, namespace);
+        AccessControlLists.removeNamespacePermissions(conf, namespace,
+            ctx.getEnvironment().getTable(AccessControlLists.ACL_TABLE_NAME));
         return null;
       }
     });
@@ -2241,7 +2249,8 @@ public class AccessController extends BaseMasterAndRegionObserver
         User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm);
+            AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm,
+                regionEnv.getTable(AccessControlLists.ACL_TABLE_NAME));
             return null;
           }
         });
@@ -2292,7 +2301,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
-            AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm);
+            AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm, null);
             return null;
           }
         });

http://git-wip-us.apache.org/repos/asf/hbase/blob/342169cc/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
index 926dacf..9b35ed1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -115,9 +117,12 @@ public class TestTablePermissions {
   @After
   public void tearDown() throws Exception {
     Configuration conf = UTIL.getConfiguration();
-    AccessControlLists.removeTablePermissions(conf, TEST_TABLE);
-    AccessControlLists.removeTablePermissions(conf, TEST_TABLE2);
-    AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE_NAME);
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      AccessControlLists.removeTablePermissions(conf, TEST_TABLE, table);
+      AccessControlLists.removeTablePermissions(conf, TEST_TABLE2, table);
+      AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE_NAME, table);
+    }
   }
 
   /**
@@ -172,18 +177,20 @@ public class TestTablePermissions {
   @Test
   public void testBasicWrite() throws Exception {
     Configuration conf = UTIL.getConfiguration();
-    // add some permissions
-    AccessControlLists.addUserPermission(conf,
-            new UserPermission(Bytes.toBytes("george"), TEST_TABLE, null, (byte[])null,
-            UserPermission.Action.READ, UserPermission.Action.WRITE));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE, null, (byte[])null,
-            UserPermission.Action.READ));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("humphrey"),
-            TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-            UserPermission.Action.READ));
-
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      // add some permissions
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("george"), TEST_TABLE, null, (byte[])null,
+              UserPermission.Action.READ, UserPermission.Action.WRITE), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE, null, (byte[])null,
+              UserPermission.Action.READ), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("humphrey"),
+              TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+              UserPermission.Action.READ), table);
+    }
     // retrieve the same
     ListMultimap<String,TablePermission> perms =
         AccessControlLists.getTablePermissions(conf, TEST_TABLE);
@@ -236,10 +243,12 @@ public class TestTablePermissions {
     assertFalse(actions.contains(TablePermission.Action.WRITE));
 
     // table 2 permissions
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE2, null, (byte[])null,
-            TablePermission.Action.READ, TablePermission.Action.WRITE));
-
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE2, null, (byte[])null,
+              TablePermission.Action.READ, TablePermission.Action.WRITE), table);
+    }
     // check full load
     Map<byte[], ListMultimap<String,TablePermission>> allPerms =
         AccessControlLists.loadAll(conf);
@@ -268,22 +277,24 @@ public class TestTablePermissions {
   @Test
   public void testPersistence() throws Exception {
     Configuration conf = UTIL.getConfiguration();
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("albert"), TEST_TABLE, null,
-                           (byte[])null, TablePermission.Action.READ));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("betty"), TEST_TABLE, null,
-                           (byte[])null, TablePermission.Action.READ,
-                           TablePermission.Action.WRITE));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("clark"),
-                           TEST_TABLE, TEST_FAMILY,
-                           TablePermission.Action.READ));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("dwight"),
-                           TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-                           TablePermission.Action.WRITE));
-
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("albert"), TEST_TABLE, null,
+              (byte[])null, TablePermission.Action.READ), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("betty"), TEST_TABLE, null,
+              (byte[])null, TablePermission.Action.READ,
+              TablePermission.Action.WRITE), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("clark"),
+              TEST_TABLE, TEST_FAMILY,
+              TablePermission.Action.READ), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("dwight"),
+              TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+              TablePermission.Action.WRITE), table);
+    }
     // verify permissions survive changes in table metadata
     ListMultimap<String,TablePermission> preperms =
         AccessControlLists.getTablePermissions(conf, TEST_TABLE);
@@ -396,16 +407,18 @@ public class TestTablePermissions {
     Configuration conf = UTIL.getConfiguration();
 
     // add some permissions
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("user1"),
-            Permission.Action.READ, Permission.Action.WRITE));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("user2"),
-            Permission.Action.CREATE));
-    AccessControlLists.addUserPermission(conf,
-        new UserPermission(Bytes.toBytes("user3"),
-            Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE));
-
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("user1"),
+              Permission.Action.READ, Permission.Action.WRITE), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("user2"),
+              Permission.Action.CREATE), table);
+      AccessControlLists.addUserPermission(conf,
+          new UserPermission(Bytes.toBytes("user3"),
+              Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE), table);
+    }
     ListMultimap<String,TablePermission> perms = AccessControlLists.getTablePermissions(conf, null);
     List<TablePermission> user1Perms = perms.get("user1");
     assertEquals("Should have 1 permission for user1", 1, user1Perms.size());
@@ -438,12 +451,15 @@ public class TestTablePermissions {
     // currently running user is the system user and should have global admin perms
     User currentUser = User.getCurrent();
     assertTrue(authManager.authorize(currentUser, Permission.Action.ADMIN));
-    for (int i=1; i<=50; i++) {
-      AccessControlLists.addUserPermission(conf, new UserPermission(Bytes.toBytes("testauth"+i),
-          Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.WRITE));
-      // make sure the system user still shows as authorized
-      assertTrue("Failed current user auth check on iter "+i,
-          authManager.authorize(currentUser, Permission.Action.ADMIN));
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      for (int i=1; i<=50; i++) {
+        AccessControlLists.addUserPermission(conf, new UserPermission(Bytes.toBytes("testauth"+i),
+            Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.WRITE), table);
+        // make sure the system user still shows as authorized
+        assertTrue("Failed current user auth check on iter "+i,
+            authManager.authorize(currentUser, Permission.Action.ADMIN));
+      }
     }
   }
 }