You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by rm...@apache.org on 2018/10/10 17:51:37 UTC

ranger git commit: RANGER-1958 [HBase] Implement getUserPermissions API of AccessControlService.Interface to allow clients to access HBase permissions stored in Ranger (Ankit Singhal)

Repository: ranger
Updated Branches:
  refs/heads/master 234fb4823 -> 1869f2a76


RANGER-1958 [HBase] Implement getUserPermissions API of AccessControlService.Interface to allow clients to access HBase permissions stored in Ranger (Ankit Singhal)

Signed-off-by: rmani <rm...@hortonworks.com>


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

Branch: refs/heads/master
Commit: 1869f2a761405aa62bfc8533a38255f69d95301e
Parents: 234fb48
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Oct 4 16:24:06 2018 -0700
Committer: rmani <rm...@hortonworks.com>
Committed: Wed Oct 10 10:51:19 2018 -0700

----------------------------------------------------------------------
 .../hbase/AuthorizationSession.java             |   3 +-
 .../hbase/RangerAuthorizationCoprocessor.java   | 122 ++++++++++++++++++-
 .../hbase/HBaseRangerAuthorizationTest.java     |  71 ++++++++++-
 .../src/test/resources/hbase-policies.json      |  58 +++++++++
 4 files changed, 250 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ranger/blob/1869f2a7/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/AuthorizationSession.java
----------------------------------------------------------------------
diff --git a/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/AuthorizationSession.java b/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/AuthorizationSession.java
index cdaad00..74293fb 100644
--- a/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/AuthorizationSession.java
+++ b/hbase-agent/src/main/java/org/apache/ranger/authorization/hbase/AuthorizationSession.java
@@ -174,7 +174,8 @@ public class AuthorizationSession {
 				StringUtils.equals(_operation, "deleteNamespace") ||
 				StringUtils.equals(_operation, "modifyNamespace") ||
 				StringUtils.equals(_operation, "setUserNamespaceQuota") ||
-				StringUtils.equals(_operation, "setNamespaceQuota");
+				StringUtils.equals(_operation, "setNamespaceQuota") ||
+				StringUtils.equals(_operation, "getUserPermissionForNamespace");
 	}
 
 	AuthorizationSession buildRequest() {

http://git-wip-us.apache.org/repos/asf/ranger/blob/1869f2a7/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 d85339a..ddb6d9b 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
@@ -19,6 +19,7 @@
 package org.apache.ranger.authorization.hbase;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -29,6 +30,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.Set;
 
@@ -36,6 +38,7 @@ 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.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -83,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.security.access.RangerAccessControlLists;
@@ -97,13 +101,19 @@ import org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants;
 import org.apache.ranger.authorization.utils.StringUtil;
 import org.apache.ranger.plugin.audit.RangerDefaultAuditHandler;
 import org.apache.ranger.plugin.policyengine.RangerAccessRequest;
+import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl;
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl;
 import org.apache.ranger.plugin.policyengine.RangerAccessResultProcessor;
+import org.apache.ranger.plugin.policyengine.RangerResourceACLs;
+import org.apache.ranger.plugin.policyengine.RangerResourceACLs.AccessResult;
+import org.apache.ranger.plugin.policyevaluator.RangerPolicyEvaluator;
 import org.apache.ranger.plugin.service.RangerBasePlugin;
 import org.apache.ranger.plugin.util.GrantRevokeRequest;
 
 import com.google.common.base.Objects;
 import com.google.common.collect.Lists;
 import com.google.common.collect.MapMaker;
+import com.google.common.collect.Sets;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -1272,8 +1282,116 @@ public class RangerAuthorizationCoprocessor extends RangerAuthorizationCoprocess
 	}
 
 	@Override
