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 2015/04/02 09:11:16 UTC

incubator-ranger git commit: RANGER-357: Ranger HDFS plugin updated to use HDFS Authorization API

Repository: incubator-ranger
Updated Branches:
  refs/heads/master c934b7906 -> 49bebb59f


RANGER-357: Ranger HDFS plugin updated to use HDFS Authorization API


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

Branch: refs/heads/master
Commit: 49bebb59f793fa550242bbc34c2bb3173b6be2b5
Parents: c934b79
Author: Madhan Neethiraj <ma...@apache.org>
Authored: Tue Mar 31 16:50:27 2015 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Apr 2 00:04:31 2015 -0700

----------------------------------------------------------------------
 .../RangerPathResourceMatcher.java              |  39 +-
 hdfs-agent/conf/hdfs-site-changes.cfg           |   1 +
 hdfs-agent/conf/xasecure-hadoop-env.sh          |  40 --
 .../namenode/RangerFSPermissionChecker.java     | 328 -------------
 .../hadoop/RangerHdfsAuthorizer.java            | 482 +++++++++++++++++++
 .../agent/TestRangerFSPermissionChecker.java    |  83 ----
 src/main/assembly/hdfs-agent.xml                |   6 -
 7 files changed, 505 insertions(+), 474 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
index 4a60281..947c1ed 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
@@ -21,6 +21,7 @@ package org.apache.ranger.plugin.resourcematcher;
 
 
 import org.apache.commons.io.FilenameUtils;
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,8 +44,8 @@ public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
 			LOG.debug("==> RangerPathResourceMatcher.init(" + resourceDef + ", " + policyResource + ")");
 		}
 
-		policyIsRecursive     = policyResource == null ? false : policyResource.getIsRecursive();
-		pathSeparatorChar     = getCharOption(OPTION_PATH_SEPERATOR, DEFAULT_PATH_SEPERATOR_CHAR);
+		policyIsRecursive = policyResource == null ? false : policyResource.getIsRecursive();
+		pathSeparatorChar = getCharOption(OPTION_PATH_SEPERATOR, DEFAULT_PATH_SEPERATOR_CHAR);
 
 		super.init(resourceDef, policyResource);
 
