You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/08/12 03:00:48 UTC

[1/2] hbase git commit: HBASE-13755 Provide single super user check implementation (Mikhail Antonov)

Repository: hbase
Updated Branches:
  refs/heads/0.98 f014ff766 -> 2eb2017c1


HBASE-13755 Provide single super user check implementation (Mikhail Antonov)

Conflicts:
	hbase-common/src/main/java/org/apache/hadoop/hbase/AuthUtil.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 2ad8da901975200adc835e4d9bfc1ed5199fa6df
Parents: f014ff7
Author: Srikanth Srungarapu <ss...@cloudera.com>
Authored: Thu Jun 11 20:51:05 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 11 17:35:44 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/AuthUtil.java  |  36 ++++++
 .../hadoop/hbase/security/Superusers.java       | 122 +++++++++++++++++++
 .../org/apache/hadoop/hbase/security/User.java  |   3 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +
 .../security/access/AccessControlLists.java     |  35 +-----
 .../hbase/security/access/AccessController.java |  26 ++--
 .../hbase/security/access/TableAuthManager.java |  24 ++--
 .../DefaultVisibilityLabelServiceImpl.java      |  34 ++----
 .../visibility/VisibilityController.java        |  27 +---
 .../visibility/VisibilityLabelsCache.java       |   6 +-
 .../security/visibility/VisibilityUtils.java    |  34 ------
 .../hbase/regionserver/TestPriorityRpc.java     |   2 +-
 .../hbase/regionserver/TestQosFunction.java     |   8 +-
 .../hbase/security/access/SecureTestUtil.java   |   6 +-
 .../security/access/TestAccessController.java   |   9 +-
 .../security/access/TestAccessController2.java  |   3 +-
 .../security/access/TestNamespaceCommands.java  |   9 +-
 .../ExpAsStringVisibilityLabelServiceImpl.java  |  58 ++-------
 18 files changed, 227 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/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 ec77b79..ec9f796 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
