You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/05/12 12:25:23 UTC

svn commit: r1337499 - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/security/access/ main/ruby/ main/ruby/hbase/ main/ruby/shell/commands/ test/java/org/apache/hadoop/hbase/security/access/

Author: tedyu
Date: Sat May 12 10:25:22 2012
New Revision: 1337499

URL: http://svn.apache.org/viewvc?rev=1337499&view=rev
Log:
HBASE-5342 Grant/Revoke global permissions (Matteo Bertozzi)

Added:
    hbase/trunk/src/main/ruby/shell/commands/whoami.rb
Modified:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControllerProtocol.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
    hbase/trunk/src/main/ruby/hbase/security.rb
    hbase/trunk/src/main/ruby/shell.rb
    hbase/trunk/src/main/ruby/shell/commands/grant.rb
    hbase/trunk/src/main/ruby/shell/commands/revoke.rb
    hbase/trunk/src/main/ruby/shell/commands/user_permission.rb
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java Sat May 12 10:25:22 2012
@@ -79,6 +79,7 @@ public class AccessControlLists {
   /** Internal storage table for access control lists */
   public static final String ACL_TABLE_NAME_STR = "_acl_";
   public static final byte[] ACL_TABLE_NAME = Bytes.toBytes(ACL_TABLE_NAME_STR);
+  public static final byte[] ACL_GLOBAL_NAME = ACL_TABLE_NAME;
   /** Column family used to store ACL grants */
   public static final String ACL_LIST_FAMILY_STR = "l";
   public static final byte[] ACL_LIST_FAMILY = Bytes.toBytes(ACL_LIST_FAMILY_STR);
@@ -117,31 +118,20 @@ public class AccessControlLists {
   }
 
   /**
-   * Stores a new table permission grant in the access control lists table.
+   * Stores a new user permission grant in the access control lists table.
    * @param conf the configuration
-   * @param tableName the table to which access is being granted
-   * @param username the user or group being granted the permission
-   * @param perm the details of the permission being granted
+   * @param userPerm the details of the permission to be granted
    * @throws IOException in the case of an error accessing the metadata table
    */
-  static void addTablePermission(Configuration conf,
-      byte[] tableName, String username, TablePermission perm)
-    throws IOException {
+  static void addUserPermission(Configuration conf, UserPermission userPerm)
+      throws IOException {
+    Permission.Action[] actions = userPerm.getActions();
 
-    Put p = new Put(tableName);
-    byte[] key = Bytes.toBytes(username);
-    if (perm.getFamily() != null && perm.getFamily().length > 0) {
-      key = Bytes.add(key,
-          Bytes.add(new byte[]{ACL_KEY_DELIMITER}, perm.getFamily()));
-      if (perm.getQualifier() != null && perm.getQualifier().length > 0) {
-        key = Bytes.add(key,
-            Bytes.add(new byte[]{ACL_KEY_DELIMITER}, perm.getQualifier()));
-      }
-    }
+    Put p = new Put(userPerm.isGlobal() ? ACL_GLOBAL_NAME : userPerm.getTable());
+    byte[] key = userPermissionKey(userPerm);
 
-    TablePermission.Action[] actions = perm.getActions();
     if ((actions == null) || (actions.length == 0)) {
-      LOG.warn("No actions associated with user '"+username+"'");
+      LOG.warn("No actions associated with user '"+Bytes.toString(userPerm.getUser())+"'");
       return;
     }
 
@@ -152,7 +142,7 @@ public class AccessControlLists {
     p.add(ACL_LIST_FAMILY, key, value);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Writing permission for table "+
-          Bytes.toString(tableName)+" "+
+          Bytes.toString(userPerm.getTable())+" "+
           Bytes.toString(key)+": "+Bytes.toStringBinary(value)
       );
     }
@@ -175,34 +165,17 @@ public class AccessControlLists {
    * column qualifier "info:colA") will have no effect.
    *
    * @param conf the configuration
-   * @param tableName the table of the current permission grant
-   * @param userName the user or group currently granted the permission
-   * @param perm the details of the permission to be revoked
+   * @param userPerm the details of the permission to be revoked
    * @throws IOException if there is an error accessing the metadata table
    */
-  static void removeTablePermission(Configuration conf,
-      byte[] tableName, String userName, TablePermission perm)
-    throws IOException {
+  static void removeUserPermission(Configuration conf, UserPermission userPerm)
+      throws IOException {
+
+    Delete d = new Delete(userPerm.isGlobal() ? ACL_GLOBAL_NAME : userPerm.getTable());
+    byte[] key = userPermissionKey(userPerm);
 
-    Delete d = new Delete(tableName);
-    byte[] key = null;
-    if (perm.getFamily() != null && perm.getFamily().length > 0) {
-      key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
-          Bytes.toString(perm.getFamily()));
-      if (perm.getQualifier() != null && perm.getQualifier().length > 0) {
-       key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
-          Bytes.toString(perm.getFamily()) + ACL_KEY_DELIMITER +
-          Bytes.toString(perm.getQualifier()));
-      } else {
-        key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
-          Bytes.toString(perm.getFamily()));
-      }
-    } else {
-      key = Bytes.toBytes(userName);
-    }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Removing permission for user '" + userName+ "': "+
-          perm.toString());
+      LOG.debug("Removing permission "+ userPerm.toString());
     }
     d.deleteColumns(ACL_LIST_FAMILY, key);
     HTable acls = null;
@@ -215,6 +188,48 @@ public class AccessControlLists {
   }
 
   /**
+   * Remove specified table from the _acl_ table.
+   */
+  static void removeTablePermissions(Configuration conf, byte[] tableName) 
+      throws IOException{    
+
+    Delete d = new Delete(tableName);
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing permissions of removed table "+ Bytes.toString(tableName));
+    }
+
+    HTable acls = null;
+    try {
+      acls = new HTable(conf, ACL_TABLE_NAME);
+      acls.delete(d);
+    } finally {
+      if (acls != null) acls.close();
+    } 
+  }
+
+  /**
+   * Build qualifier key from user permission:
+   *  username
+   *  username,family
+   *  username,family,qualifier
+   */
+  static byte[] userPermissionKey(UserPermission userPerm) {
+    byte[] qualifier = userPerm.getQualifier();
+    byte[] family = userPerm.getFamily();
+    byte[] key = userPerm.getUser();
+
+    if (family != null && family.length > 0) {
+      key = Bytes.add(key, Bytes.add(new byte[]{ACL_KEY_DELIMITER}, family));
+      if (qualifier != null && qualifier.length > 0) {
+        key = Bytes.add(key, Bytes.add(new byte[]{ACL_KEY_DELIMITER}, qualifier));
+      }
+    }
+
+    return key;
+  }
+
+  /**
    * Returns {@code true} if the given region is part of the {@code _acl_}
    * metadata table.
    */
