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/10/20 07:26:36 UTC

[ranger] 02/02: RANGER-3048: updated HDFS plugin to include user action (like list/mkdir/delete) in audit logs

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

commit 4c2540e765b1682bdcf839aafa3ff318b47762ee
Author: Ramesh Mani <rm...@cloudera.com>
AuthorDate: Mon Sep 28 16:06:20 2020 -0700

    RANGER-3048: updated HDFS plugin to include user action (like list/mkdir/delete) in audit logs
---
 .../hadoop/constants/RangerHadoopConstants.java    |   5 +
 .../plugin/util/RangerAccessRequestUtil.java       |   1 +
 .../authorization/hadoop/RangerHdfsAuthorizer.java | 126 ++++++++++++++++++++-
 .../services/hdfs/RangerHdfsAuthorizerTest.java    |  59 ++++++++--
 .../src/main/webapp/scripts/utils/XAViewUtils.js   |   5 +-
 5 files changed, 182 insertions(+), 14 deletions(-)

diff --git a/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/constants/RangerHadoopConstants.java b/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/constants/RangerHadoopConstants.java
index 6d9fe26..31e4c0f 100644
--- a/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/constants/RangerHadoopConstants.java
+++ b/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/constants/RangerHadoopConstants.java
@@ -28,6 +28,11 @@ public class RangerHadoopConstants {
 	public static final String WRITE_ACCCESS_TYPE = "write";
 	public static final String EXECUTE_ACCCESS_TYPE = "execute";
 
+	public static final String READ_EXECUTE_PERM  = "READ_EXECUTE";
+	public static final String WRITE_EXECUTE_PERM = "WRITE_EXECUTE";
+	public static final String READ_WRITE_PERM    = "READ_WRITE";
+	public static final String ALL_PERM           = "ALL";
+
 	public static final String HDFS_ROOT_FOLDER_PATH_ALT = "";
 	public static final String HDFS_ROOT_FOLDER_PATH = "/";
 	
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
index bc52bdb..a22027a 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
@@ -43,6 +43,7 @@ public class RangerAccessRequestUtil {
 	public static final String KEY_USER = "USER";
 	public static final String KEY_OWNER = "OWNER";
 	public static final String KEY_ROLES = "ROLES";
+	public static final String KEY_CONTEXT_ACCESSTYPES = "ACCESSTYPES";
 
 	public static void setRequestTagsInContext(Map<String, Object> context, Set<RangerTagForEval> tags) {
 		if(CollectionUtils.isEmpty(tags)) {
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 d8bcac7..44fec7b 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
@@ -23,11 +23,16 @@ import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConst
 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;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.READ_EXECUTE_PERM;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.WRITE_EXECUTE_PERM;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.READ_WRITE_PERM;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.ALL_PERM;
 
 import java.net.InetAddress;
 import java.security.SecureRandom;
 import java.util.*;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -42,6 +47,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.ipc.CallerContext;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -219,24 +225,47 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 		}
 
 		@Override
+		public void checkPermissionWithContext(AuthorizationContext authzContext) throws AccessControlException {
+			checkRangerPermission(authzContext.getFsOwner(), authzContext.getSupergroup(),
+					authzContext.getCallerUgi(), authzContext.getInodeAttrs(),
+					authzContext.getInodes(), authzContext.getPathByNameArr(),
+					authzContext.getSnapshotId(), authzContext.getPath(),
+					authzContext.getAncestorIndex(), authzContext.isDoCheckOwner(),
+					authzContext.getAncestorAccess(), authzContext.getParentAccess(),
+					authzContext.getAccess(), authzContext.getSubAccess(),
+					authzContext.isIgnoreEmptyDir(), authzContext.getOperationName(),
+					authzContext.getCallerContext());
+		}
+
+		@Override
 		public void checkPermission(String fsOwner, String superGroup, UserGroupInformation ugi,
 									INodeAttributes[] inodeAttrs, INode[] inodes, byte[][] pathByNameArr,
 									int snapshotId, String path, int ancestorIndex, boolean doCheckOwner,
 									FsAction ancestorAccess, FsAction parentAccess, FsAction access,
 									FsAction subAccess, boolean ignoreEmptyDir) throws AccessControlException {
+			checkRangerPermission(fsOwner, superGroup, ugi, inodeAttrs, inodes, pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner, ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir, null, null);
+		}
+
+		private void checkRangerPermission(String fsOwner, String superGroup, UserGroupInformation ugi,
+									INodeAttributes[] inodeAttrs, INode[] inodes, byte[][] pathByNameArr,
+									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;
 
 			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
 						+ ", doCheckOwner="+ doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
-						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir + ")");
+						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir + ", operationName=" + operationName
+						+ ", callerContext=" + callerctxt +")");
 			}
 
 			RangerPerfTracer perf = null;
@@ -320,7 +349,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(resourcePath, isTraverseOnlyCheck, rangerPlugin.getHadoopModuleName(), rangerPlugin.getExcludedUsers());
+					auditHandler = doNotGenerateAuditRecord ? null : new RangerHdfsAuditHandler(resourcePath, isTraverseOnlyCheck, rangerPlugin.getHadoopModuleName(), rangerPlugin.getExcludedUsers(), operationName, callerctxt);
 
 					/* 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
@@ -671,6 +700,9 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 			for(String accessType : accessTypes) {
 				RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(inode, path, pathOwner, access, accessType, user, groups);
 
+				Map<String, Object> requestContext = request.getContext();
+				requestContext.put(RangerAccessRequestUtil.KEY_CONTEXT_ACCESSTYPES, accessTypes);
+
 				RangerAccessResult result = plugin.isAccessAllowed(request, auditHandler);
 
 				if (result == null || !result.getIsAccessDetermined()) {
@@ -732,6 +764,9 @@ public class RangerHdfsAuthorizer extends INodeAttributeProvider {
 				for (String accessType : accessTypes) {
 					RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(null, subDirPath, pathOwner, access, accessType, user, groups);
 
+					Map<String, Object> requestContext = request.getContext();
+					requestContext.put(RangerAccessRequestUtil.KEY_CONTEXT_ACCESSTYPES, accessTypes);
+
 					RangerAccessResult result = plugin.isAccessAllowed(request, null);
 
 					if (result == null || !result.getIsAccessDetermined()) {
@@ -900,12 +935,16 @@ 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) {
+	public RangerHdfsAuditHandler(String pathToBeValidated,boolean auditOnlyIfDenied, String hadoopModuleName, Set<String> excludedUsers, String operationName, String callerContext) {
 		this.pathToBeValidated = pathToBeValidated;
 		this.auditOnlyIfDenied = auditOnlyIfDenied;
 		this.hadoopModuleName  = hadoopModuleName;
 		this.excludeUsers      = excludedUsers;
+		this.operationName     = operationName;
+		this.callerContext     = callerContext;
 	}
 
 	@Override
@@ -937,6 +976,10 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 				auditEvent.setAccessResult((short) (result.getIsAllowed() ? 1 : 0));
 				auditEvent.setPolicyId(result.getPolicyId());
 				auditEvent.setPolicyVersion(result.getPolicyVersion());
+				addOperationNameToAuditEvent();
+
+				auditEvent.setAction(getAccessType(request.getAccessType()));
+				auditEvent.setAdditionalInfo(getAdditionalInfo(request));
 
 				Set<String> tags = getTags(request);
 				if (tags != null) {
@@ -950,6 +993,22 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 		}
 	}
 
+	@Override
+	public 	String getAdditionalInfo(RangerAccessRequest request) {
+		StringBuilder 			sb   = null;
+		String        additionalInfo = super.getAdditionalInfo(request);
+		if (additionalInfo == null) {
+			sb = new StringBuilder("");
+		} else {
+			sb = new StringBuilder(additionalInfo);
+		}
+		String accessTypes = getAccessTypesAsString(request);
+		if (accessTypes != null) {
+			sb.append(", \"accessTypes\":[").append(accessTypes).append("]");
+		}
+		return sb.toString();
+	}
+
 	public void logHadoopEvent(String path, FsAction action, boolean accessGranted) {
 		if(LOG.isDebugEnabled()) {
 			LOG.debug("==> RangerHdfsAuditHandler.logHadoopEvent(" + path + ", " + action + ", " + accessGranted + ")");
@@ -958,9 +1017,14 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 		if(auditEvent != null) {
 			auditEvent.setResultReason(path);
 			auditEvent.setAccessResult((short) (accessGranted ? 1 : 0));
-			auditEvent.setAccessType(action == null ? null : action.toString());
+			String accessType = (action == null) ? null : action.toString();
+			auditEvent.setAccessType(accessType);
 			auditEvent.setAclEnforcer(hadoopModuleName);
 			auditEvent.setPolicyId(-1);
+			if (accessType != null) {
+				auditEvent.setAction(getAccessType(accessType));
+			}
+			addOperationNameToAuditEvent();
 		}
 
 		if(LOG.isDebugEnabled()) {
@@ -987,5 +1051,59 @@ class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
 			LOG.debug("<== RangerHdfsAuditHandler.flushAudit(" + isAuditEnabled + ", " + auditEvent + ")");
 		}
 	}
+
+	private String getAccessType(String accessType) {
+		String ret = accessType;
+
+		switch (accessType) {
+			case READ_EXECUTE_PERM:
+				ret = READ_ACCCESS_TYPE;
+				break;
+			case WRITE_EXECUTE_PERM:
+			case READ_WRITE_PERM:
+			case ALL_PERM:
+				ret = WRITE_ACCCESS_TYPE;
+				break;
+			default:
+				break;
+		}
+		return ret.toLowerCase();
+	}
+
+	private String getAccessTypesAsString(RangerAccessRequest request) {
+		String             ret         = null;
+		Map<String,Object> context     = request.getContext();
+		Set<String>        accessTypes = null;
+
+		Object val = context.get(RangerAccessRequestUtil.KEY_CONTEXT_ACCESSTYPES);
+		if (val instanceof Set<?>) {
+			try {
+				accessTypes = (Set<String>) val;
+				ret = getFormattedAccessType(accessTypes);
+			} catch (Throwable t) {
+				LOG.error("getAccessTypesAsString(): failed to get accessTypes from context", t);
+			}
+		}
+		return ret;
+	}
+
+	private String getFormattedAccessType(Set<String> accessTypes) {
+		String ret = null;
+		if (CollectionUtils.isNotEmpty(accessTypes)) {
+			ret = String.join(", ", accessTypes);
+		}
+		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);
+		}
+	}
 }
 
diff --git a/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerHdfsAuthorizerTest.java b/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerHdfsAuthorizerTest.java
index cf9bad5..e96ea5c 100644
--- a/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerHdfsAuthorizerTest.java
+++ b/hdfs-agent/src/test/java/org/apache/ranger/services/hdfs/RangerHdfsAuthorizerTest.java
@@ -30,6 +30,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.security.AccessControlException;
@@ -91,11 +92,31 @@ public class RangerHdfsAuthorizerTest {
          */
         public void checkDirAccess(FsAction access, String userName, String... groups) throws AccessControlException {
             final UserGroupInformation user = UserGroupInformation.createUserForTesting(userName, groups);
-            rangerControlEnforcer.checkPermission(FILE_OWNER, FILE_GROUP, user,
-                    Arrays.copyOf(attributes, attributes.length - 1), Arrays.copyOf(nodes, nodes.length - 1),
-                    new byte[0][0], SNAPSHOT_ID, path, ancestorIndex - 1, false /* doCheckOwner */,
-                    null /* ancestorAccess */, null /* parentAccess */ , access, null /* subAccess */ ,
-                    false /* ignoreEmptyDir */);
+
+            INodeAttributeProvider.AuthorizationContext.Builder builder =
+                    new  INodeAttributeProvider.AuthorizationContext.Builder()
+                    .fsOwner(FILE_OWNER)
+                    .supergroup(FILE_GROUP)
+                    .callerUgi(user)
+                    .inodeAttrs(Arrays.copyOf(attributes, attributes.length - 1))
+                    .inodes(Arrays.copyOf(nodes, nodes.length - 1))
+                    .pathByNameArr(new byte[0][0])
+                    .snapshotId(SNAPSHOT_ID)
+                    .path(path)
+                    .ancestorIndex(ancestorIndex - 1)
+                    .doCheckOwner(false)
+                    .ancestorAccess(null)
+                    .parentAccess(null)
+                    .access(access)
+                    .subAccess(null)
+                    .ignoreEmptyDir(false)
+                    .operationName(null)
+                    .callerContext(null);
+
+            INodeAttributeProvider.AuthorizationContext authorizationContext
+                    = new INodeAttributeProvider.AuthorizationContext(builder);
+
+            rangerControlEnforcer.checkPermissionWithContext(authorizationContext);
         }
 
         /**
@@ -104,9 +125,31 @@ public class RangerHdfsAuthorizerTest {
          */
         public void checkAccess(FsAction access, String userName, String... groups) throws AccessControlException {
             final UserGroupInformation user = UserGroupInformation.createUserForTesting(userName, groups);
-            rangerControlEnforcer.checkPermission(FILE_OWNER, FILE_GROUP, user, attributes, nodes, new byte[0][0],
-                    SNAPSHOT_ID, path, ancestorIndex, false /* doCheckOwner */, null /* ancestorAccess */,
-                    null /* parentAccess */ , access, null /* subAccess */ , false /* ignoreEmptyDir */);
+
+            INodeAttributeProvider.AuthorizationContext.Builder builder =
+                    new  INodeAttributeProvider.AuthorizationContext.Builder()
+                            .fsOwner(FILE_OWNER)
+                            .supergroup(FILE_GROUP)
+                            .callerUgi(user)
+                            .inodeAttrs(attributes)
+                            .inodes(nodes)
+                            .pathByNameArr(new byte[0][0])
+                            .snapshotId(SNAPSHOT_ID)
+                            .path(path)
+                            .ancestorIndex(ancestorIndex - 1)
+                            .doCheckOwner(false)
+                            .ancestorAccess(null)
+                            .parentAccess(null)
+                            .access(access)
+                            .subAccess(null)
+                            .ignoreEmptyDir(false)
+                            .operationName(null)
+                            .callerContext(null);
+
+            INodeAttributeProvider.AuthorizationContext authorizationContext
+                    = new INodeAttributeProvider.AuthorizationContext(builder);
+
+            rangerControlEnforcer.checkPermissionWithContext(authorizationContext);
         }
 
         /**
diff --git a/security-admin/src/main/webapp/scripts/utils/XAViewUtils.js b/security-admin/src/main/webapp/scripts/utils/XAViewUtils.js
index 8b67452..e9edc9a 100644
--- a/security-admin/src/main/webapp/scripts/utils/XAViewUtils.js
+++ b/security-admin/src/main/webapp/scripts/utils/XAViewUtils.js
@@ -31,7 +31,7 @@ define(function(require) {
     XAViewUtil.resourceTypeFormatter = function(rawValue, model){
         var resourcePath = _.isUndefined(model.get('resourcePath')) ? undefined : model.get('resourcePath');
         var resourceType = _.isUndefined(model.get('resourceType')) ? undefined : model.get('resourceType');
-        if((model.get('serviceType') === XAEnums.ServiceType.Service_HIVE.label || model.get('serviceType') === XAEnums.ServiceType.Service_HBASE.label || model.get('serviceType') === XAEnums.ServiceType.Service_SOLR.label)
+        if((model.get('serviceType') === XAEnums.ServiceType.Service_HIVE.label || model.get('serviceType') === XAEnums.ServiceType.Service_HBASE.label || model.get('serviceType') === XAEnums.ServiceType.Service_SOLR.label || model.get('serviceType') === XAEnums.ServiceType.Service_HDFS.label)
             && model.get('aclEnforcer') === "ranger-acl"
             && model.get('requestData')){
             if(resourcePath && !_.isEmpty(model.get('requestData'))) {
@@ -65,13 +65,14 @@ define(function(require) {
     };
 
     XAViewUtil.showQueryPopup = function(model, that){
-        if((model.get('serviceType') === XAEnums.ServiceType.Service_HIVE.label || model.get('serviceType') === XAEnums.ServiceType.Service_HBASE.label || model.get('serviceType') === XAEnums.ServiceType.Service_SOLR.label)
+        if((model.get('serviceType') === XAEnums.ServiceType.Service_HIVE.label || model.get('serviceType') === XAEnums.ServiceType.Service_HBASE.label || model.get('serviceType') === XAEnums.ServiceType.Service_SOLR.label || model.get('serviceType') === XAEnums.ServiceType.Service_HDFS.label)
             && model.get('aclEnforcer') === "ranger-acl"
             && model.get('requestData') && !_.isEmpty(model.get('requestData'))){
             var titleMap = {};
             titleMap[XAEnums.ServiceType.Service_HIVE.label] = 'Hive Query';
             titleMap[XAEnums.ServiceType.Service_HBASE.label] = 'HBase Audit Data';
             titleMap[XAEnums.ServiceType.Service_SOLR.label] = 'Solr Query';
+            titleMap[XAEnums.ServiceType.Service_HDFS.label] = 'HDFS Operation Name';
             var msg = '<div class="pull-right link-tag query-icon copyQuery btn btn-sm" title="Copy Query"><i class="fa-fw fa fa-copy"></i></div><div class="query-content">'+model.get('requestData')+'</div>';
             var $elements = that.$el.find('table [data-name = "queryInfo"][data-id = "'+model.id+'"]');
             $elements.popover({