@@ -39,6 +39,14 @@ import org.apache.hadoop.security.UserGroupInformation;
 @InterfaceStability.Evolving
 public class AuthUtil {
   private static final Log LOG = LogFactory.getLog(AuthUtil.class);
+
+  /** Prefix character to denote group names */
+  public static final String GROUP_PREFIX = "@";
+
+  private AuthUtil() {
+    super();
+  }
+
   /**
    * Checks if security is enabled and if so, launches chore for refreshing kerberos ticket.
    */
@@ -95,4 +103,32 @@ public class AuthUtil {
     // Start the chore for refreshing credentials
     Threads.setDaemonThreadRunning(refreshCredentials.getThread());
   }
+
+  /**
+   * Returns whether or not the given name should be interpreted as a group
+   * principal.  Currently this simply checks if the name starts with the
+   * special group prefix character ("@").
+   */
+  public static boolean isGroupPrincipal(String name) {
+    return name != null && name.startsWith(GROUP_PREFIX);
+  }
+
+  /**
+   * Returns the actual name for a group principal (stripped of the
+   * group prefix).
+   */
+  public static String getGroupName(String aclKey) {
+    if (!isGroupPrincipal(aclKey)) {
+      return aclKey;
+    }
+
+    return aclKey.substring(GROUP_PREFIX.length());
+  }
+
+  /**
+   * Returns the group entry with the group prefix for a group principal.
+   */
+  public static String toGroupEntry(String name) {
+    return GROUP_PREFIX + name;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
new file mode 100644
index 0000000..3374343
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
@@ -0,0 +1,122 @@
+/*
+ *
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Keeps lists of superusers and super groups loaded from HBase configuration,
+ * checks if certain user is regarded as superuser.
+ */
+@InterfaceAudience.Private
+public final class Superusers {
+  private static final Log LOG = LogFactory.getLog(Superusers.class);
+
+  /** Configuration key for superusers */
+  public static final String SUPERUSER_CONF_KEY = "hbase.superuser"; // Not getting a name
+
+  private static List<String> superUsers;
+  private static List<String> superGroups;
+
+  private Superusers(){}
+
+  /**
+   * Should be called only once to pre-load list of super users and super
+   * groups from Configuration. This operation is idempotent.
+   * @param conf configuration to load users from
+   * @throws IOException if unable to initialize lists of superusers or super groups
+   * @throws IllegalStateException if current user is null
+   */
+  public static void initialize(Configuration conf) throws IOException {
+    superUsers = new ArrayList<String>();
+    superGroups = new ArrayList<String>();
+    User user = User.getCurrent();
+
+    if (user == null) {
+      throw new IllegalStateException("Unable to obtain the current user, "
+        + "authorization checks for internal operations will not work correctly!");
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Current user name is " + user.getShortName());
+    }
+    String currentUser = user.getShortName();
+    String[] superUserList = conf.getStrings(SUPERUSER_CONF_KEY, new String[0]);
+    for (String name : superUserList) {
+      if (AuthUtil.isGroupPrincipal(name)) {
+        superGroups.add(AuthUtil.getGroupName(name));
+      } else {
+        superUsers.add(name);
+      }
+    }
+    superUsers.add(currentUser);
+  }
+
+  /**
+   * @return true if current user is a super user (whether as user running process,
+   * declared as individual superuser or member of supergroup), false otherwise.
+   * @param user to check
+   * @throws IllegalStateException if lists of superusers/super groups
+   *   haven't been initialized properly
+   */
+  public static boolean isSuperUser(User user) {
+    if (superUsers == null) {
+      throw new IllegalStateException("Super users/super groups lists"
+        + " haven't been initialized properly.");
+    }
+    if (superUsers.contains(user.getShortName())) {
+      return true;
+    }
+
+    for (String group : user.getGroupNames()) {
+      if (superGroups.contains(group)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @return true if user is a super user, false otherwise.
+   * @param user to check
+   * @throws IllegalStateException if lists of superusers/super groups
+   *   haven't been initialized properly
+   * @deprecated Use {@link #isSuperUser(User)}
+   */
+  @Deprecated
+  public static boolean isSuperUser(String user) {
+    if (superUsers == null) {
+      throw new IllegalStateException("Super users/super groups lists"
+        + " haven't been initialized properly.");
+    }
+    if (superUsers.contains(user)) {
+      return true;
+    }
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
index 27195d0..c4d6af4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
@@ -234,7 +234,8 @@ public abstract class User {
    */
   public static User createUserForTesting(Configuration conf,
       String name, String[] groups) {
-    return SecureHadoopUser.createUserForTesting(conf, name, groups);
+    User userForTesting = SecureHadoopUser.createUserForTesting(conf, name, groups);
+    return userForTesting;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index c36a5d4..95aa124 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -221,6 +221,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -564,6 +565,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
     checkCodecs(this.conf);
     this.userProvider = UserProvider.instantiate(conf);
 
+    Superusers.initialize(conf);
+
     FSUtils.setupShortCircuitRead(this.conf);
 
     // Config'ed params

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/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 1b1afb1..41f7b63 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
@@ -35,6 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -111,10 +112,6 @@ public class AccessControlLists {
    * Delimiter to separate user, column family, and qualifier in
    * _acl_ table info: column keys */
   public static final char ACL_KEY_DELIMITER = ',';
-  /** Prefix character to denote group names */
-  public static final String GROUP_PREFIX = "@";
-  /** Configuration key for superusers */
-  public static final String SUPERUSER_CONF_KEY = "hbase.superuser";
 
   private static Log LOG = LogFactory.getLog(AccessControlLists.class);
 
@@ -623,34 +620,6 @@ public class AccessControlLists {
     }
   }
 
-  /**
-   * Returns whether or not the given name should be interpreted as a group
-   * principal.  Currently this simply checks if the name starts with the
-   * special group prefix character ("@").
-   */
-  public static boolean isGroupPrincipal(String name) {
-    return name != null && name.startsWith(GROUP_PREFIX);
-  }
-
-  /**
-   * Returns the actual name for a group principal (stripped of the
-   * group prefix).
-   */
-  public static String getGroupName(String aclKey) {
-    if (!isGroupPrincipal(aclKey)) {
-      return aclKey;
-    }
-
-    return aclKey.substring(GROUP_PREFIX.length());
-  }
-
-  /**
-   * Returns the group entry with the group prefix for a group principal.
-   */
-  public static String toGroupEntry(String name) {
-    return GROUP_PREFIX + name;
-  }
-
   public static boolean isNamespaceEntry(String entryName) {
     return entryName.charAt(0) == NAMESPACE_PREFIX;
   }
@@ -713,7 +682,7 @@ public class AccessControlLists {
          String groupNames[] = user.getGroupNames();
          if (groupNames != null) {
            for (String group : groupNames) {
-             List<Permission> groupPerms = kvPerms.get(GROUP_PREFIX + group);
+             List<Permission> groupPerms = kvPerms.get(AuthUtil.toGroupEntry(group));
              if (results != null) {
                results.addAll(groupPerms);
              }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/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 d0b9e50..11caabc 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
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
+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.security.access.Permission.Action;
@@ -182,9 +183,6 @@ public class AccessController extends BaseMasterAndRegionObserver
   /** Provider for mapping principal names to Users */
   private UserProvider userProvider;
 
-  /** The list of users with superuser authority */
-  private List<String> superusers;
-
   /** if we are active, usually true, only not true if "hbase.security.authorization"
    has been set to false in site configuration */
   boolean authorizationEnabled;
@@ -889,7 +887,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       return;
     }
     // Superusers are allowed to store cells unconditionally.
-    if (superusers.contains(user.getShortName())) {
+    if (Superusers.isSuperUser(user)) {
       m.setAttribute(TAG_CHECK_PASSED, TRUE);
       return;
     }
@@ -953,11 +951,6 @@ public class AccessController extends BaseMasterAndRegionObserver
     // set the user-provider.
     this.userProvider = UserProvider.instantiate(env.getConfiguration());
 
-    // set up the list of users with superuser privilege
-    User user = userProvider.getCurrent();
-    superusers = Lists.asList(user.getShortName(),
-      conf.getStrings(AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
-
     // If zk is null or IOException while obtaining auth manager,
     // throw RuntimeException so that the coprocessor is unloaded.
     if (zk != null) {
@@ -1312,7 +1305,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     } else {
       HRegionInfo regionInfo = region.getRegionInfo();
       if (regionInfo.getTable().isSystemTable()) {
-        isSystemOrSuperUser(regionEnv.getConfiguration());
+        checkSystemOrSuperUser();
       } else {
         requirePermission("preOpen", Action.ADMIN);
       }
@@ -2366,20 +2359,15 @@ public class AccessController extends BaseMasterAndRegionObserver
     requirePermission("preClose", Action.ADMIN);
   }
 
-  private void isSystemOrSuperUser(Configuration conf) throws IOException {
+  private void checkSystemOrSuperUser() throws IOException {
     // No need to check if we're not going to throw
     if (!authorizationEnabled) {
       return;
     }
-    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!");
-    }
     User activeUser = getActiveUser();
-    if (!(superusers.contains(activeUser.getShortName()))) {
-      throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null") +
-        "is not system or super user.");
+    if (!Superusers.isSuperUser(activeUser)) {
+      throw new AccessDeniedException("User '" + (activeUser != null ?
+        activeUser.getShortName() : "null") + "is not system or super user.");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/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
index 8e0933c..4a98d02 100644
--- 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
@@ -26,11 +26,13 @@ import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+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;
@@ -78,13 +80,13 @@ public class TableAuthManager {
 
     /**
      * Returns a combined map of user and group permissions, with group names prefixed by
-     * {@link AccessControlLists#GROUP_PREFIX}.
+     * {@link AuthUtil#GROUP_PREFIX}.
      */
     public ListMultimap<String,T> getAllPermissions() {
       ListMultimap<String,T> tmp = ArrayListMultimap.create();
       tmp.putAll(userCache);
       for (String group : groupCache.keySet()) {
-        tmp.putAll(AccessControlLists.GROUP_PREFIX + group, groupCache.get(group));
+        tmp.putAll(AuthUtil.toGroupEntry(group), groupCache.get(group));
       }
       return tmp;
     }
@@ -138,11 +140,11 @@ public class TableAuthManager {
 
     // the system user is always included
     List<String> superusers = Lists.asList(currentUser, conf.getStrings(
-        AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
+        Superusers.SUPERUSER_CONF_KEY, new String[0]));
     if (superusers != null) {
       for (String name : superusers) {
-        if (AccessControlLists.isGroupPrincipal(name)) {
-          newCache.putGroup(AccessControlLists.getGroupName(name),
+        if (AuthUtil.isGroupPrincipal(name)) {
+          newCache.putGroup(AuthUtil.getGroupName(name),
               new Permission(Permission.Action.values()));
         } else {
           newCache.putUser(name, new Permission(Permission.Action.values()));
@@ -204,8 +206,8 @@ public class TableAuthManager {
     try {
       newCache = initGlobal(conf);
       for (Map.Entry<String,TablePermission> entry : userPerms.entries()) {
-        if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
-          newCache.putGroup(AccessControlLists.getGroupName(entry.getKey()),
+        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()));
@@ -232,8 +234,8 @@ public class TableAuthManager {
     PermissionCache<TablePermission> newTablePerms = new PermissionCache<TablePermission>();
 
     for (Map.Entry<String,TablePermission> entry : tablePerms.entries()) {
-      if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
-        newTablePerms.putGroup(AccessControlLists.getGroupName(entry.getKey()), entry.getValue());
+      if (AuthUtil.isGroupPrincipal(entry.getKey())) {
+        newTablePerms.putGroup(AuthUtil.getGroupName(entry.getKey()), entry.getValue());
       } else {
         newTablePerms.putUser(entry.getKey(), entry.getValue());
       }
@@ -256,8 +258,8 @@ public class TableAuthManager {
     PermissionCache<TablePermission> newTablePerms = new PermissionCache<TablePermission>();
 
     for (Map.Entry<String, TablePermission> entry : tablePerms.entries()) {
-      if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
-        newTablePerms.putGroup(AccessControlLists.getGroupName(entry.getKey()), entry.getValue());
+      if (AuthUtil.isGroupPrincipal(entry.getKey())) {
+        newTablePerms.putGroup(AuthUtil.getGroupName(entry.getKey()), entry.getValue());
       } else {
         newTablePerms.putUser(entry.getKey(), entry.getValue());
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
index 9ff6254..f498301 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
@@ -41,6 +41,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
@@ -57,8 +58,8 @@ import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.OperationStatus;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -78,8 +79,6 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
   private HRegion labelsRegion;
   private VisibilityLabelsCache labelsCache;
   private List<ScanLabelGenerator> scanLabelGenerators;
-  private List<String> superUsers;
-  private List<String> superGroups;
 
   static {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -116,10 +115,6 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
       throw ioe;
     }
     this.scanLabelGenerators = VisibilityUtils.getScanLabelGenerators(this.conf);
-    Pair<List<String>, List<String>> superUsersAndGroups =
-        VisibilityUtils.getSystemAndSuperUsers(this.conf);
-    this.superUsers = superUsersAndGroups.getFirst();
-    this.superGroups = superUsersAndGroups.getSecond();
     if (e.getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
       this.labelsRegion = e.getRegion();
       Pair<Map<String, Integer>, Map<String, List<Integer>>> labelsAndUserAuths =
@@ -264,8 +259,8 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
     assert labelsRegion != null;
     OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
     List<String> currentAuths;
-    if (AccessControlLists.isGroupPrincipal(Bytes.toString(user))) {
-      String group = AccessControlLists.getGroupName(Bytes.toString(user));
+    if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
+      String group = AuthUtil.getGroupName(Bytes.toString(user));
       currentAuths = this.getGroupAuths(new String[]{group}, true);
     }
     else {
@@ -370,7 +365,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
     Scan s = new Scan();
     if (groups != null && groups.length > 0) {
       for (String group : groups) {
-        s.addColumn(LABELS_TABLE_FAMILY, Bytes.toBytes(AccessControlLists.toGroupEntry(group)));
+        s.addColumn(LABELS_TABLE_FAMILY, Bytes.toBytes(AuthUtil.toGroupEntry(group)));
       }
     }
     Filter filter = VisibilityUtils.createVisibilityLabelFilter(this.labelsRegion,
@@ -541,7 +536,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
   @Deprecated
   public boolean havingSystemAuth(byte[] user) throws IOException {
     // Implementation for backward compatibility
-    if (this.superUsers.contains(Bytes.toString(user))) {
+    if (Superusers.isSuperUser(Bytes.toString(user))) {
       return true;
     }
     List<String> auths = this.getUserAuths(user, true);
@@ -554,7 +549,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
   @Override
   public boolean havingSystemAuth(User user) throws IOException {
     // A super user has 'system' auth.
-    if (isSystemOrSuperUser(user)) {
+    if (Superusers.isSuperUser(user)) {
       return true;
     }
     // A user can also be explicitly granted 'system' auth.
@@ -572,21 +567,6 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
     return auths.contains(SYSTEM_LABEL);
   }
 
-  private boolean isSystemOrSuperUser(User user) throws IOException {
-    if (this.superUsers.contains(user.getShortName())) {
-      return true;
-    }
-    String[] groups = user.getGroupNames();
-    if (groups != null && groups.length > 0) {
-      for (String group : groups) {
-        if (this.superGroups.contains(group)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
   @Override
   public boolean matchVisibility(List<Tag> putVisTags, Byte putTagsFormat, List<Tag> deleteVisTags,
       Byte deleteTagsFormat) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 348e229..c21a457 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -99,8 +100,8 @@ import org.apache.hadoop.hbase.regionserver.OperationStatus;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -135,8 +136,6 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   private Map<InternalScanner,String> scannerOwners =
       new MapMaker().weakKeys().makeMap();
 
-  private List<String> superUsers;
-  private List<String> superGroups;
   private VisibilityLabelService visibilityLabelService;
 
   /** if we are active, usually true, only not true if "hbase.security.authorization"
@@ -175,10 +174,6 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       visibilityLabelService = VisibilityLabelServiceManager.getInstance()
           .getVisibilityLabelService(this.conf);
     }
-    Pair<List<String>, List<String>> superUsersAndGroups =
-        VisibilityUtils.getSystemAndSuperUsers(this.conf);
-    this.superUsers = superUsersAndGroups.getFirst();
-    this.superGroups = superUsersAndGroups.getSecond();
   }
 
   @Override
@@ -696,19 +691,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   }
 
   private boolean isSystemOrSuperUser() throws IOException {
-    User activeUser = VisibilityUtils.getActiveUser();
-    if (this.superUsers.contains(activeUser.getShortName())) {
-      return true;
-    }
-    String[] groups = activeUser.getGroupNames();
-    if (groups != null && groups.length > 0) {
-      for (String group : groups) {
-        if (this.superGroups.contains(group)) {
-          return true;
-        }
-      }
-    }
-    return false;
+    return Superusers.isSuperUser(VisibilityUtils.getActiveUser());
   }
 
   @Override
@@ -944,7 +927,8 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
               + (requestingUser != null ? requestingUser.getShortName() : "null")
               + "' is not authorized to perform this action.");
         }
-        if (AccessControlLists.isGroupPrincipal(Bytes.toString(user))) {
+        if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
+          String group = AuthUtil.getGroupName(Bytes.toString(user));
           // For backward compatibility. Previous custom visibilityLabelService
           // implementation may not have getGroupAuths
           try {
@@ -956,7 +940,6 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
             throw new AccessDeniedException(
                 "Get group auth is not supported in this implementation");
           }
-          String group = AccessControlLists.getGroupName(Bytes.toString(user));
           labels = this.visibilityLabelService.getGroupAuths(new String[]{group}, false);
         }
         else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
index ed2998a..4cf39c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsCache.java
@@ -30,12 +30,12 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -145,8 +145,8 @@ public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
       this.groupAuths.clear();
       for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
         String user = Bytes.toString(userAuths.getUser().toByteArray());
-        if (AccessControlLists.isGroupPrincipal(user)) {
-          this.groupAuths.put(AccessControlLists.getGroupName(user),
+        if (AuthUtil.isGroupPrincipal(user)) {
+          this.groupAuths.put(AuthUtil.getGroupName(user),
             new HashSet<Integer>(userAuths.getAuthList()));
         } else {
           this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
index 40aa4d9..26772ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
@@ -62,7 +61,6 @@ import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.SimpleByteRange;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /**
@@ -101,38 +99,6 @@ public class VisibilityUtils {
   }
 
   /**
-   * Get the super users and groups defined in the configuration.
-   * The user running the hbase server is always included.
-   * @param conf
-   * @return Pair of super user list and super group list.
-   * @throws IOException
-   */
-  public static Pair<List<String>, List<String>> getSystemAndSuperUsers(Configuration conf)
-      throws IOException {
-    ArrayList<String> superUsers = new ArrayList<String>();
-    ArrayList<String> superGroups = new ArrayList<String>();
-    User user = User.getCurrent();
-    if (user == null) {
-      throw new IOException("Unable to obtain the current user, "
-          + "authorization checks for internal operations will not work correctly!");
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Current user name is " + user.getShortName());
-    }
-    String currentUser = user.getShortName();
-    String[] superUserList = conf.getStrings(AccessControlLists.SUPERUSER_CONF_KEY, new String[0]);
-    for (String name : superUserList) {
-      if (AccessControlLists.isGroupPrincipal(name)) {
-        superGroups.add(AccessControlLists.getGroupName(name));
-      } else {
-        superUsers.add(name);
-      }
-    }
-    superUsers.add(currentUser);
-    return new Pair<List<String>, List<String>>(superUsers, superGroups);
-  }
-
-  /**
    * Creates the user auth data to be written to zookeeper.
    * @param userAuths
    * @return Bytes form of user auths details to be written to zookeeper.

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
index f898d7d..fa02fca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
@@ -24,12 +24,12 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
index d5f890c..6cce075 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java
@@ -1,4 +1,3 @@
-package org.apache.hadoop.hbase.regionserver;
 /**
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
@@ -16,6 +15,9 @@ package org.apache.hadoop.hbase.regionserver;
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
+
+package org.apache.hadoop.hbase.regionserver;
+
 import static org.junit.Assert.assertEquals;
 
 import static org.mockito.Mockito.when;
@@ -39,6 +41,8 @@ import org.mockito.Mockito;
 
 import com.google.protobuf.Message;
 
+import java.io.IOException;
+
 /**
  * Basic test that qos function is sort of working; i.e. a change in method naming style
  * over in pb doesn't break it.
@@ -60,7 +64,7 @@ public class TestQosFunction {
   }
 
   @Test
-  public void testRegionInTransition() {
+  public void testRegionInTransition() throws IOException {
     Configuration conf = HBaseConfiguration.create();
     HRegionServer rs = Mockito.mock(HRegionServer.class);
     when(rs.getConfiguration()).thenReturn(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index fce5026..3d415b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -623,11 +623,7 @@ public class SecureTestUtil {
     return AccessControlLists.NAMESPACE_PREFIX + namespace;
   }
 
-  public static String convertToGroup(String group) {
-    return AccessControlLists.GROUP_PREFIX + group;
-  }
-
-  public void checkGlobalPerms(HBaseTestingUtility testUtil, Permission.Action... actions)
+  public static void checkGlobalPerms(HBaseTestingUtility testUtil, Permission.Action... actions)
       throws IOException {
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 56af4a3..a7ec07f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.security.access;
 
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -275,10 +276,10 @@ public class TestAccessController extends SecureTestUtil {
       TEST_TABLE, TEST_FAMILY,
       null, Permission.Action.ADMIN, Permission.Action.CREATE);
 
-    grantGlobal(TEST_UTIL, convertToGroup(GROUP_ADMIN), Permission.Action.ADMIN);
-    grantGlobal(TEST_UTIL, convertToGroup(GROUP_CREATE), Permission.Action.CREATE);
-    grantGlobal(TEST_UTIL, convertToGroup(GROUP_READ), Permission.Action.READ);
-    grantGlobal(TEST_UTIL, convertToGroup(GROUP_WRITE), Permission.Action.WRITE);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_CREATE), Permission.Action.CREATE);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_READ), Permission.Action.READ);
+    grantGlobal(TEST_UTIL, toGroupEntry(GROUP_WRITE), Permission.Action.WRITE);
 
     assertEquals(5, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
index a296d89..b43ace4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -98,7 +99,7 @@ public class TestAccessController2 extends SecureTestUtil {
     // Wait for the ACL table to become available
     TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
 
-    TESTGROUP_1_NAME = convertToGroup(TESTGROUP_1);
+    TESTGROUP_1_NAME = AuthUtil.toGroupEntry(TESTGROUP_1);
     TESTGROUP1_USER1 =
         User.createUserForTesting(conf, "testgroup1_user1", new String[] { TESTGROUP_1 });
     TESTGROUP2_USER1 =

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index b283afb..f80bceb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.security.access;
 
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -118,10 +119,10 @@ public class TestNamespaceCommands extends SecureTestUtil {
     grantOnNamespace(UTIL, USER_NSP_ADMIN.getShortName(), TEST_NAMESPACE, Permission.Action.ADMIN);
     grantOnNamespace(UTIL, USER_NSP_ADMIN.getShortName(), TEST_NAMESPACE2, Permission.Action.ADMIN);
 
-    grantGlobal(UTIL, convertToGroup(GROUP_ADMIN), Permission.Action.ADMIN);
-    grantGlobal(UTIL, convertToGroup(GROUP_CREATE), Permission.Action.CREATE);
-    grantGlobal(UTIL, convertToGroup(GROUP_READ), Permission.Action.READ);
-    grantGlobal(UTIL, convertToGroup(GROUP_WRITE), Permission.Action.WRITE);
+    grantGlobal(UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+    grantGlobal(UTIL, toGroupEntry(GROUP_CREATE), Permission.Action.CREATE);
+    grantGlobal(UTIL, toGroupEntry(GROUP_READ), Permission.Action.READ);
+    grantGlobal(UTIL, toGroupEntry(GROUP_WRITE), Permission.Action.WRITE);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/2ad8da90/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
index c1c690b..8b1f2a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java
@@ -35,6 +35,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
@@ -49,16 +50,14 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.OperationStatus;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
 import org.apache.hadoop.hbase.security.visibility.expression.Operator;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.common.collect.Lists;
-
 /**
  * This is a VisibilityLabelService where labels in Mutation's visibility
  * expression will be persisted as Strings itself rather than ordinals in
@@ -80,8 +79,6 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
   private Configuration conf;
   private HRegion labelsRegion;
   private List<ScanLabelGenerator> scanLabelGenerators;
-  private List<String> superUsers;
-  private List<String> superGroups;
 
   @Override
   public OperationStatus[] addLabels(List<byte[]> labels) throws IOException {
@@ -116,8 +113,8 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
     assert labelsRegion != null;
     OperationStatus[] finalOpStatus = new OperationStatus[authLabels.size()];
     List<String> currentAuths;
-    if (AccessControlLists.isGroupPrincipal(Bytes.toString(user))) {
-      String group = AccessControlLists.getGroupName(Bytes.toString(user));
+    if (AuthUtil.isGroupPrincipal(Bytes.toString(user))) {
+      String group = AuthUtil.getGroupName(Bytes.toString(user));
       currentAuths = this.getGroupAuths(new String[]{group}, true);
     }
     else {
@@ -189,7 +186,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
     List<String> auths = new ArrayList<String>();
     if (groups != null && groups.length > 0) {
       for (String group : groups) {
-        Get get = new Get(Bytes.toBytes(AccessControlLists.toGroupEntry(group)));
+        Get get = new Get(Bytes.toBytes(AuthUtil.toGroupEntry(group)));
         List<Cell> cells = null;
         if (labelsRegion == null) {
           HTable table = null;
@@ -389,57 +386,16 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
   @Override
   public void init(RegionCoprocessorEnvironment e) throws IOException {
     this.scanLabelGenerators = VisibilityUtils.getScanLabelGenerators(this.conf);
-    initSystemAndSuperUsers();
     if (e.getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
       this.labelsRegion = e.getRegion();
     }
   }
 
-  private void initSystemAndSuperUsers() throws IOException {
-    this.superUsers = new ArrayList<String>();
-    this.superGroups = new ArrayList<String>();
-    User user = User.getCurrent();
-    if (user == null) {
-      throw new IOException("Unable to obtain the current user, "
-          + "authorization checks for internal operations will not work correctly!");
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Current user name is " + user.getShortName());
-    }
-    String currentUser = user.getShortName();
-    List<String> superUserList = Lists.asList(currentUser,
-        this.conf.getStrings(AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
-    if (superUserList != null) {
-      for (String name : superUserList) {
-        if (AccessControlLists.isGroupPrincipal(name)) {
-          this.superGroups.add(AccessControlLists.getGroupName(name));
-        } else {
-          this.superUsers.add(name);
-        }
-      }
-    };
-  }
-
-  protected boolean isSystemOrSuperUser(User user) throws IOException {
-    if (this.superUsers.contains(user.getShortName())) {
-      return true;
-    }
-    String[] groups = user.getGroupNames();
-    if (groups != null) {
-      for (String group : groups) {
-        if (this.superGroups.contains(group)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
   @Override
   @Deprecated
   public boolean havingSystemAuth(byte[] user) throws IOException {
     // Implementation for backward compatibility
-    if (this.superUsers.contains(Bytes.toString(user))) {
+    if (Superusers.isSuperUser(Bytes.toString(user))) {
       return true;
     }
     List<String> auths = this.getUserAuths(user, true);
@@ -451,7 +407,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
 
   @Override
   public boolean havingSystemAuth(User user) throws IOException {
-    if (isSystemOrSuperUser(user)) {
+    if (Superusers.isSuperUser(user)) {
       return true;
     }
     Set<String> auths = new HashSet<String>();


[2/2] hbase git commit: HBASE-14057 HBase shell user_permission should list super users defined on hbase-site.xml

Posted by ap...@apache.org.
HBASE-14057 HBase shell user_permission should list super users defined on hbase-site.xml

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java


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

Branch: refs/heads/0.98
Commit: 2eb2017c1b7f9e17dcd95e3f1eacdc17bd735b03
Parents: 2ad8da9
Author: Srikanth Srungarapu <ss...@cloudera.com>
Authored: Tue Jul 28 11:43:24 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Aug 11 17:39:29 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/security/Superusers.java       |  4 +++
 .../hbase/security/access/AccessController.java | 19 +++++++++----
 .../security/access/TestAccessController.java   | 30 ++++++++++++++------
 3 files changed, 39 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2eb2017c/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
index 3374343..b196961 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/Superusers.java
@@ -119,4 +119,8 @@ public final class Superusers {
     }
     return false;
   }
+
+  public static List<String> getSuperUsers() {
+    return superUsers;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2eb2017c/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 11caabc..36394a3 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
@@ -1113,7 +1113,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       @Override
       public Void run() throws Exception {
         UserPermission userperm = new UserPermission(Bytes.toBytes(owner),
-          htd.getTableName(), null, Action.values());
+            htd.getTableName(), null, Action.values());
         AccessControlLists.addUserPermission(conf, userperm);
         return null;
       }
@@ -1130,7 +1130,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
       HColumnDescriptor descriptor) throws IOException {
     requirePermission("modifyColumn", tableName, descriptor.getName(), null, Action.ADMIN,
-      Action.CREATE);
+        Action.CREATE);
   }
 
   @Override
@@ -1284,7 +1284,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     });
     this.authManager.getZKPermissionWatcher().deleteNamespaceACLNode(namespace);
-    LOG.info(namespace + " entry deleted in "+AccessControlLists.ACL_TABLE_NAME+" table.");
+    LOG.info(namespace + " entry deleted in " + AccessControlLists.ACL_TABLE_NAME + " table.");
   }
 
   @Override
@@ -1710,7 +1710,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
     User user = getActiveUser();
     AuthResult authResult = permissionGranted(OpType.CHECK_AND_DELETE, user, env, families,
-      Action.READ, Action.WRITE);
+        Action.READ, Action.WRITE);
     logResult(authResult);
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
@@ -1762,7 +1762,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
     User user = getActiveUser();
     AuthResult authResult = permissionGranted(OpType.INCREMENT_COLUMN_VALUE, user, env, families,
-      Action.WRITE);
+        Action.WRITE);
     if (!authResult.isAllowed() && cellFeaturesEnabled && !compatibleEarlyTermination) {
       authResult.setAllowed(checkCoveringPermission(OpType.INCREMENT_COLUMN_VALUE, env, row,
         families, HConstants.LATEST_TIMESTAMP, Action.WRITE));
@@ -1948,7 +1948,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           LOG.trace("Carrying forward ACLs from " + oldCell + ": " + perms);
         }
         tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE,
-          ProtobufUtil.toUsersAndPermissions(perms).toByteArray()));
+            ProtobufUtil.toUsersAndPermissions(perms).toByteArray()));
       }
     }
 
@@ -2237,6 +2237,13 @@ public class AccessController extends BaseMasterAndRegionObserver
               return AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), null);
             }
           });
+          // Adding superusers explicitly to the result set as AccessControlLists do not store 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(user.getBytes(), AccessControlLists.ACL_TABLE_NAME, null,
+                Action.values()));
+          }
         }
         response = ResponseConverter.buildGetUserPermissionsResponse(perms);
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2eb2017c/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index a7ec07f..0e33aac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -94,6 +95,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -419,7 +421,7 @@ public class TestAccessController extends SecureTestUtil {
     };
 
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_ADMIN_CF,
-      USER_GROUP_CREATE, USER_GROUP_ADMIN);
+        USER_GROUP_CREATE, USER_GROUP_ADMIN);
     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
   }
 
@@ -663,7 +665,7 @@ public class TestAccessController extends SecureTestUtil {
 
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
-      USER_GROUP_WRITE, USER_GROUP_CREATE);
+        USER_GROUP_WRITE, USER_GROUP_CREATE);
   }
 
   @Test
@@ -739,7 +741,7 @@ public class TestAccessController extends SecureTestUtil {
   private void verifyReadWrite(AccessTestAction action) throws Exception {
     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW);
     verifyDenied(action, USER_NONE, USER_RO, USER_GROUP_ADMIN, USER_GROUP_CREATE, USER_GROUP_READ,
-      USER_GROUP_WRITE);
+        USER_GROUP_WRITE);
   }
 
   @Test
@@ -1100,7 +1102,7 @@ public class TestAccessController extends SecureTestUtil {
 
     verifyAllowed(grantAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
     verifyDenied(grantAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
-      USER_GROUP_WRITE, USER_GROUP_CREATE);
+        USER_GROUP_WRITE, USER_GROUP_CREATE);
     try {
       verifyAllowed(revokeAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
       verifyDenied(revokeAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
@@ -1391,6 +1393,11 @@ public class TestAccessController extends SecureTestUtil {
     }
   }
 
+  private boolean hasFoundUserPermission(List<UserPermission> userPermissions,
+                                         List<UserPermission> perms) {
+    return perms.containsAll(userPermissions);
+  }
+
   private boolean hasFoundUserPermission(UserPermission userPermission, List<UserPermission> perms) {
     return perms.contains(userPermission);
   }
@@ -1661,10 +1668,17 @@ public class TestAccessController extends SecureTestUtil {
     } finally {
       acl.close();
     }
-    UserPermission adminPerm = new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
-      AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW"));
-    assertTrue("Only global users and user admin has permission on table _acl_ per setup",
-      perms.size() == 5 && hasFoundUserPermission(adminPerm, perms));
+    List<UserPermission> adminPerms = new ArrayList<UserPermission>();
+    adminPerms.add(new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
+      AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW")));
+    List<String> superUsers = Superusers.getSuperUsers();
+    for(String user: superUsers) {
+      adminPerms.add(new UserPermission(Bytes.toBytes(user), AccessControlLists.ACL_TABLE_NAME,
+          null, null, Action.values()));
+    }
+    assertTrue("Only super users, global users and user admin has permission on table hbase:acl " +
+        "per setup", perms.size() == 5 + superUsers.size() &&
+        hasFoundUserPermission(adminPerms, perms));
   }
 
   /** global operations */