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 2020/12/03 22:32:53 UTC

[ranger] branch master updated: RANGER-3097: updated HDFS plugin to make operationName available to policy engine

This is an automated email from the ASF dual-hosted git repository.

madhan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ranger.git


The following commit(s) were added to refs/heads/master by this push:
     new bebbd6b  RANGER-3097: updated HDFS plugin to make operationName available to policy engine
bebbd6b is described below

commit bebbd6ba7a2cadbaf306e6c8c2d678cb942c1176
Author: Madhan Neethiraj <ma...@apache.org>
AuthorDate: Tue Dec 1 13:23:05 2020 -0800

    RANGER-3097: updated HDFS plugin to make operationName available to policy engine
---
 .../RangerAuditPolicyEvaluator.java                |   2 +-
 .../authorization/hadoop/RangerHdfsAuthorizer.java | 199 +++++++++++++--------
 2 files changed, 122 insertions(+), 79 deletions(-)

diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerAuditPolicyEvaluator.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerAuditPolicyEvaluator.java
index 81efa71..108856f 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerAuditPolicyEvaluator.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerAuditPolicyEvaluator.java
@@ -111,7 +111,7 @@ public class RangerAuditPolicyEvaluator extends RangerDefaultPolicyEvaluator {
 
         Map<String, Collection<String>> impliedAccessGrants = getImpliedAccessGrants(serviceDef);
 
-        if(impliedAccessGrants == null || impliedAccessGrants.isEmpty()) {
+        if (impliedAccessGrants == null || impliedAccessGrants.isEmpty()) {
             return;
         }
 
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 347abe7..63e8472 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
@@ -251,21 +251,17 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 									int snapshotId, String path, int ancestorIndex, boolean doCheckOwner,
 									FsAction ancestorAccess, FsAction parentAccess, FsAction access,
 									FsAction subAccess, boolean ignoreEmptyDir, String operationName, CallerContext callerContext ) throws AccessControlException {
-			AuthzStatus            authzStatus = AuthzStatus.NOT_DETERMINED;
-			RangerHdfsPlugin       plugin        = rangerPlugin;
-			RangerHdfsAuditHandler auditHandler  = null;
-			String                 user          = ugi != null ? ugi.getShortUserName() : null;
-			Set<String>            groups        = ugi != null ? Sets.newHashSet(ugi.getGroupNames()) : null;
-			String                 resourcePath  = path;
-			String                 callerctxt    = callerContext != null ? callerContext.toString() : null;
+			AuthzStatus  authzStatus  = AuthzStatus.NOT_DETERMINED;
+			String       resourcePath = path;
+			AuthzContext context      = new AuthzContext(rangerPlugin, ugi, operationName, access == null && parentAccess == null && ancestorAccess == null && subAccess == null);
 
 			if(LOG.isDebugEnabled()) {
 				LOG.debug("==> RangerAccessControlEnforcer.checkPermission("
 						+ "fsOwner=" + fsOwner + "; superGroup=" + superGroup + ", inodesCount=" + (inodes != null ? inodes.length : 0)
-						+ ", snapshotId=" + snapshotId + ", user=" + user + ", provided-path=" + path + ", ancestorIndex=" + ancestorIndex
+						+ ", snapshotId=" + snapshotId + ", user=" + context.user + ", provided-path=" + path + ", ancestorIndex=" + ancestorIndex
 						+ ", doCheckOwner="+ doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
 						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir + ", operationName=" + operationName
-						+ ", callerContext=" + callerctxt +")");
+						+ ", callerContext=" + callerContext +")");
 			}
 
 			RangerPerfTracer perf = null;
@@ -275,7 +271,6 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			try {
-				final boolean isTraverseOnlyCheck = access == null && parentAccess == null && ancestorAccess == null && subAccess == null;
 				INode   ancestor            = null;
 				INode   parent              = null;
 				INode   inode               = null;
@@ -284,7 +279,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				boolean useDefaultAuthorizerOnly = false;
 				boolean doNotGenerateAuditRecord = false;
 
-				if(plugin != null && !ArrayUtils.isEmpty(inodes)) {
+				if (context.plugin != null && !ArrayUtils.isEmpty(inodes)) {
 					int sz = inodeAttrs.length;
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("Size of INodeAttrs array:[" + sz + "]");
@@ -350,7 +345,7 @@ 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 = doNotGenerateAuditRecord ? null : new RangerHdfsAuditHandler(providedPath, isTraverseOnlyCheck, rangerPlugin.getHadoopModuleName(), rangerPlugin.getExcludedUsers(), operationName, callerctxt);
+					context.auditHandler = doNotGenerateAuditRecord ? null : new RangerHdfsAuditHandler(providedPath, context.isTraverseOnlyCheck, context.plugin.getHadoopModuleName(), context.plugin.getExcludedUsers(), callerContext != null ? callerContext.toString() : null);
 
 					/* Hadoop versions prior to 2.8.0 didn't ask for authorization of parent/ancestor traversal for
 					 * reading or writing a file. However, Hadoop version 2.8.0 and later ask traversal authorization for
@@ -367,15 +362,15 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					 * This approach would ensure that Ranger authorization will continue to work with existing policies,
 					 * without requiring policy migration/update, for the changes in behaviour in Hadoop 2.8.0.
 					 */
-					if(authzStatus == AuthzStatus.ALLOW && isTraverseOnlyCheck) {
-						authzStatus = traverseOnlyCheck(inode, inodeAttrs, resourcePath, components, parent, ancestor, ancestorIndex, user, groups, plugin, auditHandler);
+					if (authzStatus == AuthzStatus.ALLOW && context.isTraverseOnlyCheck) {
+						authzStatus = traverseOnlyCheck(inode, inodeAttrs, resourcePath, components, parent, ancestor, ancestorIndex, context);
 					}
 
 					// checkStickyBit
 					if (authzStatus == AuthzStatus.ALLOW && parentAccess != null && parentAccess.implies(FsAction.WRITE) && parent != null && inode != null) {
 						if (parent.getFsPermission() != null && parent.getFsPermission().getStickyBit()) {
 						    // user should be owner of the parent or the inode
-						    authzStatus = (StringUtils.equals(parent.getUserName(), user) || StringUtils.equals(inode.getUserName(), user)) ? AuthzStatus.ALLOW : AuthzStatus.NOT_DETERMINED;
+						    authzStatus = (StringUtils.equals(parent.getUserName(), context.user) || StringUtils.equals(inode.getUserName(), context.user)) ? AuthzStatus.ALLOW : AuthzStatus.NOT_DETERMINED;
 						}
 					}
 
@@ -384,12 +379,12 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						INodeAttributes ancestorAttribs = inodeAttrs.length > ancestorIndex ? inodeAttrs[ancestorIndex] : null;
 						String ancestorPath = ancestorAttribs != null ? DFSUtil.byteArray2PathString(components, 0, ancestorIndex + 1) : null;
 
-						authzStatus = isAccessAllowed(ancestor, ancestorAttribs, ancestorPath, ancestorAccess, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(ancestor, ancestorAttribs, ancestorPath, ancestorAccess, context);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
 											ancestorAccess, null, null, null, ignoreEmptyDir,
-											isTraverseOnlyCheck, ancestor, parent, inode, auditHandler);
+											ancestor, parent, inode, context);
 						}
 					}
 
@@ -398,12 +393,12 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						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, parentPath, parentAccess, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(parent, parentAttribs, parentPath, parentAccess, context);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
 											null, parentAccess, null, null, ignoreEmptyDir,
-											isTraverseOnlyCheck, ancestor, parent, inode, auditHandler);
+											ancestor, parent, inode, context);
 						}
 					}
 