@@ -328,12 +343,13 @@ public class AccessControlLists {
   static ListMultimap<String,TablePermission> getTablePermissions(
       Configuration conf, byte[] tableName)
   throws IOException {
+    if (tableName == null) tableName = ACL_TABLE_NAME;
+
     /* TODO: -ROOT- and .META. cannot easily be handled because they must be
      * online before _acl_ table.  Can anything be done here?
      */
     if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
-        Bytes.equals(tableName, HConstants.META_TABLE_NAME) ||
-        Bytes.equals(tableName, AccessControlLists.ACL_TABLE_NAME)) {
+        Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
       return ArrayListMultimap.create(0,0);
     }
 

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Sat May 12 10:25:22 2012
@@ -14,17 +14,19 @@
 
 package org.apache.hadoop.hbase.security.access;
 
+import java.io.*;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
+import java.util.TreeSet;
 import java.util.List;
 import java.util.Map;
 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.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -188,12 +190,10 @@ public class AccessController extends Ba
     for (Map.Entry<byte[],ListMultimap<String,TablePermission>> t:
       tables.entrySet()) {
       byte[] table = t.getKey();
-      String tableName = Bytes.toString(table);
       ListMultimap<String,TablePermission> perms = t.getValue();
       byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms,
           regionEnv.getConfiguration());
-      this.authManager.getZKPermissionWatcher().writeToZookeeper(tableName,
-        serialized);
+      this.authManager.getZKPermissionWatcher().writeToZookeeper(table, serialized);
     }
   }
 
@@ -204,31 +204,28 @@ public class AccessController extends Ba
    */
   void updateACL(RegionCoprocessorEnvironment e,
       final Map<byte[], List<KeyValue>> familyMap) {
-    Set<String> tableSet = new HashSet<String>();
+    Set<byte[]> tableSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
     for (Map.Entry<byte[], List<KeyValue>> f : familyMap.entrySet()) {
       List<KeyValue> kvs = f.getValue();
       for (KeyValue kv: kvs) {
-        if (Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(),
+        if (Bytes.equals(kv.getBuffer(), kv.getFamilyOffset(),
             kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
-            AccessControlLists.ACL_LIST_FAMILY.length) == 0) {
-          String tableName = Bytes.toString(kv.getRow());
-          tableSet.add(tableName);
+            AccessControlLists.ACL_LIST_FAMILY.length)) {
+          tableSet.add(kv.getRow());
         }
       }
     }
 
-    for (String tableName: tableSet) {
+    ZKPermissionWatcher zkw = this.authManager.getZKPermissionWatcher();
+    Configuration conf = regionEnv.getConfiguration();
+    for (byte[] tableName: tableSet) {
       try {
         ListMultimap<String,TablePermission> perms =
-          AccessControlLists.getTablePermissions(regionEnv.getConfiguration(),
-              Bytes.toBytes(tableName));
-        byte[] serialized = AccessControlLists.writePermissionsAsBytes(
-            perms, regionEnv.getConfiguration());
-        this.authManager.getZKPermissionWatcher().writeToZookeeper(tableName,
-          serialized);
+          AccessControlLists.getTablePermissions(conf, tableName);
+        byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
+        zkw.writeToZookeeper(tableName, serialized);
       } catch (IOException ex) {
-        LOG.error("Failed updating permissions mirror for '" + tableName +
-          "'", ex);
+        LOG.error("Failed updating permissions mirror for '" + tableName + "'", ex);
       }
     }
   }
@@ -256,29 +253,40 @@ public class AccessController extends Ba
 
     // 1. All users need read access to .META. and -ROOT- tables.
     // this is a very common operation, so deal with it quickly.
-    if ((hri.isRootRegion() || hri.isMetaRegion()) &&
-        (permRequest == TablePermission.Action.READ)) {
-      return AuthResult.allow("All users allowed", user, permRequest,
-          hri.getTableName());
+    if (hri.isRootRegion() || hri.isMetaRegion()) {
+      if (permRequest == TablePermission.Action.READ) {
+        return AuthResult.allow("All users allowed", user, permRequest, tableName);
+      }
     }
 
     if (user == null) {
-      return AuthResult.deny("No user associated with request!", null,
-          permRequest, hri.getTableName());
+      return AuthResult.deny("No user associated with request!", null, permRequest, tableName);
+    }
+
+    // Users with CREATE/ADMIN rights need to modify .META. and _acl_ table
+    // e.g. When a new table is created a new entry in .META. is added,
+    // so the user need to be allowed to write on it.
+    // e.g. When a table is removed an entry is removed from .META. and _acl_
+    // and the user need to be allowed to write on both tables.
+    if (permRequest == TablePermission.Action.WRITE &&
+       (hri.isRootRegion() || hri.isMetaRegion() ||
+        Bytes.equals(tableName, AccessControlLists.ACL_GLOBAL_NAME)) &&
+       (authManager.authorize(user, Permission.Action.CREATE) ||
+        authManager.authorize(user, Permission.Action.ADMIN)))
+    {
+       return AuthResult.allow("Table permission granted", user, permRequest, tableName);
     }
 
     // 2. The table owner has full privileges
     String owner = htd.getOwnerString();
     if (user.getShortName().equals(owner)) {
       // owner of the table has full access
-      return AuthResult.allow("User is table owner", user, permRequest,
-          hri.getTableName());
+      return AuthResult.allow("User is table owner", user, permRequest, tableName);
     }
 
     // 3. check for the table-level, if successful we can short-circuit
     if (authManager.authorize(user, tableName, (byte[])null, permRequest)) {
-      return AuthResult.allow("Table permission granted", user,
-          permRequest, tableName);
+      return AuthResult.allow("Table permission granted", user, permRequest, tableName);
     }
 
     // 4. check permissions against the requested families
@@ -350,6 +358,7 @@ public class AccessController extends Ba
       // for non-rpc handling, fallback to system user
       user = User.getCurrent();
     }
+
     return user;
   }
 
@@ -497,10 +506,16 @@ public class AccessController extends Ba
       desc.setOwner(owner);
     }
   }
+  @Override
+  public void preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
 
   @Override
   public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
+  @Override
+  public void postCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
 
   @Override
   public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
@@ -512,18 +527,31 @@ public class AccessController extends Ba
     }
   }
   @Override
