You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2014/11/07 20:49:14 UTC

hbase git commit: Reduce the effective scope of CREATE and ADMIN permissions

Repository: hbase
Updated Branches:
  refs/heads/0.94 4160f8b83 -> eed062217


Reduce the effective scope of CREATE and ADMIN permissions


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

Branch: refs/heads/0.94
Commit: eed062217cfc483fdd7146880abf69afa71798b6
Parents: 4160f8b
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Oct 31 13:53:28 2014 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Fri Nov 7 11:46:42 2014 -0800

----------------------------------------------------------------------
 .../hbase/security/access/AccessController.java | 130 ++++++++++------
 .../security/access/TestAccessController.java   | 147 +++++++++++++++++++
 .../org/apache/hadoop/hbase/security/User.java  |  21 +++
 3 files changed, 251 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eed06221/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 3b6869d..6161382 100644
--- a/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -16,6 +16,7 @@ package org.apache.hadoop.hbase.security.access;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -286,20 +287,6 @@ public class AccessController extends BaseRegionObserver
       return AuthResult.deny(request, "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(request, "Table permission granted", user, permRequest, tableName);
-    }
-
     // 2. check for the table-level, if successful we can short-circuit
     if (authManager.authorize(user, tableName, (byte[])null, permRequest)) {
       return AuthResult.allow(request, "Table permission granted", user, permRequest, tableName);
@@ -573,14 +560,20 @@ public class AccessController extends BaseRegionObserver
 
   @Override
   public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+      final HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
     if (!AccessControlLists.isAclTable(desc)) {
-      String owner = desc.getOwnerString();
-      // default the table owner to current user, if not specified.
-      if (owner == null) owner = getActiveUser().getShortName();
-      UserPermission userperm = new UserPermission(Bytes.toBytes(owner), desc.getName(), null,
-          Action.values());
-      AccessControlLists.addUserPermission(c.getEnvironment().getConfiguration(), userperm);
+      final Configuration conf = c.getEnvironment().getConfiguration();
+      final String owner = (desc.getOwnerString() != null) ? desc.getOwnerString() : 
+        getActiveUser().getShortName();
+      User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          UserPermission userperm = new UserPermission(Bytes.toBytes(owner), desc.getName(), null,
+              Action.values());
+          AccessControlLists.addUserPermission(conf, userperm);
+          return null;
+        }
+      });
     }
   }
 
@@ -592,8 +585,15 @@ public class AccessController extends BaseRegionObserver
 
   @Override
   public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      byte[] tableName) throws IOException {
-    AccessControlLists.removeTablePermissions(c.getEnvironment().getConfiguration(), tableName);
+      final byte[] tableName) throws IOException {
+    final Configuration conf = c.getEnvironment().getConfiguration();
+    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        AccessControlLists.removeTablePermissions(conf, tableName);
+        return null;
+      }
+    });
   }
 
   @Override
@@ -604,13 +604,19 @@ public class AccessController extends BaseRegionObserver
 
   @Override
   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      byte[] tableName, HTableDescriptor htd) throws IOException {
-    String owner = htd.getOwnerString();
-    // default the table owner to current user, if not specified.
-    if (owner == null) owner = getActiveUser().getShortName();
-    UserPermission userperm = new UserPermission(Bytes.toBytes(owner), htd.getName(), null,
-        Action.values());
-    AccessControlLists.addUserPermission(c.getEnvironment().getConfiguration(), userperm);
+      byte[] tableName, final HTableDescriptor htd) throws IOException {
+    final Configuration conf = c.getEnvironment().getConfiguration();
+    final String owner = (htd.getOwnerString() != null) ? htd.getOwnerString() : 
+      getActiveUser().getShortName();
+    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        UserPermission userperm = new UserPermission(Bytes.toBytes(owner), htd.getName(), null,
+            Action.values());
+        AccessControlLists.addUserPermission(conf, userperm);
+        return null;
+      }
+    });
   }
 
   @Override
@@ -641,9 +647,15 @@ public class AccessController extends BaseRegionObserver
 
   @Override
   public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
-      byte[] tableName, byte[] col) throws IOException {
-    AccessControlLists.removeTablePermissions(c.getEnvironment().getConfiguration(),
-                                              tableName, col);
+      final byte[] tableName, final byte[] col) throws IOException {
+    final Configuration conf = c.getEnvironment().getConfiguration();
+    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        AccessControlLists.removeTablePermissions(conf, tableName, col);
+        return null;
+      }
+    });
   }
 
   @Override