@@ -411,12 +406,12 @@ 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, resourcePath, access, user, groups, plugin, auditHandler);
+						authzStatus = isAccessAllowed(inode, inodeAttribs, resourcePath, access, context);
 						if (authzStatus == AuthzStatus.NOT_DETERMINED) {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
 											null, null, access, null, ignoreEmptyDir,
-											isTraverseOnlyCheck, ancestor, parent, inode, auditHandler);
+											ancestor, parent, inode, context);
 						}
 					}
 
@@ -431,7 +426,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 							if (!(cList.isEmpty() && ignoreEmptyDir)) {
 								INodeAttributes dirAttribs = data.dir.getSnapshotINode(snapshotId);
 
-								authzStatus = isAccessAllowed(data.dir, dirAttribs, data.resourcePath, subAccess, user, groups, plugin, auditHandler);
+								authzStatus = isAccessAllowed(data.dir, dirAttribs, data.resourcePath, subAccess, context);
 
 								if(authzStatus != AuthzStatus.ALLOW) {
 									break;
@@ -442,7 +437,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 								boolean optimizeSubAccessAuthEnabled = rangerPlugin.isOptimizeSubAccessAuthEnabled();
 
 								if (optimizeSubAccessAuthEnabled) {
-									subDirAuthStatus = isAccessAllowedForHierarchy(data.dir, dirAttribs, data.resourcePath, subAccess, user, groups, plugin);
+									subDirAuthStatus = isAccessAllowedForHierarchy(data.dir, dirAttribs, data.resourcePath, subAccess, context);
 								}
 
 								if (subDirAuthStatus != AuthzStatus.ALLOW) {
@@ -459,7 +454,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 							authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 											pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
 											null, null, null, subAccess, ignoreEmptyDir,
-											isTraverseOnlyCheck, ancestor, parent, inode, auditHandler);
+											ancestor, parent, inode, context);
 
 						}
 					}