+  public void preDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {}
+  @Override
   public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {
+    AccessControlLists.removeTablePermissions(c.getEnvironment().getConfiguration(), tableName);
+  }
+  @Override
+  public void postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName) throws IOException {}
 
-
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName, HTableDescriptor htd) throws IOException {
     requirePermission(Permission.Action.CREATE);
   }
   @Override
+  public void preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HTableDescriptor htd) throws IOException {}
+  @Override
   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName, HTableDescriptor htd) throws IOException {}
+  @Override
+  public void postModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HTableDescriptor htd) throws IOException {}
 
 
   @Override
@@ -532,9 +560,14 @@ public class AccessController extends Ba
     requirePermission(Permission.Action.CREATE);
   }
   @Override
+  public void preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HColumnDescriptor column) throws IOException {}
+  @Override
   public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName, HColumnDescriptor column) throws IOException {}
-
+  @Override
+  public void postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HColumnDescriptor column) throws IOException {}
 
   @Override
   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c,
@@ -542,8 +575,14 @@ public class AccessController extends Ba
     requirePermission(Permission.Action.CREATE);
   }
   @Override
+  public void preModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HColumnDescriptor descriptor) throws IOException {}
+  @Override
   public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName, HColumnDescriptor descriptor) throws IOException {}
+  @Override
+  public void postModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, HColumnDescriptor descriptor) throws IOException {}
 
 
   @Override
@@ -552,9 +591,14 @@ public class AccessController extends Ba
     requirePermission(Permission.Action.CREATE);
   }
   @Override
+  public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, byte[] col) throws IOException {}
+  @Override
   public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName, byte[] col) throws IOException {}
-
+  @Override
+  public void postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName, byte[] col) throws IOException {}
 
   @Override
   public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c,
@@ -566,8 +610,14 @@ public class AccessController extends Ba
     }
   }
   @Override
+  public void preEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {}
+  @Override
   public void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName) throws IOException {}
+  @Override
+  public void postEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {}
 
   @Override
   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c,
@@ -579,8 +629,14 @@ public class AccessController extends Ba
     }
   }
   @Override
+  public void preDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {}
+  @Override
   public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> c,
       byte[] tableName) throws IOException {}
+  @Override
+  public void postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      byte[] tableName) throws IOException {}
 
   @Override
   public void preMove(ObserverContext<MasterCoprocessorEnvironment> c,
@@ -892,10 +948,10 @@ public class AccessController extends Ba
   private void requireScannerOwner(InternalScanner s)
       throws AccessDeniedException {
     if (RequestContext.isInRequestContext()) {
+      String requestUserName = RequestContext.getRequestUserName();
       String owner = scannerOwners.get(s);
-      if (owner != null && !owner.equals(RequestContext.getRequestUserName())) {
-        throw new AccessDeniedException("User '"+
-            RequestContext.getRequestUserName()+"' is not the scanner owner!");
+      if (owner != null && !owner.equals(requestUserName)) {
+        throw new AccessDeniedException("User '"+ requestUserName +"' is not the scanner owner!");
       }
     }
   }
@@ -906,24 +962,20 @@ public class AccessController extends Ba
    * This will be restricted by both client side and endpoint implementations.
    */
   @Override
-  public void grant(byte[] user, TablePermission permission)
+  public void grant(UserPermission userPermission)
       throws IOException {
     // verify it's only running at .acl.
     if (aclRegion) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Received request to grant access permission to '"
-            + Bytes.toString(user) + "'. "
-            + permission.toString());
+        LOG.debug("Received request to grant access permission " + userPermission.toString());
       }
 
       requirePermission(Permission.Action.ADMIN);
 
-      AccessControlLists.addTablePermission(regionEnv.getConfiguration(),
-          permission.getTable(), Bytes.toString(user), permission);
+      AccessControlLists.addUserPermission(regionEnv.getConfiguration(), userPermission);
       if (AUDITLOG.isTraceEnabled()) {
         // audit log should store permission changes in addition to auth results
-        AUDITLOG.trace("Granted user '" + Bytes.toString(user) + "' permission "
-            + permission.toString());
+        AUDITLOG.trace("Granted permission " + userPermission.toString());
       }
     } else {
       throw new CoprocessorException(AccessController.class, "This method " +
@@ -933,24 +985,29 @@ public class AccessController extends Ba
   }
 
   @Override
-  public void revoke(byte[] user, TablePermission permission)
+  @Deprecated
+  public void grant(byte[] user, TablePermission permission)
+      throws IOException {
+    grant(new UserPermission(user, permission.getTable(),
+            permission.getFamily(), permission.getQualifier(),
+            permission.getActions()));
+  }
+
+  @Override
+  public void revoke(UserPermission userPermission)
       throws IOException{
     // only allowed to be called on _acl_ region
     if (aclRegion) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Received request to revoke access permission for '"
-            + Bytes.toString(user) + "'. "
-            + permission.toString());
+        LOG.debug("Received request to revoke access permission " + userPermission.toString());
       }
 
       requirePermission(Permission.Action.ADMIN);
 