@@ -1098,16 +1110,24 @@ public class AccessController extends BaseRegionObserver
 
   private AuthResult hasSomeAccess(RegionCoprocessorEnvironment e, String request, Action action) throws IOException {
     User requestUser = getActiveUser();
-    byte[] tableName = e.getRegion().getTableDesc().getName();
+    final byte[] tableName = e.getRegion().getTableDesc().getName();
     AuthResult authResult = permissionGranted(request, requestUser,
         action, e, Collections.EMPTY_MAP);
     if (!authResult.isAllowed()) {
-      for(UserPermission userPerm:
-          AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), tableName)) {
-        for(Permission.Action userAction: userPerm.getActions()) {
-          if(userAction.equals(action)) {
+      final Configuration conf = e.getConfiguration();
+      // hasSomeAccess is called from bulkload pre hooks
+      List<UserPermission> perms =
+        User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+          @Override
+          public List<UserPermission> run() throws Exception {
+            return AccessControlLists.getUserPermissions(conf, tableName);
+          }
+        });
+      for (UserPermission userPerm: perms) {
+        for (Action userAction: userPerm.getActions()) {
+          if (userAction.equals(action)) {
             return AuthResult.allow(request, "Access allowed", requestUser,
-                action, tableName);
+              action, tableName);
           }
         }
       }
@@ -1152,7 +1172,7 @@ public class AccessController extends BaseRegionObserver
    * This will be restricted by both client side and endpoint implementations.
    */
   @Override
-  public void grant(UserPermission perm) throws IOException {
+  public void grant(final UserPermission perm) throws IOException {
     // verify it's only running at .acl.
     if (aclRegion) {
       if (LOG.isDebugEnabled()) {
@@ -1161,7 +1181,14 @@ public class AccessController extends BaseRegionObserver
 
       requirePermission("grant", perm.getTable(), perm.getFamily(), perm.getQualifier(), Action.ADMIN);
 
-      AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm);
+      User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm);
+          return null;
+        }
+      });
+
       if (AUDITLOG.isTraceEnabled()) {
         // audit log should store permission changes in addition to auth results
         AUDITLOG.trace("Granted permission " + perm.toString());
@@ -1182,7 +1209,7 @@ public class AccessController extends BaseRegionObserver
   }
 
   @Override
