You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ma...@apache.org on 2017/01/23 19:50:03 UTC

incubator-ranger git commit: RANGER-1314: updated HBase plugin to support authorization of namespace operations

Repository: incubator-ranger
Updated Branches:
  refs/heads/master 11aa58b63 -> eaef96be8


RANGER-1314: updated HBase plugin to support authorization of namespace operations

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: eaef96be8f4ac96522cd25173146e3f3fbfa3b37
Parents: 11aa58b
Author: Abhay Kulkarni <ak...@hortonworks.com>
Authored: Thu Jan 12 18:20:36 2017 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Mon Jan 23 11:28:47 2017 -0800

----------------------------------------------------------------------
 .../hbase/RangerAuthorizationCoprocessor.java   |   8 +-
 .../hbase/HBaseRangerAuthorizationTest.java     | 232 ++++++++++++++-----
 .../src/test/resources/hbase-policies.json      |  55 +++++
 3 files changed, 239 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/eaef96be/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/RangerAuthorizationCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/RangerAuthorizationCoprocessor.java b/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/RangerAuthorizationCoprocessor.java
index cc61a83..5dd727b 100644
--- a/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/RangerAuthorizationCoprocessor.java
+++ b/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/RangerAuthorizationCoprocessor.java
@@ -752,7 +752,7 @@ public class RangerAuthorizationCoprocessor extends RangerAuthorizationCoprocess
 	}
 	@Override
 	public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
-		requirePermission("cloneSnapshot", Permission.Action.ADMIN);
+		requirePermission("cloneSnapshot", hTableDescriptor.getTableName().getName(), Permission.Action.ADMIN);
 	}
 	@Override
 	public void preClose(ObserverContext<RegionCoprocessorEnvironment> e, boolean abortRequested) throws IOException {
@@ -770,7 +770,7 @@ public class RangerAuthorizationCoprocessor extends RangerAuthorizationCoprocess
 
 	@Override
 	public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c, HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-		requirePermission("createTable", desc.getName(), Permission.Action.CREATE);
+		requirePermission("createTable", desc.getTableName().getName(), Permission.Action.CREATE);
 	}
 	@Override
 	public void preDelete(ObserverContext<RegionCoprocessorEnvironment> c, Delete delete, WALEdit edit, Durability durability) throws IOException {
@@ -879,7 +879,7 @@ public class RangerAuthorizationCoprocessor extends RangerAuthorizationCoprocess
 	}
 	@Override
 	public void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
-		requirePermission("restoreSnapshot", hTableDescriptor.getName(), Permission.Action.ADMIN);
+		requirePermission("restoreSnapshot", hTableDescriptor.getTableName().getName(), Permission.Action.ADMIN);
 	}
 
 	@Override
@@ -928,7 +928,7 @@ public class RangerAuthorizationCoprocessor extends RangerAuthorizationCoprocess
 	}
 	@Override
 	public void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, HTableDescriptor hTableDescriptor) throws IOException {
-		requirePermission("snapshot", hTableDescriptor.getName(), Permission.Action.ADMIN);
+		requirePermission("snapshot", hTableDescriptor.getTableName().getName(), Permission.Action.ADMIN);
 	}
 	@Override
 	public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/eaef96be/hbase-agent/src/test/java/org/apache/ranger/authorization/hbase/HBaseRangerAuthorizationTest.java
----------------------------------------------------------------------
diff --git a/hbase-agent/src/test/java/org/apache/ranger/authorization/hbase/HBaseRangerAuthorizationTest.java b/hbase-agent/src/test/java/org/apache/ranger/authorization/hbase/HBaseRangerAuthorizationTest.java
index 8fe6284..7aec352 100644
--- a/hbase-agent/src/test/java/org/apache/ranger/authorization/hbase/HBaseRangerAuthorizationTest.java
+++ b/hbase-agent/src/test/java/org/apache/ranger/authorization/hbase/HBaseRangerAuthorizationTest.java
@@ -21,12 +21,17 @@ import java.io.IOException;
 import java.net.ServerSocket;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
+import java.util.List;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
@@ -36,6 +41,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
@@ -47,13 +53,16 @@ import org.junit.Test;
  * 
  * a) The "logged in" user can do anything
  * b) The IT group can read and write to the "temp" table, but only the "colfam1" column family.