-      AccessControlLists.removeTablePermission(regionEnv.getConfiguration(),
-          permission.getTable(), Bytes.toString(user), permission);
+      AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), userPermission);
       if (AUDITLOG.isTraceEnabled()) {
         // audit log should record all permission changes
-        AUDITLOG.trace("Revoked user '" + Bytes.toString(user) + "' permission "
-            + permission.toString());
+        AUDITLOG.trace("Revoked permission " + userPermission.toString());
       }
     } else {
       throw new CoprocessorException(AccessController.class, "This method " +
@@ -960,6 +1017,15 @@ public class AccessController extends Ba
   }
 
   @Override
+  @Deprecated
+  public void revoke(byte[] user, TablePermission permission)
+      throws IOException {
+    revoke(new UserPermission(user, permission.getTable(),
+            permission.getFamily(), permission.getQualifier(),
+            permission.getActions()));
+  }
+
+  @Override
   public List<UserPermission> getUserPermissions(final byte[] tableName)
       throws IOException {
     // only allowed to be called on _acl_ region
@@ -1038,7 +1104,7 @@ public class AccessController extends Ba
     return tableName;
   }
 
-  private String getTableOwner(MasterCoprocessorEnvironment e, 
+  private String getTableOwner(MasterCoprocessorEnvironment e,
       byte[] tableName) throws IOException {
     HTableDescriptor htd = e.getTable(tableName).getTableDescriptor();
     return htd.getOwnerString();
@@ -1049,84 +1115,4 @@ public class AccessController extends Ba
     String activeUser = getActiveUser().getShortName();
     return activeUser.equals(getTableOwner(e, tableName));
   }
-
-  @Override
-  public void preCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-  }
-
-  @Override
-  public void postCreateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-  }
-
-  @Override
-  public void preDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
-
-  @Override
-  public void postDeleteTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
-
-  @Override
-  public void preModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HTableDescriptor htd) throws IOException {
-  }
-
-  @Override
-  public void postModifyTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HTableDescriptor htd) throws IOException {
-  }
-
-  @Override
-  public void preAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException {
-  }
-
-  @Override
-  public void postAddColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException {
-  }
-
-  @Override
-  public void preModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException {
-  }
-
-  @Override
-  public void postModifyColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException {
-  }
-
-  @Override
-  public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, byte[] c) throws IOException {
-  }
-
-  @Override
-  public void postDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, byte[] c) throws IOException {
-  }
-
-  @Override
-  public void preEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
-
-  @Override
-  public void postEnableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
-
-  @Override
-  public void preDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
-
-  @Override
-  public void postDisableTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
-  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControllerProtocol.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControllerProtocol.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControllerProtocol.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/AccessControllerProtocol.java Sat May 12 10:25:22 2012
@@ -32,12 +32,22 @@ public interface AccessControllerProtoco
 
   /**
    * Grants the given user or group the privilege to perform the given actions
+   * @param userPermission the details of the provided user permissions
+   * @throws IOException if the grant could not be applied
+   */
+  public void grant(UserPermission userPermission)
+      throws IOException;
+
+  /**
+   * Grants the given user or group the privilege to perform the given actions
    * over the specified scope contained in {@link TablePermission}
    * @param user the user name, or, if prefixed with "@", group name receiving
    * the grant
    * @param permission the details of the provided permissions
    * @throws IOException if the grant could not be applied
+   * @deprecated Use {@link #revoke(UserPermission userPermission)} instead
    */
+  @Deprecated
   public void grant(byte[] user, TablePermission permission)
       throws IOException;
 
@@ -49,11 +59,27 @@ public interface AccessControllerProtoco
    * table, column family and column qualifier must all be specified.
    * Attempting to revoke permissions over just the "data" table will have
    * no effect.
+   * @param permission the details of the previously granted permission to revoke
+   * @throws IOException if the revocation could not be performed
+   */
+  public void revoke(UserPermission userPermission)
+      throws IOException;
+
+  /**
+   * Revokes a previously granted privilege from a user or group.
+   * Note that the provided {@link TablePermission} details must exactly match
+   * a stored grant.  For example, if user "bob" has been granted "READ" access
+   * to table "data", over column family and qualifer "info:colA", then the
+   * table, column family and column qualifier must all be specified.
+   * Attempting to revoke permissions over just the "data" table will have
+   * no effect.
    * @param user the user name, or, if prefixed with "@", group name whose
    * privileges are being revoked
    * @param permission the details of the previously granted permission to revoke
    * @throws IOException if the revocation could not be performed
+   * @deprecated Use {@link #revoke(UserPermission userPermission)} instead
    */
+  @Deprecated
   public void revoke(byte[] user, TablePermission permission)
       throws IOException;
 
@@ -81,5 +107,4 @@ public interface AccessControllerProtoco
    */
   public void checkPermissions(Permission[] permissions)
       throws IOException;