@@ -469,7 +464,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						INodeAttributes inodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
 						String          owner        = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 
-						authzStatus = StringUtils.equals(user, owner) ? AuthzStatus.ALLOW : AuthzStatus.NOT_DETERMINED;
+						authzStatus = StringUtils.equals(context.user, owner) ? AuthzStatus.ALLOW : AuthzStatus.NOT_DETERMINED;
 					}
 				}
 
@@ -477,7 +472,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					authzStatus = checkDefaultEnforcer(fsOwner, superGroup, ugi, inodeAttrs, inodes,
 									pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
 									ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir,
-									isTraverseOnlyCheck, ancestor, parent, inode, auditHandler);
+									ancestor, parent, inode, context);
 				}
 
 				if(authzStatus != AuthzStatus.ALLOW) {
@@ -493,17 +488,17 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						}
 					}
 
-					throw new RangerAccessControlException("Permission denied: user=" + user + ", access=" + action + ", inode=\"" + resourcePath + "\"");
+					throw new RangerAccessControlException("Permission denied: user=" + context.user + ", access=" + action + ", inode=\"" + resourcePath + "\"");
 				}
 			} finally {
-				if(auditHandler != null) {
-					auditHandler.flushAudit();
+				if (context.auditHandler != null) {
+					context.auditHandler.flushAudit();
 				}
 
 				RangerPerfTracer.log(perf);
 
 				if(LOG.isDebugEnabled()) {
-					LOG.debug("<== RangerAccessControlEnforcer.checkPermission(" + resourcePath + ", " + access + ", user=" + user + ") : " + authzStatus);
+					LOG.debug("<== RangerAccessControlEnforcer.checkPermission(" + resourcePath + ", " + access + ", user=" + context.user + ") : " + authzStatus);
 				}
 			}
 		}