-	public void getUserPermissions(RpcController controller, AccessControlProtos.GetUserPermissionsRequest request, RpcCallback<AccessControlProtos.GetUserPermissionsResponse> done) {
-		LOG.debug("getUserPermissions(): ");
+	public void getUserPermissions(RpcController controller, AccessControlProtos.GetUserPermissionsRequest request,
+			RpcCallback<AccessControlProtos.GetUserPermissionsResponse> done) {
+		AccessControlProtos.GetUserPermissionsResponse response = null;
+		try {
+			String operation = "userPermissions";
+			final RangerAccessResourceImpl resource = new RangerAccessResourceImpl();
+			User user = getActiveUser();
+			Set<String> groups = _userUtils.getUserGroups(user);
+			if (groups.isEmpty() && user.getUGI() != null) {
+				String[] groupArray = user.getUGI().getGroupNames();
+				if (groupArray != null) {
+					groups = Sets.newHashSet(groupArray);
+				}
+			}
+			RangerAccessRequestImpl rangerAccessrequest = new RangerAccessRequestImpl(resource, null,
+					_userUtils.getUserAsString(user), groups);
+			rangerAccessrequest.setAction(operation);
+			rangerAccessrequest.setClientIPAddress(getRemoteAddress());
+			rangerAccessrequest.setResourceMatchingScope(RangerAccessRequest.ResourceMatchingScope.SELF);
+			rangerAccessrequest.setClusterName(hbasePlugin.getClusterName());
+			List<UserPermission> perms = null;
+			if (request.getType() == AccessControlProtos.Permission.Type.Table) {
+				final TableName table = request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName())
+						: null;
+				requirePermission(operation, table.getName(), Action.ADMIN);
+				resource.setValue(RangerHBaseResource.KEY_TABLE, table.getNameAsString());
+				perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+					@Override
+					public List<UserPermission> run() throws Exception {
+						return getUserPrermissions(
+								hbasePlugin.getCurrentRangerAuthContext().getResourceACLs(rangerAccessrequest),
+								table.getNameAsString(), false);
+					}
+				});
+			} else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
+				final String namespace = request.getNamespaceName().toStringUtf8();
+				requireGlobalPermission("getUserPermissionForNamespace", namespace, Action.ADMIN);
+				resource.setValue(RangerHBaseResource.KEY_TABLE, namespace + RangerHBaseResource.NAMESPACE_SEPARATOR);
+				rangerAccessrequest.setRequestData(namespace);
+				perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+					@Override
+					public List<UserPermission> run() throws Exception {
+						return getUserPrermissions(
+								hbasePlugin.getCurrentRangerAuthContext().getResourceACLs(rangerAccessrequest),
+								namespace, true);
+					}
+				});
+			} else {
+				requirePermission("userPermissions", Action.ADMIN);
+				perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+					@Override
+					public List<UserPermission> run() throws Exception {
+						return getUserPrermissions(
+								hbasePlugin.getCurrentRangerAuthContext().getResourceACLs(rangerAccessrequest), null,
+								false);
+					}
+				});
+				if (_userUtils.isSuperUser(user)) {
+					perms.add(new UserPermission(Bytes.toBytes(_userUtils.getUserAsString(user)),
+							AccessControlLists.ACL_TABLE_NAME, null, Action.values()));
+				}
+			}
+			response = ResponseConverter.buildGetUserPermissionsResponse(perms);
+		} catch (IOException ioe) {
+			// pass exception back up
+			ResponseConverter.setControllerException(controller, ioe);
+		}
+		done.run(response);
+	}
+
+	private List<UserPermission> getUserPrermissions(RangerResourceACLs rangerResourceACLs, String resource,
+			boolean isNamespace) {
+		List<UserPermission> userPermissions = new ArrayList<UserPermission>();
+		Action[] hbaseActions = Action.values();
+		List<String> hbaseActionsList = new ArrayList<String>();
+		for (Action action : hbaseActions) {
+			hbaseActionsList.add(action.name());
+		}
+		addPermission(rangerResourceACLs.getUserACLs(), isNamespace, hbaseActionsList, userPermissions, resource,
+				false);
+		addPermission(rangerResourceACLs.getGroupACLs(), isNamespace, hbaseActionsList, userPermissions, resource,
+				true);
+		return userPermissions;
+	}
+
+	private void addPermission(Map<String, Map<String, AccessResult>> acls, boolean isNamespace,
+			List<String> hbaseActionsList, List<UserPermission> userPermissions, String resource, boolean isGroup) {
+		for (Entry<String, Map<String, AccessResult>> userAcls : acls.entrySet()) {
+			String user = !isGroup ? userAcls.getKey() : AuthUtil.toGroupEntry(userAcls.getKey());
+			List<Action> allowedPermissions = new ArrayList<Action>();
+			for (Entry<String, AccessResult> permissionAccess : userAcls.getValue().entrySet()) {
+				String permission = permissionAccess.getKey().toUpperCase();
+				if (hbaseActionsList.contains(permission)
+						&& permissionAccess.getValue().getResult() == RangerPolicyEvaluator.ACCESS_ALLOWED) {
+					allowedPermissions.add(Action.valueOf(permission));
+				}
+
+			}
+			if (!allowedPermissions.isEmpty()) {
+				UserPermission up = null;
+				if (isNamespace) {
+					up = new UserPermission(Bytes.toBytes(user), resource,
+							allowedPermissions.toArray(new Action[allowedPermissions.size()]));
+				} else {
+					up = new UserPermission(Bytes.toBytes(user), TableName.valueOf(resource), null, null,
+							allowedPermissions.toArray(new Action[allowedPermissions.size()]));
+				}
+				userPermissions.add(up);
+			}
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/ranger/blob/1869f2a7/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 3840885..f1cd893 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
@@ -42,6 +42,9 @@ 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.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
@@ -116,6 +119,15 @@ public class HBaseRangerAuthorizationTest {
             admin.createTable(tableDescriptor);
         }
 
+		if (!admin.tableExists(TableName.valueOf("default:temp5"))) {
+			HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf("default:temp5"));
+
+			// Adding column families to table descriptor
+			tableDescriptor.addFamily(new HColumnDescriptor("colfam1"));
+
+			admin.createTable(tableDescriptor);
+		}
+
         // Add a new row
         Put put = new Put(Bytes.toBytes("row1"));
         put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("col1"), Bytes.toBytes("val1"));