-
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java Sat May 12 10:25:22 2012
@@ -40,8 +40,6 @@ import java.util.concurrent.ConcurrentSk
  * Performs authorization checks for a given user's assigned permissions
  */
 public class TableAuthManager {
-  /** Key for the user and group cache maps for globally assigned permissions */
-  private static final String GLOBAL_CACHE_KEY = ".access.";
   private static Log LOG = LogFactory.getLog(TableAuthManager.class);
 
   private static TableAuthManager instance;
@@ -103,15 +101,37 @@ public class TableAuthManager {
 
   public void refreshCacheFromWritable(byte[] table, byte[] data) throws IOException {
     if (data != null && data.length > 0) {
-      DataInput in = new DataInputStream( new ByteArrayInputStream(data) );
+      DataInput in = new DataInputStream(new ByteArrayInputStream(data));
       ListMultimap<String,TablePermission> perms = AccessControlLists.readPermissions(in, conf);
-      cache(table, perms);
+      if (perms != null) {
+        if (Bytes.equals(table, AccessControlLists.ACL_GLOBAL_NAME)) {
+          updateGlobalCache(perms);
+        } else {
+          updateTableCache(table, perms);
+        }
+      }
     } else {
       LOG.debug("Skipping permission cache refresh because writable data is empty");
     }
   }
 
   /**
+   * Updates the internal global permissions cache
+   *
+   * @param userPerms
+   */
+  private void updateGlobalCache(ListMultimap<String,TablePermission> userPerms) {
+    for (Map.Entry<String,TablePermission> entry : userPerms.entries()) {
+      if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
+        GROUP_CACHE.put(AccessControlLists.getGroupName(entry.getKey()),
+                        new Permission(entry.getValue().getActions()));
+      } else {
+        USER_CACHE.put(entry.getKey(), new Permission(entry.getValue().getActions()));
+      }
+    }
+  }
+
+  /**
    * Updates the internal permissions cache for a single table, splitting
    * the permissions listed into separate caches for users and groups to optimize
    * group lookups.
@@ -119,26 +139,22 @@ public class TableAuthManager {
    * @param table
    * @param tablePerms
    */
-  private void cache(byte[] table,
-      ListMultimap<String,TablePermission> tablePerms) {
+  private void updateTableCache(byte[] table, ListMultimap<String,TablePermission> tablePerms) {
     // split user from group assignments so we don't have to prepend the group
     // prefix every time we query for groups
     ListMultimap<String,TablePermission> userPerms = ArrayListMultimap.create();
     ListMultimap<String,TablePermission> groupPerms = ArrayListMultimap.create();
 
-    if (tablePerms != null) {
-      for (Map.Entry<String,TablePermission> entry : tablePerms.entries()) {
-        if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
-          groupPerms.put(
-              entry.getKey().substring(AccessControlLists.GROUP_PREFIX.length()),
-              entry.getValue());
-        } else {
-          userPerms.put(entry.getKey(), entry.getValue());
-        }
+    for (Map.Entry<String,TablePermission> entry : tablePerms.entries()) {
+      if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
+        groupPerms.put(AccessControlLists.getGroupName(entry.getKey()), entry.getValue());
+      } else {
+        userPerms.put(entry.getKey(), entry.getValue());
       }
-      TABLE_GROUP_CACHE.put(table, groupPerms);
-      TABLE_USER_CACHE.put(table, userPerms);
     }
+
+    TABLE_GROUP_CACHE.put(table, groupPerms);
+    TABLE_USER_CACHE.put(table, userPerms);
   }
 
   private List<TablePermission> getUserPermissions(String username, byte[] table) {
@@ -464,7 +480,7 @@ public class TableAuthManager {
       }
     }
     byte[] serialized = AccessControlLists.writePermissionsAsBytes(tmp, conf);
-    zkperms.writeToZookeeper(Bytes.toString(table), serialized);
+    zkperms.writeToZookeeper(table, serialized);
   }
 
   static Map<ZooKeeperWatcher,TableAuthManager> managerMap =

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java Sat May 12 10:25:22 2012
@@ -41,6 +41,27 @@ public class UserPermission extends Tabl
   }
 
   /**
+   * Creates a new instance for the given user.
+   * @param user the user
+   * @param assigned the list of allowed actions
+   */
+  public UserPermission(byte[] user, Action... assigned) {
+    super(null, null, null, assigned);
+    this.user = user;
+  }
+
+  /**
+   * 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
+   */
+  public UserPermission(byte[] user, byte[] actionCodes) {
+    super(null, null, null, actionCodes);
+    this.user = user;
+  }
+
+  /**
    * Creates a new instance for the given user, table and column family.
    * @param user the user
    * @param table the table
@@ -92,6 +113,14 @@ public class UserPermission extends Tabl
     return user;
   }
 
+  /**
+   * Returns true if this permission describes a global user permission.
+   */
+  public boolean isGlobal() {
+    byte[] tableName = getTable();
+    return(tableName == null || tableName.length == 0);
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (!(obj instanceof UserPermission)) {

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java Sat May 12 10:25:22 2012
@@ -146,18 +146,16 @@ public class ZKPermissionWatcher extends
    * @param tableName
    * @param permsData
    */
-  public void writeToZookeeper(String tableName, 
-      byte[] permsData) {
-    String zkNode =
-        ZKUtil.joinZNode(ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE),
-          tableName);
+  public void writeToZookeeper(byte[] tableName, byte[] parmsData) {    
+    String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE);
+    zkNode = ZKUtil.joinZNode(zkNode, Bytes.toString(tableName));
+
     try {
       ZKUtil.createWithParents(watcher, zkNode);
-      ZKUtil.updateExistingNodeData(watcher, zkNode,
-        permsData, -1);
+      ZKUtil.updateExistingNodeData(watcher, zkNode, parmsData, -1);
     } catch (KeeperException e) {
-      LOG.error("Failed updating permissions for table '" + tableName +
-          "'", e);
+      LOG.error("Failed updating permissions for table '" + 
+                Bytes.toString(tableName) + "'", e);
       watcher.abort("Failed writing node "+zkNode+" to zookeeper", e);
     }
   }

Modified: hbase/trunk/src/main/ruby/hbase/security.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/hbase/security.rb?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/hbase/security.rb (original)
+++ hbase/trunk/src/main/ruby/hbase/security.rb Sat May 12 10:25:22 2012
@@ -31,62 +31,117 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
-    def grant(user, permissions, table_name, family=nil, qualifier=nil)
+    def grant(user, permissions, table_name=nil, family=nil, qualifier=nil)
       security_available?
 
-      # Table should exist
-      raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
+      # TODO: need to validate user name
 
-      htd = @admin.getTableDescriptor(table_name.to_java_bytes)
+      if (table_name != nil)
+        # Table should exist
+        raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-      if (family != nil)
-        raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
-      end
+        htd = @admin.getTableDescriptor(table_name.to_java_bytes)
+
+        if (family != nil)
+          raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
+        end
 
-      #TODO: need to validate user name
+        # invoke cp endpoint to perform access controlse
+        fambytes = family.to_java_bytes if (family != nil)
+        qualbytes = qualifier.to_java_bytes if (qualifier != nil)
+        user_permission = org.apache.hadoop.hbase.security.access.UserPermission.new(
+                                                user.to_java_bytes, table_name.to_java_bytes, 
+                                                fambytes, qualbytes, permissions.to_java_bytes)
+      else
+        user_permission = org.apache.hadoop.hbase.security.access.UserPermission.new(
+                                                user.to_java_bytes, permissions.to_java_bytes)
+      end
 
-      # invoke cp endpoint to perform access control
-      fambytes = family.to_java_bytes if (family != nil)
-      qualbytes = qualifier.to_java_bytes if (qualifier != nil)
-      tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, permissions.to_java_bytes)
-      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
-      protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
+      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config,
+                      org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
+      protocol = meta_table.coprocessorProxy(
+                      org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
                                              org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
-      protocol.grant(user.to_java_bytes, tp)
+      begin
+        protocol.grant(user_permission)
+      rescue java.io.IOException => e
+        if !(e.message.include? "java.lang.NoSuchMethodException")
+          raise e
+        end
+
+        # Server has not the new API, try the old one
+        if (table_name == nil)
+          raise "Global permissions not supported by HBase Server"
+        end
+
+        tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, permissions.to_java_bytes)
+        protocol.grant(user.to_java_bytes, tp)
+      end
     end
 
     #----------------------------------------------------------------------------------------------
-    def revoke(user, table_name, family=nil, qualifier=nil)
+    def revoke(user, table_name=nil, family=nil, qualifier=nil)
       security_available?
 
-      # Table should exist
-      raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
+      # TODO: need to validate user name
+
+      if (table_name != nil)
+        # Table should exist
+        raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-      htd = @admin.getTableDescriptor(table_name.to_java_bytes)
+        htd = @admin.getTableDescriptor(table_name.to_java_bytes)
 
-      if (family != nil)
-        raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
+        if (family != nil)
+          raise(ArgumentError, "Can't find family: #{family}") unless htd.hasFamily(family.to_java_bytes)
+        end
+
+        # invoke cp endpoint to perform access control
+        fambytes = family.to_java_bytes if (family != nil)
+        qualbytes = qualifier.to_java_bytes if (qualifier != nil)
+        user_permission = org.apache.hadoop.hbase.security.access.UserPermission.new(
+                                                    user.to_java_bytes, table_name.to_java_bytes,
+                                                    fambytes, qualbytes, "".to_java_bytes)
+      else
+        user_permission = org.apache.hadoop.hbase.security.access.UserPermission.new(
+                                                    user.to_java_bytes, "".to_java_bytes)
       end
 
