You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ab...@apache.org on 2018/09/27 04:47:04 UTC

[2/2] ranger git commit: RANGER-2183: Use INodeAttribute information to authorize HDFS access

RANGER-2183: Use INodeAttribute information to authorize HDFS access


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

Branch: refs/heads/ranger-0.7
Commit: 29801e0e51b752530be7ce36e786111459614387
Parents: 2d35834
Author: Abhay Kulkarni <ak...@hortonworks.com>
Authored: Thu Aug 9 14:08:11 2018 -0700
Committer: Abhay Kulkarni <ak...@hortonworks.com>
Committed: Wed Sep 26 21:46:42 2018 -0700

----------------------------------------------------------------------
 .../hadoop/RangerHdfsAuthorizer.java            | 181 +++++++++++++++----
 1 file changed, 145 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ranger/blob/29801e0e/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java
----------------------------------------------------------------------
diff --git a/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java b/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java
index b37d0ff..22de0e8 100644
--- a/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java
+++ b/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java
@@ -20,6 +20,7 @@
 package org.apache.ranger.authorization.hadoop;
 
 import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.EXECUTE_ACCCESS_TYPE;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.HDFS_ROOT_FOLDER_PATH;
 import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.READ_ACCCESS_TYPE;
 import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.WRITE_ACCCESS_TYPE;
 
@@ -37,6 +38,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
@@ -196,6 +198,16 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 		}
 