@@ -174,7 +186,7 @@ public class HBaseRangerAuthorizationTest {
         for (HTableDescriptor desc : tableDescriptors) {
             LOG.info("Found table:[" + desc.getTableName().getNameAsString() + "]");
         }
-        Assert.assertEquals(2, tableDescriptors.length);
+        Assert.assertEquals(3, tableDescriptors.length);
 
         conn.close();
     }
@@ -961,6 +973,63 @@ public class HBaseRangerAuthorizationTest {
         conn.close();
     }
 
+	@Test
+	public void testGetUserPermission() throws Throwable {
+		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");
+		String user = "IT";
+		UserGroupInformation ugi = UserGroupInformation.createUserForTesting(user, new String[] { "IT" });
+		ugi.doAs(new PrivilegedExceptionAction<Void>() {
+			public Void run() throws Exception {
+				try (Connection conn = ConnectionFactory.createConnection(conf)) {
+					AccessControlClient.getUserPermissions(conn, "temp");
+					Assert.fail();
+				} catch (Throwable e) {
+					// expected
+				}
+				return null;
+			}
+
+		});
+
+		user = "QA";
+		ugi = UserGroupInformation.createUserForTesting(user, new String[] { "QA" });
+		ugi.doAs(new PrivilegedExceptionAction<Void>() {
+			public Void run() throws Exception {
+				List<UserPermission> userPermissions;
+				try (Connection conn = ConnectionFactory.createConnection(conf)) {
+					userPermissions = AccessControlClient.getUserPermissions(conn, "@test_namespace");
+				} catch (Throwable e) {
+					throw new Exception(e);
+				}
+				boolean found = false;
+				for (UserPermission namespacePermission : userPermissions) {
+					if (namespacePermission.hasNamespace()) {
+						found = Bytes.equals(namespacePermission.getUser(), Bytes.toBytes("@QA"));
+						if (found) {
+							break;
+						}
+					}
+				}
+				Assert.assertTrue("QA is not found", found);
+				return null;
+			}
+		});
+
+		List<UserPermission> userPermissions;
+		try (Connection conn = ConnectionFactory.createConnection(conf)) {
+			userPermissions = AccessControlClient.getUserPermissions(conn, "temp5");
+		} catch (Throwable e) {
+			throw new Exception(e);
+		}
+		UserPermission userPermission = new UserPermission(Bytes.toBytes("@IT"), TableName.valueOf("temp5"), null,
+				Permission.Action.READ, Permission.Action.WRITE);
+		Assert.assertTrue("@IT permission should be there", userPermissions.contains(userPermission));
+
+	}
+
     private static int getFreePort() throws IOException {
         ServerSocket serverSocket = new ServerSocket(0);
         int port = serverSocket.getLocalPort();

http://git-wip-us.apache.org/repos/asf/ranger/blob/1869f2a7/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 b7b44c9..6213a0e 100644
--- a/hbase-agent/src/test/resources/hbase-policies.json
+++ b/hbase-agent/src/test/resources/hbase-policies.json
@@ -132,6 +132,64 @@
     },
     {
       "service": "cl1_hbase",
+      "name": "TempPolicy",
+      "policyType": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "column-family": {
+          "values": [
+            "*"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "*"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "temp5"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [
+        {
+          "accesses": [
+            {
+              "type": "read",
+              "isAllowed": true
+            },
+            {
+              "type": "write",
+              "isAllowed": true
+            }
+          ],
+          "users": [],
+          "groups": [
+            "IT"
+          ],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [],
+      "id": 33,
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "cl1_hbase",
       "name": "HBASETest-3-namespace",
       "description": "Default Policy for Service: HBASETest for namespace test_namespace",
       "isAuditEnabled": true,