- * 
+ * c) The QA group can read and write to tables in "test_namespace" namespace.
+ *
  * Policies available from admin via:
  * 
  * http://localhost:6080/service/plugins/policies/download/HBASETest
  */
 public class HBaseRangerAuthorizationTest {
-    
+
+    private static final Log LOG = LogFactory.getLog(HBaseRangerAuthorizationTest.class.getName());
+
     private static int port;
     private static HBaseTestingUtility utility;
     
@@ -110,6 +119,32 @@ public class HBaseRangerAuthorizationTest {
         put.addColumn(Bytes.toBytes("colfam2"), Bytes.toBytes("col1"), Bytes.toBytes("val2"));
         table.put(put);
 
+        // Create a namespace
+        NamespaceDescriptor ns = NamespaceDescriptor.create("test_namespace").build();
+        admin.createNamespace(ns);
+
+        // Create a table
+        if (!admin.tableExists(TableName.valueOf("test_namespace", "temp"))) {
+            HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf("test_namespace", "temp"));
+
+            // Adding column families to table descriptor
+            tableDescriptor.addFamily(new HColumnDescriptor("colfam1"));
+            tableDescriptor.addFamily(new HColumnDescriptor("colfam2"));
+
+            admin.createTable(tableDescriptor);
+        }
+
+        table = conn.getTable(TableName.valueOf("test_namespace", "temp"));
+
+        // Add a new row
+        put = new Put(Bytes.toBytes("row1"));
+        put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("col1"), Bytes.toBytes("val1"));
+        table.put(put);
+
+        put = new Put(Bytes.toBytes("row1"));
+        put.addColumn(Bytes.toBytes("colfam2"), Bytes.toBytes("col1"), Bytes.toBytes("val2"));
+        table.put(put);
+
         conn.close();
     }
     
@@ -129,12 +164,15 @@ public class HBaseRangerAuthorizationTest {
         Admin admin = conn.getAdmin();
 
         HTableDescriptor[] tableDescriptors = admin.listTables();
-        Assert.assertEquals(1, tableDescriptors.length);
+        for (HTableDescriptor desc : tableDescriptors) {
+            LOG.info("Found table:[" + desc.getTableName().getNameAsString() + "]");
+        }
+        Assert.assertEquals(2, tableDescriptors.length);
 
         conn.close();
     }
     