-      fambytes = family.to_java_bytes if (family != nil)
-      qualbytes = qualifier.to_java_bytes if (qualifier != nil)
-      tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, "".to_java_bytes)
-      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
-      protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
+      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config,
+                        org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
+      protocol = meta_table.coprocessorProxy(
+                        org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
                                              org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
-      protocol.revoke(user.to_java_bytes, tp)
+      begin
+        protocol.revoke(user_permission)
+      rescue java.io.IOException => e
+        if !(e.message.include? "java.lang.NoSuchMethodException")
+          raise e
+        end
+
+        # Server has not the new API, try the old one
+        if (table_name == nil)
+          raise "Global permissions not supported by HBase Server"
+        end
+
+        tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, "".to_java_bytes)
+        protocol.revoke(user.to_java_bytes, tp)
+      end
     end
 
     #----------------------------------------------------------------------------------------------
-    def user_permission(table_name)
+    def user_permission(table_name=nil)
       security_available?
 
-      raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
+      if (table_name != nil)
+        raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
+      end
 
-      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
-      protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
-                                             org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
-      perms = protocol.getUserPermissions(table_name.to_java_bytes)
+      meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, 
+                        org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
+      protocol = meta_table.coprocessorProxy(
+                      org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
+                      org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
+      perms = protocol.getUserPermissions(table_name != nil ? table_name.to_java_bytes : nil)
 
       res = {}
       count  = 0

Modified: hbase/trunk/src/main/ruby/shell.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell.rb?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/shell.rb (original)
+++ hbase/trunk/src/main/ruby/shell.rb Sat May 12 10:25:22 2012
@@ -226,6 +226,7 @@ Shell.load_command_group(
     status
     version
     table_help
+    whoami
   ]
 )
 

Modified: hbase/trunk/src/main/ruby/shell/commands/grant.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/grant.rb?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/grant.rb (original)
+++ hbase/trunk/src/main/ruby/shell/commands/grant.rb Sat May 12 10:25:22 2012
@@ -21,7 +21,7 @@ module Shell
     class Grant < Command
       def help
         return <<-EOF
-Grant users specific rights to tables.
+Grant users specific rights.
 Syntax : grant <user> <permissions> <table> <column family> <column qualifier>
 
 permissions is either zero or more letters from the set "RWXCA".
@@ -29,11 +29,12 @@ READ('R'), WRITE('W'), EXEC('X'), CREATE
 
 For example:
 
+    hbase> grant 'bobsmith', 'RWXCA'
     hbase> grant 'bobsmith', 'RW', 't1', 'f1', 'col1'
 EOF
       end
 
-      def command(user, rights, table_name, family=nil, qualifier=nil)
+      def command(user, rights, table_name=nil, family=nil, qualifier=nil)
         format_simple_command do
           security_admin.grant(user, rights, table_name, family, qualifier)
         end

Modified: hbase/trunk/src/main/ruby/shell/commands/revoke.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/revoke.rb?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/revoke.rb (original)
+++ hbase/trunk/src/main/ruby/shell/commands/revoke.rb Sat May 12 10:25:22 2012
@@ -21,15 +21,15 @@ module Shell
     class Revoke < Command
       def help
         return <<-EOF
-Revoke a user's access rights to tables.
-Syntax : revoke <user> <table> <column family>
+Revoke a user's access rights.
+Syntax : revoke <user> <table> <column family> <column qualifier>
 For example:
 
-    hbase> revoke 'bobsmith', 't1', 'f1'
+    hbase> revoke 'bobsmith', 't1', 'f1', 'col1'
 EOF
       end
 
-      def command(user, table_name, family=nil, qualifier=nil)
+      def command(user, table_name=nil, family=nil, qualifier=nil)
         format_simple_command do
           security_admin.revoke(user, table_name, family, qualifier)
         end

Modified: hbase/trunk/src/main/ruby/shell/commands/user_permission.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/user_permission.rb?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/user_permission.rb (original)
+++ hbase/trunk/src/main/ruby/shell/commands/user_permission.rb Sat May 12 10:25:22 2012
@@ -21,15 +21,16 @@ module Shell
     class UserPermission < Command
       def help
         return <<-EOF
-Show all table access permissions for the particular user.
+Show all permissions for the particular user.
 Syntax : user_permission <table>
 For example:
 
+    hbase> user_permission
     hbase> user_permission 'table1'
 EOF
       end
 
-      def command(table)
+      def command(table=nil)
         #format_simple_command do
         #admin.user_permission(table)
         now = Time.now

Added: hbase/trunk/src/main/ruby/shell/commands/whoami.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/ruby/shell/commands/whoami.rb?rev=1337499&view=auto
==============================================================================
--- hbase/trunk/src/main/ruby/shell/commands/whoami.rb (added)
+++ hbase/trunk/src/main/ruby/shell/commands/whoami.rb Sat May 12 10:25:22 2012
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class Whoami < Command
+      def help
+        return <<-EOF
+Show the current hbase user.
+Syntax : whoami
+For example:
+
+    hbase> whoami
+EOF
+      end
+
+      def command()
+        puts "#{org.apache.hadoop.hbase.security.User.getCurrent().toString()}"
+      end
+    end
+  end
+end

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java Sat May 12 10:25:22 2012
@@ -95,10 +95,12 @@ public class TestAccessControlFilter {
             AccessControlLists.ACL_TABLE_NAME);
         AccessControllerProtocol acls = aclmeta.coprocessorProxy(
             AccessControllerProtocol.class, Bytes.toBytes("testtable"));