@@ -513,11 +508,11 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 		    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) {
+											  AuthzContext context) {
 
 			if (LOG.isDebugEnabled()) {
 				LOG.debug("==> RangerAccessControlEnforcer.traverseOnlyCheck("
-						+ "path=" + path + ", user=" + user + ", groups=" + groups + ")");
+						+ "path=" + path + ", user=" + context.user + ", groups=" + context.userGroups + ")");
 			}
 			final AuthzStatus ret;
 
@@ -545,23 +540,22 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						resourcePath = resourcePath.substring(0, resourcePath.length()-1);
 					}
 				}
-				ret = isAccessAllowedForTraversal(nodeToCheck, nodeAttribs, resourcePath, user, groups, plugin, auditHandler, skipAuditOnAllow);
+				ret = isAccessAllowedForTraversal(nodeToCheck, nodeAttribs, resourcePath, skipAuditOnAllow, context);
 			} else {
 				ret = AuthzStatus.ALLOW;
 			}
 			if (LOG.isDebugEnabled()) {
 				LOG.debug("<== RangerAccessControlEnforcer.traverseOnlyCheck("
-						+ "path=" + path + ", resourcePath=" + resourcePath + ", user=" + user + ", groups=" + groups + ") : " + ret);
+						+ "path=" + path + ", resourcePath=" + resourcePath + ", user=" + context.user + ", groups=" + context.userGroups + ") : " + ret);
 			}
 			return ret;
 		}
 
-		private AuthzStatus isAccessAllowedForTraversal(INode inode, INodeAttributes inodeAttribs, String path, String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler, boolean skipAuditOnAllow) {
+		private AuthzStatus isAccessAllowedForTraversal(INode inode, INodeAttributes inodeAttribs, String path, boolean skipAuditOnAllow, AuthzContext context) {
 			final AuthzStatus ret;
 			String pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 			FsAction access = FsAction.EXECUTE;
 
-
 			if (pathOwner == null) {
 				pathOwner = inode.getUserName();
 			}
@@ -571,12 +565,14 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + user + ", " + skipAuditOnAllow + ")");
+				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + context.user + ", " + skipAuditOnAllow + ")");
 			}
 
-			RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, EXECUTE_ACCCESS_TYPE, user, groups);
+			RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, EXECUTE_ACCCESS_TYPE, context.operationName, context.user, context.userGroups);
+
+			RangerAccessResult result = context.plugin.isAccessAllowed(request, null);
 
-			RangerAccessResult result = plugin.isAccessAllowed(request, null);
+			context.saveResult(result);
 
 			if (result != null && result.getIsAccessDetermined() && !result.getIsAllowed()) {
 				ret = AuthzStatus.DENY;
@@ -585,11 +581,13 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if (ret == AuthzStatus.DENY || (!skipAuditOnAllow && result != null && result.getIsAccessDetermined())) {
-				auditHandler.processResult(result);
+				if (context.auditHandler != null) {
+					context.auditHandler.processResult(result);
+				}
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + user + ", " + skipAuditOnAllow + "): " + ret);
+				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowedForTraversal(" + path + ", " + access + ", " + context.user + ", " + skipAuditOnAllow + "): " + ret);
 			}
 
 			return ret;
@@ -600,8 +598,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 									int snapshotId, String path, int ancestorIndex, boolean doCheckOwner,
 									FsAction ancestorAccess, FsAction parentAccess, FsAction access,
 									FsAction subAccess, boolean ignoreEmptyDir,
