You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2018/12/07 22:28:45 UTC

[02/51] [abbrv] hbase git commit: HBASE-21255 [acl] Refactor TablePermission into three classes (Global, Namespace, Table)

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/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 b6d8fe9..34480d3 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
@@ -131,14 +131,15 @@ public class AccessControlLists {
    * @throws IOException in the case of an error accessing the metadata table
    */
   static void addUserPermission(Configuration conf, UserPermission userPerm, Table t,
-                                boolean mergeExistingPermissions) throws IOException {
-    Permission.Action[] actions = userPerm.getActions();
-    byte[] rowKey = userPermissionRowKey(userPerm);
+      boolean mergeExistingPermissions) throws IOException {
+    Permission permission = userPerm.getPermission();
+    Permission.Action[] actions = permission.getActions();
+    byte[] rowKey = userPermissionRowKey(permission);
     Put p = new Put(rowKey);
     byte[] key = userPermissionKey(userPerm);
 
     if ((actions == null) || (actions.length == 0)) {
-      String msg = "No actions associated with user '" + Bytes.toString(userPerm.getUser()) + "'";
+      String msg = "No actions associated with user '" + userPerm.getUser() + "'";
       LOG.warn(msg);
       throw new IOException(msg);
     }
@@ -148,16 +149,14 @@ public class AccessControlLists {
       List<UserPermission> perms = getUserPermissions(conf, rowKey, null, null, null, false);
       UserPermission currentPerm = null;
       for (UserPermission perm : perms) {
-        if (Bytes.equals(perm.getUser(), userPerm.getUser())
-                && ((userPerm.isGlobal() && ACL_TABLE_NAME.equals(perm.getTableName()))
-                || perm.tableFieldsEqual(userPerm))) {
+        if (userPerm.equalsExceptActions(perm)) {
           currentPerm = perm;
           break;
         }
       }
 
-      if(currentPerm != null && currentPerm.getActions() != null){
-        actionSet.addAll(Arrays.asList(currentPerm.getActions()));
+      if (currentPerm != null && currentPerm.getPermission().getActions() != null){
+        actionSet.addAll(Arrays.asList(currentPerm.getPermission().getActions()));
       }
     }
 
@@ -225,24 +224,27 @@ public class AccessControlLists {
    */
   static void removeUserPermission(Configuration conf, UserPermission userPerm, Table t)
       throws IOException {
-    if (null == userPerm.getActions()) {
+    if (null == userPerm.getPermission().getActions() ||
+        userPerm.getPermission().getActions().length == 0) {
       removePermissionRecord(conf, userPerm, t);
     } else {
       // Get all the global user permissions from the acl table
       List<UserPermission> permsList =
-          getUserPermissions(conf, userPermissionRowKey(userPerm), null, null, null, false);
+        getUserPermissions(conf, userPermissionRowKey(userPerm.getPermission()),
+          null, null, null, false);
       List<Permission.Action> remainingActions = new ArrayList<>();
-      List<Permission.Action> dropActions = Arrays.asList(userPerm.getActions());
+      List<Permission.Action> dropActions = Arrays.asList(userPerm.getPermission().getActions());
       for (UserPermission perm : permsList) {
         // Find the user and remove only the requested permissions
-        if (Bytes.toString(perm.getUser()).equals(Bytes.toString(userPerm.getUser()))) {
-          for (Permission.Action oldAction : perm.getActions()) {
+        if (perm.getUser().equals(userPerm.getUser())) {
+          for (Permission.Action oldAction : perm.getPermission().getActions()) {
             if (!dropActions.contains(oldAction)) {
               remainingActions.add(oldAction);
             }
           }
           if (!remainingActions.isEmpty()) {
-            perm.setActions(remainingActions.toArray(new Permission.Action[remainingActions.size()]));
+            perm.getPermission().setActions(
+              remainingActions.toArray(new Permission.Action[remainingActions.size()]));
             addUserPermission(conf, perm, t);
           } else {
             removePermissionRecord(conf, userPerm, t);
@@ -258,7 +260,7 @@ public class AccessControlLists {
 
   private static void removePermissionRecord(Configuration conf, UserPermission userPerm, Table t)
       throws IOException {
-    Delete d = new Delete(userPermissionRowKey(userPerm));
+    Delete d = new Delete(userPermissionRowKey(userPerm.getPermission()));
     d.addColumns(ACL_LIST_FAMILY, userPermissionKey(userPerm));
     try {
       t.delete(d);
@@ -348,14 +350,17 @@ public class AccessControlLists {
     removeTablePermissions(tableName, column, t, true);
   }
 
-  static byte[] userPermissionRowKey(UserPermission userPerm) {
+  static byte[] userPermissionRowKey(Permission permission) {
     byte[] row;
-    if(userPerm.hasNamespace()) {
-      row = Bytes.toBytes(toNamespaceEntry(userPerm.getNamespace()));
-    } else if(userPerm.isGlobal()) {
-      row = ACL_GLOBAL_NAME;
+    if (permission instanceof TablePermission) {
+      TablePermission tablePerm = (TablePermission) permission;
+      row = tablePerm.getTableName().getName();
+    } else if (permission instanceof NamespacePermission) {
+      NamespacePermission nsPerm = (NamespacePermission) permission;
+      row = Bytes.toBytes(toNamespaceEntry(nsPerm.getNamespace()));
     } else {
-      row = userPerm.getTableName().getName();
+      // permission instanceof TablePermission
+      row = ACL_GLOBAL_NAME;
     }
     return row;
   }
@@ -366,10 +371,15 @@ public class AccessControlLists {
    *  username,family
    *  username,family,qualifier
    */
-  static byte[] userPermissionKey(UserPermission userPerm) {
-    byte[] qualifier = userPerm.getQualifier();
-    byte[] family = userPerm.getFamily();
-    byte[] key = userPerm.getUser();
+  static byte[] userPermissionKey(UserPermission permission) {
+    byte[] key = Bytes.toBytes(permission.getUser());
+    byte[] qualifier = null;
+    byte[] family = null;
+    if (permission.getPermission().getAccessScope() == Permission.Scope.TABLE) {
+      TablePermission tablePermission = (TablePermission) permission.getPermission();
+      family = tablePermission.getFamily();
+      qualifier = tablePermission.getQualifier();
+    }
 
     if (family != null && family.length > 0) {
       key = Bytes.add(key, Bytes.add(new byte[]{ACL_KEY_DELIMITER}, family));
@@ -404,14 +414,15 @@ public class AccessControlLists {
    * @return a map of the permissions for this table.
    * @throws IOException
    */
-  static Map<byte[], ListMultimap<String,TablePermission>> loadAll(Region aclRegion)
+  static Map<byte[], ListMultimap<String, UserPermission>> loadAll(Region aclRegion)
       throws IOException {
 
     if (!isAclRegion(aclRegion)) {
       throw new IOException("Can only load permissions from "+ACL_TABLE_NAME);
     }
 
-    Map<byte[], ListMultimap<String, TablePermission>> allPerms = new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
+    Map<byte[], ListMultimap<String, UserPermission>> allPerms =
+      new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
 
     // do a full scan of _acl_ table
 
@@ -426,18 +437,18 @@ public class AccessControlLists {
         List<Cell> row = new ArrayList<>();
 
         boolean hasNext = iScanner.next(row);
-        ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
+        ListMultimap<String, UserPermission> perms = ArrayListMultimap.create();
         byte[] entry = null;
         for (Cell kv : row) {
           if (entry == null) {
             entry = CellUtil.cloneRow(kv);
           }
-          Pair<String, TablePermission> permissionsOfUserOnTable =
+          Pair<String, Permission> permissionsOfUserOnTable =
               parsePermissionRecord(entry, kv, null, null, false, null);
           if (permissionsOfUserOnTable != null) {
             String username = permissionsOfUserOnTable.getFirst();
-            TablePermission permissions = permissionsOfUserOnTable.getSecond();
-            perms.put(username, permissions);
+            Permission permission = permissionsOfUserOnTable.getSecond();
+            perms.put(username, new UserPermission(username, permission));
           }
         }
         if (entry != null) {
@@ -460,9 +471,10 @@ public class AccessControlLists {
    * Load all permissions from the region server holding {@code _acl_},
    * primarily intended for testing purposes.
    */
-  static Map<byte[], ListMultimap<String,TablePermission>> loadAll(
+  static Map<byte[], ListMultimap<String, UserPermission>> loadAll(
       Configuration conf) throws IOException {
-    Map<byte[], ListMultimap<String,TablePermission>> allPerms = new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
+    Map<byte[], ListMultimap<String, UserPermission>> allPerms =
+      new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
 
     // do a full scan of _acl_, filtering on only first table region rows
 
@@ -476,7 +488,7 @@ public class AccessControlLists {
         scanner = table.getScanner(scan);
         try {
           for (Result row : scanner) {
-            ListMultimap<String, TablePermission> resultPerms =
+            ListMultimap<String, UserPermission> resultPerms =
                 parsePermissions(row.getRow(), row, null, null, null, false);
             allPerms.put(row.getRow(), resultPerms);
           }
@@ -489,14 +501,14 @@ public class AccessControlLists {
     return allPerms;
   }
 
-  public static ListMultimap<String, TablePermission> getTablePermissions(Configuration conf,
+  public static ListMultimap<String, UserPermission> getTablePermissions(Configuration conf,
       TableName tableName) throws IOException {
     return getPermissions(conf, tableName != null ? tableName.getName() : null, null, null, null,
       null, false);
   }
 
   @VisibleForTesting
-  public static ListMultimap<String, TablePermission> getNamespacePermissions(Configuration conf,
+  public static ListMultimap<String, UserPermission> getNamespacePermissions(Configuration conf,
       String namespace) throws IOException {
     return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null, null, null, null,
       false);
@@ -509,11 +521,11 @@ public class AccessControlLists {
    * See {@link AccessControlLists class documentation} for the key structure used for storage.
    * </p>
    */
-  static ListMultimap<String, TablePermission> getPermissions(Configuration conf, byte[] entryName,
+  static ListMultimap<String, UserPermission> getPermissions(Configuration conf, byte[] entryName,
       Table t, byte[] cf, byte[] cq, String user, boolean hasFilterUser) 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();
+    ListMultimap<String, UserPermission> perms = ArrayListMultimap.create();
     Get get = new Get(entryName);
     get.addFamily(ACL_LIST_FAMILY);
     Result row = null;
@@ -570,25 +582,12 @@ public class AccessControlLists {
    */
   static List<UserPermission> getUserPermissions(Configuration conf, byte[] entryName, byte[] cf,
       byte[] cq, String user, boolean hasFilterUser) throws IOException {
-    ListMultimap<String, TablePermission> allPerms =
+    ListMultimap<String, UserPermission> allPerms =
         getPermissions(conf, entryName, null, cf, cq, user, hasFilterUser);
-
     List<UserPermission> perms = new ArrayList<>();
-    if (isNamespaceEntry(entryName)) { // Namespace
-      for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
-        UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()),
-            entry.getValue().getNamespace(), entry.getValue().getActions());
-        perms.add(up);
-      }
-    } else { // Table
-      for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
-        UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()),
-            entry.getValue().getTableName(), entry.getValue().getFamily(),
-            entry.getValue().getQualifier(), entry.getValue().getActions());
-        perms.add(up);
-      }
+    for (Map.Entry<String, UserPermission> entry : allPerms.entries()) {
+      perms.add(entry.getValue());
     }
-
     return perms;
   }
 
@@ -596,25 +595,25 @@ public class AccessControlLists {
    * Parse and filter permission based on the specified column family, column qualifier and user
    * name.
    */
-  private static ListMultimap<String, TablePermission> parsePermissions(byte[] entryName,
+  private static ListMultimap<String, UserPermission> parsePermissions(byte[] entryName,
       Result result, byte[] cf, byte[] cq, String user, boolean hasFilterUser) {
-    ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
+    ListMultimap<String, UserPermission> perms = ArrayListMultimap.create();
     if (result != null && result.size() > 0) {
       for (Cell kv : result.rawCells()) {
-        Pair<String, TablePermission> permissionsOfUserOnTable =
+        Pair<String, Permission> permissionsOfUserOnTable =
             parsePermissionRecord(entryName, kv, cf, cq, hasFilterUser, user);
 
         if (permissionsOfUserOnTable != null) {
           String username = permissionsOfUserOnTable.getFirst();
-          TablePermission permissions = permissionsOfUserOnTable.getSecond();
-          perms.put(username, permissions);
+          Permission permission = permissionsOfUserOnTable.getSecond();
+          perms.put(username, new UserPermission(username, permission));
         }
       }
     }
     return perms;
   }
 
-  private static Pair<String, TablePermission> parsePermissionRecord(byte[] entryName, Cell kv,
+  private static Pair<String, Permission> parsePermissionRecord(byte[] entryName, Cell kv,
       byte[] cf, byte[] cq, boolean filterPerms, String filterUser) {
     // return X given a set of permissions encoded in the permissionRecord kv.
     byte[] family = CellUtil.cloneFamily(kv);
@@ -625,9 +624,10 @@ public class AccessControlLists {
     byte[] key = CellUtil.cloneQualifier(kv);
     byte[] value = CellUtil.cloneValue(kv);
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Read acl: kv ["+
-          Bytes.toStringBinary(key)+": "+
-          Bytes.toStringBinary(value)+"]");
+      LOG.debug("Read acl: entry[" +
+        Bytes.toStringBinary(entryName) + "], kv [" +
+        Bytes.toStringBinary(key) + ": " +
+        Bytes.toStringBinary(value)+"]");
     }
 
     // check for a column family appended to the key
@@ -652,11 +652,20 @@ public class AccessControlLists {
       }
 
       return new Pair<>(username,
-          new TablePermission(Bytes.toString(fromNamespaceEntry(entryName)), value));
+        new NamespacePermission(Bytes.toString(fromNamespaceEntry(entryName)), value));
     }
 
-    //Handle table and global entry
-    //TODO global entry should be handled differently
+    // Handle global entry
+    if (isGlobalEntry(entryName)) {
+      // Filter the permissions cell record if client query
+      if (filterPerms && !validateFilterUser(username, filterUser, filterUserGroups)) {
+        return null;
+      }
+
+      return new Pair<>(username, new GlobalPermission(value));
+    }
+
+    // Handle table entry
     int idx = username.indexOf(ACL_KEY_DELIMITER);
     byte[] permFamily = null;
     byte[] permQualifier = null;
@@ -738,7 +747,7 @@ public class AccessControlLists {
    * Writes a set of permissions as {@link org.apache.hadoop.io.Writable} instances and returns the
    * resulting byte array. Writes a set of permission [user: table permission]
    */
-  public static byte[] writePermissionsAsBytes(ListMultimap<String, TablePermission> perms,
+  public static byte[] writePermissionsAsBytes(ListMultimap<String, UserPermission> perms,
       Configuration conf) {
     return ProtobufUtil.prependPBMagic(AccessControlUtil.toUserTablePermissions(perms).toByteArray());
   }
@@ -750,11 +759,11 @@ public class AccessControlLists {
 
   private static final int WRITABLE_NOT_ENCODED = 0;
 
-  private static List<TablePermission> readWritablePermissions(DataInput in, Configuration conf)
-      throws IOException, ClassNotFoundException {
+  private static List<Permission> readWritableUserPermission(DataInput in,
+      Configuration conf) throws IOException, ClassNotFoundException {
     assert WritableUtils.readVInt(in) == LIST_CODE;
     int length = in.readInt();
-    List<TablePermission> list = new ArrayList<>(length);
+    List<Permission> list = new ArrayList<>(length);
     for (int i = 0; i < length; i++) {
       assert WritableUtils.readVInt(in) == WRITABLE_CODE;
       assert WritableUtils.readVInt(in) == WRITABLE_NOT_ENCODED;
@@ -762,38 +771,67 @@ public class AccessControlLists {
       Class<? extends Writable> clazz = conf.getClassByName(className).asSubclass(Writable.class);
       Writable instance = WritableFactories.newInstance(clazz, conf);
       instance.readFields(in);
-      list.add((TablePermission) instance);
+      list.add((Permission) instance);
     }
     return list;
   }
 
-  /**
-   * Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances from the input
-   * stream.
-   */
-  public static ListMultimap<String, TablePermission> readPermissions(byte[] data,
+  @VisibleForTesting
+  public static ListMultimap<String, UserPermission> readUserPermission(byte[] data,
       Configuration conf) throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(data)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
       try {
         AccessControlProtos.UsersAndPermissions.Builder builder =
-            AccessControlProtos.UsersAndPermissions.newBuilder();
+          AccessControlProtos.UsersAndPermissions.newBuilder();
         ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
-        return AccessControlUtil.toUserTablePermissions(builder.build());
+        return AccessControlUtil.toUserPermission(builder.build());
       } catch (IOException e) {
         throw new DeserializationException(e);
       }
     } else {
       // TODO: We have to re-write non-PB data as PB encoded. Otherwise we will carry old Writables
       // forever (here and a couple of other places).
-      ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
+      ListMultimap<String, UserPermission> userPermission = ArrayListMultimap.create();
       try {
         DataInput in = new DataInputStream(new ByteArrayInputStream(data));
         int length = in.readInt();
         for (int i = 0; i < length; i++) {
           String user = Text.readString(in);
-          List<TablePermission> userPerms = readWritablePermissions(in, conf);
-          perms.putAll(user, userPerms);
+          List<Permission> perms = readWritableUserPermission(in, conf);
+          for (Permission p : perms) {
+            userPermission.put(user, new UserPermission(user, p));
+          }
+        }
+      } catch (IOException | ClassNotFoundException e) {
+        throw new DeserializationException(e);
+      }
+      return userPermission;
+    }
+  }
+
+  public static ListMultimap<String, Permission> readPermissions(byte[] data,
+      Configuration conf) throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(data)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      try {
+        AccessControlProtos.UsersAndPermissions.Builder builder =
+          AccessControlProtos.UsersAndPermissions.newBuilder();
+        ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
+        return AccessControlUtil.toPermission(builder.build());
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+    } else {
+      // TODO: We have to re-write non-PB data as PB encoded. Otherwise we will carry old Writables
+      // forever (here and a couple of other places).
+      ListMultimap<String, Permission> perms = ArrayListMultimap.create();
+      try {
+        DataInput in = new DataInputStream(new ByteArrayInputStream(data));
+        int length = in.readInt();
+        for (int i = 0; i < length; i++) {
+          String user = Text.readString(in);
+          perms.putAll(user, readWritableUserPermission(in, conf));
         }
       } catch (IOException | ClassNotFoundException e) {
         throw new DeserializationException(e);
@@ -802,6 +840,10 @@ public class AccessControlLists {
     }
   }
 
+  public static boolean isGlobalEntry(byte[] entryName) {
+    return entryName != null && TableName.valueOf(entryName).equals(ACL_TABLE_NAME);
+  }
+
   public static boolean isNamespaceEntry(String entryName) {
     return entryName != null && entryName.charAt(0) == NAMESPACE_PREFIX;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/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 1100500..1a84bfd 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
@@ -237,20 +237,20 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     return regionEnv != null ? regionEnv.getRegion() : null;
   }
 
-  public TableAuthManager getAuthManager() {
+  public AuthManager getAuthManager() {
     return accessChecker.getAuthManager();
   }
 
   private void initialize(RegionCoprocessorEnvironment e) throws IOException {
     final Region region = e.getRegion();
     Configuration conf = e.getConfiguration();
-    Map<byte[], ListMultimap<String, TablePermission>> tables = AccessControlLists.loadAll(region);
+    Map<byte[], ListMultimap<String, UserPermission>> tables = AccessControlLists.loadAll(region);
     // For each table, write out the table's permissions to the respective
     // znode for that table.
-    for (Map.Entry<byte[], ListMultimap<String,TablePermission>> t:
+    for (Map.Entry<byte[], ListMultimap<String, UserPermission>> t:
       tables.entrySet()) {
       byte[] entry = t.getKey();
-      ListMultimap<String,TablePermission> perms = t.getValue();
+      ListMultimap<String, UserPermission> perms = t.getValue();
       byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
       getAuthManager().getZKPermissionWatcher().writeToZookeeper(entry, serialized);
     }
@@ -286,7 +286,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     try (Table t = e.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME)) {
       for (byte[] entry : entries) {
         currentEntry = entry;
-        ListMultimap<String, TablePermission> perms =
+        ListMultimap<String, UserPermission> perms =
             AccessControlLists.getPermissions(conf, entry, t, null, null, null, false);
         byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
         zkw.writeToZookeeper(entry, serialized);
@@ -330,7 +330,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     }
 
     // 2. check for the table-level, if successful we can short-circuit
-    if (getAuthManager().authorize(user, tableName, (byte[])null, permRequest)) {
+    if (getAuthManager().authorizeUserTable(user, tableName, permRequest)) {
       return AuthResult.allow(request, "Table permission granted", user,
         permRequest, tableName, families);
     }
@@ -340,7 +340,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       // all families must pass
       for (Map.Entry<byte [], ? extends Collection<?>> family : families.entrySet()) {
         // a) check for family level access
-        if (getAuthManager().authorize(user, tableName, family.getKey(),
+        if (getAuthManager().authorizeUserTable(user, tableName, family.getKey(),
             permRequest)) {
           continue;  // family-level permission overrides per-qualifier
         }
@@ -351,17 +351,17 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
             // for each qualifier of the family
             Set<byte[]> familySet = (Set<byte[]>)family.getValue();
             for (byte[] qualifier : familySet) {
-              if (!getAuthManager().authorize(user, tableName, family.getKey(),
-                                         qualifier, permRequest)) {
+              if (!getAuthManager().authorizeUserTable(user, tableName,
+                    family.getKey(), qualifier, permRequest)) {
                 return AuthResult.deny(request, "Failed qualifier check", user,
-                    permRequest, tableName, makeFamilyMap(family.getKey(), qualifier));
+                  permRequest, tableName, makeFamilyMap(family.getKey(), qualifier));
               }
             }
           } else if (family.getValue() instanceof List) { // List<Cell>
             List<Cell> cellList = (List<Cell>)family.getValue();
             for (Cell cell : cellList) {
-              if (!getAuthManager().authorize(user, tableName, family.getKey(),
-                CellUtil.cloneQualifier(cell), permRequest)) {
+              if (!getAuthManager().authorizeUserTable(user, tableName, family.getKey(),
+                  CellUtil.cloneQualifier(cell), permRequest)) {
                 return AuthResult.deny(request, "Failed qualifier check", user, permRequest,
                   tableName, makeFamilyMap(family.getKey(), CellUtil.cloneQualifier(cell)));
               }
@@ -370,7 +370,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         } else {
           // no qualifiers and family-level check already failed
           return AuthResult.deny(request, "Failed family check", user, permRequest,
-              tableName, makeFamilyMap(family.getKey(), null));
+            tableName, makeFamilyMap(family.getKey(), null));
         }
       }
 
@@ -487,14 +487,13 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
           familyMap.entrySet()) {
         if (family.getValue() != null && !family.getValue().isEmpty()) {
           for (byte[] qualifier : family.getValue()) {
-            if (getAuthManager().matchPermission(user, tableName,
-                family.getKey(), qualifier, perm)) {
+            if (getAuthManager().authorizeUserTable(user, tableName,
+                  family.getKey(), qualifier, perm)) {
               return true;
             }
           }
         } else {
-          if (getAuthManager().matchPermission(user, tableName, family.getKey(),
-              perm)) {
+          if (getAuthManager().authorizeUserFamily(user, tableName, family.getKey(), perm)) {
             return true;
           }
         }
@@ -683,7 +682,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
           foundColumn = true;
           for (Action action: actions) {
             // Are there permissions for this user for the cell?
-            if (!getAuthManager().authorize(user, getTableName(e), cell, action)) {
+            if (!getAuthManager().authorizeCell(user, getTableName(e), cell, action)) {
               // We can stop if the cell ACL denies access
               return false;
             }
@@ -798,7 +797,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
     // set the user-provider.
     this.userProvider = UserProvider.instantiate(env.getConfiguration());
-    // Throws RuntimeException if fails to load TableAuthManager so that coprocessor is unloaded.
+    // Throws RuntimeException if fails to load AuthManager so that coprocessor is unloaded.
     accessChecker = new AccessChecker(env.getConfiguration(), zk);
     tableAcls = new MapMaker().weakValues().makeMap();
   }
@@ -886,8 +885,8 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         // default the table owner to current user, if not specified.
         if (owner == null)
           owner = getActiveUser(c).getShortName();
-        final UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
-            desc.getTableName(), null, Action.values());
+        final UserPermission userPermission = new UserPermission(owner,
+          desc.getTableName(), Action.values());
         // switch to the real hbase master user for doing the RPC on the ACL table
         User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
           @Override
@@ -895,7 +894,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
             try (Table table = c.getEnvironment().getConnection().
                 getTable(AccessControlLists.ACL_TABLE_NAME)) {
               AccessControlLists.addUserPermission(c.getEnvironment().getConfiguration(),
-                  userperm, table);
+                userPermission, table);
             }
             return null;
           }
@@ -988,8 +987,8 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
-            currentDesc.getTableName(), null, Action.values());
+        UserPermission userperm = new UserPermission(owner,
+          currentDesc.getTableName(), Action.values());
         try (Table table = c.getEnvironment().getConnection().
             getTable(AccessControlLists.ACL_TABLE_NAME)) {
           AccessControlLists.addUserPermission(conf, userperm, table);
@@ -2044,19 +2043,24 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         if (!initialized) {
           throw new CoprocessorException("AccessController not yet initialized");
         }
+        User caller = RpcServer.getRequestUser().orElse(null);
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Received request to grant access permission " + perm.toString());
+          LOG.debug("Received request from {} to grant access permission {}",
+            caller.getName(), perm.toString());
         }
-        User caller = RpcServer.getRequestUser().orElse(null);
 
         switch(request.getUserPermission().getPermission().getType()) {
           case Global :
+            accessChecker.requireGlobalPermission(caller, "grant", Action.ADMIN, "");
+            break;
           case Table :
-            accessChecker.requirePermission(caller, "grant", perm.getTableName(),
-                perm.getFamily(), perm.getQualifier(), null, Action.ADMIN);
+            TablePermission tablePerm = (TablePermission) perm.getPermission();
+            accessChecker.requirePermission(caller, "grant", tablePerm.getTableName(),
+              tablePerm.getFamily(), tablePerm.getQualifier(), null, Action.ADMIN);
             break;
           case Namespace :
-            accessChecker.requireNamespacePermission(caller, "grant", perm.getNamespace(),
+            NamespacePermission namespacePer = (NamespacePermission) perm.getPermission();
+            accessChecker.requireNamespacePermission(caller, "grant", namespacePer.getNamespace(),
                 null, Action.ADMIN);
            break;
         }
@@ -2102,20 +2106,25 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         if (!initialized) {
           throw new CoprocessorException("AccessController not yet initialized");
         }
+        User caller = RpcServer.getRequestUser().orElse(null);
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Received request to revoke access permission " + perm.toString());
+          LOG.debug("Received request from {} to revoke access permission {}",
+            caller.getShortName(), perm.toString());
         }
-        User caller = RpcServer.getRequestUser().orElse(null);
 
         switch(request.getUserPermission().getPermission().getType()) {
           case Global :
+            accessChecker.requireGlobalPermission(caller, "revoke", Action.ADMIN, "");
+            break;
           case Table :
-            accessChecker.requirePermission(caller, "revoke", perm.getTableName(), perm.getFamily(),
-              perm.getQualifier(), null, Action.ADMIN);
+            TablePermission tablePerm = (TablePermission) perm.getPermission();
+            accessChecker.requirePermission(caller, "revoke", tablePerm.getTableName(),
+              tablePerm.getFamily(), tablePerm.getQualifier(), null, Action.ADMIN);
             break;
           case Namespace :
-            accessChecker.requireNamespacePermission(caller, "revoke", perm.getNamespace(),
-                null, Action.ADMIN);
+            NamespacePermission namespacePer = (NamespacePermission) perm.getPermission();
+            accessChecker.requireNamespacePermission(caller, "revoke",
+              namespacePer.getNamespace(), null, Action.ADMIN);
             break;
         }
 
@@ -2189,7 +2198,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         } else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
           final String namespace = request.getNamespaceName().toStringUtf8();
           accessChecker.requireNamespacePermission(caller, "userPermissions",
-            namespace,userName, Action.ADMIN);
+            namespace, userName, Action.ADMIN);
           perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
             @Override
             public List<UserPermission> run() throws Exception {
@@ -2225,8 +2234,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
             // them. Also using acl as table name to be inline with the results of global admin and
             // will help in avoiding any leakage of information about being superusers.
             for (String user : Superusers.getSuperUsers()) {
-              perms.add(new UserPermission(Bytes.toBytes(user), AccessControlLists.ACL_TABLE_NAME,
-                  null, Action.values()));
+              perms.add(new UserPermission(user, Action.values()));
             }
           }
         }
@@ -2295,7 +2303,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
           for (Action action : permission.getActions()) {
             AuthResult result;
-            if (getAuthManager().authorize(user, action)) {
+            if (getAuthManager().authorizeUserGlobal(user, action)) {
               result = AuthResult.allow("checkPermissions", "Global action allowed", user,
                 action, null, null);
             } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthManager.java
new file mode 100644
index 0000000..8da9a82
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AuthManager.java
@@ -0,0 +1,608 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.log.HBaseMarkers;
+import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
+/**
+ * Performs authorization checks for a given user's assigned permissions.
+ * <p>
+ *   There're following scopes: <b>Global</b>, <b>Namespace</b>, <b>Table</b>, <b>Family</b>,
+ *   <b>Qualifier</b>, <b>Cell</b>.
+ *   Generally speaking, higher scopes can overrides lower scopes,
+ *   except for Cell permission can be granted even a user has not permission on specified table,
+ *   which means the user can get/scan only those granted cells parts.
+ * </p>
+ * e.g, if user A has global permission R(ead), he can
+ * read table T without checking table scope permission, so authorization checks alway starts from
+ * Global scope.
+ * <p>
+ *   For each scope, not only user but also groups he belongs to will be checked.
+ * </p>
+ */
+@InterfaceAudience.Private
+public final class AuthManager implements Closeable {
+
+  /**
+   * Cache of permissions, it is thread safe.
+   * @param <T> T extends Permission
+   */
+  private static class PermissionCache<T extends Permission> {
+    private final Object mutex = new Object();
+    private Map<String, Set<T>> cache = new HashMap<>();
+
+    void put(String name, T perm) {
+      synchronized (mutex) {
+        Set<T> perms = cache.getOrDefault(name, new HashSet<>());
+        perms.add(perm);
+        cache.put(name, perms);
+      }
+    }
+
+    Set<T> get(String name) {
+      synchronized (mutex) {
+        return cache.get(name);
+      }
+    }
+
+    void clear() {
+      synchronized (mutex) {
+        for (Map.Entry<String, Set<T>> entry : cache.entrySet()) {
+          entry.getValue().clear();
+        }
+        cache.clear();
+      }
+    }
+  }
+  PermissionCache<NamespacePermission> NS_NO_PERMISSION = new PermissionCache<>();
+  PermissionCache<TablePermission> TBL_NO_PERMISSION = new PermissionCache<>();
+
+  /**
+   * Cache for global permission.
+   * Since every user/group can only have one global permission, no need to user PermissionCache.
+   */
+  private volatile Map<String, GlobalPermission> globalCache;
+  /** Cache for namespace permission. */
+  private ConcurrentHashMap<String, PermissionCache<NamespacePermission>> namespaceCache =
+    new ConcurrentHashMap<>();
+  /** Cache for table permission. */
+  private ConcurrentHashMap<TableName, PermissionCache<TablePermission>> tableCache =
+    new ConcurrentHashMap<>();
+
+  private static final Logger LOG = LoggerFactory.getLogger(AuthManager.class);
+
+  private Configuration conf;
+  private ZKPermissionWatcher zkperms;
+  private final AtomicLong mtime = new AtomicLong(0L);
+
+  private AuthManager(ZKWatcher watcher, Configuration conf)
+      throws IOException {
+    this.conf = conf;
+    // initialize global permissions based on configuration
+    globalCache = initGlobal(conf);
+
+    this.zkperms = new ZKPermissionWatcher(watcher, this, conf);
+    try {
+      this.zkperms.start();
+    } catch (KeeperException ke) {
+      LOG.error("ZooKeeper initialization failed", ke);
+    }
+  }
+
+  @Override
+  public void close() {
+    this.zkperms.close();
+  }
+
+  /**
+   * Initialize with global permission assignments
+   * from the {@code hbase.superuser} configuration key.
+   */
+  private Map<String, GlobalPermission> initGlobal(Configuration conf) throws IOException {
+    UserProvider userProvider = UserProvider.instantiate(conf);
+    User user = userProvider.getCurrent();
+    if (user == null) {
+      throw new IOException("Unable to obtain the current user, " +
+        "authorization checks for internal operations will not work correctly!");
+    }
+    String currentUser = user.getShortName();
+
+    Map<String, GlobalPermission> global = new HashMap<>();
+    // the system user is always included
+    List<String> superusers = Lists.asList(currentUser, conf.getStrings(
+        Superusers.SUPERUSER_CONF_KEY, new String[0]));
+    for (String name : superusers) {
+      GlobalPermission globalPermission = new GlobalPermission(Permission.Action.values());
+      global.put(name, globalPermission);
+    }
+    return global;
+  }
+
+  public ZKPermissionWatcher getZKPermissionWatcher() {
+    return this.zkperms;
+  }
+
+  /**
+   * Update acl info for table.
+   * @param table name of table
+   * @param data updated acl data
+   * @throws IOException exception when deserialize data
+   */
+  public void refreshTableCacheFromWritable(TableName table, byte[] data) throws IOException {
+    if (data != null && data.length > 0) {
+      try {
+        ListMultimap<String, Permission> perms =
+          AccessControlLists.readPermissions(data, conf);
+        if (perms != null) {
+          if (Bytes.equals(table.getName(), AccessControlLists.ACL_GLOBAL_NAME)) {
+            updateGlobalCache(perms);
+          } else {
+            updateTableCache(table, perms);
+          }
+        }
+      } catch (DeserializationException e) {
+        throw new IOException(e);
+      }
+    } else {
+      LOG.info("Skipping permission cache refresh because writable data is empty");
+    }
+  }
+
+  /**
+   * Update acl info for namespace.
+   * @param namespace namespace
+   * @param data updated acl data
+   * @throws IOException exception when deserialize data
+   */
+  public void refreshNamespaceCacheFromWritable(String namespace, byte[] data) throws IOException {
+    if (data != null && data.length > 0) {
+      try {
+        ListMultimap<String, Permission> perms =
+          AccessControlLists.readPermissions(data, conf);
+        if (perms != null) {
+          updateNamespaceCache(namespace, perms);
+        }
+      } catch (DeserializationException e) {
+        throw new IOException(e);
+      }
+    } else {
+      LOG.debug("Skipping permission cache refresh because writable data is empty");
+    }
+  }
+
+  /**
+   * Updates the internal global permissions cache.
+   * @param globalPerms new global permissions
+   */
+  private void updateGlobalCache(ListMultimap<String, Permission> globalPerms) {
+    try {
+      Map<String, GlobalPermission> global = initGlobal(conf);
+      for (String name : globalPerms.keySet()) {
+        for (Permission permission : globalPerms.get(name)) {
+          global.put(name, (GlobalPermission) permission);
+        }
+      }
+      globalCache = global;
+      mtime.incrementAndGet();
+    } catch (Exception e) {
+      // Never happens
+      LOG.error("Error occurred while updating the global cache", e);
+    }
+  }
+
+  /**
+   * Updates the internal table permissions cache for specified table.
+   * @param table updated table name
+   * @param tablePerms new table permissions
+   */
+  private void updateTableCache(TableName table, ListMultimap<String, Permission> tablePerms) {
+    PermissionCache<TablePermission> cacheToUpdate =
+      tableCache.getOrDefault(table, new PermissionCache<>());
+    clearCache(cacheToUpdate);
+    updateCache(tablePerms, cacheToUpdate);
+    tableCache.put(table, cacheToUpdate);
+    mtime.incrementAndGet();
+  }
+
+  /**
+   * Updates the internal namespace permissions cache for specified namespace.
+   * @param namespace updated namespace
+   * @param nsPerms new namespace permissions
+   */
+  private void updateNamespaceCache(String namespace,
+      ListMultimap<String, Permission> nsPerms) {
+    PermissionCache<NamespacePermission> cacheToUpdate =
+      namespaceCache.getOrDefault(namespace, new PermissionCache<>());
+    clearCache(cacheToUpdate);
+    updateCache(nsPerms, cacheToUpdate);
+    namespaceCache.put(namespace, cacheToUpdate);
+    mtime.incrementAndGet();
+  }
+
+  private void clearCache(PermissionCache cacheToUpdate) {
+    cacheToUpdate.clear();
+  }
+
+  @SuppressWarnings("unchecked")
+  private void updateCache(ListMultimap<String, ? extends Permission> newPermissions,
+      PermissionCache cacheToUpdate) {
+    for (String name : newPermissions.keySet()) {
+      for (Permission permission : newPermissions.get(name)) {
+        cacheToUpdate.put(name, permission);
+      }
+    }
+  }
+
+  /**
+   * Check if user has given action privilige in global scope.
+   * @param user user name
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserGlobal(User user, Permission.Action action) {
+    if (user == null) {
+      return false;
+    }
+    if (authorizeGlobal(globalCache.get(user.getShortName()), action)) {
+      return true;
+    }
+    for (String group : user.getGroupNames()) {
+      if (authorizeGlobal(globalCache.get(AuthUtil.toGroupEntry(group)), action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean authorizeGlobal(GlobalPermission permissions, Permission.Action action) {
+    return permissions != null && permissions.implies(action);
+  }
+
+  /**
+   * Check if user has given action privilige in namespace scope.
+   * @param user user name
+   * @param namespace namespace
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserNamespace(User user, String namespace, Permission.Action action) {
+    if (user == null) {
+      return false;
+    }
+    if (authorizeUserGlobal(user, action)) {
+      return true;
+    }
+    PermissionCache<NamespacePermission> nsPermissions = namespaceCache.getOrDefault(namespace,
+      NS_NO_PERMISSION);
+    if (authorizeNamespace(nsPermissions.get(user.getShortName()), namespace, action)) {
+      return true;
+    }
+    for (String group : user.getGroupNames()) {
+      if (authorizeNamespace(nsPermissions.get(AuthUtil.toGroupEntry(group)), namespace, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean authorizeNamespace(Set<NamespacePermission> permissions,
+      String namespace, Permission.Action action) {
+    if (permissions == null) {
+      return false;
+    }
+    for (NamespacePermission permission : permissions) {
+      if (permission.implies(namespace, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Checks if the user has access to the full table or at least a family/qualifier
+   * for the specified action.
+   * @param user user name
+   * @param table table name
+   * @param action action in one of [Read, Write, Create, Exec, Admin]
+   * @return true if the user has access to the table, false otherwise
+   */
+  public boolean accessUserTable(User user, TableName table, Permission.Action action) {
+    if (user == null) {
+      return false;
+    }
+    if (table == null) {
+      table = AccessControlLists.ACL_TABLE_NAME;
+    }
+    if (authorizeUserNamespace(user, table.getNamespaceAsString(), action)) {
+      return true;
+    }
+    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
+      TBL_NO_PERMISSION);
+    if (hasAccessTable(tblPermissions.get(user.getShortName()), action)) {
+      return true;
+    }
+    for (String group : user.getGroupNames()) {
+      if (hasAccessTable(tblPermissions.get(AuthUtil.toGroupEntry(group)), action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean hasAccessTable(Set<TablePermission> permissions, Permission.Action action) {
+    if (permissions == null) {
+      return false;
+    }
+    for (TablePermission permission : permissions) {
+      if (permission.implies(action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check if user has given action privilige in table scope.
+   * @param user user name
+   * @param table table name
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserTable(User user, TableName table, Permission.Action action) {
+    return authorizeUserTable(user, table, null, null, action);
+  }
+
+  /**
+   * Check if user has given action privilige in table:family scope.
+   * @param user user name
+   * @param table table name
+   * @param family family name
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserTable(User user, TableName table, byte[] family,
+      Permission.Action action) {
+    return authorizeUserTable(user, table, family, null, action);
+  }
+
+  /**
+   * Check if user has given action privilige in table:family:qualifier scope.
+   * @param user user name
+   * @param table table name
+   * @param family family name
+   * @param qualifier qualifier name
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserTable(User user, TableName table, byte[] family,
+      byte[] qualifier, Permission.Action action) {
+    if (user == null) {
+      return false;
+    }
+    if (table == null) {
+      table = AccessControlLists.ACL_TABLE_NAME;
+    }
+    if (authorizeUserNamespace(user, table.getNamespaceAsString(), action)) {
+      return true;
+    }
+    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
+      TBL_NO_PERMISSION);
+    if (authorizeTable(tblPermissions.get(user.getShortName()), table, family, qualifier, action)) {
+      return true;
+    }
+    for (String group : user.getGroupNames()) {
+      if (authorizeTable(tblPermissions.get(AuthUtil.toGroupEntry(group)),
+          table, family, qualifier, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean authorizeTable(Set<TablePermission> permissions,
+      TableName table, byte[] family, byte[] qualifier, Permission.Action action) {
+    if (permissions == null) {
+      return false;
+    }
+    for (TablePermission permission : permissions) {
+      if (permission.implies(table, family, qualifier, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check if user has given action privilige in table:family scope.
+   * This method is for backward compatibility.
+   * @param user user name
+   * @param table table name
+   * @param family family names
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeUserFamily(User user, TableName table,
+      byte[] family, Permission.Action action) {
+    PermissionCache<TablePermission> tblPermissions = tableCache.getOrDefault(table,
+      TBL_NO_PERMISSION);
+    if (authorizeFamily(tblPermissions.get(user.getShortName()), table, family, action)) {
+      return true;
+    }
+    for (String group : user.getGroupNames()) {
+      if (authorizeFamily(tblPermissions.get(AuthUtil.toGroupEntry(group)),
+          table, family, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean authorizeFamily(Set<TablePermission> permissions,
+      TableName table, byte[] family, Permission.Action action) {
+    if (permissions == null) {
+      return false;
+    }
+    for (TablePermission permission : permissions) {
+      if (permission.implies(table, family, action)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Check if user has given action privilige in cell scope.
+   * @param user user name
+   * @param table table name
+   * @param cell cell to be checked
+   * @param action one of action in [Read, Write, Create, Exec, Admin]
+   * @return true if user has, false otherwise
+   */
+  public boolean authorizeCell(User user, TableName table, Cell cell, Permission.Action action) {
+    try {
+      List<Permission> perms = AccessControlLists.getCellPermissionsForUser(user, cell);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Perms for user " + user.getShortName() + " in cell " + cell + ": " +
+          (perms != null ? perms : ""));
+      }
+      if (perms != null) {
+        for (Permission p: perms) {
+          if (p.implies(action)) {
+            return true;
+          }
+        }
+      }
+    } catch (IOException e) {
+      // We failed to parse the KV tag
+      LOG.error("Failed parse of ACL tag in cell " + cell);
+      // Fall through to check with the table and CF perms we were able
+      // to collect regardless
+    }
+    return false;
+  }
+
+  /**
+   * Remove given namespace from AuthManager's namespace cache.
+   * @param ns namespace
+   */
+  public void removeNamespace(byte[] ns) {
+    namespaceCache.remove(Bytes.toString(ns));
+  }
+
+  /**
+   * Remove given table from AuthManager's table cache.
+   * @param table table name
+   */
+  public void removeTable(TableName table) {
+    tableCache.remove(table);
+  }
+
+  /**
+   * Last modification logical time
+   * @return time
+   */
+  public long getMTime() {
+    return mtime.get();
+  }
+
+  private static Map<ZKWatcher, AuthManager> managerMap = new HashMap<>();
+
+  private static Map<AuthManager, Integer> refCount = new HashMap<>();
+
+  /**
+   * Returns a AuthManager from the cache. If not cached, constructs a new one.
+   * Returned instance should be released back by calling {@link #release(AuthManager)}.
+   * @param watcher zk watcher
+   * @param conf configuration
+   * @return an AuthManager
+   * @throws IOException zookeeper initialization failed
+   */
+  public synchronized static AuthManager getOrCreate(
+      ZKWatcher watcher, Configuration conf) throws IOException {
+    AuthManager instance = managerMap.get(watcher);
+    if (instance == null) {
+      instance = new AuthManager(watcher, conf);
+      managerMap.put(watcher, instance);
+    }
+    int ref = refCount.get(instance) == null ? 0 : refCount.get(instance);
+    refCount.put(instance, ref + 1);
+    return instance;
+  }
+
+  @VisibleForTesting
+  public static int getTotalRefCount() {
+    int total = 0;
+    for (int count : refCount.values()) {
+      total += count;
+    }
+    return total;
+  }
+
+  /**
+   * Releases the resources for the given AuthManager if the reference count is down to 0.
+   * @param instance AuthManager to be released
+   */
+  public synchronized static void release(AuthManager instance) {
+    if (refCount.get(instance) == null || refCount.get(instance) < 1) {
+      String msg = "Something wrong with the AuthManager reference counting: " + instance
+          + " whose count is " + refCount.get(instance);
+      LOG.error(HBaseMarkers.FATAL, msg);
+      instance.close();
+      managerMap.remove(instance.getZKPermissionWatcher().getWatcher());
+      instance.getZKPermissionWatcher().getWatcher().abort(msg, null);
+    } else {
+      int ref = refCount.get(instance);
+      --ref;
+      refCount.put(instance, ref);
+      if (ref == 0) {
+        instance.close();
+        managerMap.remove(instance.getZKPermissionWatcher().getWatcher());
+        refCount.remove(instance);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
deleted file mode 100644
index 76feff4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ /dev/null
@@ -1,787 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.security.access;
-
-import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
-import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Performs authorization checks for a given user's assigned permissions
- */
-@InterfaceAudience.Private
-public class TableAuthManager implements Closeable {
-  private static class PermissionCache<T extends Permission> {
-    /** Cache of user permissions */
-    private ListMultimap<String,T> userCache = ArrayListMultimap.create();
-    /** Cache of group permissions */
-    private ListMultimap<String,T> groupCache = ArrayListMultimap.create();
-
-    public List<T> getUser(String user) {
-      return userCache.get(user);
-    }
-
-    public void putUser(String user, T perm) {
-      userCache.put(user, perm);
-    }
-
-    public List<T> replaceUser(String user, Iterable<? extends T> perms) {
-      return userCache.replaceValues(user, perms);
-    }
-
-    public List<T> getGroup(String group) {
-      return groupCache.get(group);
-    }
-
-    public void putGroup(String group, T perm) {
-      groupCache.put(group, perm);
-    }
-
-    public List<T> replaceGroup(String group, Iterable<? extends T> perms) {
-      return groupCache.replaceValues(group, perms);
-    }
-
-    /**
-     * Returns a combined map of user and group permissions, with group names
-     * distinguished according to {@link AuthUtil#isGroupPrincipal(String)}.
-     */
-    public ListMultimap<String,T> getAllPermissions() {
-      ListMultimap<String,T> tmp = ArrayListMultimap.create();
-      tmp.putAll(userCache);
-      for (String group : groupCache.keySet()) {
-        tmp.putAll(AuthUtil.toGroupEntry(group), groupCache.get(group));
-      }
-      return tmp;
-    }
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(TableAuthManager.class);
-
-  /** Cache of global permissions */
-  private volatile PermissionCache<Permission> globalCache;
-
-  private ConcurrentSkipListMap<TableName, PermissionCache<TablePermission>> tableCache =
-      new ConcurrentSkipListMap<>();
-
-  private ConcurrentSkipListMap<String, PermissionCache<TablePermission>> nsCache =
-    new ConcurrentSkipListMap<>();
-
-  private Configuration conf;
-  private ZKPermissionWatcher zkperms;
-  private final AtomicLong mtime = new AtomicLong(0L);
-
-  private TableAuthManager(ZKWatcher watcher, Configuration conf)
-      throws IOException {
-    this.conf = conf;
-
-    // initialize global permissions based on configuration
-    globalCache = initGlobal(conf);
-
-    this.zkperms = new ZKPermissionWatcher(watcher, this, conf);
-    try {
-      this.zkperms.start();
-    } catch (KeeperException ke) {
-      LOG.error("ZooKeeper initialization failed", ke);
-    }
-  }
-
-  @Override
-  public void close() {
-    this.zkperms.close();
-  }
-
-  /**
-   * Returns a new {@code PermissionCache} initialized with permission assignments
-   * from the {@code hbase.superuser} configuration key.
-   */
-  private PermissionCache<Permission> initGlobal(Configuration conf) throws IOException {
-    UserProvider userProvider = UserProvider.instantiate(conf);
-    User user = userProvider.getCurrent();
-    if (user == null) {
-      throw new IOException("Unable to obtain the current user, " +
-          "authorization checks for internal operations will not work correctly!");
-    }
-    PermissionCache<Permission> newCache = new PermissionCache<>();
-    String currentUser = user.getShortName();
-
-    // the system user is always included
-    List<String> superusers = Lists.asList(currentUser, conf.getStrings(
-        Superusers.SUPERUSER_CONF_KEY, new String[0]));
-    if (superusers != null) {
-      for (String name : superusers) {
-        if (AuthUtil.isGroupPrincipal(name)) {
-          newCache.putGroup(AuthUtil.getGroupName(name),
-              new Permission(Permission.Action.values()));
-        } else {
-          newCache.putUser(name, new Permission(Permission.Action.values()));
-        }
-      }
-    }
-    return newCache;
-  }
-
-  public ZKPermissionWatcher getZKPermissionWatcher() {
-    return this.zkperms;
-  }
-
-  public void refreshTableCacheFromWritable(TableName table,
-                                       byte[] data) throws IOException {
-    if (data != null && data.length > 0) {
-      ListMultimap<String,TablePermission> perms;
-      try {
-        perms = AccessControlLists.readPermissions(data, conf);
-      } catch (DeserializationException e) {
-        throw new IOException(e);
-      }
-
-      if (perms != null) {
-        if (Bytes.equals(table.getName(), AccessControlLists.ACL_GLOBAL_NAME)) {
-          updateGlobalCache(perms);
-        } else {
-          updateTableCache(table, perms);
-        }
-      }
-    } else {
-      LOG.debug("Skipping permission cache refresh because writable data is empty");
-    }
-  }
-
-  public void refreshNamespaceCacheFromWritable(String namespace, byte[] data) throws IOException {
-    if (data != null && data.length > 0) {
-      ListMultimap<String,TablePermission> perms;
-      try {
-        perms = AccessControlLists.readPermissions(data, conf);
-      } catch (DeserializationException e) {
-        throw new IOException(e);
-      }
-      if (perms != null) {
-        updateNsCache(namespace, perms);
-      }
-    } else {
-      LOG.debug("Skipping permission cache refresh because writable data is empty");
-    }
-  }
-
-  /**
-   * Updates the internal global permissions cache
-   *
-   * @param userPerms
-   */
-  private void updateGlobalCache(ListMultimap<String,TablePermission> userPerms) {
-    PermissionCache<Permission> newCache = null;
-    try {
-      newCache = initGlobal(conf);
-      for (Map.Entry<String,TablePermission> entry : userPerms.entries()) {
-        if (AuthUtil.isGroupPrincipal(entry.getKey())) {
-          newCache.putGroup(AuthUtil.getGroupName(entry.getKey()),
-              new Permission(entry.getValue().getActions()));
-        } else {
-          newCache.putUser(entry.getKey(), new Permission(entry.getValue().getActions()));
-        }
-      }
-      globalCache = newCache;
-      mtime.incrementAndGet();
-    } catch (IOException e) {
-      // Never happens
-      LOG.error("Error occurred while updating the global cache", e);
-    }
-  }
-
-  /**
-   * Updates the internal permissions cache for a single table, splitting
-   * the permissions listed into separate caches for users and groups to optimize
-   * group lookups.
-   *
-   * @param table
-   * @param tablePerms
-   */
-  private void updateTableCache(TableName table,
-                                ListMultimap<String,TablePermission> tablePerms) {
-    PermissionCache<TablePermission> newTablePerms = new PermissionCache<>();
-
-    for (Map.Entry<String,TablePermission> entry : tablePerms.entries()) {
-      if (AuthUtil.isGroupPrincipal(entry.getKey())) {
-        newTablePerms.putGroup(AuthUtil.getGroupName(entry.getKey()), entry.getValue());
-      } else {
-        newTablePerms.putUser(entry.getKey(), entry.getValue());
-      }
-    }
-
-    tableCache.put(table, newTablePerms);
-    mtime.incrementAndGet();
-  }
-
-  /**
-   * Updates the internal permissions cache for a single table, splitting
-   * the permissions listed into separate caches for users and groups to optimize
-   * group lookups.
-   *
-   * @param namespace
-   * @param tablePerms
-   */
-  private void updateNsCache(String namespace,
-                             ListMultimap<String, TablePermission> tablePerms) {
-    PermissionCache<TablePermission> newTablePerms = new PermissionCache<>();
-
-    for (Map.Entry<String, TablePermission> entry : tablePerms.entries()) {
-      if (AuthUtil.isGroupPrincipal(entry.getKey())) {
-        newTablePerms.putGroup(AuthUtil.getGroupName(entry.getKey()), entry.getValue());
-      } else {
-        newTablePerms.putUser(entry.getKey(), entry.getValue());
-      }
-    }
-
-    nsCache.put(namespace, newTablePerms);
-    mtime.incrementAndGet();
-  }
-
-  private PermissionCache<TablePermission> getTablePermissions(TableName table) {
-    return computeIfAbsent(tableCache, table, PermissionCache::new);
-  }
-
-  private PermissionCache<TablePermission> getNamespacePermissions(String namespace) {
-    return computeIfAbsent(nsCache, namespace, PermissionCache::new);
-  }
-
-  /**
-   * Authorizes a global permission
-   * @param perms
-   * @param action
-   * @return true if authorized, false otherwise
-   */
-  private boolean authorize(List<Permission> perms, Permission.Action action) {
-    if (perms != null) {
-      for (Permission p : perms) {
-        if (p.implies(action)) {
-          return true;
-        }
-      }
-    } else if (LOG.isDebugEnabled()) {
-      LOG.debug("No permissions found for " + action);
-    }
-
-    return false;
-  }
-
-  /**
-   * Authorize a global permission based on ACLs for the given user and the
-   * user's groups.
-   * @param user
-   * @param action
-   * @return true if known and authorized, false otherwise
-   */
-  public boolean authorize(User user, Permission.Action action) {
-    if (user == null) {
-      return false;
-    }
-
-    if (authorize(globalCache.getUser(user.getShortName()), action)) {
-      return true;
-    }
-
-    String[] groups = user.getGroupNames();
-    if (groups != null) {
-      for (String group : groups) {
-        if (authorize(globalCache.getGroup(group), action)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  private boolean authorize(List<TablePermission> perms,
-                            TableName table, byte[] family,
-                            byte[] qualifier, Permission.Action action) {
-    if (perms != null) {
-      for (TablePermission p : perms) {
-        if (p.implies(table, family, qualifier, action)) {
-          return true;
-        }
-      }
-    } else if (LOG.isDebugEnabled()) {
-      LOG.debug("No permissions found for table="+table);
-    }
-    return false;
-  }
-
-  private boolean hasAccess(List<TablePermission> perms,
-                            TableName table, Permission.Action action) {
-    if (perms != null) {
-      for (TablePermission p : perms) {
-        if (p.implies(action)) {
-          return true;
-        }
-      }
-    } else if (LOG.isDebugEnabled()) {
-      LOG.debug("No permissions found for table="+table);
-    }
-    return false;
-  }
-
-  /**
-   * Authorize a user for a given KV. This is called from AccessControlFilter.
-   */
-  public boolean authorize(User user, TableName table, Cell cell, Permission.Action action) {
-    try {
-      List<Permission> perms = AccessControlLists.getCellPermissionsForUser(user, cell);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Perms for user " + user.getShortName() + " in cell " + cell + ": " +
-          (perms != null ? perms : ""));
-      }
-      if (perms != null) {
-        for (Permission p: perms) {
-          if (p.implies(action)) {
-            return true;
-          }
-        }
-      }
-    } catch (IOException e) {
-      // We failed to parse the KV tag
-      LOG.error("Failed parse of ACL tag in cell " + cell);
-      // Fall through to check with the table and CF perms we were able
-      // to collect regardless
-    }
-    return false;
-  }
-
-  public boolean authorize(User user, String namespace, Permission.Action action) {
-    // Global authorizations supercede namespace level
-    if (authorize(user, action)) {
-      return true;
-    }
-    // Check namespace permissions
-    PermissionCache<TablePermission> tablePerms = nsCache.get(namespace);
-    if (tablePerms != null) {
-      List<TablePermission> userPerms = tablePerms.getUser(user.getShortName());
-      if (authorize(userPerms, namespace, action)) {
-        return true;
-      }
-      String[] groupNames = user.getGroupNames();
-      if (groupNames != null) {
-        for (String group : groupNames) {
-          List<TablePermission> groupPerms = tablePerms.getGroup(group);
-          if (authorize(groupPerms, namespace, action)) {
-            return true;
-          }
-        }
-      }
-    }
-    return false;
-  }
-
-  private boolean authorize(List<TablePermission> perms, String namespace,
-                            Permission.Action action) {
-    if (perms != null) {
-      for (TablePermission p : perms) {
-        if (p.implies(namespace, action)) {
-          return true;
-        }
-      }
-    } else if (LOG.isDebugEnabled()) {
-      LOG.debug("No permissions for authorize() check, table=" + namespace);
-    }
-
-    return false;
-  }
-
-  /**
-   * Checks authorization to a given table and column family for a user, based on the
-   * stored user permissions.
-   *
-   * @param user
-   * @param table
-   * @param family
-   * @param action
-   * @return true if known and authorized, false otherwise
-   */
-  public boolean authorizeUser(User user, TableName table, byte[] family,
-      Permission.Action action) {
-    return authorizeUser(user, table, family, null, action);
-  }
-
-  public boolean authorizeUser(User user, TableName table, byte[] family,
-      byte[] qualifier, Permission.Action action) {
-    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
-    // Global and namespace authorizations supercede table level
-    if (authorize(user, table.getNamespaceAsString(), action)) {
-      return true;
-    }
-    // Check table permissions
-    return authorize(getTablePermissions(table).getUser(user.getShortName()), table, family,
-        qualifier, action);
-  }
-
-  /**
-   * Checks if the user has access to the full table or at least a family/qualifier
-   * for the specified action.
-   *
-   * @param user
-   * @param table
-   * @param action
-   * @return true if the user has access to the table, false otherwise
-   */
-  public boolean userHasAccess(User user, TableName table, Permission.Action action) {
-    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
-    // Global and namespace authorizations supercede table level
-    if (authorize(user, table.getNamespaceAsString(), action)) {
-      return true;
-    }
-    // Check table permissions
-    return hasAccess(getTablePermissions(table).getUser(user.getShortName()), table, action);
-  }
-
-  /**
-   * Checks global authorization for a given action for a group, based on the stored
-   * permissions.
-   */
-  public boolean authorizeGroup(String groupName, Permission.Action action) {
-    List<Permission> perms = globalCache.getGroup(groupName);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("authorizing " + (perms != null && !perms.isEmpty() ? perms.get(0) : "") +
-        " for " + action);
-    }
-    return authorize(perms, action);
-  }
-
-  /**
-   * Checks authorization to a given table, column family and column for a group, based
-   * on the stored permissions.
-   * @param groupName
-   * @param table
-   * @param family
-   * @param qualifier
-   * @param action
-   * @return true if known and authorized, false otherwise
-   */
-  public boolean authorizeGroup(String groupName, TableName table, byte[] family,
-      byte[] qualifier, Permission.Action action) {
-    // Global authorization supercedes table level
-    if (authorizeGroup(groupName, action)) {
-      return true;
-    }
-    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
-    // Namespace authorization supercedes table level
-    String namespace = table.getNamespaceAsString();
-    if (authorize(getNamespacePermissions(namespace).getGroup(groupName), namespace, action)) {
-      return true;
-    }
-    // Check table level
-    List<TablePermission> tblPerms = getTablePermissions(table).getGroup(groupName);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("authorizing " + (tblPerms != null && !tblPerms.isEmpty() ? tblPerms.get(0) : "") +
-        " for " +groupName + " on " + table + "." + Bytes.toString(family) + "." +
-        Bytes.toString(qualifier) + " with " + action);
-    }
-    return authorize(tblPerms, table, family, qualifier, action);
-  }
-
-  /**
-   * Checks if the user has access to the full table or at least a family/qualifier
-   * for the specified action.
-   * @param groupName
-   * @param table
-   * @param action
-   * @return true if the group has access to the table, false otherwise
-   */
-  public boolean groupHasAccess(String groupName, TableName table, Permission.Action action) {
-    // Global authorization supercedes table level
-    if (authorizeGroup(groupName, action)) {
-      return true;
-    }
-    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
-    // Namespace authorization supercedes table level
-    if (hasAccess(getNamespacePermissions(table.getNamespaceAsString()).getGroup(groupName),
-        table, action)) {
-      return true;
-    }
-    // Check table level
-    return hasAccess(getTablePermissions(table).getGroup(groupName), table, action);
-  }
-
-  public boolean authorize(User user, TableName table, byte[] family,
-      byte[] qualifier, Permission.Action action) {
-    if (authorizeUser(user, table, family, qualifier, action)) {
-      return true;
-    }
-
-    String[] groups = user.getGroupNames();
-    if (groups != null) {
-      for (String group : groups) {
-        if (authorizeGroup(group, table, family, qualifier, action)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  public boolean hasAccess(User user, TableName table, Permission.Action action) {
-    if (userHasAccess(user, table, action)) {
-      return true;
-    }
-
-    String[] groups = user.getGroupNames();
-    if (groups != null) {
-      for (String group : groups) {
-        if (groupHasAccess(group, table, action)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  public boolean authorize(User user, TableName table, byte[] family,
-      Permission.Action action) {
-    return authorize(user, table, family, null, action);
-  }
-
-  /**
-   * Returns true if the given user has a {@link TablePermission} matching up
-   * to the column family portion of a permission.  Note that this permission
-   * may be scoped to a given column qualifier and does not guarantee that
-   * authorize() on the same column family would return true.
-   */
-  public boolean matchPermission(User user,
-      TableName table, byte[] family, Permission.Action action) {
-    PermissionCache<TablePermission> tablePerms = tableCache.get(table);
-    if (tablePerms != null) {
-      List<TablePermission> userPerms = tablePerms.getUser(user.getShortName());
-      if (userPerms != null) {
-        for (TablePermission p : userPerms) {
-          if (p.matchesFamily(table, family, action)) {
-            return true;
-          }
-        }
-      }
-
-      String[] groups = user.getGroupNames();
-      if (groups != null) {
-        for (String group : groups) {
-          List<TablePermission> groupPerms = tablePerms.getGroup(group);
-          if (groupPerms != null) {
-            for (TablePermission p : groupPerms) {
-              if (p.matchesFamily(table, family, action)) {
-                return true;
-              }
-            }
-          }
-        }
-      }
-    }
-
-    return false;
-  }
-
-  public boolean matchPermission(User user,
-      TableName table, byte[] family, byte[] qualifier,
-      Permission.Action action) {
-    PermissionCache<TablePermission> tablePerms = tableCache.get(table);
-    if (tablePerms != null) {
-      List<TablePermission> userPerms = tablePerms.getUser(user.getShortName());
-      if (userPerms != null) {
-        for (TablePermission p : userPerms) {
-          if (p.matchesFamilyQualifier(table, family, qualifier, action)) {
-            return true;
-          }
-        }
-      }
-
-      String[] groups = user.getGroupNames();
-      if (groups != null) {
-        for (String group : groups) {
-          List<TablePermission> groupPerms = tablePerms.getGroup(group);
-          if (groupPerms != null) {
-            for (TablePermission p : groupPerms) {
-              if (p.matchesFamilyQualifier(table, family, qualifier, action)) {
-                return true;
-              }
-            }
-          }
-        }
-      }
-    }
-    return false;
-  }
-
-  public void removeNamespace(byte[] ns) {
-    nsCache.remove(Bytes.toString(ns));
-  }
-
-  public void removeTable(TableName table) {
-    tableCache.remove(table);
-  }
-
-  /**
-   * Overwrites the existing permission set for a given user for a table, and
-   * triggers an update for zookeeper synchronization.
-   * @param username
-   * @param table
-   * @param perms
-   */
-  public void setTableUserPermissions(String username, TableName table,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
-    tablePerms.replaceUser(username, perms);
-    writeTableToZooKeeper(table, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a group and triggers an update
-   * for zookeeper synchronization.
-   * @param group
-   * @param table
-   * @param perms
-   */
-  public void setTableGroupPermissions(String group, TableName table,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
-    tablePerms.replaceGroup(group, perms);
-    writeTableToZooKeeper(table, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a given user for a table, and
-   * triggers an update for zookeeper synchronization.
-   * @param username
-   * @param namespace
-   * @param perms
-   */
-  public void setNamespaceUserPermissions(String username, String namespace,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
-    tablePerms.replaceUser(username, perms);
-    writeNamespaceToZooKeeper(namespace, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a group and triggers an update
-   * for zookeeper synchronization.
-   * @param group
-   * @param namespace
-   * @param perms
-   */
-  public void setNamespaceGroupPermissions(String group, String namespace,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
-    tablePerms.replaceGroup(group, perms);
-    writeNamespaceToZooKeeper(namespace, tablePerms);
-  }
-
-  public void writeTableToZooKeeper(TableName table,
-      PermissionCache<TablePermission> tablePerms) {
-    byte[] serialized = new byte[0];
-    if (tablePerms != null) {
-      serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
-    }
-    zkperms.writeToZookeeper(table.getName(), serialized);
-  }
-
-  public void writeNamespaceToZooKeeper(String namespace,
-      PermissionCache<TablePermission> tablePerms) {
-    byte[] serialized = new byte[0];
-    if (tablePerms != null) {
-      serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
-    }
-    zkperms.writeToZookeeper(Bytes.toBytes(AccessControlLists.toNamespaceEntry(namespace)),
-        serialized);
-  }
-
-  public long getMTime() {
-    return mtime.get();
-  }
-
-  private static Map<ZKWatcher,TableAuthManager> managerMap = new HashMap<>();
-
-  private static Map<TableAuthManager, Integer> refCount = new HashMap<>();
-
-  /** Returns a TableAuthManager from the cache. If not cached, constructs a new one. Returned
-   * instance should be released back by calling {@link #release(TableAuthManager)}. */
-  public synchronized static TableAuthManager getOrCreate(
-          ZKWatcher watcher, Configuration conf) throws IOException {
-    TableAuthManager instance = managerMap.get(watcher);
-    if (instance == null) {
-      instance = new TableAuthManager(watcher, conf);
-      managerMap.put(watcher, instance);
-    }
-    int ref = refCount.get(instance) == null ? 0 : refCount.get(instance).intValue();
-    refCount.put(instance, ref + 1);
-    return instance;
-  }
-
-  @VisibleForTesting
-  public static int getTotalRefCount() {
-    int total = 0;
-    for (int count : refCount.values()) {
-      total += count;
-    }
-    return total;
-  }
-
-  /**
-   * Releases the resources for the given TableAuthManager if the reference count is down to 0.
-   * @param instance TableAuthManager to be released
-   */
-  public synchronized static void release(TableAuthManager instance) {
-    if (refCount.get(instance) == null || refCount.get(instance) < 1) {
-      String msg = "Something wrong with the TableAuthManager reference counting: " + instance
-          + " whose count is " + refCount.get(instance);
-      LOG.error(HBaseMarkers.FATAL, msg);
-      instance.close();
-      managerMap.remove(instance.getZKPermissionWatcher().getWatcher());
-      instance.getZKPermissionWatcher().getWatcher().abort(msg, null);
-    } else {
-      int ref = refCount.get(instance);
-      refCount.put(instance, ref-1);
-      if (ref-1 == 0) {
-        instance.close();
-        managerMap.remove(instance.getZKPermissionWatcher().getWatcher());
-        refCount.remove(instance);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
index 2266e86..fa3c30f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
@@ -49,21 +49,21 @@ import java.util.concurrent.RejectedExecutionException;
  * {@code /hbase/acl/tablename}, with the znode data containing a serialized
  * list of the permissions granted for the table.  The {@code AccessController}
  * instances on all other cluster hosts watch the znodes for updates, which
- * trigger updates in the {@link TableAuthManager} permission cache.
+ * trigger updates in the {@link AuthManager} permission cache.
  */
 @InterfaceAudience.Private
 public class ZKPermissionWatcher extends ZKListener implements Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(ZKPermissionWatcher.class);
   // parent node for permissions lists
   static final String ACL_NODE = "acl";
-  private final TableAuthManager authManager;
+  private final AuthManager authManager;
   private final String aclZNode;
   private final CountDownLatch initialized = new CountDownLatch(1);
   private final ExecutorService executor;
   private Future<?> childrenChangedFuture;
 
   public ZKPermissionWatcher(ZKWatcher watcher,
-      TableAuthManager authManager, Configuration conf) {
+      AuthManager authManager, Configuration conf) {
     super(watcher);
     this.authManager = authManager;
     String aclZnodeParent = conf.get("zookeeper.znode.acl.parent", ACL_NODE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 7edf734..6aa378c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -858,15 +859,16 @@ public class RestoreSnapshotHelper {
       Configuration conf) throws IOException {
     if (snapshot.hasUsersAndPermissions() && snapshot.getUsersAndPermissions() != null) {
       LOG.info("Restore snapshot acl to table. snapshot: " + snapshot + ", table: " + newTableName);
-      ListMultimap<String, TablePermission> perms =
+      ListMultimap<String, Permission> perms =
           ShadedAccessControlUtil.toUserTablePermissions(snapshot.getUsersAndPermissions());
       try (Connection conn = ConnectionFactory.createConnection(conf)) {
-        for (Entry<String, TablePermission> e : perms.entries()) {
+        for (Entry<String, Permission> e : perms.entries()) {
           String user = e.getKey();
-          TablePermission perm = e.getValue();
-          perm.setTableName(newTableName);
-          AccessControlClient.grant(conn, perm.getTableName(), user, perm.getFamily(),
-            perm.getQualifier(), perm.getActions());
+          TablePermission tablePerm = (TablePermission) e.getValue();
+          TablePermission newPerm = new TablePermission(newTableName,
+            tablePerm.getFamily(), tablePerm.getQualifier(), tablePerm.getActions());
+          AccessControlClient.grant(conn, newPerm.getTableName(), user, newPerm.getFamily(),
+            newPerm.getQualifier(), newPerm.getActions());
         }
       } catch (Throwable e) {
         throw new IOException("Grant acl into newly creatd table failed. snapshot: " + snapshot