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:46 UTC

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

HBASE-21255 [acl] Refactor TablePermission into three classes (Global, Namespace, Table)

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-20952
Commit: 130057f13774f6b213cdb06952c805a29d59396e
Parents: 9e42a9e
Author: Reid Chan <re...@apache.org>
Authored: Wed Nov 14 11:12:14 2018 +0800
Committer: Reid Chan <re...@apache.org>
Committed: Thu Nov 15 11:34:16 2018 +0800

----------------------------------------------------------------------
 .../security/access/AccessControlUtil.java      | 202 +++--
 .../hbase/security/access/GlobalPermission.java |  67 ++
 .../security/access/NamespacePermission.java    | 121 +++
 .../hbase/security/access/Permission.java       | 166 ++--
 .../access/ShadedAccessControlUtil.java         | 136 ++--
 .../hbase/security/access/TablePermission.java  | 323 +++-----
 .../hbase/security/access/UserPermission.java   | 191 ++---
 .../java/org/apache/hadoop/hbase/AuthUtil.java  |   2 +-
 .../hbase/rsgroup/TestRSGroupsWithACL.java      |   4 +-
 .../hbase/security/access/AccessChecker.java    |  24 +-
 .../security/access/AccessControlFilter.java    |  16 +-
 .../security/access/AccessControlLists.java     | 206 +++--
 .../hbase/security/access/AccessController.java |  86 +-
 .../hbase/security/access/AuthManager.java      | 608 ++++++++++++++
 .../hbase/security/access/TableAuthManager.java | 787 -------------------
 .../security/access/ZKPermissionWatcher.java    |   6 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |  14 +-
 .../snapshot/SnapshotDescriptionUtils.java      |   8 +-
 .../security/access/TestAccessController.java   |  56 +-
 .../security/access/TestAccessController2.java  |  12 +-
 .../security/access/TestAccessController3.java  |   2 +-
 .../security/access/TestNamespaceCommands.java  |  18 +-
 .../security/access/TestRpcAccessChecks.java    |   6 +-
 .../security/access/TestTablePermissions.java   | 190 +++--
 .../access/TestZKPermissionWatcher.java         | 104 ++-
 25 files changed, 1635 insertions(+), 1720 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