-                                    boolean isTraverseOnlyCheck, INode ancestor,
-												 INode parent, INode inode, RangerHdfsAuditHandler auditHandler
+                                    INode ancestor, INode parent, INode inode, AuthzContext context
 												 ) throws AccessControlException {
 			if (LOG.isDebugEnabled()) {
 				LOG.debug("==> RangerAccessControlEnforcer.checkDefaultEnforcer("
@@ -609,7 +606,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 						+ ", snapshotId=" + snapshotId + ", path=" + path + ", ancestorIndex=" + ancestorIndex
 						+ ", doCheckOwner=" + doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
 						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir
-						+ ", isTraverseOnlyCheck=" + isTraverseOnlyCheck + ",ancestor=" + (ancestor == null ? null : ancestor.getFullPathName())
+						+ ", isTraverseOnlyCheck=" + context.isTraverseOnlyCheck + ",ancestor=" + (ancestor == null ? null : ancestor.getFullPathName())
 						+ ", parent=" + (parent == null ? null : parent.getFullPathName()) + ", inode=" + (inode == null ? null : inode.getFullPathName())
 						+ ")");
 			}
@@ -630,10 +627,10 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 
 					authzStatus = AuthzStatus.ALLOW;
 				} finally {
-					if (auditHandler != null) {
+					if (context.auditHandler != null) {
 						INode nodeChecked = inode;
 						FsAction action = access;
-						if (isTraverseOnlyCheck) {
+						if (context.isTraverseOnlyCheck) {
 							if (nodeChecked == null || nodeChecked.isFile()) {
 								if (parent != null) {
 									nodeChecked = parent;
@@ -655,9 +652,20 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 							}
 						}
 
-						String pathChecked = nodeChecked != null ? nodeChecked.getFullPathName() : path;
+						String             pathChecked = nodeChecked != null ? nodeChecked.getFullPathName() : path;
+						boolean            isAllowed   = authzStatus == AuthzStatus.ALLOW;
+						RangerAccessResult lastResult  = context.getLastResult();
+
+						if (lastResult != null) {
+							lastResult.setIsAllowed(isAllowed);
+							lastResult.setIsAccessDetermined(true);
+
+							context.plugin.evalAuditPolicies(lastResult);
+
+							context.auditHandler.processResult(lastResult);
+						}
 
-						auditHandler.logHadoopEvent(pathChecked, action, authzStatus == AuthzStatus.ALLOW);
+						context.auditHandler.logHadoopEvent(pathChecked, action, isAllowed);
 					}
 					RangerPerfTracer.log(hadoopAuthPerf);
 				}
@@ -667,14 +675,14 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 					+ ", snapshotId=" + snapshotId + ", path=" + path + ", ancestorIndex=" + ancestorIndex
 					+ ", doCheckOwner="+ doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
 					+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir
-					+ ", isTraverseOnlyCheck=" + isTraverseOnlyCheck + ",ancestor=" + (ancestor == null ? null : ancestor.getFullPathName())
+					+ ", isTraverseOnlyCheck=" + context.isTraverseOnlyCheck + ",ancestor=" + (ancestor == null ? null : ancestor.getFullPathName())
 					+ ", parent=" + (parent == null ? null : parent.getFullPathName()) + ", inode=" + (inode == null ? null : inode.getFullPathName())
 					+ ") : " + authzStatus );
 
 			return authzStatus;
 		}
 
-		private AuthzStatus isAccessAllowed(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler) {
+		private AuthzStatus isAccessAllowed(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, AuthzContext context) {
 			AuthzStatus ret       = null;
 			String      pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 
@@ -687,24 +695,26 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if(LOG.isDebugEnabled()) {
-				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + user + ")");
+				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + context.user + ")");
 			}
 
 			Set<String> accessTypes = access2ActionListMapper.get(access);
 
 			if(accessTypes == null) {
-				LOG.warn("RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + user + "): no Ranger accessType found for " + access);
+				LOG.warn("RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + context.user + "): no Ranger accessType found for " + access);
 
 				accessTypes = access2ActionListMapper.get(FsAction.NONE);
 			}
 
 			for(String accessType : accessTypes) {
-				RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, accessType, user, groups);
+				RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, accessType, context.operationName, context.user, context.userGroups);
 
 				Map<String, Object> requestContext = request.getContext();
 				requestContext.put(RangerAccessRequestUtil.KEY_CONTEXT_ACCESSTYPES, accessTypes);
 
