You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2014/12/09 13:12:44 UTC

[3/4] hbase git commit: HBASE-12564 consolidate the getTableDescriptors() semantic

http://git-wip-us.apache.org/repos/asf/hbase/blob/09617cc2/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
index ae4af26..4ed81dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
@@ -223,7 +223,7 @@ public class TableAuthManager {
    * Updates the internal permissions cache for a single table, splitting
    * the permissions listed into separate caches for users and groups to optimize
    * group lookups.
-   * 
+   *
    * @param table
    * @param tablePerms
    */
@@ -349,6 +349,20 @@ public class TableAuthManager {
     return false;
   }
 
+  private boolean hasAccess(List<TablePermission> perms,
+                            TableName table, Permission.Action action) {
+    if (perms != null) {
+      for (TablePermission p : perms) {
+        if (p.implies(action)) {
+          return true;
+        }
+      }
+    } else if (LOG.isDebugEnabled()) {
+      LOG.debug("No permissions found for table="+table);
+    }
+    return false;
+  }
+
   /**
    * Authorize a user for a given KV. This is called from AccessControlFilter.
    */
@@ -442,7 +456,7 @@ public class TableAuthManager {
       byte[] qualifier, Permission.Action action) {
     if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
     // Global and namespace authorizations supercede table level
-    if (authorize(user, table.getNamespaceAsString(), action)) {    
+    if (authorize(user, table.getNamespaceAsString(), action)) {
       return true;
     }
     // Check table permissions
@@ -451,6 +465,25 @@ public class TableAuthManager {
   }
 
   /**
+   * Checks if the user has access to the full table or at least a family/qualifier
+   * for the specified action.
+   *
+   * @param user
+   * @param table
+   * @param action
+   * @return true if the user has access to the table, false otherwise
+   */
+  public boolean userHasAccess(User user, TableName table, Permission.Action action) {
+    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
+    // Global and namespace authorizations supercede table level
+    if (authorize(user, table.getNamespaceAsString(), action)) {
+      return true;
+    }
+    // Check table permissions
+    return hasAccess(getTablePermissions(table).getUser(user.getShortName()), table, action);
+  }
+
+  /**
    * Checks global authorization for a given action for a group, based on the stored
    * permissions.
    */
@@ -460,7 +493,7 @@ public class TableAuthManager {
 
   /**
    * Checks authorization to a given table and column family for a group, based
-   * on the stored permissions. 
+   * on the stored permissions.
    * @param groupName
    * @param table
    * @param family
@@ -483,6 +516,29 @@ public class TableAuthManager {
     return authorize(getTablePermissions(table).getGroup(groupName), table, family, action);
   }
 
+  /**
+   * Checks if the user has access to the full table or at least a family/qualifier
+   * for the specified action.
+   * @param groupName
+   * @param table
+   * @param action
+   * @return true if the group has access to the table, false otherwise
+   */
+  public boolean groupHasAccess(String groupName, TableName table, Permission.Action action) {
+    // Global authorization supercedes table level
+    if (authorizeGroup(groupName, action)) {
+      return true;
+    }
+    if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
+    // Namespace authorization supercedes table level
+    if (hasAccess(getNamespacePermissions(table.getNamespaceAsString()).getGroup(groupName),
+        table, action)) {
+      return true;
+    }
+    // Check table level
+    return hasAccess(getTablePermissions(table).getGroup(groupName), table, action);
+  }
+
   public boolean authorize(User user, TableName table, byte[] family,
       byte[] qualifier, Permission.Action action) {
     if (authorizeUser(user, table, family, qualifier, action)) {
@@ -500,6 +556,22 @@ public class TableAuthManager {
     return false;
   }
 
+  public boolean hasAccess(User user, TableName table, Permission.Action action) {
+    if (userHasAccess(user, table, action)) {
+      return true;
+    }
+
+    String[] groups = user.getGroupNames();
+    if (groups != null) {
+      for (String group : groups) {
+        if (groupHasAccess(group, table, action)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
   public boolean authorize(User user, TableName table, byte[] family,
       Permission.Action action) {
     return authorize(user, table, family, null, action);

http://git-wip-us.apache.org/repos/asf/hbase/blob/09617cc2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 22e9e67..dc4253e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
@@ -146,6 +147,8 @@ public class TestMasterObserver {
     private boolean postModifyTableHandlerCalled;
     private boolean preGetTableDescriptorsCalled;
     private boolean postGetTableDescriptorsCalled;
+    private boolean postGetTableNamesCalled;
+    private boolean preGetTableNamesCalled;
 
     public void enableBypass(boolean bypass) {
       this.bypass = bypass;
@@ -218,6 +221,8 @@ public class TestMasterObserver {
       postModifyTableHandlerCalled = false;
       preGetTableDescriptorsCalled = false;
       postGetTableDescriptorsCalled = false;
+      postGetTableNamesCalled = false;
+      preGetTableNamesCalled = false;
     }
 
     @Override
@@ -656,11 +661,11 @@ public class TestMasterObserver {
         ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
       preMasterInitializationCalled = true;
     }
-    
+
     public boolean wasMasterInitializationCalled(){
       return preMasterInitializationCalled;
     }
-    
+
     @Override
     public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
         throws IOException {
@@ -984,14 +989,25 @@ public class TestMasterObserver {
 
     @Override
     public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        List<TableName> tableNamesList, List<HTableDescriptor> descriptors)
-        throws IOException {
-      preGetTableDescriptorsCalled = true;
+        List<TableName> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
     }
 
     @Override
     public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
         List<HTableDescriptor> descriptors) throws IOException {
+    }
+
+    @Override
+    public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+        String regex) throws IOException {
+      preGetTableDescriptorsCalled = true;
+    }
+
+    @Override
+    public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+        String regex) throws IOException {
       postGetTableDescriptorsCalled = true;
     }
 
@@ -1000,6 +1016,22 @@ public class TestMasterObserver {
     }
 
     @Override
+    public void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<HTableDescriptor> descriptors, String regex) throws IOException {
+      preGetTableNamesCalled = true;
+    }
+
+    @Override
+    public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<HTableDescriptor> descriptors, String regex) throws IOException {
+      postGetTableNamesCalled = true;
+    }
+
+    public boolean wasGetTableNamesCalled() {
+      return preGetTableNamesCalled && postGetTableNamesCalled;
+    }
+
+    @Override
     public void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
         TableName tableName) throws IOException {
     }
@@ -1465,4 +1497,19 @@ public class TestMasterObserver {
       cp.wasGetTableDescriptorsCalled());
   }
 
+  @Test
+  public void testTableNamesEnumeration() throws Exception {
+    MiniHBaseCluster cluster = UTIL.getHBaseCluster();
+
+    HMaster master = cluster.getMaster();
+    MasterCoprocessorHost host = master.getMasterCoprocessorHost();
+    CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
+        CPMasterObserver.class.getName());
+    cp.resetStates();
+
+    master.getMasterRpcServices().getTableNames(null,
+        GetTableNamesRequest.newBuilder().build());
+    assertTrue("Coprocessor should be called on table names request",
+      cp.wasGetTableNamesCalled());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09617cc2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index af734df..b5d7e3d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -20,10 +20,13 @@ package org.apache.hadoop.hbase.security.access;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -259,7 +262,7 @@ public class TestAccessController extends SecureTestUtil {
     try {
       assertEquals(4, AccessControlClient.getUserPermissions(conf, TEST_TABLE.toString()).size());
     } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.getUserPermissions. " + e.getStackTrace());
+      LOG.error("error during call of AccessControlClient.getUserPermissions. ", e);
     }
   }
 
@@ -1992,11 +1995,14 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction listTablesAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Admin admin = TEST_UTIL.getHBaseAdmin();
+        Connection unmanagedConnection =
+          ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Admin admin = unmanagedConnection.getAdmin();
         try {
           admin.listTables();
         } finally {
           admin.close();
+          unmanagedConnection.close();
         }
         return null;
       }
@@ -2005,24 +2011,48 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction getTableDescAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        Admin admin = TEST_UTIL.getHBaseAdmin();
+        Connection unmanagedConnection =
+          ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Admin admin = unmanagedConnection.getAdmin();
         try {
           admin.getTableDescriptor(TEST_TABLE.getTableName());
         } finally {
           admin.close();
+          unmanagedConnection.close();
         }
         return null;
       }
     };
 
-    verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN);
-    verifyDenied(listTablesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, TABLE_ADMIN);
+    verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN, USER_CREATE, TABLE_ADMIN);
+    verifyDenied(listTablesAction, USER_RW, USER_RO, USER_NONE);
 
     verifyAllowed(getTableDescAction, SUPERUSER, USER_ADMIN, USER_CREATE, TABLE_ADMIN);
     verifyDenied(getTableDescAction, USER_RW, USER_RO, USER_NONE);
   }
 
   @Test