-        TablePermission perm = new TablePermission(TABLE, null, Permission.Action.READ);
-        acls.grant(Bytes.toBytes(READER.getShortName()), perm);
-        perm = new TablePermission(TABLE, FAMILY, PUBLIC_COL, Permission.Action.READ);
-        acls.grant(Bytes.toBytes(LIMITED.getShortName()), perm);
+        UserPermission perm = new UserPermission(Bytes.toBytes(READER.getShortName()), 
+                                                 TABLE, null, Permission.Action.READ);
+        acls.grant(perm);
+        perm = new UserPermission(Bytes.toBytes(LIMITED.getShortName()), 
+                                  TABLE, FAMILY, PUBLIC_COL, Permission.Action.READ);
+        acls.grant(perm);
         return null;
       }
     });

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Sat May 12 10:25:22 2012
@@ -72,6 +72,8 @@ public class TestAccessController {
 
   // user with all permissions
   private static User SUPERUSER;
+  // user granted with all global permission
+  private static User USER_ADMIN;
   // table owner user
   private static User USER_OWNER;
   // user with rw permissions
@@ -104,6 +106,7 @@ public class TestAccessController {
 
     // create a set of test users
     SUPERUSER = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
+    USER_ADMIN = User.createUserForTesting(conf, "admin2", new String[0]);
     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
     USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
     USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
@@ -119,12 +122,16 @@ public class TestAccessController {
     HTable meta = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
     AccessControllerProtocol protocol =
         meta.coprocessorProxy(AccessControllerProtocol.class, TEST_TABLE);
-    protocol.grant(Bytes.toBytes(USER_RW.getShortName()),
-        new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ,
-            Permission.Action.WRITE));
 
-    protocol.grant(Bytes.toBytes(USER_RO.getShortName()),
-        new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ));
+    protocol.grant(new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
+                      Permission.Action.ADMIN, Permission.Action.CREATE,
+                      Permission.Action.READ, Permission.Action.WRITE));
+
+    protocol.grant(new UserPermission(Bytes.toBytes(USER_RW.getShortName()),
+        TEST_TABLE, TEST_FAMILY, Permission.Action.READ, Permission.Action.WRITE));
+
+    protocol.grant(new UserPermission(Bytes.toBytes(USER_RO.getShortName()),
+                   TEST_TABLE, TEST_FAMILY, Permission.Action.READ));
   }
 
   @AfterClass
@@ -195,6 +202,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, createTable);
+    verifyAllowed(USER_ADMIN, createTable);
 
     // all others should be denied
     verifyDenied(USER_OWNER, createTable);
@@ -223,6 +231,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, modifyTable);
+    verifyAllowed(USER_ADMIN, modifyTable);
   }
 
   @Test
@@ -242,6 +251,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, deleteTable);
+    verifyAllowed(USER_ADMIN, deleteTable);
   }
 
   @Test
@@ -262,6 +272,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -283,6 +294,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -302,6 +314,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -321,6 +334,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, disableTable);
+    verifyAllowed(USER_ADMIN, disableTable);
   }
 
   @Test
@@ -340,6 +354,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, enableTable);
+    verifyAllowed(USER_ADMIN, enableTable);
   }
 
   @Test
@@ -365,6 +380,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -390,6 +406,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -415,6 +432,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -434,6 +452,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -453,6 +472,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -472,6 +492,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   @Test
@@ -491,6 +512,7 @@ public class TestAccessController {
 
     // verify that superuser can create tables
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
   }
 
   private void verifyWrite(PrivilegedExceptionAction action) throws Exception {
@@ -500,6 +522,7 @@ public class TestAccessController {
 
     // should be allowed
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
     verifyAllowed(USER_OWNER, action);
     verifyAllowed(USER_RW, action);
   }
@@ -510,6 +533,7 @@ public class TestAccessController {
 
     // should be allowed
     verifyAllowed(SUPERUSER, action);
+    verifyAllowed(USER_ADMIN, action);
     verifyAllowed(USER_OWNER, action);
     verifyAllowed(USER_RW, action);
     verifyAllowed(USER_RO, action);
@@ -720,8 +744,8 @@ public class TestAccessController {
     verifyDenied(user, deleteAction2);
 
     // grant table read permission
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, null, Permission.Action.READ));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, null, Permission.Action.READ));
     Thread.sleep(100);
     // check
     verifyAllowed(user, getActionAll);
@@ -737,8 +761,8 @@ public class TestAccessController {
     verifyDenied(user, deleteAction2);
 
     // grant table write permission
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, null, Permission.Action.WRITE));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, null, Permission.Action.WRITE));
     Thread.sleep(100);
     verifyDenied(user, getActionAll);
     verifyDenied(user, getAction1);
@@ -753,12 +777,11 @@ public class TestAccessController {
     verifyAllowed(user, deleteAction2);
 
     // revoke table permission
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, null, Permission.Action.READ,
-        Permission.Action.WRITE));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, null, Permission.Action.READ, Permission.Action.WRITE));
 
-    protocol.revoke(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, null));
+    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()),
+                    tableName, null));
     Thread.sleep(100);
     verifyDenied(user, getActionAll);
     verifyDenied(user, getAction1);
@@ -773,8 +796,8 @@ public class TestAccessController {
     verifyDenied(user, deleteAction2);
 
     // grant column family read permission
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, family1, Permission.Action.READ));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, family1, Permission.Action.READ));
     Thread.sleep(100);
 
     verifyAllowed(user, getActionAll);
@@ -790,8 +813,8 @@ public class TestAccessController {
     verifyDenied(user, deleteAction2);
 
     // grant column family write permission
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, family2, Permission.Action.WRITE));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, family2, Permission.Action.WRITE));
     Thread.sleep(100);
 
     verifyAllowed(user, getActionAll);
@@ -807,8 +830,8 @@ public class TestAccessController {
     verifyAllowed(user, deleteAction2);
 
     // revoke column family permission
-    protocol.revoke(Bytes.toBytes(user.getShortName()),
-      new TablePermission(tableName, family2));
+    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()),
+                    tableName, family2));
     Thread.sleep(100);
 
     verifyAllowed(user, getActionAll);
@@ -890,15 +913,14 @@ public class TestAccessController {
       }
     };
 
-    protocol.revoke(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, family1));
+    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()),
+                    tableName, family1));
     verifyDenied(user, getQualifierAction);
     verifyDenied(user, putQualifierAction);
     verifyDenied(user, deleteQualifierAction);
 
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, family1, qualifier,
-            Permission.Action.READ));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, family1, qualifier, Permission.Action.READ));
     Thread.sleep(100);
 
     verifyAllowed(user, getQualifierAction);
@@ -907,9 +929,8 @@ public class TestAccessController {
 
     // only grant write permission
     // TODO: comment this portion after HBASE-3583
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, family1, qualifier,
-            Permission.Action.WRITE));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, family1, qualifier, Permission.Action.WRITE));
     Thread.sleep(100);
 
     verifyDenied(user, getQualifierAction);
@@ -917,9 +938,9 @@ public class TestAccessController {
     verifyAllowed(user, deleteQualifierAction);
 
     // grant both read and write permission.
-    protocol.grant(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, family1, qualifier,
-            Permission.Action.READ, Permission.Action.WRITE));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()),
+                   tableName, family1, qualifier,
+                   Permission.Action.READ, Permission.Action.WRITE));
     Thread.sleep(100);
 
     verifyAllowed(user, getQualifierAction);
