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

[3/3] git commit: HBASE-11886 The creator of the table should have all permissions on the table (Devaraj Das and Andrew Purtell)

HBASE-11886 The creator of the table should have all permissions on the table (Devaraj Das and Andrew Purtell)


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

Branch: refs/heads/0.98
Commit: f463c4eb27f05df0048f2bb2704549177495f385
Parents: da3ce2c
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Sep 3 18:30:54 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Sep 3 18:30:54 2014 -0700

----------------------------------------------------------------------
 .../master/handler/CreateTableHandler.java      |  23 ++++-
 .../security/access/TestAccessController2.java  | 102 +++++++++++++++++++
 2 files changed, 123 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f463c4eb/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
index e643395..757db4a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.handler;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.ipc.RequestContext;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -46,6 +48,8 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@@ -65,6 +69,7 @@ public class CreateTableHandler extends EventHandler {
   private final TableLockManager tableLockManager;
   private final HRegionInfo [] newRegions;
   private final TableLock tableLock;
+  private User activeUser;
 
   public CreateTableHandler(Server server, MasterFileSystem fileSystemManager,
       HTableDescriptor hTableDescriptor, Configuration conf, HRegionInfo [] newRegions,
@@ -91,6 +96,14 @@ public class CreateTableHandler extends EventHandler {
       if(catalogTracker.waitForMeta(timeout) == null) {
         throw new NotAllMetaRegionsOnlineException();
       }
+      // If we are creating the table in service to an RPC request, record the
+      // active user for later, so proper permissions will be applied to the
+      // new table by the AccessController if it is active
+      if (RequestContext.isInRequestContext()) {
+        this.activeUser = RequestContext.getRequestUser();
+      } else {
+        this.activeUser = UserProvider.instantiate(conf).getCurrent();
+      }
     } catch (InterruptedException e) {
       LOG.warn("Interrupted waiting for meta availability", e);
       InterruptedIOException ie = new InterruptedIOException(e.getMessage());
@@ -148,14 +161,20 @@ public class CreateTableHandler extends EventHandler {
     LOG.info("Create table " + tableName);
 
     try {
-      MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
+      final MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
       if (cpHost != null) {
         cpHost.preCreateTableHandler(this.hTableDescriptor, this.newRegions);
       }
       handleCreateTable(tableName);
       completed(null);
       if (cpHost != null) {
-        cpHost.postCreateTableHandler(this.hTableDescriptor, this.newRegions);
+        this.activeUser.runAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            cpHost.postCreateTableHandler(hTableDescriptor, newRegions);
+            return null;
+          }
+        });
       }
     } catch (Throwable e) {
       LOG.error("Error trying to create the table " + tableName, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f463c4eb/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
new file mode 100644
index 0000000..864739c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.security.access;
+
+import static org.junit.Assert.*;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.TestTableName;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestAccessController2 extends SecureTestUtil {
+
+  private static final byte[] TEST_FAMILY = Bytes.toBytes("f");
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf;
+
+  @Rule public TestTableName TEST_TABLE = new TestTableName();
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    // Enable security
+    enableSecurity(conf);
+    // Verify enableSecurity sets up what we require
+    verifyConfiguration(conf);
+    TEST_UTIL.startMiniCluster();
+    // Wait for the ACL table to become available
+    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testCreateWithCorrectOwner() throws Exception {
+    // Create a test user
+    User testUser = User.createUserForTesting(TEST_UTIL.getConfiguration(), "TestUser",
+      new String[0]);
+    // Grant the test user the ability to create tables
+    SecureTestUtil.grantGlobal(TEST_UTIL, testUser.getShortName(), Action.CREATE);
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTableDescriptor desc = new HTableDescriptor(TEST_TABLE.getTableName());
+        desc.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        HBaseAdmin admin = new HBaseAdmin(conf);
+        try {
+          admin.createTable(desc);
+        } finally {
+          admin.close();
+        }
+        return null;
+      }
+    }, testUser);
+    TEST_UTIL.waitTableEnabled(TEST_TABLE.getTableName().getName());
+    // Verify that owner permissions have been granted to the test user on the
+    // table just created
+    List<TablePermission> perms = AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName())
+       .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));
+  }
+
+}