+  public void testTableNameEnumeration() throws Exception {
+    AccessTestAction listTablesAction = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Connection unmanagedConnection =
+            ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+        Admin admin = unmanagedConnection.getAdmin();
+        try {
+          return Arrays.asList(admin.listTableNames());
+        } finally {
+          admin.close();
+          unmanagedConnection.close();
+        }
+      }
+    };
+
+    verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO);
+    verifyDenied(listTablesAction, USER_NONE);
+  }
+
+  @Test
   public void testTableDeletion() throws Exception {
     User TABLE_ADMIN = User.createUserForTesting(conf, "TestUser", new String[0]);
 
@@ -2099,7 +2129,7 @@ public class TestAccessController extends SecureTestUtil {
       grantOnTableUsingAccessControlClient(TEST_UTIL, conf, testGrantRevoke.getShortName(),
           TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
     } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.grant. " + e.getStackTrace());
+      LOG.error("error during call of AccessControlClient.grant. ", e);
     }
 
     // Now testGrantRevoke should be able to read also
@@ -2110,7 +2140,7 @@ public class TestAccessController extends SecureTestUtil {
       revokeFromTableUsingAccessControlClient(TEST_UTIL, conf, testGrantRevoke.getShortName(),
           TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
     } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.revoke " + e.getStackTrace());
+      LOG.error("error during call of AccessControlClient.revoke ", e);
     }
 
     // Now testGrantRevoke shouldn't be able read
@@ -2140,7 +2170,7 @@ public class TestAccessController extends SecureTestUtil {
       grantOnNamespaceUsingAccessControlClient(TEST_UTIL, conf, testNS.getShortName(),
           TEST_TABLE.getTableName().getNamespaceAsString(), Permission.Action.READ);
     } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.grant. " + e.getStackTrace());
+      LOG.error("error during call of AccessControlClient.grant. ", e);
     }
 
     // Now testNS should be able to read also