-				RangerAccessResult result = plugin.isAccessAllowed(request, auditHandler);
+				RangerAccessResult result = context.plugin.isAccessAllowed(request, context.auditHandler);
+
+				context.saveResult(result);
 
 				if (result == null || !result.getIsAccessDetermined()) {
 					ret = AuthzStatus.NOT_DETERMINED;
@@ -724,13 +734,13 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if(LOG.isDebugEnabled()) {
-				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + user + "): " + ret);
+				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + context.user + "): " + ret);
 			}
 
 			return ret;
 		}
 
-		private AuthzStatus isAccessAllowedForHierarchy(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, String user, Set<String> groups, RangerHdfsPlugin plugin) {
+		private AuthzStatus isAccessAllowedForHierarchy(INode inode, INodeAttributes inodeAttribs, String path, FsAction access, AuthzContext context) {
 			AuthzStatus ret   = null;
 			String  pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
 
@@ -743,7 +753,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + user + ")");
+				LOG.debug("==> RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + context.user + ")");
 			}
 
 			if (path != null) {
@@ -751,7 +761,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				Set<String> accessTypes = access2ActionListMapper.get(access);
 
 				if (accessTypes == null) {
-					LOG.warn("RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + user + "): no Ranger accessType found for " + access);
+					LOG.warn("RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + context.user + "): no Ranger accessType found for " + access);
 
 					accessTypes = access2ActionListMapper.get(FsAction.NONE);
 				}
@@ -763,12 +773,14 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				subDirPath = subDirPath + rangerPlugin.getRandomizedWildcardPathName();
 
 				for (String accessType : accessTypes) {
-					RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(null, subDirPath, pathOwner, access, accessType, user, groups);
+					RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(null, subDirPath, pathOwner, access, accessType, context.operationName, context.user, context.userGroups);
 
 					Map<String, Object> requestContext = request.getContext();
 					requestContext.put(RangerAccessRequestUtil.KEY_CONTEXT_ACCESSTYPES, accessTypes);
 
-					RangerAccessResult result = plugin.isAccessAllowed(request, null);
+					RangerAccessResult result = context.plugin.isAccessAllowed(request, null);
+
+					context.saveResult(result);
 
 					if (result == null || !result.getIsAccessDetermined()) {
 						ret = AuthzStatus.NOT_DETERMINED;
@@ -789,7 +801,7 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			}
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + user + "): " + ret);
+				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowedForHierarchy(" + path + ", " + access + ", " + context.user + "): " + ret);
 			}
 
 			return ret;