@@ -927,8 +948,8 @@ public class TestAccessController {
     verifyAllowed(user, deleteQualifierAction);
 
     // revoke family level permission won't impact column level.
-    protocol.revoke(Bytes.toBytes(user.getShortName()),
-        new TablePermission(tableName, family1, qualifier));
+    protocol.revoke(new UserPermission(Bytes.toBytes(user.getShortName()),
+                    tableName, family1, qualifier));
     Thread.sleep(100);
 
     verifyDenied(user, getQualifierAction);
@@ -974,7 +995,7 @@ public class TestAccessController {
     // grant read permission
     UserPermission upToSet = new UserPermission(user,
         tableName, family1, qualifier, Permission.Action.READ);
-    protocol.grant(user, upToSet);
+    protocol.grant(upToSet);
     perms = protocol.getUserPermissions(tableName);
 
     UserPermission upToVerify = new UserPermission(user,
@@ -990,7 +1011,7 @@ public class TestAccessController {
     // grant read+write
     upToSet = new UserPermission(user, tableName, family1, qualifier,
         Permission.Action.WRITE, Permission.Action.READ);
-    protocol.grant(user, upToSet);
+    protocol.grant(upToSet);
     perms = protocol.getUserPermissions(tableName);
 
     upToVerify = new UserPermission(user, tableName, family1, qualifier,
@@ -998,7 +1019,7 @@ public class TestAccessController {
     assertTrue("User should be granted permission: " + upToVerify.toString(),
             hasFoundUserPermission(upToVerify, perms));
 
-    protocol.revoke(user, upToSet);
+    protocol.revoke(upToSet);
     perms = protocol.getUserPermissions(tableName);
     assertFalse("User should not be granted permission: " + upToVerify.toString(),
       hasFoundUserPermission(upToVerify, perms));
@@ -1053,7 +1074,7 @@ public class TestAccessController {
 
   public void grant(AccessControllerProtocol protocol, User user, byte[] t, byte[] f,
       byte[] q, Permission.Action... actions) throws IOException {
-    protocol.grant(Bytes.toBytes(user.getShortName()), new TablePermission(t, f, q, actions));
+    protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()), t, f, q, actions));
   }
 
   @Test

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java?rev=1337499&r1=1337498&r2=1337499&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java Sat May 12 10:25:22 2012
@@ -103,15 +103,16 @@ public class TestTablePermissions {
   public void testBasicWrite() throws Exception {
     Configuration conf = UTIL.getConfiguration();
     // add some permissions
-    AccessControlLists.addTablePermission(conf, TEST_TABLE,
-        "george", new TablePermission(TEST_TABLE, null,
-            TablePermission.Action.READ, TablePermission.Action.WRITE));
-    AccessControlLists.addTablePermission(conf, TEST_TABLE,
-        "hubert", new TablePermission(TEST_TABLE, null,
-            TablePermission.Action.READ));
-    AccessControlLists.addTablePermission(conf, TEST_TABLE,
-        "humphrey", new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-            TablePermission.Action.READ));
+    AccessControlLists.addUserPermission(conf,
+            new UserPermission(Bytes.toBytes("george"), TEST_TABLE, null, (byte[])null,
+            UserPermission.Action.READ, UserPermission.Action.WRITE));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE, null, (byte[])null,
+            UserPermission.Action.READ));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("humphrey"),
+            TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+            UserPermission.Action.READ));
 
     // retrieve the same
     ListMultimap<String,TablePermission> perms =
@@ -165,8 +166,8 @@ public class TestTablePermissions {
     assertFalse(actions.contains(TablePermission.Action.WRITE));
 
     // table 2 permissions
-    AccessControlLists.addTablePermission(conf, TEST_TABLE2, "hubert",
-        new TablePermission(TEST_TABLE2, null,
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE2, null, (byte[])null,
             TablePermission.Action.READ, TablePermission.Action.WRITE));
 
     // check full load
@@ -197,16 +198,21 @@ public class TestTablePermissions {
   @Test
   public void testPersistence() throws Exception {
     Configuration conf = UTIL.getConfiguration();
-    AccessControlLists.addTablePermission(conf, TEST_TABLE, "albert",
-        new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
-    AccessControlLists.addTablePermission(conf, TEST_TABLE, "betty",
-        new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
-            TablePermission.Action.WRITE));
-    AccessControlLists.addTablePermission(conf, TEST_TABLE, "clark",
-        new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ));
-    AccessControlLists.addTablePermission(conf, TEST_TABLE, "dwight",
-        new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-            TablePermission.Action.WRITE));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("albert"), TEST_TABLE, null,
+                           (byte[])null, TablePermission.Action.READ));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("betty"), TEST_TABLE, null,
+                           (byte[])null, TablePermission.Action.READ,
+                           TablePermission.Action.WRITE));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("clark"),
+                           TEST_TABLE, TEST_FAMILY,
+                           TablePermission.Action.READ));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("dwight"),
+                           TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
+                           TablePermission.Action.WRITE));
 
     // verify permissions survive changes in table metadata
     ListMultimap<String,TablePermission> preperms =
@@ -313,4 +319,41 @@ public class TestTablePermissions {
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
   }
+
+  @Test
+  public void testGlobalPermission() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+
+    // add some permissions
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("user1"),
+            Permission.Action.READ, Permission.Action.WRITE));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("user2"),
+            Permission.Action.CREATE));
+    AccessControlLists.addUserPermission(conf,
+        new UserPermission(Bytes.toBytes("user3"),
+            Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE));
+
+    ListMultimap<String,TablePermission> perms = AccessControlLists.getTablePermissions(conf, null);
+    List<TablePermission> user1Perms = perms.get("user1");
+    assertEquals("Should have 1 permission for user1", 1, user1Perms.size());
+    assertEquals("user1 should have WRITE permission",
+                 new Permission.Action[] { Permission.Action.READ, Permission.Action.WRITE },
+                 user1Perms.get(0).getActions());
+
+    List<TablePermission> user2Perms = perms.get("user2");
+    assertEquals("Should have 1 permission for user2", 1, user2Perms.size());
+    assertEquals("user2 should have CREATE permission",
+                 new Permission.Action[] { Permission.Action.CREATE },
+                 user2Perms.get(0).getActions());
+
+    List<TablePermission> user3Perms = perms.get("user3");
+    assertEquals("Should have 1 permission for user3", 1, user3Perms.size());
+    assertEquals("user3 should have ADMIN, READ, CREATE permission",
+                 new Permission.Action[] {
+                    Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE
+                 },
+                 user3Perms.get(0).getActions());
+  }
 }