@@ -2151,7 +2181,7 @@ public class TestAccessController extends SecureTestUtil {
       revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, conf, testNS.getShortName(),
           TEST_TABLE.getTableName().getNamespaceAsString(), Permission.Action.READ);
     } catch (Throwable e) {
-      LOG.error("error during call of AccessControlClient.revoke " + e.getStackTrace());
+      LOG.error("error during call of AccessControlClient.revoke ", e);
     }
 
     // Now testNS shouldn't be able read
@@ -2262,7 +2292,7 @@ public class TestAccessController extends SecureTestUtil {
         try {
           KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
             HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
-            new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE, 
+            new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE,
               ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(),
                 new Permission(Permission.Action.READ)).toByteArray()) });
           t.put(new Put(TEST_ROW).add(kv));
@@ -2314,4 +2344,88 @@ public class TestAccessController extends SecureTestUtil {
       assertEquals(existingPerms.size(), perms.size());
     }
   }
+
+  private PrivilegedAction<List<UserPermission>> getPrivilegedAction(final String regex) {
+    return new PrivilegedAction<List<UserPermission>>() {
+      @Override
+      public List<UserPermission> run() {
+        try {
+          return AccessControlClient.getUserPermissions(conf, regex);
+        } catch (Throwable e) {
+          LOG.error("error during call of AccessControlClient.getUserPermissions.", e);
+          return null;
+        }
+      }
+    };
+  }
+
+  @Test
+  public void testAccessControlClientUserPerms() throws Exception {
+    // adding default prefix explicitly as it is not included in the table name.
+    assertEquals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR,
+                 TEST_TABLE.getTableName().getNamespaceAsString());
+    final String regex = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR +
+      TableName.NAMESPACE_DELIM + TEST_TABLE.getTableName().getNameAsString();
+    User testUserPerms = User.createUserForTesting(conf, "testUserPerms", new String[0]);
+    assertEquals(0, testUserPerms.runAs(getPrivilegedAction(regex)).size());
+    // Grant TABLE ADMIN privs to testUserPerms
+    grantOnTable(TEST_UTIL, testUserPerms.getShortName(), TEST_TABLE.getTableName(), null,
+      null, Action.ADMIN);
+    List<UserPermission> perms = testUserPerms.runAs(getPrivilegedAction(regex));
+    assertNotNull(perms);
+    // USER_ADMIN, USER_CREATE, USER_RW, USER_RO, testUserPerms has row each.
+    assertEquals(5, perms.size());
+  }
+
+  @Test
+  public void testAccessControllerUserPermsRegexHandling() throws Exception {
+    User testRegexHandler = User.createUserForTesting(conf, "testRegexHandling", new String[0]);
+
+    final String REGEX_ALL_TABLES = ".*";
+    final String tableName = "testRegex";
+    final TableName table1 = TableName.valueOf(tableName);
+    final byte[] family = Bytes.toBytes("f1");
+
+    // create table in default ns
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor htd = new HTableDescriptor(table1);
+    htd.addFamily(new HColumnDescriptor(family));
+    admin.createTable(htd);
+    TEST_UTIL.waitUntilAllRegionsAssigned(table1);
+
+    // creating the ns and table in it
+    String ns = "testNamespace";
+    NamespaceDescriptor desc = NamespaceDescriptor.create(ns).build();
+    final TableName table2 = TableName.valueOf(ns, tableName);
+    TEST_UTIL.getMiniHBaseCluster().getMaster().createNamespace(desc);
+    htd = new HTableDescriptor(table2);
+    htd.addFamily(new HColumnDescriptor(family));
+    admin.createTable(htd);
+    TEST_UTIL.waitUntilAllRegionsAssigned(table2);
+
+    // Verify that we can read sys-tables
+    String aclTableName = AccessControlLists.ACL_TABLE_NAME.getNameAsString();
+    assertEquals(1, SUPERUSER.runAs(getPrivilegedAction(aclTableName)).size());
+    assertEquals(0, testRegexHandler.runAs(getPrivilegedAction(aclTableName)).size());
+
+    // Grant TABLE ADMIN privs to testUserPerms
+    assertEquals(0, testRegexHandler.runAs(getPrivilegedAction(REGEX_ALL_TABLES)).size());
+    grantOnTable(TEST_UTIL, testRegexHandler.getShortName(), table1, null, null, Action.ADMIN);
+    assertEquals(2, testRegexHandler.runAs(getPrivilegedAction(REGEX_ALL_TABLES)).size());
+    grantOnTable(TEST_UTIL, testRegexHandler.getShortName(), table2, null, null, Action.ADMIN);
+    assertEquals(4, testRegexHandler.runAs(getPrivilegedAction(REGEX_ALL_TABLES)).size());
+
+    // USER_ADMIN, testUserPerms must have a row each.
+    assertEquals(2, testRegexHandler.runAs(getPrivilegedAction(tableName)).size());
+    assertEquals(2, testRegexHandler.runAs(getPrivilegedAction(
+          NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + TableName.NAMESPACE_DELIM + tableName)
+        ).size());
+    assertEquals(2, testRegexHandler.runAs(getPrivilegedAction(
+        ns + TableName.NAMESPACE_DELIM + tableName)).size());
+    assertEquals(0, testRegexHandler.runAs(getPrivilegedAction("notMatchingAny")).size());
+
+    TEST_UTIL.deleteTable(table1);
+    TEST_UTIL.deleteTable(table2);
+    TEST_UTIL.getMiniHBaseCluster().getMaster().deleteNamespace(ns);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09617cc2/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 2c8828e..748a41c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -43,7 +43,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Returns a list of tables in hbase
     def list(regex = ".*")
-      @admin.getTableNames(regex).to_a
+      @admin.listTableNames(regex).map { |t| t.getNameAsString }
     end
 
     #----------------------------------------------------------------------------------------------