index 1b5a70c..b37440c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
@@ -47,7 +47,7 @@ public class AccessControlUtil {
   private AccessControlUtil() {}
 
   /**
-   * Create a request to grant user permissions.
+   * Create a request to grant user table permissions.
    *
    * @param username the short user name who to grant permissions
    * @param tableName optional table name the permissions apply
@@ -88,7 +88,7 @@ public class AccessControlUtil {
   }
 
   /**
-   * Create a request to grant user permissions.
+   * Create a request to grant user namespace permissions.
    *
    * @param username the short user name who to grant permissions
    * @param namespace optional table name the permissions apply
@@ -119,7 +119,7 @@ public class AccessControlUtil {
   }
 
   /**
-   * Create a request to revoke user permissions.
+   * Create a request to revoke user global permissions.
    *
    * @param username the short user name whose permissions to be revoked
    * @param actions the permissions to be revoked
@@ -145,7 +145,7 @@ public class AccessControlUtil {
   }
 
   /**
-   * Create a request to revoke user permissions.
+   * Create a request to revoke user namespace permissions.
    *
    * @param username the short user name whose permissions to be revoked
    * @param namespace optional table name the permissions apply
@@ -176,7 +176,7 @@ public class AccessControlUtil {
   }
 
   /**
-   * Create a request to grant user permissions.
+   * Create a request to grant user global permissions.
    *
    * @param username the short user name who to grant permissions
    * @param actions the permissions to be granted
@@ -240,23 +240,6 @@ public class AccessControlUtil {
     return result;
   }
 
-
-  /**
-   * Converts a Permission proto to a client Permission object.
-   *
-   * @param proto the protobuf Permission
-   * @return the converted Permission
-   */
-  public static Permission toPermission(AccessControlProtos.Permission proto) {
-    if (proto.getType() != AccessControlProtos.Permission.Type.Global) {
-      return toTablePermission(proto);
-    } else {
-      List<Permission.Action> actions = toPermissionActions(
-          proto.getGlobalPermission().getActionList());
-      return new Permission(actions.toArray(new Permission.Action[actions.size()]));
-    }
-  }
-
   /**
    * Converts a TablePermission proto to a client TablePermission object.
    * @param proto the protobuf TablePermission
@@ -282,48 +265,45 @@ public class AccessControlUtil {
   }
 
   /**
-   * Converts a Permission proto to a client TablePermission object.
+   * Converts a Permission proto to a client Permission object.
    * @param proto the protobuf Permission
-   * @return the converted TablePermission
+   * @return the converted Permission
    */
-  public static TablePermission toTablePermission(AccessControlProtos.Permission proto) {
-    if(proto.getType() == AccessControlProtos.Permission.Type.Global) {
+  public static Permission toPermission(AccessControlProtos.Permission proto) {
+    if (proto.getType() == AccessControlProtos.Permission.Type.Global) {
       AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
-
-      return new TablePermission(null, null, null,
-          actions.toArray(new Permission.Action[actions.size()]));
+      return new GlobalPermission(actions.toArray(new Permission.Action[actions.size()]));
     }
-    if(proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
+    if (proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
       AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
-
-      if(!proto.hasNamespacePermission()) {
+      if (!proto.hasNamespacePermission()) {
         throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
       }
-      String namespace = perm.getNamespaceName().toStringUtf8();
-      return new TablePermission(namespace, actions.toArray(new Permission.Action[actions.size()]));
+      return new NamespacePermission(perm.getNamespaceName().toStringUtf8(),
+        actions.toArray(new Permission.Action[actions.size()]));
     }
-    if(proto.getType() == AccessControlProtos.Permission.Type.Table) {
+    if (proto.getType() == AccessControlProtos.Permission.Type.Table) {
       AccessControlProtos.TablePermission perm = proto.getTablePermission();
       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
-
       byte[] qualifier = null;
       byte[] family = null;
       TableName table = null;
-
       if (!perm.hasTableName()) {
         throw new IllegalStateException("TableName cannot be empty");
       }
       table = ProtobufUtil.toTableName(perm.getTableName());
-
-      if (perm.hasFamily()) family = perm.getFamily().toByteArray();
-      if (perm.hasQualifier()) qualifier = perm.getQualifier().toByteArray();
-
+      if (perm.hasFamily()) {
+        family = perm.getFamily().toByteArray();
+      }
+      if (perm.hasQualifier()) {
+        qualifier = perm.getQualifier().toByteArray();
+      }
       return new TablePermission(table, family, qualifier,
-          actions.toArray(new Permission.Action[actions.size()]));
+        actions.toArray(new Permission.Action[actions.size()]));
     }
-    throw new IllegalStateException("Unrecognize Perm Type: "+proto.getType());
+    throw new IllegalStateException("Unrecognize Perm Type: " + proto.getType());
   }
 
   /**
@@ -334,56 +314,51 @@ public class AccessControlUtil {
    */
   public static AccessControlProtos.Permission toPermission(Permission perm) {
     AccessControlProtos.Permission.Builder ret = AccessControlProtos.Permission.newBuilder();
-    if (perm instanceof TablePermission) {
-      TablePermission tablePerm = (TablePermission)perm;
-      if(tablePerm.hasNamespace()) {
-        ret.setType(AccessControlProtos.Permission.Type.Namespace);
-
-        AccessControlProtos.NamespacePermission.Builder builder =
-            AccessControlProtos.NamespacePermission.newBuilder();
-        builder.setNamespaceName(ByteString.copyFromUtf8(tablePerm.getNamespace()));
-        Permission.Action[] actions = perm.getActions();
-        if (actions != null) {
-          for (Permission.Action a : actions) {
-            builder.addAction(toPermissionAction(a));
-          }
-        }
-        ret.setNamespacePermission(builder);
-        return ret.build();
-      } else if (tablePerm.hasTable()) {
-        ret.setType(AccessControlProtos.Permission.Type.Table);
-
-        AccessControlProtos.TablePermission.Builder builder =
-            AccessControlProtos.TablePermission.newBuilder();
-        builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
-        if (tablePerm.hasFamily()) {
-          builder.setFamily(ByteStringer.wrap(tablePerm.getFamily()));
-        }
-        if (tablePerm.hasQualifier()) {
-          builder.setQualifier(ByteStringer.wrap(tablePerm.getQualifier()));
+    if (perm instanceof NamespacePermission) {
+      NamespacePermission namespace = (NamespacePermission) perm;
+      ret.setType(AccessControlProtos.Permission.Type.Namespace);
+      AccessControlProtos.NamespacePermission.Builder builder =
+        AccessControlProtos.NamespacePermission.newBuilder();
+      builder.setNamespaceName(ByteString.copyFromUtf8(namespace.getNamespace()));
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a : actions) {
+          builder.addAction(toPermissionAction(a));
         }
-        Permission.Action actions[] = perm.getActions();
-        if (actions != null) {
-          for (Permission.Action a : actions) {
-            builder.addAction(toPermissionAction(a));
-          }
+      }
+      ret.setNamespacePermission(builder);
+    } else if (perm instanceof TablePermission) {
+      TablePermission table = (TablePermission) perm;
+      ret.setType(AccessControlProtos.Permission.Type.Table);
+      AccessControlProtos.TablePermission.Builder builder =
+        AccessControlProtos.TablePermission.newBuilder();
+      builder.setTableName(ProtobufUtil.toProtoTableName(table.getTableName()));
+      if (table.hasFamily()) {
+        builder.setFamily(ByteStringer.wrap(table.getFamily()));
+      }
+      if (table.hasQualifier()) {
+        builder.setQualifier(ByteStringer.wrap(table.getQualifier()));
+      }
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a : actions) {
+          builder.addAction(toPermissionAction(a));
         }
-        ret.setTablePermission(builder);
-        return ret.build();
       }
-    }
-
-    ret.setType(AccessControlProtos.Permission.Type.Global);
-
-    AccessControlProtos.GlobalPermission.Builder builder =
+      ret.setTablePermission(builder);
+    } else {
+      // perm instanceof GlobalPermission
+      ret.setType(AccessControlProtos.Permission.Type.Global);
+      AccessControlProtos.GlobalPermission.Builder builder =
         AccessControlProtos.GlobalPermission.newBuilder();
-    Permission.Action actions[] = perm.getActions();
-    if (actions != null) {
-      for (Permission.Action a: actions) {
-        builder.addAction(toPermissionAction(a));
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a: actions) {
+          builder.addAction(toPermissionAction(a));
+        }
       }
+      ret.setGlobalPermission(builder);
     }
-    ret.setGlobalPermission(builder);
     return ret.build();
   }
 
@@ -456,8 +431,8 @@ public class AccessControlUtil {
    */
   public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
     return AccessControlProtos.UserPermission.newBuilder()
-        .setUser(ByteStringer.wrap(perm.getUser()))
-        .setPermission(toPermission(perm))
+        .setUser(ByteString.copyFromUtf8(perm.getUser()))
+        .setPermission(toPermission(perm.getPermission()))
         .build();
   }
 
@@ -480,8 +455,7 @@ public class AccessControlUtil {
    * @return the converted UserPermission
    */
   public static UserPermission toUserPermission(AccessControlProtos.UserPermission proto) {
-    return new UserPermission(proto.getUser().toByteArray(),
-        toTablePermission(proto.getPermission()));
+    return new UserPermission(proto.getUser().toStringUtf8(), toPermission(proto.getPermission()));
   }
 
   /**
@@ -492,15 +466,15 @@ public class AccessControlUtil {
    * @return the protobuf UserTablePermissions
    */
   public static AccessControlProtos.UsersAndPermissions toUserTablePermissions(
-      ListMultimap<String, TablePermission> perm) {
+      ListMultimap<String, UserPermission> perm) {
     AccessControlProtos.UsersAndPermissions.Builder builder =
         AccessControlProtos.UsersAndPermissions.newBuilder();
-    for (Map.Entry<String, Collection<TablePermission>> entry : perm.asMap().entrySet()) {
+    for (Map.Entry<String, Collection<UserPermission>> entry : perm.asMap().entrySet()) {
       AccessControlProtos.UsersAndPermissions.UserPermissions.Builder userPermBuilder =
           AccessControlProtos.UsersAndPermissions.UserPermissions.newBuilder();
       userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
-      for (TablePermission tablePerm: entry.getValue()) {
-        userPermBuilder.addPermissions(toPermission(tablePerm));
+      for (UserPermission userPerm: entry.getValue()) {
+        userPermBuilder.addPermissions(toPermission(userPerm.getPermission()));
       }
       builder.addUserPermissions(userPermBuilder.build());
     }
@@ -844,28 +818,46 @@ public class AccessControlUtil {
   }
 
   /**
-   * Convert a protobuf UserTablePermissions to a
-   * ListMultimap&lt;String, TablePermission&gt; where key is username.
-   *
-   * @param proto the protobuf UserPermission
-   * @return the converted UserPermission
+   * Convert a protobuf UserTablePermissions to a ListMultimap&lt;Username, UserPermission&gt
+   * @param proto the proto UsersAndPermissions
+   * @return a ListMultimap with user and its permissions
+   */
+  public static ListMultimap<String, UserPermission> toUserPermission(
+      AccessControlProtos.UsersAndPermissions proto) {
+    ListMultimap<String, UserPermission> userPermission = ArrayListMultimap.create();
+    AccessControlProtos.UsersAndPermissions.UserPermissions userPerm;
+    for (int i = 0; i < proto.getUserPermissionsCount(); i++) {
+      userPerm = proto.getUserPermissions(i);
+      String username = userPerm.getUser().toStringUtf8();
+      for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
+        userPermission.put(username,
+          new UserPermission(username, toPermission(userPerm.getPermissions(j))));
+      }
+    }
+    return userPermission;
+  }
+
+  /**
+   * Convert a protobuf UserTablePermissions to a ListMultimap&lt;Username, Permission&gt
+   * @param proto the proto UsersAndPermissions
+   * @return a ListMultimap with user and its permissions
    */
-  public static ListMultimap<String, TablePermission> toUserTablePermissions(
+  public static ListMultimap<String, Permission> toPermission(
       AccessControlProtos.UsersAndPermissions proto) {
-    ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
+    ListMultimap<String, Permission> perms = ArrayListMultimap.create();
     AccessControlProtos.UsersAndPermissions.UserPermissions userPerm;
     for (int i = 0; i < proto.getUserPermissionsCount(); i++) {
       userPerm = proto.getUserPermissions(i);
+      String username = userPerm.getUser().toStringUtf8();
       for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
-        TablePermission tablePerm = toTablePermission(userPerm.getPermissions(j));
-        perms.put(userPerm.getUser().toStringUtf8(), tablePerm);
+        perms.put(username, toPermission(userPerm.getPermissions(j)));
       }
     }
     return perms;
   }
 
   /**
-   * Create a request to revoke user permissions.
+   * Create a request to revoke user table permissions.
    *
    * @param username the short user name whose permissions to be revoked
    * @param tableName optional table name the permissions apply

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
new file mode 100644
index 0000000..b29317a
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
@@ -0,0 +1,67 @@
+/*
+ * 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 org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Represents an authorization for access whole cluster.
+ */
+@InterfaceAudience.Private
+public class GlobalPermission extends Permission {
+
+  /** Default constructor for Writable, do not use */
+  public GlobalPermission() {
+    super();
+    this.scope = Scope.EMPTY;
+  }
+
+  /**
+   * Construct a global permission.
+   * @param assigned assigned actions
+   */
+  GlobalPermission(Action... assigned) {
+    super(assigned);
+    this.scope = Scope.GLOBAL;
+  }
+
+  /**
+   * Construct a global permission.
+   * @param actionCode assigned actions
+   */
+  GlobalPermission(byte[] actionCode) {
+    super(actionCode);
+    this.scope = Scope.GLOBAL;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj instanceof GlobalPermission && super.equals(obj);
+  }
+
+  @Override
+  public String toString() {
+    return "[GlobalPermission: " + rawExpression() + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
new file mode 100644
index 0000000..c7ede96
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
@@ -0,0 +1,121 @@
+/*
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Represents an authorization for access for the given namespace.
+ */
+@InterfaceAudience.Private
+public class NamespacePermission extends Permission {
+
+  private String namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
+
+  /** Default constructor for Writable, do not use */
+  public NamespacePermission() {
+    super();
+    this.scope = Scope.EMPTY;
+  }
+
+  /**
+   * Construct a namespace permission.
+   * @param namespace namespace's name
+   * @param assigned assigned actions
+   */
+  public NamespacePermission(String namespace, Action... assigned) {
+    super(assigned);
+    this.namespace = namespace;
+    this.scope = Scope.NAMESPACE;
+  }
+
+  /**
+   * Construct a namespace permission.
+   * @param namespace namespace's name
+   * @param actionCode assigned actions
+   */
+  public NamespacePermission(String namespace, byte[] actionCode) {
+    super(actionCode);
+    this.namespace = namespace;
+    this.scope = Scope.NAMESPACE;
+  }
+
+  public String getNamespace() {
+    return namespace;
+  }
+
+  /**
+   * check if given action is granted in given namespace.
+   * @param namespace namespace's name
+   * @param action action to be checked
+   * @return true if granted, false otherwise
+   */
+  public boolean implies(String namespace, Action action) {
+    return namespace.equals(this.namespace) && implies(action);
+  }
+
+  @Override
+  public boolean equalsExceptActions(Object obj) {
+    if (!(obj instanceof NamespacePermission)) {
+      return false;
+    }
+    NamespacePermission gp = (NamespacePermission) obj;
+    return namespace.equals(gp.namespace);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(namespace) + super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return equalsExceptActions(obj) && super.equals(obj);
+  }
+
+  @Override
+  public String toString() {
+    return "[NamespacePermission: " + rawExpression() + "]";
+  }
+
+  @Override
+  protected String rawExpression() {
+    StringBuilder raw = new StringBuilder("namespace=").append(namespace).append(", ");
+    return raw.toString() + super.rawExpression();
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    namespace = Bytes.toString(Bytes.readByteArray(in));
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    Bytes.writeByteArray(out, Bytes.toBytes(namespace));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
index 1e9e60c..d448d3a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.Map;
 
 import org.apache.yetus.audience.InterfaceAudience;
@@ -30,7 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.VersionedWritable;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 
 /**
  * Base permissions instance representing the ability to perform a given set
@@ -48,21 +49,49 @@ public class Permission extends VersionedWritable {
 
     private final byte code;
     Action(char code) {
-      this.code = (byte)code;
+      this.code = (byte) code;
     }
 
     public byte code() { return code; }
   }
 
+  @InterfaceAudience.Private
+  protected enum Scope {
+    GLOBAL('G'), NAMESPACE('N'), TABLE('T'), EMPTY('E');
+
+    private final byte code;
+    Scope(char code) {
+      this.code = (byte) code;
+    }
+
+    public byte code() {
+      return code;
+    }
+  }
+
   private static final Logger LOG = LoggerFactory.getLogger(Permission.class);
-  protected static final Map<Byte,Action> ACTION_BY_CODE = Maps.newHashMap();
 
-  protected Action[] actions;
+  protected static final Map<Byte, Action> ACTION_BY_CODE;
+  protected static final Map<Byte, Scope> SCOPE_BY_CODE;
+
+  protected EnumSet<Action> actions = EnumSet.noneOf(Action.class);
+  protected Scope scope = Scope.EMPTY;
 
   static {
-    for (Action a : Action.values()) {
-      ACTION_BY_CODE.put(a.code(), a);
-    }
+    ACTION_BY_CODE = ImmutableMap.of(
+      Action.READ.code, Action.READ,
+      Action.WRITE.code, Action.WRITE,
+      Action.EXEC.code, Action.EXEC,
+      Action.CREATE.code, Action.CREATE,
+      Action.ADMIN.code, Action.ADMIN
+    );
+
+    SCOPE_BY_CODE = ImmutableMap.of(
+      Scope.GLOBAL.code, Scope.GLOBAL,
+      Scope.NAMESPACE.code, Scope.NAMESPACE,
+      Scope.TABLE.code, Scope.TABLE,
+      Scope.EMPTY.code, Scope.EMPTY
+    );
   }
 
   /** Empty constructor for Writable implementation.  <b>Do not use.</b> */
@@ -72,75 +101,72 @@ public class Permission extends VersionedWritable {
 
   public Permission(Action... assigned) {
     if (assigned != null && assigned.length > 0) {
-      actions = Arrays.copyOf(assigned, assigned.length);
+      actions.addAll(Arrays.asList(assigned));
     }
   }
 
   public Permission(byte[] actionCodes) {
     if (actionCodes != null) {
-      Action acts[] = new Action[actionCodes.length];
-      int j = 0;
-      for (int i=0; i<actionCodes.length; i++) {
-        byte b = actionCodes[i];
-        Action a = ACTION_BY_CODE.get(b);
-        if (a == null) {
-          LOG.error("Ignoring unknown action code '"+
-              Bytes.toStringBinary(new byte[]{b})+"'");
+      for (byte code : actionCodes) {
+        Action action = ACTION_BY_CODE.get(code);
+        if (action == null) {
+          LOG.error("Ignoring unknown action code '" +
+            Bytes.toStringBinary(new byte[] { code }) + "'");
           continue;
         }
-        acts[j++] = a;
+        actions.add(action);
       }
-      this.actions = Arrays.copyOf(acts, j);
     }
   }
 
   public Action[] getActions() {
-    return actions;
+    return actions.toArray(new Action[actions.size()]);
   }
 
+  /**
+   * check if given action is granted
+   * @param action action to be checked
+   * @return true if granted, false otherwise
+   */
   public boolean implies(Action action) {
-    if (this.actions != null) {
-      for (Action a : this.actions) {
-        if (a == action) {
-          return true;
-        }
-      }
-    }
-
-    return false;
+    return actions.contains(action);
   }
 
   public void setActions(Action[] assigned) {
     if (assigned != null && assigned.length > 0) {
-      actions = Arrays.copyOf(assigned, assigned.length);
+      // setActions should cover the previous actions,
+      // so we call clear here.
+      actions.clear();
+      actions.addAll(Arrays.asList(assigned));
     }
   }
 
+  /**
+   * Check if two permission equals regardless of actions. It is useful when
+   * merging a new permission with an existed permission which needs to check two permissions's
+   * fields.
+   * @param obj instance
+   * @return true if equals, false otherwise
+   */
+  public boolean equalsExceptActions(Object obj) {
+    return obj instanceof Permission;
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (!(obj instanceof Permission)) {
       return false;
     }
-    Permission other = (Permission)obj;
-    // check actions
-    if (actions == null && other.getActions() == null) {
-      return true;
-    } else if (actions != null && other.getActions() != null) {
-      Action[] otherActions = other.getActions();
-      if (actions.length != otherActions.length) {
-        return false;
-      }
 
-      outer:
-      for (Action a : actions) {
-        for (Action oa : otherActions) {
-          if (a == oa) continue outer;
-        }
+    Permission other = (Permission) obj;
+    if (actions.isEmpty() && other.actions.isEmpty()) {
+      return true;
+    } else if (!actions.isEmpty() && !other.actions.isEmpty()) {
+      if (actions.size() != other.actions.size()) {
         return false;
       }
-      return true;
+      return actions.containsAll(other.actions);
     }
-
     return false;
   }
 
@@ -151,26 +177,28 @@ public class Permission extends VersionedWritable {
     for (Action a : actions) {
       result = prime * result + a.code();
     }
+    result = prime * result + scope.code();
     return result;
   }
 
   @Override
   public String toString() {
-    StringBuilder str = new StringBuilder("[Permission: ")
-        .append("actions=");
+    return "[Permission: " + rawExpression() + "]";
+  }
+
+  protected String rawExpression() {
+    StringBuilder raw = new StringBuilder("actions=");
     if (actions != null) {
-      for (int i=0; i<actions.length; i++) {
-        if (i > 0)
-          str.append(",");
-        if (actions[i] != null)
-          str.append(actions[i].toString());
-        else
-          str.append("NULL");
+      int i = 0;
+      for (Action action : actions) {
+        if (i > 0) {
+          raw.append(",");
+        }
+        raw.append(action != null ? action.toString() : "NULL");
+        i++;
       }
     }
-    str.append("]");
-
-    return str.toString();
+    return raw.toString();
   }
 
   /** @return the object version number */
@@ -182,31 +210,35 @@ public class Permission extends VersionedWritable {
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    int length = (int)in.readByte();
+    int length = (int) in.readByte();
+    actions = EnumSet.noneOf(Action.class);
     if (length > 0) {
-      actions = new Action[length];
       for (int i = 0; i < length; i++) {
         byte b = in.readByte();
-        Action a = ACTION_BY_CODE.get(b);
-        if (a == null) {
-          throw new IOException("Unknown action code '"+
-              Bytes.toStringBinary(new byte[]{b})+"' in input");
+        Action action = ACTION_BY_CODE.get(b);
+        if (action == null) {
+          throw new IOException("Unknown action code '" +
+            Bytes.toStringBinary(new byte[] { b }) + "' in input");
         }
-        this.actions[i] = a;
+        actions.add(action);
       }
-    } else {
-      actions = new Action[0];
     }
+    scope = SCOPE_BY_CODE.get(in.readByte());
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    out.writeByte(actions != null ? actions.length : 0);
+    out.writeByte(actions != null ? actions.size() : 0);
     if (actions != null) {
       for (Action a: actions) {
         out.writeByte(a.code());
       }
     }
+    out.writeByte(scope.code());
+  }
+
+  public Scope getAccessScope() {
+    return scope;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
index 5a94805..7e36656 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
@@ -119,14 +119,13 @@ public class ShadedAccessControlUtil {
    * @param proto the protobuf Permission
    * @return the converted TablePermission
    */
-  public static TablePermission toTablePermission(AccessControlProtos.Permission proto) {
+  public static Permission toPermission(AccessControlProtos.Permission proto) {
 
     if (proto.getType() == AccessControlProtos.Permission.Type.Global) {
       AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
       List<Action> actions = toPermissionActions(perm.getActionList());
 
-      return new TablePermission(null, null, null,
-          actions.toArray(new Permission.Action[actions.size()]));
+      return new GlobalPermission(actions.toArray(new Permission.Action[actions.size()]));
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
       AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
@@ -135,8 +134,8 @@ public class ShadedAccessControlUtil {
       if (!proto.hasNamespacePermission()) {
         throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
       }
-      String namespace = perm.getNamespaceName().toStringUtf8();
-      return new TablePermission(namespace, actions.toArray(new Permission.Action[actions.size()]));
+      String ns = perm.getNamespaceName().toStringUtf8();
+      return new NamespacePermission(ns, actions.toArray(new Permission.Action[actions.size()]));
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Table) {
       AccessControlProtos.TablePermission perm = proto.getTablePermission();
@@ -144,12 +143,11 @@ public class ShadedAccessControlUtil {
 
       byte[] qualifier = null;
       byte[] family = null;
-      TableName table = null;
 
       if (!perm.hasTableName()) {
         throw new IllegalStateException("TableName cannot be empty");
       }
-      table = toTableName(perm.getTableName());
+      TableName table = toTableName(perm.getTableName());
 
       if (perm.hasFamily()) family = perm.getFamily().toByteArray();
       if (perm.hasQualifier()) qualifier = perm.getQualifier().toByteArray();
@@ -170,63 +168,58 @@ public class ShadedAccessControlUtil {
     org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder ret =
         org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission
             .newBuilder();
-    if (perm instanceof TablePermission) {
-      TablePermission tablePerm = (TablePermission) perm;
-      if (tablePerm.hasNamespace()) {
-        ret.setType(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Namespace);
-
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission.Builder builder =
-            org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission
-                .newBuilder();
-        builder.setNamespaceName(org.apache.hbase.thirdparty.com.google.protobuf.ByteString
-            .copyFromUtf8(tablePerm.getNamespace()));
-        Permission.Action[] actions = perm.getActions();
-        if (actions != null) {
-          for (Permission.Action a : actions) {
-            builder.addAction(toPermissionAction(a));
-          }
-        }
-        ret.setNamespacePermission(builder);
-        return ret.build();
-      } else if (tablePerm.hasTable()) {
-        ret.setType(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Table);
-
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission.Builder builder =
-            org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission
-                .newBuilder();
-        builder.setTableName(toProtoTableName(tablePerm.getTableName()));
-        if (tablePerm.hasFamily()) {
-          builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
-        }
-        if (tablePerm.hasQualifier()) {
-          builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
+    if (perm instanceof NamespacePermission) {
+      NamespacePermission nsPerm = (NamespacePermission) perm;
+      ret.setType(
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Namespace);
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission.Builder builder =
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission
+          .newBuilder();
+      builder.setNamespaceName(org.apache.hbase.thirdparty.com.google.protobuf.ByteString
+        .copyFromUtf8(nsPerm.getNamespace()));
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a : actions) {
+          builder.addAction(toPermissionAction(a));
         }
-        Permission.Action actions[] = perm.getActions();
-        if (actions != null) {
-          for (Permission.Action a : actions) {
-            builder.addAction(toPermissionAction(a));
-          }
+      }
+      ret.setNamespacePermission(builder);
+    } else if (perm instanceof TablePermission) {
+      TablePermission tablePerm = (TablePermission) perm;
+      ret.setType(
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Table);
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission.Builder builder =
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission
+          .newBuilder();
+      builder.setTableName(toProtoTableName(tablePerm.getTableName()));
+      if (tablePerm.hasFamily()) {
+        builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
+      }
+      if (tablePerm.hasQualifier()) {
+        builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
+      }
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a : actions) {
+          builder.addAction(toPermissionAction(a));
         }
-        ret.setTablePermission(builder);
-        return ret.build();
       }
-    }
-
-    ret.setType(
-      org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Global);
-
-    org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission.Builder builder =
+      ret.setTablePermission(builder);
+    } else {
+      // perm.getAccessScope() == Permission.Scope.GLOBAL
+      ret.setType(
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type.Global);
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission.Builder builder =
         org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission
-            .newBuilder();
-    Permission.Action actions[] = perm.getActions();
-    if (actions != null) {
-      for (Permission.Action a : actions) {
-        builder.addAction(toPermissionAction(a));
+          .newBuilder();
+      Permission.Action[] actions = perm.getActions();
+      if (actions != null) {
+        for (Permission.Action a : actions) {
+          builder.addAction(toPermissionAction(a));
+        }
       }
+      ret.setGlobalPermission(builder);
     }
-    ret.setGlobalPermission(builder);
     return ret.build();
   }
 
@@ -236,15 +229,15 @@ public class ShadedAccessControlUtil {
    * @param proto the protobuf UserPermission
    * @return the converted UserPermission
    */
-  public static ListMultimap<String, TablePermission> toUserTablePermissions(
+  public static ListMultimap<String, Permission> toUserTablePermissions(
       org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions proto) {
-    ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
+    ListMultimap<String, Permission> perms = ArrayListMultimap.create();
     org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions userPerm;
     for (int i = 0; i < proto.getUserPermissionsCount(); i++) {
       userPerm = proto.getUserPermissions(i);
       for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
-        TablePermission tablePerm = toTablePermission(userPerm.getPermissions(j));
-        perms.put(userPerm.getUser().toStringUtf8(), tablePerm);
+        Permission perm = toPermission(userPerm.getPermissions(j));
+        perms.put(userPerm.getUser().toStringUtf8(), perm);
       }
     }
     return perms;
@@ -258,31 +251,20 @@ public class ShadedAccessControlUtil {
    */
   public static
       org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions
-      toUserTablePermissions(ListMultimap<String, TablePermission> perm) {
+      toUserTablePermissions(ListMultimap<String, UserPermission> perm) {
     org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.Builder builder =
         org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions
             .newBuilder();
-    for (Map.Entry<String, Collection<TablePermission>> entry : perm.asMap().entrySet()) {
+    for (Map.Entry<String, Collection<UserPermission>> entry : perm.asMap().entrySet()) {
       org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions.Builder userPermBuilder =
           org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions
               .newBuilder();
       userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
-      for (TablePermission tablePerm : entry.getValue()) {
-        userPermBuilder.addPermissions(toPermission(tablePerm));
+      for (UserPermission userPerm : entry.getValue()) {
+        userPermBuilder.addPermissions(toPermission(userPerm.getPermission()));
       }
       builder.addUserPermissions(userPermBuilder.build());
     }
     return builder.build();
   }
-
-  /**
-   * Converts a user permission proto to a client user permission object.
-   *
-   * @param proto the protobuf UserPermission
-   * @return the converted UserPermission
-   */
-  public static UserPermission toUserPermission(org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission proto) {
-    return new UserPermission(proto.getUser().toByteArray(),
-        toTablePermission(proto.getPermission()));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
index dd0e71d..36ed8e4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 /**
  * Represents an authorization for access for the given actions, optionally
  * restricted to the given column family or column qualifier, over the
- * given table.  If the family property is <code>null</code>, it implies
+ * given table. If the family property is <code>null</code>, it implies
  * full table access.
  */
 @InterfaceAudience.Private
@@ -41,114 +41,78 @@ public class TablePermission extends Permission {
   private byte[] family;
   private byte[] qualifier;
 
-  //TODO refactor this class
-  //we need to refacting this into three classes (Global, Table, Namespace)
-  private String namespace;
-
   /** Nullary constructor for Writable, do not use */
   public TablePermission() {
     super();
+    this.scope = Scope.EMPTY;
+  }
+
+  /**
+   * Construct a table permission.
+   * @param table table name
+   * @param assigned assigned actions
+   */
+  public TablePermission(TableName table, Action... assigned) {
+    this(table, null, null, assigned);
   }
 
   /**
-   * Create a new permission for the given table and (optionally) column family,
-   * allowing the given actions.
-   * @param table the table
-   * @param family the family, can be null if a global permission on the table
-   * @param assigned the list of allowed actions
+   * Construct a table:family permission.
+   * @param table table name
+   * @param family family name
+   * @param assigned assigned actions
    */
   public TablePermission(TableName table, byte[] family, Action... assigned) {
     this(table, family, null, assigned);
   }
 
   /**
-   * Creates a new permission for the given table, restricted to the given
-   * column family and qualifier, allowing the assigned actions to be performed.
-   * @param table the table
-   * @param family the family, can be null if a global permission on the table
-   * @param assigned the list of allowed actions
+   * Construct a table:family:qualifier permission.
+   * @param table table name
+   * @param family family name
+   * @param qualifier qualifier name
+   * @param assigned assigned actions
    */
-  public TablePermission(TableName table, byte[] family, byte[] qualifier,
-      Action... assigned) {
+  public TablePermission(TableName table, byte[] family, byte[] qualifier, Action... assigned) {
     super(assigned);
     this.table = table;
     this.family = family;
     this.qualifier = qualifier;
+    this.scope = Scope.TABLE;
   }
 
   /**
-   * Creates a new permission for the given table, family and column qualifier,
-   * allowing the actions matching the provided byte codes to be performed.
-   * @param table the table
-   * @param family the family, can be null if a global permission on the table
-   * @param actionCodes the list of allowed action codes
+   * Construct a table permission.
+   * @param table table name
+   * @param actionCodes assigned actions
    */
-  public TablePermission(TableName table, byte[] family, byte[] qualifier,
-      byte[] actionCodes) {
-    super(actionCodes);
-    this.table = table;
-    this.family = family;
-    this.qualifier = qualifier;
+  public TablePermission(TableName table, byte[] actionCodes) {
+    this(table, null, null, actionCodes);
   }
 
   /**
-   * Creates a new permission for the given namespace or table, restricted to the given
-   * column family and qualifier, allowing the assigned actions to be performed.
-   * @param namespace
-   * @param table the table
-   * @param family the family, can be null if a global permission on the table
-   * @param assigned the list of allowed actions
+   * Construct a table:family permission.
+   * @param table table name
+   * @param family family name
+   * @param actionCodes assigned actions
    */
-  public TablePermission(String namespace, TableName table, byte[] family, byte[] qualifier,
-      Action... assigned) {
-    super(assigned);
-    this.namespace = namespace;
-    this.table = table;
-    this.family = family;
-    this.qualifier = qualifier;
+  public TablePermission(TableName table, byte[] family, byte[] actionCodes) {
+    this(table, family, null, actionCodes);
   }
 
   /**
-   * Creates a new permission for the given namespace or table, family and column qualifier,
-   * allowing the actions matching the provided byte codes to be performed.
-   * @param namespace
-   * @param table the table
-   * @param family the family, can be null if a global permission on the table
-   * @param actionCodes the list of allowed action codes
+   * Construct a table:family:qualifier permission.
+   * @param table table name
+   * @param family family name
+   * @param qualifier qualifier name
+   * @param actionCodes assigned actions
    */
-  public TablePermission(String namespace, TableName table, byte[] family, byte[] qualifier,
-      byte[] actionCodes) {
+  public TablePermission(TableName table, byte[] family, byte[] qualifier, byte[] actionCodes) {
     super(actionCodes);
-    this.namespace = namespace;
     this.table = table;
     this.family = family;
     this.qualifier = qualifier;
-  }
-
-  /**
-   * Creates a new permission for the given namespace,
-   * allowing the actions matching the provided byte codes to be performed.
-   * @param namespace
-   * @param actionCodes the list of allowed action codes
-   */
-  public TablePermission(String namespace, byte[] actionCodes) {
-    super(actionCodes);
-    this.namespace = namespace;
-  }
-
-  /**
-   * Create a new permission for the given namespace,
-   * allowing the given actions.
-   * @param namespace
-   * @param assigned the list of allowed actions
-   */
-  public TablePermission(String namespace, Action... assigned) {
-    super(assigned);
-    this.namespace = namespace;
-  }
-
-  public boolean hasTable() {
-    return table != null;
+    this.scope = Scope.TABLE;
   }
 
   public TableName getTableName() {
@@ -175,65 +139,58 @@ public class TablePermission extends Permission {
     return qualifier;
   }
 
-  public boolean hasNamespace() {
-    return namespace != null;
-  }
-
   public String getNamespace() {
-    return namespace;
+    return table.getNamespaceAsString();
   }
 
   /**
-   * Checks that a given table operation is authorized by this permission
-   * instance.
-   *
-   * @param namespace the namespace where the operation is being performed
-   * @param action the action being requested
-   * @return <code>true</code> if the action within the given scope is allowed
-   *   by this permission, <code>false</code>
+   * Check if given action can performs on given table:family:qualifier.
+   * @param table table name
+   * @param family family name
+   * @param qualifier qualifier name
+   * @param action one of [Read, Write, Create, Exec, Admin]
+   * @return true if can, false otherwise
    */
-  public boolean implies(String namespace, Action action) {
-    if (this.namespace == null || !this.namespace.equals(namespace)) {
+  public boolean implies(TableName table, byte[] family, byte[] qualifier, Action action) {
+    if (failCheckTable(table)) {
       return false;
     }
-
-    // check actions
-    return super.implies(action);
+    if (failCheckFamily(family)) {
+      return false;
+    }
+    if (failCheckQualifier(qualifier)) {
+      return false;
+    }
+    return implies(action);
   }
 
   /**
-   * Checks that a given table operation is authorized by this permission
-   * instance.
-   *
-   * @param table the table where the operation is being performed
-   * @param family the column family to which the operation is restricted,
-   *   if <code>null</code> implies "all"
-   * @param qualifier the column qualifier to which the action is restricted,
-   *   if <code>null</code> implies "all"
-   * @param action the action being requested
-   * @return <code>true</code> if the action within the given scope is allowed
-   *   by this permission, <code>false</code>
+   * Check if given action can performs on given table:family.
+   * @param table table name
+   * @param family family name
+   * @param action one of [Read, Write, Create, Exec, Admin]
+   * @return true if can, false otherwise
    */
-  public boolean implies(TableName table, byte[] family, byte[] qualifier,
-      Action action) {
-    if (this.table == null || !this.table.equals(table)) {
+  public boolean implies(TableName table, byte[] family, Action action) {
+    if (failCheckTable(table)) {
       return false;
     }
-
-    if (this.family != null &&
-        (family == null ||
-         !Bytes.equals(this.family, family))) {
+    if (failCheckFamily(family)) {
       return false;
     }
+    return implies(action);
+  }
 
-    if (this.qualifier != null &&
-        (qualifier == null ||
-         !Bytes.equals(this.qualifier, qualifier))) {
-      return false;
-    }
+  private boolean failCheckTable(TableName table) {
+    return this.table == null || !this.table.equals(table);
+  }
 
-    // check actions
-    return super.implies(action);
+  private boolean failCheckFamily(byte[] family) {
+    return this.family != null && (family == null || !Bytes.equals(this.family, family));
+  }
+
+  private boolean failCheckQualifier(byte[] qual) {
+    return this.qualifier != null && (qual == null || !Bytes.equals(this.qualifier, qual));
   }
 
   /**
@@ -246,7 +203,7 @@ public class TablePermission extends Permission {
    *   by this permission, otherwise <code>false</code>
    */
   public boolean implies(TableName table, KeyValue kv, Action action) {
-    if (this.table == null || !this.table.equals(table)) {
+    if (failCheckTable(table)) {
       return false;
     }
 
@@ -263,82 +220,34 @@ public class TablePermission extends Permission {
   }
 
   /**
-   * Returns <code>true</code> if this permission matches the given column
-   * family at least.  This only indicates a partial match against the table
-   * and column family, however, and does not guarantee that implies() for the
-   * column same family would return <code>true</code>.  In the case of a
-   * column-qualifier specific permission, for example, implies() would still
-   * return false.
+   * Check if fields of table in table permission equals.
+   * @param tp to be checked table permission
+   * @return true if equals, false otherwise
    */
-  public boolean matchesFamily(TableName table, byte[] family, Action action) {
-    if (this.table == null || !this.table.equals(table)) {
+  public boolean tableFieldsEqual(TablePermission tp) {
+    if (tp == null) {
       return false;
     }
 
-    if (this.family != null &&
-        (family == null ||
-         !Bytes.equals(this.family, family))) {
-      return false;
-    }
-
-    // ignore qualifier
-    // check actions
-    return super.implies(action);
+    boolean tEq = (table == null && tp.table == null) || (table != null && table.equals(tp.table));
+    boolean fEq = (family == null && tp.family == null) || Bytes.equals(family, tp.family);
+    boolean qEq = (qualifier == null && tp.qualifier == null) ||
+                   Bytes.equals(qualifier, tp.qualifier);
+    return tEq && fEq && qEq;
   }
 
-  /**
-   * Returns if the given permission matches the given qualifier.
-   * @param table the table name to match
-   * @param family the column family to match
-   * @param qualifier the qualifier name to match
-   * @param action the action requested
-   * @return <code>true</code> if the table, family and qualifier match,
-   *   otherwise <code>false</code>
-   */
-  public boolean matchesFamilyQualifier(TableName table, byte[] family, byte[] qualifier,
-                                Action action) {
-    if (!matchesFamily(table, family, action)) {
-      return false;
-    } else {
-      if (this.qualifier != null &&
-          (qualifier == null ||
-           !Bytes.equals(this.qualifier, qualifier))) {
-        return false;
-      }
-    }
-    return super.implies(action);
-  }
-
-  public boolean tableFieldsEqual(TablePermission other){
-    if (!(((table == null && other.getTableName() == null) ||
-           (table != null && table.equals(other.getTableName()))) &&
-         ((family == null && other.getFamily() == null) ||
-           Bytes.equals(family, other.getFamily())) &&
-         ((qualifier == null && other.getQualifier() == null) ||
-          Bytes.equals(qualifier, other.getQualifier())) &&
-         ((namespace == null && other.getNamespace() == null) ||
-          (namespace != null && namespace.equals(other.getNamespace())))
-    )) {
+  @Override
+  public boolean equalsExceptActions(Object obj) {
+    if (!(obj instanceof TablePermission)) {
       return false;
     }
-    return true;
+    TablePermission other = (TablePermission) obj;
+    return tableFieldsEqual(other);
   }
 
   @Override
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
-    justification="Passed on construction except on constructor not to be used")
   public boolean equals(Object obj) {
-    if (!(obj instanceof TablePermission)) {
-      return false;
-    }
-    TablePermission other = (TablePermission)obj;
-
-    if(!this.tableFieldsEqual(other)){
-      return false;
-    }
-
-    // check actions
-    return super.equals(other);
+    return equalsExceptActions(obj) && super.equals(obj);
   }
 
   @Override
@@ -354,41 +263,24 @@ public class TablePermission extends Permission {
     if (qualifier != null) {
       result = prime * result + Bytes.hashCode(qualifier);
     }
-    if (namespace != null) {
-      result = prime * result + namespace.hashCode();
-    }
     return result;
   }
 
   @Override
   public String toString() {
-    StringBuilder str = new StringBuilder("[TablePermission: ");
-    if(namespace != null) {
-      str.append("namespace=").append(namespace)
+    return "[TablePermission: " + rawExpression() + "]";
+  }
+
+  @Override
+  protected String rawExpression() {
+    StringBuilder raw = new StringBuilder();
+    if (table != null) {
+      raw.append("table=").append(table)
+         .append(", family=").append(family == null ? null : Bytes.toString(family))
+         .append(", qualifier=").append(qualifier == null ? null : Bytes.toString(qualifier))
          .append(", ");
     }
-    if(table != null) {
-       str.append("table=").append(table)
-          .append(", family=")
-          .append(family == null ? null : Bytes.toString(family))
-          .append(", qualifier=")
-          .append(qualifier == null ? null : Bytes.toString(qualifier))
-          .append(", ");
-    }
-    if (actions != null) {
-      str.append("actions=");
-      for (int i=0; i<actions.length; i++) {
-        if (i > 0)
-          str.append(",");
-        if (actions[i] != null)
-          str.append(actions[i].toString());
-        else
-          str.append("NULL");
-      }
-    }
-    str.append("]");
-
-    return str.toString();
+    return raw.toString() + super.rawExpression();
   }
 
   @Override
@@ -404,16 +296,13 @@ public class TablePermission extends Permission {
     if (in.readBoolean()) {
       qualifier = Bytes.readByteArray(in);
     }
-    if(in.readBoolean()) {
-      namespace = Bytes.toString(Bytes.readByteArray(in));
-    }
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
     // Explicitly writing null to maintain se/deserialize backward compatibility.
-    Bytes.writeByteArray(out, (table == null) ? null : table.getName());
+    Bytes.writeByteArray(out, table == null ? null : table.getName());
     out.writeBoolean(family != null);
     if (family != null) {
       Bytes.writeByteArray(out, family);
@@ -422,9 +311,5 @@ public class TablePermission extends Permission {
     if (qualifier != null) {
       Bytes.writeByteArray(out, qualifier);
     }
-    out.writeBoolean(namespace != null);
-    if(namespace != null) {
-      Bytes.writeByteArray(out, Bytes.toBytes(namespace));
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
index 72bd69f..2a9a109 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
@@ -18,167 +18,152 @@
 
 package org.apache.hadoop.hbase.security.access;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+import java.util.Objects;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Represents an authorization for access over the given table, column family
- * plus qualifier, for the given user.
+ * UserPermission consists of a user name and a permission.
+ * Permission can be one of [Global, Namespace, Table] permission.
  */
 @InterfaceAudience.Private
-public class UserPermission extends TablePermission {
-  private static final Logger LOG = LoggerFactory.getLogger(UserPermission.class);
+public class UserPermission {
 
-  private byte[] user;
+  private String user;
+  private Permission permission;
 
-  /** Nullary constructor for Writable, do not use */
-  public UserPermission() {
-    super();
+  /**
+   * Construct a global user permission.
+   * @param user user name
+   * @param assigned assigned actions
+   */
+  public UserPermission(String user, Permission.Action... assigned) {
+    this.user = user;
+    this.permission = new GlobalPermission(assigned);
   }
 
   /**
-   * Creates a new instance for the given user.
-   * @param user the user
-   * @param assigned the list of allowed actions
+   * Construct a global user permission.
+   * @param user user name
+   * @param actionCode action codes
    */
-  public UserPermission(byte[] user, Action... assigned) {
-    super(null, null, null, assigned);
+  public UserPermission(String user, byte[] actionCode) {
     this.user = user;
+    this.permission = new GlobalPermission(actionCode);
   }
 
   /**
-   * Creates a new instance for the given user,
-   * matching the actions with the given codes.
-   * @param user the user
-   * @param actionCodes the list of allowed action codes
+   * Construct a namespace user permission.
+   * @param user user name
+   * @param namespace namespace
+   * @param assigned assigned actions
    */
-  public UserPermission(byte[] user, byte[] actionCodes) {
-    super(null, null, null, actionCodes);
+  public UserPermission(String user, String namespace, Permission.Action... assigned) {
     this.user = user;
+    this.permission = new NamespacePermission(namespace, assigned);
   }
 
   /**
-   * Creates a new instance for the given user.
-   * @param user the user
-   * @param namespace
-   * @param assigned the list of allowed actions
+   * Construct a table user permission.
+   * @param user user name
+   * @param tableName table name
+   * @param assigned assigned actions
    */
-  public UserPermission(byte[] user, String namespace, Action... assigned) {
-    super(namespace, assigned);
+  public UserPermission(String user, TableName tableName, Permission.Action... assigned) {
     this.user = user;
+    this.permission = new TablePermission(tableName, assigned);
   }
 
   /**
-   * Creates a new instance for the given user,
-   * matching the actions with the given codes.
-   * @param user the user
-   * @param namespace
-   * @param actionCodes the list of allowed action codes
+   * Construct a table:family user permission.
+   * @param user user name
+   * @param tableName table name
+   * @param family family name of table
+   * @param assigned assigned actions
    */
-  public UserPermission(byte[] user, String namespace, byte[] actionCodes) {
-    super(namespace, actionCodes);
-    this.user = user;
+  public UserPermission(String user, TableName tableName, byte[] family,
+    Permission.Action... assigned) {
+    this(user, tableName, family, null, assigned);
   }
 
   /**
-   * Creates a new instance for the given user, table and column family.
-   * @param user the user
-   * @param table the table
-   * @param family the family, can be null if action is allowed over the entire
-   *   table
-   * @param assigned the list of allowed actions
+   * Construct a table:family:qualifier user permission.
+   * @param user user name
+   * @param tableName table name
+   * @param family family name of table
+   * @param qualifier qualifier name of table
+   * @param assigned assigned actions
    */
-  public UserPermission(byte[] user, TableName table, byte[] family,
-                        Action... assigned) {
-    super(table, family, assigned);
+  public UserPermission(String user, TableName tableName, byte[] family, byte[] qualifier,
+      Permission.Action... assigned) {
     this.user = user;
+    this.permission = new TablePermission(tableName, family, qualifier, assigned);
   }
 
   /**
-   * Creates a new permission for the given user, table, column family and
-   * column qualifier.
-   * @param user the user
-   * @param table the table
-   * @param family the family, can be null if action is allowed over the entire
-   *   table
-   * @param qualifier the column qualifier, can be null if action is allowed
-   *   over the entire column family
-   * @param assigned the list of allowed actions
+   * Construct a table:family:qualifier user permission.
+   * @param user user name
+   * @param tableName table name
+   * @param family family name of table
+   * @param qualifier qualifier name of table
+   * @param actionCodes assigned actions
    */
-  public UserPermission(byte[] user, TableName table, byte[] family,
-                        byte[] qualifier, Action... assigned) {
-    super(table, family, qualifier, assigned);
+  public UserPermission(String user, TableName tableName, byte[] family, byte[] qualifier,
+      byte[] actionCodes) {
     this.user = user;
+    this.permission = new TablePermission(tableName, family, qualifier, actionCodes);
   }
 
   /**
-   * Creates a new instance for the given user, table, column family and
-   * qualifier, matching the actions with the given codes.
-   * @param user the user
-   * @param table the table
-   * @param family the family, can be null if action is allowed over the entire
-   *   table
-   * @param qualifier the column qualifier, can be null if action is allowed
-   *   over the entire column family
-   * @param actionCodes the list of allowed action codes
+   * Construct a user permission given permission.
+   * @param user user name
+   * @param permission one of [Global, Namespace, Table] permission
    */
-  public UserPermission(byte[] user, TableName table, byte[] family,
-                        byte[] qualifier, byte[] actionCodes) {
-    super(table, family, qualifier, actionCodes);
+  public UserPermission(String user, Permission permission) {
     this.user = user;
+    this.permission = permission;
   }
 
   /**
-   * Creates a new instance for the given user, table, column family and
-   * qualifier, matching the actions with the given codes.
-   * @param user the user
-   * @param perm a TablePermission
+   * Get this permission access scope.
+   * @return access scope
    */
-  public UserPermission(byte[] user, TablePermission perm) {
-    super(perm.getNamespace(), perm.getTableName(), perm.getFamily(), perm.getQualifier(),
-        perm.actions);
-    this.user = user;
+  public Permission.Scope getAccessScope() {
+    return permission.getAccessScope();
   }
 
-  public byte[] getUser() {
+  public String getUser() {
     return user;
   }
 
-  /**
-   * Returns true if this permission describes a global user permission.
-   */
-  public boolean isGlobal() {
-    return(!hasTable() && !hasNamespace());
+  public Permission getPermission() {
+    return permission;
   }
 
-  @Override
-  public boolean equals(Object obj) {
+  public boolean equalsExceptActions(Object obj) {
     if (!(obj instanceof UserPermission)) {
       return false;
     }
-    UserPermission other = (UserPermission)obj;
+    UserPermission other = (UserPermission) obj;
+    return user.equals(other.user) && permission.equalsExceptActions(other.permission);
+  }
 
-    if ((Bytes.equals(user, other.getUser()) &&
-        super.equals(obj))) {
-      return true;
-    } else {
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof UserPermission)) {
       return false;
     }
+    UserPermission other = (UserPermission) obj;
+    return user.equals(other.user) && permission.equals(other.permission);
   }
 
   @Override
   public int hashCode() {
     final int prime = 37;
-    int result = super.hashCode();
+    int result = permission.hashCode();
     if (user != null) {
-      result = prime * result + Bytes.hashCode(user);
+      result = prime * result + Objects.hashCode(user);
     }
     return result;
   }
@@ -186,20 +171,8 @@ public class UserPermission extends TablePermission {
   @Override
   public String toString() {
     StringBuilder str = new StringBuilder("UserPermission: ")
-        .append("user=").append(Bytes.toString(user))
-        .append(", ").append(super.toString());
+        .append("user=").append(user)
+        .append(", ").append(permission.toString());
     return str.toString();
   }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    user = Bytes.readByteArray(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    Bytes.writeByteArray(out, user);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
index 78da55d..1cf43e1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
@@ -68,7 +68,7 @@ import org.slf4j.LoggerFactory;
  * an example of configuring a user of this Auth Chore to run on a secure cluster.
  * <pre>
  * </pre>
- * This class will be internal use only from 2.2.0 version, and will transparently work
+ * This class will be internal used only from 2.2.0 version, and will transparently work
  * for kerberized applications. For more, please refer
  * <a href="http://hbase.apache.org/book.html#hbase.secure.configuration">Client-side Configuration for Secure Operation</a>
  *

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
index b6f6463..59e5601 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsWithACL.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.security.access.AuthManager;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.SecureTestUtil;
-import org.apache.hadoop.hbase.security.access.TableAuthManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -203,7 +203,7 @@ public class TestRSGroupsWithACL extends SecureTestUtil{
   public static void tearDownAfterClass() throws Exception {
     cleanUp();
     TEST_UTIL.shutdownMiniCluster();
-    int total = TableAuthManager.getTotalRefCount();
+    int total = AuthManager.getTotalRefCount();
     assertTrue("Unexpected reference count: " + total, total == 0);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
index c31658f..986efd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -50,7 +50,7 @@ public final class AccessChecker {
   // TODO: we should move to a design where we don't even instantiate an AccessChecker if
   // authorization is not enabled (like in RSRpcServices), instead of always instantiating one and
   // calling requireXXX() only to do nothing (since authorizationEnabled will be false).
-  private TableAuthManager authManager;
+  private AuthManager authManager;
 
   /** Group service to retrieve the user group information */
   private static Groups groupService;
@@ -75,7 +75,7 @@ public final class AccessChecker {
       throws RuntimeException {
     if (zkw != null) {
       try {
-        this.authManager = TableAuthManager.getOrCreate(zkw, conf);
+        this.authManager = AuthManager.getOrCreate(zkw, conf);
       } catch (IOException ioe) {
         throw new RuntimeException("Error obtaining AccessChecker", ioe);
       }
@@ -87,13 +87,13 @@ public final class AccessChecker {
   }
 
   /**
-   * Releases {@link TableAuthManager}'s reference.
+   * Releases {@link AuthManager}'s reference.
    */
   public void stop() {
-    TableAuthManager.release(authManager);
+    AuthManager.release(authManager);
   }
 
-  public TableAuthManager getAuthManager() {
+  public AuthManager getAuthManager() {
     return authManager;
   }
 
@@ -115,7 +115,7 @@ public final class AccessChecker {
     AuthResult result = null;
 
     for (Action permission : permissions) {
-      if (authManager.hasAccess(user, tableName, permission)) {
+      if (authManager.accessUserTable(user, tableName, permission)) {
         result = AuthResult.allow(request, "Table permission granted",
             user, permission, tableName, null, null);
         break;
@@ -164,7 +164,7 @@ public final class AccessChecker {
       return;
     }
     AuthResult result;
-    if (authManager.authorize(user, perm)) {
+    if (authManager.authorizeUserGlobal(user, perm)) {
       result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
     } else {
       result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
@@ -195,7 +195,7 @@ public final class AccessChecker {
       return;
     }
     AuthResult authResult;
-    if (authManager.authorize(user, perm)) {
+    if (authManager.authorizeUserGlobal(user, perm)) {
       authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
@@ -225,7 +225,7 @@ public final class AccessChecker {
     AuthResult result = null;
 
     for (Action permission : permissions) {
-      if (authManager.authorize(user, namespace, permission)) {
+      if (authManager.authorizeUserNamespace(user, namespace, permission)) {
         result =
             AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         break;
@@ -260,7 +260,7 @@ public final class AccessChecker {
     AuthResult result = null;
 
     for (Action permission : permissions) {
-      if (authManager.authorize(user, namespace, permission)) {
+      if (authManager.authorizeUserNamespace(user, namespace, permission)) {
         result =
             AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
         result.getParams().setTableName(tableName).setFamilies(familyMap);
@@ -299,7 +299,7 @@ public final class AccessChecker {
     AuthResult result = null;
 
     for (Action permission : permissions) {
-      if (authManager.authorize(user, tableName, family, qualifier, permission)) {
+      if (authManager.authorizeUserTable(user, tableName, family, qualifier, permission)) {
         result = AuthResult.allow(request, "Table permission granted",
             user, permission, tableName, family, qualifier);
         break;
@@ -337,7 +337,7 @@ public final class AccessChecker {
     AuthResult result = null;
 
     for (Action permission : permissions) {
-      if (authManager.authorize(user, tableName, null, null, permission)) {
+      if (authManager.authorizeUserTable(user, tableName, permission)) {
         result = AuthResult.allow(request, "Table permission granted",
             user, permission, tableName, null, null);
         result.getParams().setFamily(family).setQualifier(qualifier);

http://git-wip-us.apache.org/repos/asf/hbase/blob/130057f1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
index fd48641..79233df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
  *
  * <p>
  * TODO: There is room for further performance optimization here.
- * Calling TableAuthManager.authorize() per KeyValue imposes a fair amount of
+ * Calling AuthManager.authorize() per KeyValue imposes a fair amount of
  * overhead.  A more optimized solution might look at the qualifiers where
  * permissions are actually granted and explicitly limit the scan to those.
  * </p>
@@ -58,7 +58,7 @@ class AccessControlFilter extends FilterBase {
     CHECK_CELL_DEFAULT,
   }
 
-  private TableAuthManager authManager;
+  private AuthManager authManager;
   private TableName table;
   private User user;
   private boolean isSystemTable;
@@ -75,7 +75,7 @@ class AccessControlFilter extends FilterBase {
   AccessControlFilter() {
   }
 
-  AccessControlFilter(TableAuthManager mgr, User ugi, TableName tableName,
+  AccessControlFilter(AuthManager mgr, User ugi, TableName tableName,
       Strategy strategy, Map<ByteRange, Integer> cfVsMaxVersions) {
     authManager = mgr;
     table = tableName;
@@ -119,20 +119,20 @@ class AccessControlFilter extends FilterBase {
       return ReturnCode.SKIP;
     }
     // XXX: Compare in place, don't clone
-    byte[] family = CellUtil.cloneFamily(cell);
-    byte[] qualifier = CellUtil.cloneQualifier(cell);
+    byte[] f = CellUtil.cloneFamily(cell);
+    byte[] q = CellUtil.cloneQualifier(cell);
     switch (strategy) {
       // Filter only by checking the table or CF permissions
       case CHECK_TABLE_AND_CF_ONLY: {
-        if (authManager.authorize(user, table, family, qualifier, Permission.Action.READ)) {
+        if (authManager.authorizeUserTable(user, table, f, q, Permission.Action.READ)) {
           return ReturnCode.INCLUDE;
         }
       }
       break;
       // Cell permissions can override table or CF permissions
       case CHECK_CELL_DEFAULT: {
-        if (authManager.authorize(user, table, family, qualifier, Permission.Action.READ) ||
-            authManager.authorize(user, table, cell, Permission.Action.READ)) {
+        if (authManager.authorizeUserTable(user, table, f, q, Permission.Action.READ) ||
+            authManager.authorizeCell(user, table, cell, Permission.Action.READ)) {
           return ReturnCode.INCLUDE;
         }
       }