@@ -889,14 +901,18 @@ class RangerHdfsResource extends RangerAccessResourceImpl {
 
 class RangerHdfsAccessRequest extends RangerAccessRequestImpl {
 
-	public RangerHdfsAccessRequest(INode inode, String path, String pathOwner, FsAction access, String accessType, String user, Set<String> groups) {
+	public RangerHdfsAccessRequest(INode inode, String path, String pathOwner, FsAction access, String accessType, String action, String user, Set<String> groups) {
+		if (action == null && access != null) {
+			action = access.toString();
+		}
+
 		super.setResource(new RangerHdfsResource(path, pathOwner));
 		super.setAccessType(accessType);
 		super.setUser(user);
 		super.setUserGroups(groups);
 		super.setAccessTime(new Date());
 		super.setClientIPAddress(getRemoteIp());
-		super.setAction(access.toString());
+		super.setAction(action);
 		super.setForwardedAddresses(null);
 		super.setRemoteIPAddress(getRemoteIp());
 
@@ -926,6 +942,34 @@ class RangerHdfsAccessRequest extends RangerAccessRequestImpl {
 	}
 }
 
+class AuthzContext {
+	public final RangerHdfsPlugin       plugin;
+	public final String                 user;
+	public final Set<String>            userGroups;
+	public final String                 operationName;
+	public final boolean                isTraverseOnlyCheck;
+	public       RangerHdfsAuditHandler auditHandler = null;
+	private      RangerAccessResult     lastResult   = null;
+
+	public AuthzContext(RangerHdfsPlugin plugin, UserGroupInformation ugi, String operationName, boolean isTraverseOnlyCheck) {
+		this.plugin              = plugin;
+		this.user                = ugi != null ? ugi.getShortUserName() : null;
+		this.userGroups          = ugi != null ? Sets.newHashSet(ugi.getGroupNames()) : null;
+		this.operationName       = operationName;
+		this.isTraverseOnlyCheck = isTraverseOnlyCheck;
+	}
+
+	public void saveResult(RangerAccessResult result) {
+		if (result != null) {
+			this.lastResult = result;
+		}
+	}
+
+	public RangerAccessResult getLastResult() {
+		return lastResult;
+	}
+}
+
 class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 	private static final Log LOG = LogFactory.getLog(RangerHdfsAuditHandler.class);
 
@@ -936,15 +980,13 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 
 	private final String      hadoopModuleName;
 	private final Set<String> excludeUsers;
-	private final String	  operationName;
 	private final String      callerContext;
 
-	public RangerHdfsAuditHandler(String pathToBeValidated,boolean auditOnlyIfDenied, String hadoopModuleName, Set<String> excludedUsers, String operationName, String callerContext) {
+	public RangerHdfsAuditHandler(String pathToBeValidated, boolean auditOnlyIfDenied, String hadoopModuleName, Set<String> excludedUsers, String callerContext) {
 		this.pathToBeValidated = pathToBeValidated;
 		this.auditOnlyIfDenied = auditOnlyIfDenied;
 		this.hadoopModuleName  = hadoopModuleName;
 		this.excludeUsers      = excludedUsers;
-		this.operationName     = operationName;
 		this.callerContext     = callerContext;
 	}
 
@@ -977,7 +1019,7 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 				auditEvent.setAccessResult((short) (result.getIsAllowed() ? 1 : 0));
 				auditEvent.setPolicyId(result.getPolicyId());
 				auditEvent.setPolicyVersion(result.getPolicyVersion());
-				addOperationNameToAuditEvent();
+				setRequestData();
 
 				auditEvent.setAction(getAccessType(request.getAccessType()));
 				auditEvent.setAdditionalInfo(getAdditionalInfo(request));
@@ -1018,14 +1060,20 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 		if(auditEvent != null) {
 			auditEvent.setResultReason(path);
 			auditEvent.setAccessResult((short) (accessGranted ? 1 : 0));
-			String accessType = (action == null) ? null : action.toString();
-			auditEvent.setAccessType(accessType);
 			auditEvent.setAclEnforcer(hadoopModuleName);
 			auditEvent.setPolicyId(-1);
+
+			String accessType = (action == null) ? null : action.toString();
+
+			if (StringUtils.isBlank(auditEvent.getAccessType())) { // retain existing value
+				auditEvent.setAccessType(accessType);
+			}
+
 			if (accessType != null) {
 				auditEvent.setAction(getAccessType(accessType));
 			}
-			addOperationNameToAuditEvent();
+
+			setRequestData();
 		}
 
 		if(LOG.isDebugEnabled()) {
@@ -1096,14 +1144,9 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 		return ret;
 	}
 
-	private void addOperationNameToAuditEvent(){
-		if (StringUtils.isNotBlank(operationName)) {
-			String opName = operationName;
-			if (StringUtils.isNotBlank(callerContext)) {
-				opName = opName + "/" + callerContext;
-			}
-			auditEvent.setAccessType(operationName);
-			auditEvent.setRequestData(opName);
+	private void setRequestData(){
+		if (StringUtils.isNotBlank(auditEvent.getAccessType()) && StringUtils.isNotBlank(callerContext)) {
+			auditEvent.setRequestData(auditEvent.getAccessType() + "/" + callerContext);
 		}
 	}
 }