-  public void revoke(UserPermission perm) throws IOException {
+  public void revoke(final UserPermission perm) throws IOException {
     // only allowed to be called on _acl_ region
     if (aclRegion) {
       if (LOG.isDebugEnabled()) {
@@ -1192,7 +1219,14 @@ public class AccessController extends BaseRegionObserver
       requirePermission("revoke", perm.getTable(), perm.getFamily(),
                         perm.getQualifier(), Action.ADMIN);
 
-      AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm);
+      User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm);
+          return null;
+        }
+      });
+
       if (AUDITLOG.isTraceEnabled()) {
         // audit log should record all permission changes
         AUDITLOG.trace("Revoked permission " + perm.toString());
@@ -1217,10 +1251,12 @@ public class AccessController extends BaseRegionObserver
     // only allowed to be called on _acl_ region
     if (aclRegion) {
       requirePermission("userPermissions", tableName, null, null, Action.ADMIN);
-
-      List<UserPermission> perms = AccessControlLists.getUserPermissions(
-        regionEnv.getConfiguration(), tableName);
-      return perms;
+      return User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+        @Override
+        public List<UserPermission> run() throws Exception {
+          return AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), tableName);
+        }
+      });
     } else {
       throw new CoprocessorException(AccessController.class, "This method "
           + "can only execute at " + Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/eed06221/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index d696acb..688e30d 100644
--- a/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.security.access;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -1962,4 +1963,150 @@ public class TestAccessController {
     verifyAllowed(deleteTableAction, tableAdmin);
   }
 
+  @Test
+  public void testCreateWithCorrectOwner() throws Exception {
+    final byte[] tableName = Bytes.toBytes("testCreateWithCorrectOwner");
+    
+    // Create a test user
+    User testUser = User.createUserForTesting(TEST_UTIL.getConfiguration(), "TestUser",
+      new String[0]);
+
+    // Grant the test user the ability to create tables
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      AccessControllerProtocol protocol = acl.coprocessorProxy(
+          AccessControllerProtocol.class, AccessControlLists.ACL_TABLE_NAME);
+      protocol.grant(new UserPermission(Bytes.toBytes(testUser.getShortName()),
+        Permission.Action.CREATE));
+    } finally {
+      acl.close();
+    }
+
+    verifyAllowed(new PrivilegedExceptionAction() {
+      @Override
+      public Object run() throws Exception {
+        HTableDescriptor desc = new HTableDescriptor(tableName);
+        desc.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        HBaseAdmin admin = new HBaseAdmin(conf);
+        try {
+          admin.createTable(desc);
+        } finally {
+          admin.close();
+        }
+        return null;
+      }
+    }, testUser);
+    TEST_UTIL.waitTableEnabled(tableName, 5000);
+
+    // Verify that owner permissions have been granted to the test user on the
+    // table just created
+    List<TablePermission> perms = AccessControlLists.getTablePermissions(conf, tableName)
+       .get(testUser.getShortName());
+    assertNotNull(perms);
+    assertFalse(perms.isEmpty());
+    // Should be RWXCA
+    assertTrue(perms.get(0).implies(Permission.Action.READ));
+    assertTrue(perms.get(0).implies(Permission.Action.WRITE));
+    assertTrue(perms.get(0).implies(Permission.Action.EXEC));
+    assertTrue(perms.get(0).implies(Permission.Action.CREATE));
+    assertTrue(perms.get(0).implies(Permission.Action.ADMIN));
+  }
+
+  @Test
+  public void testACLTableAccess() throws Exception {
+    final Configuration conf = TEST_UTIL.getConfiguration();
+
+    final byte[] tableName = Bytes.toBytes("testACLTableAccess");
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+    admin.createTable(htd);
+    TEST_UTIL.waitTableEnabled(tableName, 5000);
+
+    // Global users
+    User globalRead = User.createUserForTesting(conf, "globalRead", new String[0]);
+    User globalWrite = User.createUserForTesting(conf, "globalWrite", new String[0]);
+    User globalCreate = User.createUserForTesting(conf, "globalCreate", new String[0]);
+    User globalAdmin = User.createUserForTesting(conf, "globalAdmin", new String[0]);
+    
+    // Table users
+    User tableRead = User.createUserForTesting(conf, "tableRead", new String[0]);
+    User tableWrite = User.createUserForTesting(conf, "tableWrite", new String[0]);
+    User tableCreate = User.createUserForTesting(conf, "tableCreate", new String[0]);
+    User tableAdmin = User.createUserForTesting(conf, "tableAdmin", new String[0]);
+
+    // Set up grants
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      AccessControllerProtocol protocol = acl.coprocessorProxy(
+        AccessControllerProtocol.class, AccessControlLists.ACL_TABLE_NAME);
+      protocol.grant(new UserPermission(Bytes.toBytes(globalRead.getShortName()), Action.READ));
+      protocol.grant(new UserPermission(Bytes.toBytes(globalWrite.getShortName()), Action.WRITE));
+      protocol.grant(new UserPermission(Bytes.toBytes(globalCreate.getShortName()),
+        Action.CREATE));
+      protocol.grant(new UserPermission(Bytes.toBytes(globalAdmin.getShortName()),
+        Action.ADMIN));
+      protocol.grant(new UserPermission(Bytes.toBytes(tableRead.getShortName()), tableName,
+        null, Action.READ));
+      protocol.grant(new UserPermission(Bytes.toBytes(tableWrite.getShortName()), tableName,
+        null, Action.WRITE));
+      protocol.grant(new UserPermission(Bytes.toBytes(tableCreate.getShortName()), tableName,
+        null, Action.CREATE));
+      protocol.grant(new UserPermission(Bytes.toBytes(tableAdmin.getShortName()), tableName,
+        null, Action.ADMIN));
+    } finally {
+      acl.close();
+    }
+
+    // Write tests
+
+    PrivilegedExceptionAction writeAction = new PrivilegedExceptionAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+        try {
+          t.put(new Put(Bytes.toBytes("test")).add(AccessControlLists.ACL_LIST_FAMILY,
+            Bytes.toBytes("q"), Bytes.toBytes("value")));
+          return null;
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    // All writes to ACL table denied except for GLOBAL WRITE permission and superuser
+
+    verifyDenied(writeAction, globalAdmin, globalCreate, globalRead);
+    verifyDenied(writeAction, tableAdmin, tableCreate, tableRead, tableWrite);
+    verifyAllowed(writeAction, SUPERUSER, globalWrite);
+
+    // Read tests
+
+    PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() {
+      @Override
+      public Object run() throws Exception {
+        HTable t = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+        try {
+          ResultScanner s = t.getScanner(new Scan());
+          try {
+            for (Result r = s.next(); r != null; r = s.next()) {
+              // do nothing
+            }
+          } finally {
+            s.close();
+          }
+          return null;
+        } finally {
+          t.close();
+        }
+      }
+    };
+
+    // All reads from ACL table denied except for GLOBAL READ and superuser
+
+    verifyDenied(scanAction, globalAdmin, globalCreate, globalWrite);
+    verifyDenied(scanAction, tableCreate, tableAdmin, tableRead, tableWrite);
+    verifyAllowed(scanAction, SUPERUSER, globalRead);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eed06221/src/main/java/org/apache/hadoop/hbase/security/User.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/security/User.java b/src/main/java/org/apache/hadoop/hbase/security/User.java
index 96e0a39..f19e7fb 100644
--- a/src/main/java/org/apache/hadoop/hbase/security/User.java
+++ b/src/main/java/org/apache/hadoop/hbase/security/User.java
@@ -159,6 +159,27 @@ public abstract class User {
   }
 
   /**
+   * Executes the given action as the login user
+   * @param action
+   * @return
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public static <T> T runAsLoginUser(PrivilegedExceptionAction<T> action) throws IOException {
+    return doAsUser(UserGroupInformation.getLoginUser(), action);
+  }
+
+  private static <T> T doAsUser(UserGroupInformation ugi,
+      PrivilegedExceptionAction<T> action) throws IOException {
+    try {
+      return ugi.doAs(action);
+    } catch (InterruptedException ie) {
+      throw new IOException(ie);
+    }
+  }
+
+  /**
    * Wraps an underlying {@code UserGroupInformation} instance.
    * @param ugi The base Hadoop user
    * @return User