-    // This should fail, as the "IT" group only has read privileges, not admin privileges, on the table "temp"
+    // This should fail as the "IT" group only has read privileges, not admin privileges, on the table "temp"
     @Test
     public void testReadTablesAsGroupIT() throws Exception {
         final Configuration conf = HBaseConfiguration.create();
@@ -142,10 +180,8 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
         
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -153,6 +189,9 @@ public class HBaseRangerAuthorizationTest {
                 Admin admin = conn.getAdmin();
                 
                 HTableDescriptor[] tableDescriptors = admin.listTables();
+                for (HTableDescriptor desc : tableDescriptors) {
+                    LOG.info("Found table:[" + desc.getTableName().getNameAsString() + "]");
+                }
                 Assert.assertEquals(0, tableDescriptors.length);
         
                 conn.close();
@@ -183,10 +222,8 @@ public class HBaseRangerAuthorizationTest {
         conn.close();
         
         // Try to disable + delete the table as the "IT" group
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -240,11 +277,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -270,11 +305,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"public"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -321,11 +354,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -368,11 +399,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -396,11 +425,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"public"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -429,11 +456,9 @@ public class HBaseRangerAuthorizationTest {
         conf.set("hbase.zookeeper.quorum", "localhost");
         conf.set("hbase.zookeeper.property.clientPort", "" + port);
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -490,10 +515,8 @@ public class HBaseRangerAuthorizationTest {
         conn.close();
         
         // Now try to read the row as group "IT" - it should fail as "IT" can only read from table "temp"
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -559,11 +582,9 @@ public class HBaseRangerAuthorizationTest {
         Put put = new Put(Bytes.toBytes("row5"));
         put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("col1"), Bytes.toBytes("val2"));
         table.put(put);
-        
-        String user = "bob";
-        if ("bob".equals(System.getProperty("user.name"))) {
-            user = "alice";
-        }
+
+        String user = getNotCurrentUser();
+
         UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"IT"});
         ugi.doAs(new PrivilegedExceptionAction<Void>() {
             public Void run() throws Exception {
@@ -590,11 +611,118 @@ public class HBaseRangerAuthorizationTest {
         
         conn.close();
     }
-    
+
+    @Test
+    public void testCloneSnapshotAsGroupQA() throws Exception {
+        final Configuration conf = HBaseConfiguration.create();
+        conf.set("hbase.zookeeper.quorum", "localhost");
+        conf.set("hbase.zookeeper.property.clientPort", "" + port);
+        conf.set("zookeeper.znode.parent", "/hbase-unsecure");
+
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Admin admin = conn.getAdmin();
+
+        List<HBaseProtos.SnapshotDescription> snapshots = admin.listSnapshots("test_snapshot");
+        if (CollectionUtils.isNotEmpty(snapshots)) {
+            admin.deleteSnapshot("test_snapshot");
+        }
+        String user = getNotCurrentUser();
+
+        UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[]{"QA"});
+
+        ugi.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                Connection conn = ConnectionFactory.createConnection(conf);
+                Admin admin = conn.getAdmin();
+                Table table = conn.getTable(TableName.valueOf("test_namespace", "temp"));
+                TableName tableName = table.getName();
+
+                admin.disableTable(tableName);
+
+                // Create a snapshot
+                admin.snapshot("test_snapshot", tableName);
+
+                // Clone snapshot
+                HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf("test_namespace", "temp_cloned"));
+                TableName newTableName = tableDescriptor.getTableName();
+                admin.cloneSnapshot("test_snapshot", newTableName);
+                admin.disableTable(newTableName);
+                admin.deleteTable(newTableName);
+
+                admin.enableTable(tableName);
+
+                conn.close();
+                return null;
+            }
+        });
+
+        snapshots = admin.listSnapshots("test_snapshot");
+        if (CollectionUtils.isNotEmpty(snapshots)) {
+            admin.deleteSnapshot("test_snapshot");
+        }
+    }
+    @Test
+    public void testCloneSnapshotAsNonQAGroup() throws Exception {
+        final Configuration conf = HBaseConfiguration.create();
+        conf.set("hbase.zookeeper.quorum", "localhost");
+        conf.set("hbase.zookeeper.property.clientPort", "" + port);
+        conf.set("zookeeper.znode.parent", "/hbase-unsecure");
+
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Admin admin = conn.getAdmin();
+        TableName tableName = conn.getTable(TableName.valueOf("test_namespace", "temp")).getName();
+
+        admin.disableTable(tableName);
+
+        // Create a snapshot
+        List<HBaseProtos.SnapshotDescription> snapshots = admin.listSnapshots("test_snapshot");
+        if (CollectionUtils.isEmpty(snapshots)) {
+            admin.snapshot("test_snapshot", tableName);
+        }
+
+        admin.enableTable(tableName);
+
+        String user = getNotCurrentUser();
+
+        UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] {"public"});
+
+        ugi.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                Connection conn = ConnectionFactory.createConnection(conf);
+                Admin admin = conn.getAdmin();
+
+                try {
+                    TableName clone = TableName.valueOf("test_namespace", "temp_cloned_public");
+                    if (admin.tableExists(clone)) {
+                        // Delete it
+                        admin.deleteTable(clone);
+                    }
+                    // Clone snapshot
+                    admin.cloneSnapshot("test_snapshot", clone);
+                    Assert.fail("Failure expected on an unauthorized group public");
+                } catch(Exception e) {
+                    // Expected
+                }
+                conn.close();
+                return null;
+            }
+        });
+        TableName clone = TableName.valueOf("test_namespace", "temp_cloned_public");
+
+        if (admin.tableExists(clone)) {
+            admin.deleteTable(clone);
+        }
+        admin.deleteSnapshot("test_snapshot");
+    }
+
     private static int getFreePort() throws IOException {
         ServerSocket serverSocket = new ServerSocket(0);
         int port = serverSocket.getLocalPort();
         serverSocket.close();
         return port;
     }
+
+    private static String getNotCurrentUser() {
+        return "not-" + System.getProperty("user.name");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/eaef96be/hbase-agent/src/test/resources/hbase-policies.json
----------------------------------------------------------------------
diff --git a/hbase-agent/src/test/resources/hbase-policies.json b/hbase-agent/src/test/resources/hbase-policies.json
index 30088c6..f113076 100644
--- a/hbase-agent/src/test/resources/hbase-policies.json
+++ b/hbase-agent/src/test/resources/hbase-policies.json
@@ -140,6 +140,61 @@
       "updateTime": "20160622-11:01:02.000-+0100",
       "version": 19
     }
+    ,
+    {
+      "service": "HBASETest",
+      "name": "HBASETest-3-namespace",
+      "description": "Default Policy for Service: HBASETest for namespace test_namespace",
+      "isAuditEnabled": true,
+      "resources": {
+        "column": {
+          "values": [
+            "*"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column-family": {
+          "values": [
+            "*"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "test_namespace:*"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [
+        {
+          "accesses": [
+            {
+              "type": "admin",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+          ],
+          "groups": [
+            "QA"
+          ],
+          "conditions": [],
+          "delegateAdmin": true
+        }
+      ],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [],
+      "id": 38,
+      "isEnabled": true,
+      "version": 1
+    }
   ],
   "serviceDef": {
     "name": "hbase",