@@ -108,27 +109,31 @@ public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
 		boolean ret = false;
 
 		if (! StringUtils.isEmpty(pathToCheck)) {
-			StringBuilder sb = new StringBuilder();
-			
-			if(pathToCheck.charAt(0) == pathSeparatorChar) {
-				sb.append(pathSeparatorChar); // preserve the initial seperator
-			}
+			String[] pathElements = StringUtils.split(pathToCheck, pathSeparatorChar);
+
+			if(! ArrayUtils.isEmpty(pathElements)) {
+				StringBuilder sb = new StringBuilder();
 
-			for(String p : StringUtils.split(pathToCheck, pathSeparatorChar)) {
-				sb.append(p);
+				if(pathToCheck.charAt(0) == pathSeparatorChar) {
+					sb.append(pathSeparatorChar); // preserve the initial pathSeparatorChar
+				}
 
-				boolean matchFound = FilenameUtils.wildcardMatch(sb.toString(), wildcardPath) ;
-				
-				if (matchFound) {
-					ret = true ;
+				for(String p : pathElements) {
+					sb.append(p);
 
-					break;
+					ret = FilenameUtils.wildcardMatch(sb.toString(), wildcardPath) ;
+
+					if (ret) {
+						break;
+					}
+
+					sb.append(pathSeparatorChar) ;
 				}
 
-				sb.append(pathSeparatorChar) ;
+				sb = null;
+			} else { // pathToCheck consists of only pathSeparatorChar
+				ret = FilenameUtils.wildcardMatch(pathToCheck, wildcardPath) ;
 			}
-
-			sb = null;
 		}
 
 		if(LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/hdfs-agent/conf/hdfs-site-changes.cfg
----------------------------------------------------------------------
diff --git a/hdfs-agent/conf/hdfs-site-changes.cfg b/hdfs-agent/conf/hdfs-site-changes.cfg
index 8059ec7..8088b43 100644
--- a/hdfs-agent/conf/hdfs-site-changes.cfg
+++ b/hdfs-agent/conf/hdfs-site-changes.cfg
@@ -18,3 +18,4 @@
 #
 dfs.permissions.enabled						true	mod	create-if-not-exists
 dfs.permissions								true	mod	create-if-not-exists
+dfs.namenode.inode.attributes.provider.class org.apache.ranger.authorization.hadoop.RangerHdfsAuthorizer mod create-if-not-exists

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/hdfs-agent/conf/xasecure-hadoop-env.sh
----------------------------------------------------------------------
diff --git a/hdfs-agent/conf/xasecure-hadoop-env.sh b/hdfs-agent/conf/xasecure-hadoop-env.sh
deleted file mode 100644
index 4e3e78e..0000000
--- a/hdfs-agent/conf/xasecure-hadoop-env.sh
+++ /dev/null
@@ -1,40 +0,0 @@
-#!/bin/bash
-
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-
-if [ -f ${HADOOP_HOME}/conf/ranger-security.xml ]
-then
-	echo "$0" | grep -q beeswax_server.sh > /dev/null 2>&1
-	if [ $? -ne 0 ]
-	then
-		XASECURE_AGENT_PATH="`ls -1 ${HADOOP_HOME}/lib/ranger-hdfs-plugin-*.jar 2> /dev/null | head -1`"
-		if [ -f "${XASECURE_AGENT_PATH}" ]
-		then
-	    	if [ "${XASECURE_INIT}" != "0" ]
-	    	then
-	        	XASECURE_INIT="0"
-	        	XASECURE_AGENT_OPTS=" -javaagent:${XASECURE_AGENT_PATH}=authagent "
-	        	echo ${HADOOP_NAMENODE_OPTS} | grep -q -- "${XASECURE_AGENT_OPTS}" > /dev/null 2>&1
-	        	if [ $? -ne 0 ]
-	        	then
-	                	export HADOOP_NAMENODE_OPTS=" ${XASECURE_AGENT_OPTS} ${HADOOP_NAMENODE_OPTS} "
-	                	export HADOOP_SECONDARYNAMENODE_OPTS=" ${XASECURE_AGENT_OPTS} ${HADOOP_SECONDARYNAMENODE_OPTS}"
-	        	fi
-	    	fi
-	    fi
-	fi
-fi

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/hdfs-agent/src/main/java/org/apache/hadoop/hdfs/server/namenode/RangerFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hdfs-agent/src/main/java/org/apache/hadoop/hdfs/server/namenode/RangerFSPermissionChecker.java b/hdfs-agent/src/main/java/org/apache/hadoop/hdfs/server/namenode/RangerFSPermissionChecker.java
deleted file mode 100644
index 0eb1435..0000000
--- a/hdfs-agent/src/main/java/org/apache/hadoop/hdfs/server/namenode/RangerFSPermissionChecker.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.*;
-
-import java.net.InetAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-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.ipc.Server;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.ranger.audit.model.AuthzAuditEvent;
-import org.apache.ranger.authorization.hadoop.config.RangerConfiguration;
-import org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants;
-import org.apache.ranger.authorization.hadoop.exceptions.RangerAccessControlException;
-import org.apache.ranger.authorization.utils.StringUtil;
-import org.apache.ranger.plugin.audit.RangerDefaultAuditHandler;
-import org.apache.ranger.plugin.model.RangerServiceDef;
-import org.apache.ranger.plugin.policyengine.RangerAccessRequest;
-import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl;
-import org.apache.ranger.plugin.policyengine.RangerAccessResult;
-import org.apache.ranger.plugin.policyengine.RangerAccessResource;
-import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl;
-import org.apache.ranger.plugin.service.RangerBasePlugin;
-
-import com.google.common.collect.Sets;
-
-
-public class RangerFSPermissionChecker {
-	private static final Log LOG = LogFactory.getLog(RangerFSPermissionChecker.class);
-
-	private static Map<FsAction, Set<String>> access2ActionListMapper = null ;
-
-	static {
-		access2ActionListMapper = new HashMap<FsAction, Set<String>>();
-
-		access2ActionListMapper.put(FsAction.NONE,          new HashSet<String>());
-		access2ActionListMapper.put(FsAction.ALL,           Sets.newHashSet(READ_ACCCESS_TYPE, WRITE_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.READ,          Sets.newHashSet(READ_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.READ_WRITE,    Sets.newHashSet(READ_ACCCESS_TYPE, WRITE_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.READ_EXECUTE,  Sets.newHashSet(READ_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.WRITE,         Sets.newHashSet(WRITE_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.WRITE_EXECUTE, Sets.newHashSet(WRITE_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
-		access2ActionListMapper.put(FsAction.EXECUTE,       Sets.newHashSet(EXECUTE_ACCCESS_TYPE));
-	}
-
-	private static volatile RangerHdfsPlugin           rangerPlugin        = null;
-	private static ThreadLocal<RangerHdfsAuditHandler> currentAuditHandler = new ThreadLocal<RangerHdfsAuditHandler>();
-
-	public static boolean check(UserGroupInformation ugi, INode inode, FsAction access) throws RangerAccessControlException {
-		if (ugi == null || inode == null || access == null) {
-			return false;
-		}
-
-		return check(ugi.getShortUserName(), Sets.newHashSet(ugi.getGroupNames()), inode, access);
-	}
-
-	public static boolean check(String user, Set<String> groups, INode inode, FsAction access) throws RangerAccessControlException {
-		if (user == null || inode == null || access == null) {
-			return false;
-		}
-
-		String path      = inode.getFullPathName();
-		String pathOwner = inode.getUserName();
-
-		boolean accessGranted =  AuthorizeAccessForUser(path, pathOwner, access, user, groups);
-
-		if (!accessGranted &&  !RangerHdfsPlugin.isHadoopAuthEnabled()) {
-			String inodeInfo = (inode.isDirectory() ? "directory" : "file") +  "="  + "\"" + path + "\""  ;
-		    throw new RangerAccessControlException("Permission denied: principal{user=" + user + ",groups: " + groups + "}, access=" + access + ", " + inodeInfo ) ; 
-		}
-
-		return accessGranted ;
-	}
-
-	public static boolean AuthorizeAccessForUser(String aPathName, String aPathOwnerName, FsAction access, String user, Set<String> groups) throws RangerAccessControlException {
-		boolean accessGranted = false;
-
-		if(aPathName != null && aPathOwnerName != null && access != null && user != null && groups != null) {
-			if (RangerHadoopConstants.HDFS_ROOT_FOLDER_PATH_ALT.equals(aPathName)) {
-				aPathName = RangerHadoopConstants.HDFS_ROOT_FOLDER_PATH;
-			}
-
-			RangerHdfsPlugin plugin = rangerPlugin;
-
-			if (plugin == null) {
-				synchronized(RangerFSPermissionChecker.class) {
-					plugin = rangerPlugin ;
-
-					if (plugin == null) {
-						try {
-							plugin = new RangerHdfsPlugin();
-							plugin.init();
-
-							rangerPlugin = plugin;
-						}
-						catch(Throwable t) {
-							LOG.error("Unable to create Authorizer", t);
-						}
-					}
-				}
-			}
-
-			if (rangerPlugin != null) {
-				Set<String> accessTypes = access2ActionListMapper.get(access);
-
-				boolean isAllowed = true;
-				for(String accessType : accessTypes) {
-					RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(aPathName, aPathOwnerName, access, accessType, user, groups);
-
-					RangerAccessResult result = rangerPlugin.isAccessAllowed(request, getCurrentAuditHandler());
-
-					isAllowed = result != null && result.getIsAllowed();
-					
-					if(!isAllowed) {
-						break;
-					}
-				}
-
-				accessGranted = isAllowed;
-			}
-		}
-
-		return accessGranted;
-	}
-
-	public static void checkPermissionPre(INodesInPath inodesInPath) {
-		String pathToBeValidated = getPath(inodesInPath);
-
-		checkPermissionPre(pathToBeValidated);
-	}
-
-	public static void checkPermissionPost(INodesInPath inodesInPath) {
-		String pathToBeValidated = getPath(inodesInPath);
-
-		checkPermissionPost(pathToBeValidated);
-	}
-
-	public static void checkPermissionPre(String pathToBeValidated) {
-		RangerHdfsAuditHandler auditHandler = new RangerHdfsAuditHandler(pathToBeValidated);
-		
-		currentAuditHandler.set(auditHandler);
-	}
-
-	public static void checkPermissionPost(String pathToBeValidated) {
-		RangerHdfsAuditHandler auditHandler = getCurrentAuditHandler();
-
-		if(auditHandler != null) {
-			auditHandler.flushAudit();
-		}
-
-		currentAuditHandler.set(null);
-	}
-
-	public static void logHadoopEvent(INode inode, boolean accessGranted) {
-		if(inode == null) {
-			return;
-		}
-
-		RangerHdfsAuditHandler auditHandler = getCurrentAuditHandler();
-
-		if(auditHandler != null) {
-			auditHandler.logHadoopEvent(inode.getFullPathName(), accessGranted);
-		}
-	}
-
-	private static RangerHdfsAuditHandler getCurrentAuditHandler() {
-		return currentAuditHandler.get();
-	}
-
-	private static String getPath(INodesInPath inodesInPath) {
-		int   length = inodesInPath.length();
-		INode last   = length > 0 ? inodesInPath.getLastINode() : null;
-
-		return last == null ? org.apache.hadoop.fs.Path.SEPARATOR : last.getFullPathName();
-	}
-}
-
-class RangerHdfsPlugin extends RangerBasePlugin {
-	private static boolean hadoopAuthEnabled = RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_DEFAULT;
-
-	public RangerHdfsPlugin() {
-		super("hdfs", "hdfs");
-	}
-	
-	public void init() {
-		super.init();
-		
-		RangerHdfsPlugin.hadoopAuthEnabled = RangerConfiguration.getInstance().getBoolean(RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_PROP, RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_DEFAULT);
-	}
-
-	public static boolean isHadoopAuthEnabled() {
-		return RangerHdfsPlugin.hadoopAuthEnabled;
-	}
-}
-
-class RangerHdfsResource extends RangerAccessResourceImpl {
-	private static final String KEY_PATH = "path";
-
-
-	public RangerHdfsResource(String path, String owner) {
-		super.setValue(KEY_PATH, path);
-		super.setOwnerUser(owner);
-	}
-}
-
-class RangerHdfsAccessRequest extends RangerAccessRequestImpl {
-	public RangerHdfsAccessRequest(String path, String pathOwner, FsAction access, String accessType, String user, Set<String> groups) {
-		super.setResource(new RangerHdfsResource(path, pathOwner));
-		super.setAccessType(accessType);
-		super.setUser(user);
-		super.setUserGroups(groups);
-		super.setAccessTime(StringUtil.getUTCDate());
-		super.setClientIPAddress(getRemoteIp());
-		super.setAction(access.toString());
-	}
-	
-	private static String getRemoteIp() {
-		String ret = null ;
-		InetAddress ip = Server.getRemoteIp() ;
-		if (ip != null) {
-			ret = ip.getHostAddress();
-		}
-		return ret ;
-	}
-}
-
-class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
-	private static final Log LOG = LogFactory.getLog(RangerHdfsAuditHandler.class);
-
-	private String          pathToBeValidated = null;
-	private boolean         isAuditEnabled    = false;
-	private AuthzAuditEvent auditEvent        = null;
-
-	private static final String    RangerModuleName = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_RANGER_MODULE_ACL_NAME_PROP , RangerHadoopConstants.DEFAULT_RANGER_MODULE_ACL_NAME) ;
-	private static final String    HadoopModuleName = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_HADOOP_MODULE_ACL_NAME_PROP , RangerHadoopConstants.DEFAULT_HADOOP_MODULE_ACL_NAME) ;
-	private static final String    excludeUserList  = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_HDFS_EXCLUDE_LIST_PROP, RangerHadoopConstants.AUDITLOG_EMPTY_STRING) ;
-	private static HashSet<String> excludeUsers     = null ;
-
-	static {
-		if (excludeUserList != null && excludeUserList.trim().length() > 0) {
-			excludeUsers = new HashSet<String>() ;
-			for(String excludeUser : excludeUserList.trim().split(",")) {
-				excludeUser = excludeUser.trim() ;
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Adding exclude user [" + excludeUser + "]");
-				}
-				excludeUsers.add(excludeUser) ;
-				}
-		}
-	}
-
-	public RangerHdfsAuditHandler(String pathToBeValidated) {
-		this.pathToBeValidated = pathToBeValidated;
-
-		auditEvent = new AuthzAuditEvent();
-	}
-
-	@Override
-	public void logAudit(RangerAccessResult result) {
-		if(! isAuditEnabled && result.getIsAudited()) {
-			isAuditEnabled = true;
-		}
-
-		RangerAccessRequest request      = result.getAccessRequest();
-		RangerServiceDef    serviceDef   = result.getServiceDef();
-		RangerAccessResource      resource     = request.getResource();
-		String              resourceType = resource != null ? resource.getLeafName(serviceDef) : null;
-		String              resourcePath = resource != null ? resource.getAsString(serviceDef) : null;
-
-		auditEvent.setUser(request.getUser());
-		auditEvent.setResourcePath(pathToBeValidated);
-		auditEvent.setResourceType(resourceType) ;
-		auditEvent.setAccessType(request.getAction());
-		auditEvent.setAccessResult((short)(result.getIsAllowed() ? 1 : 0));
-		auditEvent.setClientIP(request.getClientIPAddress());
-		auditEvent.setEventTime(request.getAccessTime());
-		auditEvent.setAclEnforcer(RangerModuleName);
-		auditEvent.setPolicyId(result.getPolicyId());
-		auditEvent.setRepositoryType(result.getServiceType());
-		auditEvent.setRepositoryName(result.getServiceName());
-		auditEvent.setResultReason(resourcePath);
-	}
-
-	public void logHadoopEvent(String path, boolean accessGranted) {
-		auditEvent.setResultReason(path);
-		auditEvent.setAccessResult((short) (accessGranted ? 1 : 0));
-		auditEvent.setAclEnforcer(HadoopModuleName);
-		auditEvent.setPolicyId(0);
-	}
-
-	public void flushAudit() {
-		String username = auditEvent.getUser();
-
-		boolean skipLog = (username != null && excludeUsers != null && excludeUsers.contains(username)) ;
-		
-		if (skipLog) {
-			return ;
-		}
-
-		if(isAuditEnabled) {
-			super.logAuthzAudit(auditEvent);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/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
new file mode 100644
index 0000000..8a73d4d
--- /dev/null
+++ b/hdfs-agent/src/main/java/org/apache/ranger/authorization/hadoop/RangerHdfsAuthorizer.java
@@ -0,0 +1,482 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+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.READ_ACCCESS_TYPE;
+import static org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants.WRITE_ACCCESS_TYPE;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+
+import org.apache.commons.lang.ArrayUtils;
+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.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.ranger.audit.model.AuthzAuditEvent;
+import org.apache.ranger.authorization.hadoop.config.RangerConfiguration;
+import org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants;
+import org.apache.ranger.authorization.hadoop.exceptions.RangerAccessControlException;
+import org.apache.ranger.authorization.utils.StringUtil;
+import org.apache.ranger.plugin.audit.RangerDefaultAuditHandler;
+import org.apache.ranger.plugin.model.RangerServiceDef;
+import org.apache.ranger.plugin.policyengine.RangerAccessRequest;
+import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl;
+import org.apache.ranger.plugin.policyengine.RangerAccessResource;
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl;
+import org.apache.ranger.plugin.policyengine.RangerAccessResult;
+import org.apache.ranger.plugin.service.RangerBasePlugin;
+
+import com.google.common.collect.Sets;
+
+public class RangerHdfsAuthorizer extends INodeAttributeProvider {
+	private static final Log LOG = LogFactory.getLog(RangerHdfsAuthorizer.class);
+
+	private RangerHdfsPlugin           rangerPlugin            = null;
+	private Map<FsAction, Set<String>> access2ActionListMapper = new HashMap<FsAction, Set<String>>();
+
+	public RangerHdfsAuthorizer() {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.RangerHdfsAuthorizer()");
+		}
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.RangerHdfsAuthorizer()");
+		}
+	}
+
+	public void start() {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.start()");
+		}
+
+		RangerHdfsPlugin plugin = new RangerHdfsPlugin();
+		plugin.init();
+
+		access2ActionListMapper.put(FsAction.NONE,          new HashSet<String>());
+		access2ActionListMapper.put(FsAction.ALL,           Sets.newHashSet(READ_ACCCESS_TYPE, WRITE_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.READ,          Sets.newHashSet(READ_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.READ_WRITE,    Sets.newHashSet(READ_ACCCESS_TYPE, WRITE_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.READ_EXECUTE,  Sets.newHashSet(READ_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.WRITE,         Sets.newHashSet(WRITE_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.WRITE_EXECUTE, Sets.newHashSet(WRITE_ACCCESS_TYPE, EXECUTE_ACCCESS_TYPE));
+		access2ActionListMapper.put(FsAction.EXECUTE,       Sets.newHashSet(EXECUTE_ACCCESS_TYPE));
+
+		rangerPlugin = plugin;
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.start()");
+		}
+	}
+
+	public void stop() {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.stop()");
+		}
+
+		RangerHdfsPlugin plugin = rangerPlugin;
+		rangerPlugin = null;
+
+		if(plugin != null) {
+			plugin.cleanup();
+		}
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.stop()");
+		}
+	}
+
+	@Override
+	public INodeAttributes getAttributes(String fullPath, INodeAttributes inode) {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.getAttributes(" + fullPath + ")");
+		}
+
+		INodeAttributes ret = inode; // return default attributes
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.getAttributes(" + fullPath + "): " + ret);
+		}
+
+		return ret;
+	}
+
+	@Override
+	public INodeAttributes getAttributes(String[] pathElements, INodeAttributes inode) {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.getAttributes(pathElementsCount=" + (pathElements == null ? 0 : pathElements.length) + ")");
+		}
+
+		INodeAttributes ret = inode; // return default attributes
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.getAttributes(pathElementsCount=" + (pathElements == null ? 0 : pathElements.length) + "): " + ret);
+		}
+
+		return ret;
+	}
+
+	@Override
+	public AccessControlEnforcer getExternalAccessControlEnforcer(AccessControlEnforcer defaultEnforcer) {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerHdfsAuthorizer.getExternalAccessControlEnforcer()");
+		}
+
+		RangerAccessControlEnforcer rangerAce = new RangerAccessControlEnforcer(defaultEnforcer);
+
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerHdfsAuthorizer.getExternalAccessControlEnforcer()");
+		}
+
+		return rangerAce;
+	}
+
+
+	class RangerAccessControlEnforcer implements AccessControlEnforcer {
+		private INodeAttributeProvider.AccessControlEnforcer defaultEnforcer = null;
+
+		public RangerAccessControlEnforcer(AccessControlEnforcer defaultEnforcer) {
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("==> RangerAccessControlEnforcer.RangerAccessControlEnforcer()");
+			}
+
+			this.defaultEnforcer = defaultEnforcer;
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("<== RangerAccessControlEnforcer.RangerAccessControlEnforcer()");
+			}
+		}
+
+		@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 {
+			boolean                accessGranted = false;
+			RangerHdfsPlugin       plugin        = rangerPlugin;
+			RangerHdfsAuditHandler auditHandler  = null;
+			String                 user          = ugi != null ? ugi.getShortUserName() : null;
+			Set<String>            groups        = ugi != null ? Sets.newHashSet(ugi.getGroupNames()) : null;
+
+			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
+						+ ", doCheckOwner="+ doCheckOwner + ", ancestorAccess=" + ancestorAccess + ", parentAccess=" + parentAccess
+						+ ", access=" + access + ", subAccess=" + subAccess + ", ignoreEmptyDir=" + ignoreEmptyDir + ")");
+			}
+
+			try {
+				if(plugin != null && (access != null || ancestorAccess != null || parentAccess != null || subAccess != null) && !ArrayUtils.isEmpty(inodes)) {
+					auditHandler = new RangerHdfsAuditHandler(path);
+
+					if(ancestorIndex >= inodes.length) {
+						ancestorIndex = inodes.length - 1;
+					}
+
+					for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null; ancestorIndex--);
+
+					accessGranted = true;
+
+					INode ancestor = inodes.length > ancestorIndex && ancestorIndex >= 0 ? inodes[ancestorIndex] : null;
+					INode parent   = inodes.length > 1 ? inodes[inodes.length - 2] : null;
+					INode inode    = inodes[inodes.length - 1];
+
+					// checkStickyBit
+					if (accessGranted && 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
+						    accessGranted = StringUtils.equals(parent.getUserName(), user) || StringUtils.equals(inode.getUserName(), user);
+						}
+					}
+
+					// checkAncestorAccess
+					if(accessGranted && ancestorAccess != null && ancestor != null) {
+						INodeAttributes ancestorAttribs = inodeAttrs.length > ancestorIndex ? inodeAttrs[ancestorIndex] : null;
+
+						accessGranted = isAccessAllowed(ancestor, ancestorAttribs, ancestorAccess, user, groups, fsOwner, superGroup, plugin, auditHandler);
+					}
+
+					// checkParentAccess
+					if(accessGranted && parentAccess != null && parent != null) {
+						INodeAttributes parentAttribs = inodeAttrs.length > 1 ? inodeAttrs[inodeAttrs.length - 2] : null;
+
+						accessGranted = isAccessAllowed(parent, parentAttribs, parentAccess, user, groups, fsOwner, superGroup, plugin, auditHandler);
+					}
+
+					// checkINodeAccess
+					if(accessGranted && access != null && inode != null) {
+						INodeAttributes inodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
+
+						accessGranted = isAccessAllowed(inode, inodeAttribs, access, user, groups, fsOwner, superGroup, plugin, auditHandler);
+					}
+
+					// checkSubAccess
+					if(accessGranted && subAccess != null && inode != null && inode.isDirectory()) {
+						Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
+
+						for(directories.push(inode.asDirectory()); !directories.isEmpty(); ) {
+							INodeDirectory      dir   = directories.pop();
+							ReadOnlyList<INode> cList = dir.getChildrenList(snapshotId);
+
+							if (!(cList.isEmpty() && ignoreEmptyDir)) {
+								INodeAttributes dirAttribs = dir.getSnapshotINode(snapshotId);
+
+								accessGranted = isAccessAllowed(dir, dirAttribs, access, user, groups, fsOwner, superGroup, plugin, auditHandler);
+
+								if(! accessGranted) {
+									break;
+								}
+							}
+
+							for(INode child : cList) {
+								if (child.isDirectory()) {
+									directories.push(child.asDirectory());
+								}
+							}
+						}
+					}
+
+					// checkOwnerAccess
+					if(accessGranted && doCheckOwner) {
+						INodeAttributes inodeAttribs = inodeAttrs.length > 0 ? inodeAttrs[inodeAttrs.length - 1] : null;
+						String          owner        = inodeAttribs != null ? inodeAttribs.getUserName() : null;
+
+						accessGranted = StringUtils.equals(user, owner);
+					}
+				}
+
+				if(! accessGranted && RangerHdfsPlugin.isHadoopAuthEnabled() && defaultEnforcer != null) {
+					try {
+						defaultEnforcer.checkPermission(fsOwner, superGroup, ugi, inodeAttrs, inodes,
+														pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
+														ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
+
+						accessGranted = true;
+					} finally {
+						if(auditHandler != null) {
+							auditHandler.logHadoopEvent(path, access, accessGranted);
+						}
+					}
+				}
+
+				if(! accessGranted) {
+					throw new RangerAccessControlException("Permission denied: principal{user=" + user + ",groups: " + groups + "}, access=" + access + ", " + path) ;
+				}
+			} finally {
+				if(auditHandler != null) {
+					auditHandler.flushAudit();
+				}
+
+				if(LOG.isDebugEnabled()) {
+					LOG.debug("<== RangerAccessControlEnforcer.checkPermission(" + path + ", " + access + ", user=" + user + ") : " + accessGranted);
+				}
+			}
+		}
+
+		private boolean isAccessAllowed(INode inode, INodeAttributes inodeAttribs, FsAction access, String user, Set<String> groups, String fsOwner, String superGroup, RangerHdfsPlugin plugin, RangerHdfsAuditHandler auditHandler) {
+			boolean ret       = false;
+			String  path      = inode != null ? inode.getFullPathName() : null;
+			String  pathOwner = inodeAttribs != null ? inodeAttribs.getUserName() : null;
+
+			if(pathOwner == null && inode != 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.isAccessAllowed(" + path + ", " + access + ", " + user + ")");
+			}
+
+			Set<String> accessTypes = access2ActionListMapper.get(access);
+
+			if(accessTypes == null) {
+				LOG.warn("RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + user + "): no Ranger accessType found for " + access);
+
+				accessTypes = access2ActionListMapper.get(FsAction.NONE);
+			}
+
+			for(String accessType : accessTypes) {
+				RangerHdfsAccessRequest request = new RangerHdfsAccessRequest(path, pathOwner, access, accessType, user, groups);
+
+				RangerAccessResult result = plugin.isAccessAllowed(request, auditHandler);
+
+				ret = result.getIsAllowed();
+
+				if(! ret) {
+					break;
+				}
+			}
+
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("<== RangerAccessControlEnforcer.isAccessAllowed(" + path + ", " + access + ", " + user + "): " + ret);
+			}
+
+			return ret;
+		}
+	}
+}
+
+
+class RangerHdfsPlugin extends RangerBasePlugin {
+	private static boolean hadoopAuthEnabled = RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_DEFAULT;
+
+	public RangerHdfsPlugin() {
+		super("hdfs", "hdfs");
+	}
+	
+	public void init() {
+		super.init();
+		
+		RangerHdfsPlugin.hadoopAuthEnabled = RangerConfiguration.getInstance().getBoolean(RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_PROP, RangerHadoopConstants.RANGER_ADD_HDFS_PERMISSION_DEFAULT);
+	}
+
+	public static boolean isHadoopAuthEnabled() {
+		return RangerHdfsPlugin.hadoopAuthEnabled;
+	}
+}
+
+class RangerHdfsResource extends RangerAccessResourceImpl {
+	private static final String KEY_PATH = "path";
+
+
+	public RangerHdfsResource(String path, String owner) {
+		super.setValue(KEY_PATH, path);
+		super.setOwnerUser(owner);
+	}
+}
+
+class RangerHdfsAccessRequest extends RangerAccessRequestImpl {
+	public RangerHdfsAccessRequest(String path, String pathOwner, FsAction access, String accessType, String user, Set<String> groups) {
+		super.setResource(new RangerHdfsResource(path, pathOwner));
+		super.setAccessType(accessType);
+		super.setUser(user);
+		super.setUserGroups(groups);
+		super.setAccessTime(StringUtil.getUTCDate());
+		super.setClientIPAddress(getRemoteIp());
+		super.setAction(access.toString());
+	}
+	
+	private static String getRemoteIp() {
+		String ret = null ;
+		InetAddress ip = Server.getRemoteIp() ;
+		if (ip != null) {
+			ret = ip.getHostAddress();
+		}
+		return ret ;
+	}
+}
+
+class RangerHdfsAuditHandler extends RangerDefaultAuditHandler {
+	private static final Log LOG = LogFactory.getLog(RangerHdfsAuditHandler.class);
+
+	private boolean         isAuditEnabled = false;
+	private AuthzAuditEvent auditEvent     = null;
+
+	private static final String    RangerModuleName = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_RANGER_MODULE_ACL_NAME_PROP , RangerHadoopConstants.DEFAULT_RANGER_MODULE_ACL_NAME) ;
+	private static final String    HadoopModuleName = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_HADOOP_MODULE_ACL_NAME_PROP , RangerHadoopConstants.DEFAULT_HADOOP_MODULE_ACL_NAME) ;
+	private static final String    excludeUserList  = RangerConfiguration.getInstance().get(RangerHadoopConstants.AUDITLOG_HDFS_EXCLUDE_LIST_PROP, RangerHadoopConstants.AUDITLOG_EMPTY_STRING) ;
+	private static HashSet<String> excludeUsers     = null ;
+
+	static {
+		if (excludeUserList != null && excludeUserList.trim().length() > 0) {
+			excludeUsers = new HashSet<String>() ;
+			for(String excludeUser : excludeUserList.trim().split(",")) {
+				excludeUser = excludeUser.trim() ;
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Adding exclude user [" + excludeUser + "]");
+				}
+				excludeUsers.add(excludeUser) ;
+				}
+		}
+	}
+
+	public RangerHdfsAuditHandler(String pathToBeValidated) {
+		auditEvent = new AuthzAuditEvent();
+		auditEvent.setResourcePath(pathToBeValidated);
+	}
+
+	@Override
+	public void logAudit(RangerAccessResult result) {
+		if(! isAuditEnabled && result.getIsAudited()) {
+			isAuditEnabled = true;
+		}
+
+		RangerAccessRequest  request      = result.getAccessRequest();
+		RangerServiceDef     serviceDef   = result.getServiceDef();
+		RangerAccessResource resource     = request.getResource();
+		String               resourceType = resource != null ? resource.getLeafName(serviceDef) : null;
+		String               resourcePath = resource != null ? resource.getAsString(serviceDef) : null;
+
+		auditEvent.setUser(request.getUser());
+		auditEvent.setResourceType(resourceType) ;
+		auditEvent.setAccessType(request.getAction());
+		auditEvent.setAccessResult((short)(result.getIsAllowed() ? 1 : 0));
+		auditEvent.setClientIP(request.getClientIPAddress());
+		auditEvent.setEventTime(request.getAccessTime());
+		auditEvent.setAclEnforcer(RangerModuleName);
+		auditEvent.setPolicyId(result.getPolicyId());
+		auditEvent.setRepositoryType(result.getServiceType());
+		auditEvent.setRepositoryName(result.getServiceName());
+		auditEvent.setResultReason(resourcePath);
+	}
+
+	public void logHadoopEvent(String path, FsAction action, boolean accessGranted) {
+		auditEvent.setResultReason(path);
+		auditEvent.setAccessResult((short) (accessGranted ? 1 : 0));
+		auditEvent.setAccessType(action == null ? null : action.toString());
+		auditEvent.setAclEnforcer(HadoopModuleName);
+		auditEvent.setPolicyId(-1);
+	}
+
+	public void flushAudit() {
+		if(! isAuditEnabled || StringUtils.isEmpty(auditEvent.getAccessType())) {
+			return;
+		}
+
+		String username = auditEvent.getUser();
+
+		boolean skipLog = (username != null && excludeUsers != null && excludeUsers.contains(username)) ;
+		
+		if (skipLog) {
+			return ;
+		}
+
+		super.logAuthzAudit(auditEvent);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/hdfs-agent/src/test/java/org/apache/ranger/authorization/hadoop/agent/TestRangerFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hdfs-agent/src/test/java/org/apache/ranger/authorization/hadoop/agent/TestRangerFSPermissionChecker.java b/hdfs-agent/src/test/java/org/apache/ranger/authorization/hadoop/agent/TestRangerFSPermissionChecker.java
deleted file mode 100644
index 91f24b1..0000000
--- a/hdfs-agent/src/test/java/org/apache/ranger/authorization/hadoop/agent/TestRangerFSPermissionChecker.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.ranger.authorization.hadoop.agent;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import java.util.Set;
-
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.RangerFSPermissionChecker;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.ranger.authorization.hadoop.exceptions.RangerAccessControlException;
-import org.junit.Test;
-
-public class TestRangerFSPermissionChecker {
-
-	@Test
-	public void nullUgiToCheckReturnsFalse() {
-
-		UserGroupInformation ugi = null;
-		INode inode = null;
-		FsAction access = null;
-		try {
-			boolean result = RangerFSPermissionChecker.check(ugi, inode, access);
-			assertFalse(result);
-		} catch (RangerAccessControlException e) {
-			fail("Unexpected exception!");
-		} 
-	}
-
-	@Test
-	public void nullUserGroupsToCheckReturnsFalse() {
-		String      user   = null;
-		Set<String> groups = null;
-		INode       inode  = null;
-		FsAction    access = null;
-
-		try {
-			boolean result = RangerFSPermissionChecker.check(user, groups, inode, access);
-
-			assertFalse(result);
-		} catch (RangerAccessControlException e) {
-			fail("Unexpected exception!");
-		}
-	}
-	
-	@Test
-	public void authorizeAccess() {
-		String aPathName = null;
-		String aPathOwnerName = null;
-		String user = null;
-		Set<String> groups = null;
-		FsAction access = null;
-		try {
-			// null access returns false! 
-			assertFalse(RangerFSPermissionChecker.AuthorizeAccessForUser(aPathName, aPathOwnerName, access, user, groups));
-			// None access type returns true!
-			access = FsAction.NONE;
-			assertFalse(RangerFSPermissionChecker.AuthorizeAccessForUser(aPathName, aPathOwnerName, access, user, groups));
-		} catch (RangerAccessControlException e) {
-			e.printStackTrace();
-			fail("Unexpected exception!");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/49bebb59/src/main/assembly/hdfs-agent.xml
----------------------------------------------------------------------
diff --git a/src/main/assembly/hdfs-agent.xml b/src/main/assembly/hdfs-agent.xml
index df3f32e..593aedb 100644
--- a/src/main/assembly/hdfs-agent.xml
+++ b/src/main/assembly/hdfs-agent.xml
@@ -126,11 +126,5 @@
 		<destName>install.properties</destName>
 		<fileMode>755</fileMode>
 	</file>
-    <file>
-		<source>hdfs-agent/conf/xasecure-hadoop-env.sh</source>
-		<outputDirectory>/install/conf.templates/enable</outputDirectory>
-		<destName>set-hdfs-plugin-env.sh</destName>
-		<fileMode>755</fileMode>
-    </file>	
   </files>
 </assembly>