+		class SubAccessData {
+			final INodeDirectory    dir;
+			final String            resourcePath;
+
+			SubAccessData(INodeDirectory dir, String resourcePath) {
+				this.dir            = dir;
+				this.resourcePath   = resourcePath;
+			}
+		}
+
 		@Override
 		public void checkPermission(String fsOwner, String superGroup, UserGroupInformation ugi,
 									INodeAttributes[] inodeAttrs, INode[] inodes, byte[][] pathByNameArr,
@@ -207,11 +219,12 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			RangerHdfsAuditHandler auditHandler  = null;
 			String                 user          = ugi != null ? ugi.getShortUserName() : null;
 			Set<String>            groups        = ugi != null ? Sets.newHashSet(ugi.getGroupNames()) : null;
+			String                 resourcePath  = path;
 
 			if(LOG.isDebugEnabled()) {
 				LOG.debug("==> RangerAccessControlEnforcer.checkPermission("
 						+ "fsOwner=" + fsOwner + "; superGroup=" + superGroup + ", inodesCount=" + (inodes != null ? inodes.length : 0)
-						+ ", snapshotId=" + snapshotId + ", user=" + user + ", path=" + path + ", ancestorIndex=" + ancestorIndex
+						+ ", snapshotId=" + snapshotId + ", user=" + user + ", provided-path=" + path + ", ancestorIndex=" + ancestorIndex
 						+ ", doCheckOwner="+ doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
 						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir + ")");
 			}
@@ -219,7 +232,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			RangerPerfTracer perf = null;
 
 			if(RangerPerfTracer.isPerfTraceEnabled(PERF_HDFSAUTH_REQUEST_LOG)) {
-				perf = RangerPerfTracer.getPerfTracer(PERF_HDFSAUTH_REQUEST_LOG, "RangerHdfsAuthorizer.checkPermission(path=" + path + ")");
+				perf = RangerPerfTracer.getPerfTracer(PERF_HDFSAUTH_REQUEST_LOG, "RangerHdfsAuthorizer.checkPermission(provided-path=" + path + ")");
 			}
 
 			try {
@@ -229,6 +242,29 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				INode   inode               = null;
 
 				if(plugin != null && !ArrayUtils.isEmpty(inodes)) {
+					int sz = inodeAttrs.length;
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Size of INodeAttrs array:[" + sz + "]");
+					}
+					byte[][] components = new byte[sz][];
+
+					int i = 0;
+					for (; i < sz; i++) {
+						if (inodeAttrs[i] != null) {
+							components[i] = inodeAttrs[i].getLocalNameBytes();
+						} else {
+							break;
+						}
+					}
+					if (i != sz) {
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Input INodeAttributes array contains null at position " + i);
+							LOG.debug("Will use only first [" + i + "] components to build resourcePath");
+						}
+					}
+
+					resourcePath = DFSUtil.byteArray2PathString(components, 0, i);
+
 					if(ancestorIndex >= inodes.length) {
 						ancestorIndex = inodes.length - 1;
 					}
@@ -241,25 +277,10 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					parent   = inodes.length > 1 ? inodes[inodes.length - 2] : null;
 					inode    = inodes[inodes.length - 1]; // could be null while creating a new file
 
-					auditHandler = new RangerHdfsAuditHandler(path, isTraverseOnlyCheck);
+					auditHandler = new RangerHdfsAuditHandler(resourcePath, isTraverseOnlyCheck);
 
 					if(isTraverseOnlyCheck) {
-						INode           nodeToCheck = inode;
-						INodeAttributes nodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
-
-						if(nodeToCheck == null || nodeToCheck.isFile()) {
-							if(parent != null) {
-								nodeToCheck = parent;
-								nodeAttribs = inodeAttrs.length > 1 ? inodeAttrs[inodeAttrs.length - 2] : null;
-							} else if(ancestor != null) {
-								nodeToCheck = ancestor;
-								nodeAttribs = inodeAttrs.length > ancestorIndex ? inodeAttrs[ancestorIndex] : null;
-							}
-						}
-
-						if(nodeToCheck != null) {
-							authzStatus = isAccessAllowed(nodeToCheck, nodeAttribs, FsAction.EXECUTE, user, groups, plugin, auditHandler);
-						}
+						authzStatus = traverseOnlyCheck(inode, inodeAttrs, resourcePath, components, parent, ancestor, ancestorIndex, user, groups, plugin, auditHandler);
 					}
 
 					// checkStickyBit
@@ -273,8 +294,9 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					// checkAncestorAccess
 					if(authzStatus == AuthzStatus.ALLOW && ancestorAccess != null && ancestor != null) {
 						INodeAttributes ancestorAttribs = inodeAttrs.length > ancestorIndex ? inodeAttrs[ancestorIndex] : null;
+						String ancestorPath = ancestorAttribs != null ? DFSUtil.byteArray2PathString(components, 0, ancestorIndex + 1) : null;
 
-						authzStatus = isAccessAllowed(ancestor, ancestorAttribs, ancestorAccess, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(ancestor, ancestorAttribs, ancestorPath, ancestorAccess, user, groups, plugin, auditHandler);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
@@ -286,8 +308,9 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					// checkParentAccess
 					if(authzStatus == AuthzStatus.ALLOW && parentAccess != null && parent != null) {
 						INodeAttributes parentAttribs = inodeAttrs.length > 1 ? inodeAttrs[inodeAttrs.length - 2] : null;
+						String parentPath = parentAttribs != null ? DFSUtil.byteArray2PathString(components, 0, inodeAttrs.length - 1) : null;
 
-						authzStatus = isAccessAllowed(parent, parentAttribs, parentAccess, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(parent, parentAttribs, parentPath, parentAccess, user, groups, plugin, auditHandler);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
@@ -300,7 +323,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					if(authzStatus == AuthzStatus.ALLOW && access != null && inode != null) {
 						INodeAttributes inodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
 
-						authzStatus = isAccessAllowed(inode, inodeAttribs, access, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(inode, inodeAttribs, resourcePath, access, user, groups, plugin, auditHandler);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
@@ -311,16 +334,16 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 
 					// checkSubAccess
 					if(authzStatus == AuthzStatus.ALLOW && subAccess != null && inode != null && inode.isDirectory()) {
-						Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
+						Stack<SubAccessData> directories = new Stack<>();
 
-						for(directories.push(inode.asDirectory()); !directories.isEmpty(); ) {
-							INodeDirectory      dir   = directories.pop();
-							ReadOnlyList<INode> cList = dir.getChildrenList(snapshotId);
+						for(directories.push(new SubAccessData(inode.asDirectory(), resourcePath)); !directories.isEmpty(); ) {
+							SubAccessData data = directories.pop();
+							ReadOnlyList<INode> cList = data.dir.getChildrenList(snapshotId);
 
 							if (!(cList.isEmpty() && ignoreEmptyDir)) {
-								INodeAttributes dirAttribs = dir.getSnapshotINode(snapshotId);
+								INodeAttributes dirAttribs = data.dir.getSnapshotINode(snapshotId);
 
-								authzStatus = isAccessAllowed(dir, dirAttribs, subAccess, user, groups, plugin, auditHandler);
+								authzStatus = isAccessAllowed(data.dir, dirAttribs, data.resourcePath, subAccess, user, groups, plugin, auditHandler);
 
 								if(authzStatus != AuthzStatus.ALLOW) {
 									break;
@@ -331,13 +354,13 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 								boolean optimizeSubAccessAuthEnabled = RangerHdfsPlugin.isOptimizeSubAccessAuthEnabled();
 
 								if (optimizeSubAccessAuthEnabled) {
-									subDirAuthStatus = isAccessAllowedForHierarchy(dir, dirAttribs, subAccess, user, groups, plugin);
+									subDirAuthStatus = isAccessAllowedForHierarchy(data.dir, dirAttribs, data.resourcePath, subAccess, user, groups, plugin);
 								}
 
 								if (subDirAuthStatus != AuthzStatus.ALLOW) {
 									for(INode child : cList) {
 										if (child.isDirectory()) {
-											directories.push(child.asDirectory());
+											directories.push(new SubAccessData(child.asDirectory(), resourcePath + org.apache.hadoop.fs.Path.SEPARATOR_CHAR + child.getLocalName()));
 										}
 									}
 								}
@@ -382,7 +405,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						}
 					}
 
-					throw new RangerAccessControlException("Permission denied: user=" + user + ", access=" + action + ", inode=\"" + path + "\"");
+					throw new RangerAccessControlException("Permission denied: user=" + user + ", access=" + action + ", inode=\"" + resourcePath + "\"");
 				}
 			} finally {
 				if(auditHandler != null) {
@@ -392,9 +415,97 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				RangerPerfTracer.log(perf);
 
 				if(LOG.isDebugEnabled()) {
-					LOG.debug("<== RangerAccessControlEnforcer.checkPermission(" + path + ", " + access + ", user=" + user + ") : " + authzStatus);
+					LOG.debug("<== RangerAccessControlEnforcer.checkPermission(" + resourcePath + ", " + access + ", user=" + user + ") : " + authzStatus);
+				}
+			}
+		}
+
+		/*
+		    Check if parent or ancestor of the file being accessed is denied EXECUTE permission. If not, assume that Ranger-acls
+		    allowed EXECUTE access. Do not audit this authorization check if resource is a file unless access is explicitly denied
+		 */
+		private AuthzStatus traverseOnlyCheck(INode inode, INodeAttributes[] inodeAttrs, String path, byte[][] components, INode parent, INode ancestor, int ancestorIndex,
+											  String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler) {
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("==> RangerAccessControlEnforcer.traverseOnlyCheck("
+						+ "path=" + path + ", user=" + user + ", groups=" + groups + ")");
+			}
+			final AuthzStatus ret;
+
+			INode nodeToCheck = inode;
+			INodeAttributes nodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
+			boolean skipAuditOnAllow = false;
+
+			String resourcePath = path;
+			if (nodeToCheck == null || nodeToCheck.isFile()) {
+				skipAuditOnAllow = true;
+				if (parent != null) {
+					nodeToCheck = parent;
+					nodeAttribs = inodeAttrs.length > 1 ? inodeAttrs[inodeAttrs.length - 2] : null;
+					resourcePath = inodeAttrs.length > 0 ? DFSUtil.byteArray2PathString(components, 0, inodeAttrs.length - 1) : HDFS_ROOT_FOLDER_PATH;
+				} else if (ancestor != null) {
+					nodeToCheck = ancestor;
+					nodeAttribs = inodeAttrs.length > ancestorIndex ? inodeAttrs[ancestorIndex] : null;
+					resourcePath = nodeAttribs != null ? DFSUtil.byteArray2PathString(components, 0, ancestorIndex+1) : HDFS_ROOT_FOLDER_PATH;
+				}
+			}
+
+			if (nodeToCheck != null) {
+				if (resourcePath.length() > 1) {
+					if (resourcePath.endsWith(HDFS_ROOT_FOLDER_PATH)) {
+						resourcePath = resourcePath.substring(0, resourcePath.length()-1);
+					}
 				}
+				ret = isAccessAllowedForTraversal(nodeToCheck, nodeAttribs, resourcePath, user, groups, plugin, auditHandler, skipAuditOnAllow);
+			} else {
+				ret = AuthzStatus.ALLOW;
 			}
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("<== RangerAccessControlEnforcer.traverseOnlyCheck("
+						+ "path=" + path + ", resourcePath=" + resourcePath + ", user=" + user + ", groups=" + groups + ") : " + ret);
+			}
+			return ret;
+		}
+
+		private AuthzStatus isAccessAllowedForTraversal(INode inode, INodeAttributes inodeAttribs, String path, String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler, boolean skipAuditOnAllow) {
+			final AuthzStatus ret;
+			String pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
+			String clusterName = plugin.getClusterName();
+			FsAction access = FsAction.EXECUTE;
+
+
+			if (pathOwner == null) {
+				pathOwner = inode.getUserName();
+			}
+
+			if (RangerHadoopConstants.HDFS_ROOT_FOLDER_PATH_ALT.equals(path)) {
+				path = RangerHadoopConstants.HDFS_ROOT_FOLDER_PATH;
+			}
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + user + ", " + skipAuditOnAllow + ")");
+			}
+
+			RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, EXECUTE_ACCCESS_TYPE, user, groups, clusterName);
+
+			RangerAccessResult result = plugin.isAccessAllowed(request, null);
+
+			if (result != null && result.getIsAccessDetermined() && !result.getIsAllowed()) {
+				ret = AuthzStatus.DENY;
+			} else {
+				ret = AuthzStatus.ALLOW;
+			}
+
+			if (ret == AuthzStatus.DENY || (!skipAuditOnAllow && result != null && result.getIsAccessDetermined())) {
+				auditHandler.processResult(result);
+			}
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + user + ", " + skipAuditOnAllow + "): " + ret);
+			}
+
+			return ret;
 		}
 
 		private AuthzStatus checkDefaultEnforcer(String fsOwner, String superGroup, UserGroupInformation ugi,
@@ -476,9 +587,8 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			return authzStatus;
 		}
 
-		private AuthzStatus isAccessAllowed(INode inode, INodeAttributes inodeAttribs, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler) {
+		private AuthzStatus isAccessAllowed(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler) {
 			AuthzStatus ret       = null;
-			String      path      = inode != null ? inode.getFullPathName() : null;
 			String      pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 			String 		clusterName = plugin.getClusterName();
 
@@ -531,9 +641,8 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			return ret;
 		}
 
-		private AuthzStatus isAccessAllowedForHierarchy(INode inode, INodeAttributes inodeAttribs, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin) {
+		private AuthzStatus isAccessAllowedForHierarchy(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin) {
 			AuthzStatus ret   = null;
-			String  path      = inode != null ? inode.getFullPathName() : null;
 			String  pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 			String 		clusterName = plugin.getClusterName();