You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ab...@apache.org on 2021/01/11 17:53:20 UTC

[ranger] branch ranger-2.2 updated: RANGER-3147: enhance resource-trie to enable finding evaluators for a given resource and its children

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

abhay pushed a commit to branch ranger-2.2
in repository https://gitbox.apache.org/repos/asf/ranger.git


The following commit(s) were added to refs/heads/ranger-2.2 by this push:
     new 0b36db3  RANGER-3147: enhance resource-trie to enable finding evaluators for a given resource and its children
0b36db3 is described below

commit 0b36db3c7e53c8e11593901859001d7d856ba188
Author: Abhay Kulkarni <ab...@apache.org>
AuthorDate: Mon Jan 11 09:20:52 2021 -0800

    RANGER-3147: enhance resource-trie to enable finding evaluators for a given resource and its children
---
 .../plugin/contextenricher/RangerTagEnricher.java  |  18 +-
 .../plugin/policyengine/RangerAccessRequest.java   |   2 +-
 .../policyengine/RangerPolicyEngineImpl.java       |  16 +-
 .../policyengine/RangerPolicyRepository.java       |  48 ++--
 .../plugin/policyengine/RangerResourceTrie.java    | 237 ++++++++++------
 .../RangerDefaultPolicyEvaluator.java              |   7 +
 .../RangerAbstractResourceMatcher.java             |   7 +
 .../resourcematcher/RangerPathResourceMatcher.java | 300 ++++++++++++++-------
 .../resourcematcher/RangerURLResourceMatcher.java  |  18 ++
 .../plugin/policyengine/TestPathResourceTrie.java  | 184 +++++++++++++
 .../plugin/policyengine/TestPolicyEngine.java      |   7 +
 .../policyengine/test_policyengine_aws.json        | 134 +++++++++
 .../apache/ranger/biz/RangerPolicyAdminImpl.java   |  11 +-
 13 files changed, 782 insertions(+), 207 deletions(-)

diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
index f31b2c5..97b9e2c 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
@@ -32,6 +32,7 @@ import org.apache.ranger.plugin.model.RangerServiceResource;
 import org.apache.ranger.plugin.model.RangerTag;
 import org.apache.ranger.plugin.model.validation.RangerServiceDefHelper;
 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.RangerResourceTrie;
@@ -500,7 +501,10 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 				LOG.debug("RangerAccessResource:[" + accessResource + "] created to represent service-resource[" + serviceResource + "] to find evaluators from trie-map");
 			}
 
-			List<RangerServiceResourceMatcher> oldMatchers = getEvaluators(accessResource, enrichedServiceTags);
+			RangerAccessRequestImpl  request = new RangerAccessRequestImpl();
+			request.setResource(accessResource);
+
+			List<RangerServiceResourceMatcher> oldMatchers = getEvaluators(request, enrichedServiceTags);
 
 			if (LOG.isDebugEnabled()) {
 				LOG.debug("Found [" + oldMatchers.size() + "] matchers for service-resource[" + serviceResource + "]");
@@ -621,7 +625,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 			ret = enrichedServiceTags.getTagsForEmptyResourceAndAnyAccess();
 		} else {
 
-			final List<RangerServiceResourceMatcher> serviceResourceMatchers = getEvaluators(resource, enrichedServiceTags);
+			final List<RangerServiceResourceMatcher> serviceResourceMatchers = getEvaluators(request, enrichedServiceTags);
 
 			if (CollectionUtils.isNotEmpty(serviceResourceMatchers)) {
 
@@ -667,12 +671,14 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		return ret;
 	}
 
-	private List<RangerServiceResourceMatcher> getEvaluators(RangerAccessResource resource, EnrichedServiceTags enrichedServiceTags) {
+	private List<RangerServiceResourceMatcher> getEvaluators(RangerAccessRequest request, EnrichedServiceTags enrichedServiceTags) {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerTagEnricher.getEvaluators(" + (resource != null ? resource.getAsString() : null) + ")");
+			LOG.debug("==> RangerTagEnricher.getEvaluators(request=" + request + ")");
 		}
 		List<RangerServiceResourceMatcher>  ret        = Collections.EMPTY_LIST;
 
+		RangerAccessResource                resource   = request.getResource();
+
 		final Map<String, RangerResourceTrie<RangerServiceResourceMatcher>> serviceResourceTrie = enrichedServiceTags.getServiceResourceTrie();
 
 		if (resource == null || resource.getKeys() == null || resource.getKeys().isEmpty() || serviceResourceTrie == null) {
@@ -696,7 +702,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 						continue;
 					}
 
-					Set<RangerServiceResourceMatcher> serviceResourceMatchersForResource = trie.getEvaluatorsForResource(resource.getValue(resourceName));
+					Set<RangerServiceResourceMatcher> serviceResourceMatchersForResource = trie.getEvaluatorsForResource(resource.getValue(resourceName), request.getResourceMatchingScope());
 					Set<RangerServiceResourceMatcher> inheritedResourceMatchers = trie.getInheritedEvaluators();
 
 					if (smallestList != null) {
@@ -754,7 +760,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerTagEnricher.getEvaluators(" + (resource != null ? resource.getAsString() : null) + "): evaluatorCount=" + ret.size());
+			LOG.debug("<== RangerTagEnricher.getEvaluators(request=" + request + "): evaluatorCount=" + ret.size());
 		}
 
 		return ret;
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerAccessRequest.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerAccessRequest.java
index 4a12168..57804ad 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerAccessRequest.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerAccessRequest.java
@@ -65,5 +65,5 @@ public interface RangerAccessRequest {
 
 	ResourceMatchingScope getResourceMatchingScope();
 
-	enum ResourceMatchingScope {SELF, SELF_OR_DESCENDANTS}
+	enum ResourceMatchingScope {SELF, SELF_OR_DESCENDANTS, SELF_OR_CHILD}
 }
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngineImpl.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngineImpl.java
index 940cc65..4e0c98e 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngineImpl.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngineImpl.java
@@ -582,7 +582,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 				ret.setIsAccessDetermined(false); // discard result by tag-policies, to evaluate resource policies for possible override
 			}
 
-			List<RangerPolicyEvaluator> evaluators = policyRepository.getLikelyMatchPolicyEvaluators(request.getResource(), policyType);
+			List<RangerPolicyEvaluator> evaluators = policyRepository.getLikelyMatchPolicyEvaluators(request, policyType);
 
 			for (RangerPolicyEvaluator evaluator : evaluators) {
 				if (!evaluator.isApplicable(accessTime)) {
@@ -645,7 +645,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 
 		Date                        accessTime       = request.getAccessTime() != null ? request.getAccessTime() : new Date();
 		Set<RangerTagForEval>       tags             = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
-		List<PolicyEvaluatorForTag> policyEvaluators = tagPolicyRepository == null ? null : tagPolicyRepository.getLikelyMatchPolicyEvaluators(tags, policyType, accessTime);
+		List<PolicyEvaluatorForTag> policyEvaluators = tagPolicyRepository == null ? null : tagPolicyRepository.getLikelyMatchPolicyEvaluators(request, tags, policyType, accessTime);
 
 		if (CollectionUtils.isNotEmpty(policyEvaluators)) {
 			final boolean useTagPoliciesFromDefaultZone = !policyEngine.isResourceZoneAssociatedWithTagService(zoneName);
@@ -793,7 +793,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 			LOG.error("policyRepository for zoneName:[" + zoneName + "],  serviceName:[" + policyEngine.getPolicyRepository().getServiceName() + "], policyVersion:[" + getPolicyVersion() + "] is null!! ERROR!");
 		} else {
 			Set<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
-			List<PolicyEvaluatorForTag> tagPolicyEvaluators = policyEngine.getTagPolicyRepository() == null ? null : policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(tags, RangerPolicy.POLICY_TYPE_ACCESS, null);
+			List<PolicyEvaluatorForTag> tagPolicyEvaluators = policyEngine.getTagPolicyRepository() == null ? null : policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(request, tags, RangerPolicy.POLICY_TYPE_ACCESS, null);
 
 			if (CollectionUtils.isNotEmpty(tagPolicyEvaluators)) {
 
@@ -832,7 +832,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 				}
 			}
 
-			List<RangerPolicyEvaluator> resourcePolicyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request.getResource(), RangerPolicy.POLICY_TYPE_ACCESS);
+			List<RangerPolicyEvaluator> resourcePolicyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request, RangerPolicy.POLICY_TYPE_ACCESS);
 
 			allEvaluators.addAll(resourcePolicyEvaluators);
 		}
@@ -854,7 +854,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 
 					for (RangerTagForEval tag : tags) {
 						RangerAccessRequest         tagEvalRequest = new RangerTagAccessRequest(tag, policyEngine.getTagPolicyRepository().getServiceDef(), request);
-						List<RangerPolicyEvaluator> evaluators     = policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(tagEvalRequest.getResource(), RangerPolicy.POLICY_TYPE_ACCESS);
+						List<RangerPolicyEvaluator> evaluators     = policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(tagEvalRequest, RangerPolicy.POLICY_TYPE_ACCESS);
 
 						for (RangerPolicyEvaluator evaluator : evaluators) {
 							String policyZoneName = evaluator.getPolicy().getZoneName();
@@ -883,7 +883,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 				}
 			}
 
-			List<RangerPolicyEvaluator> resPolicyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request.getResource(), RangerPolicy.POLICY_TYPE_ACCESS);
+			List<RangerPolicyEvaluator> resPolicyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request, RangerPolicy.POLICY_TYPE_ACCESS);
 
 			if (CollectionUtils.isNotEmpty(resPolicyEvaluators)) {
 				for (RangerPolicyEvaluator evaluator : resPolicyEvaluators) {
@@ -905,7 +905,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 
 		if (CollectionUtils.isNotEmpty(tags)) {
 			Date                        accessTime = request.getAccessTime() != null ? request.getAccessTime() : new Date();
-			List<PolicyEvaluatorForTag> evaluators = tagPolicyRepository.getLikelyMatchPolicyEvaluators(tags, RangerPolicy.POLICY_TYPE_AUDIT, accessTime);
+			List<PolicyEvaluatorForTag> evaluators = tagPolicyRepository.getLikelyMatchPolicyEvaluators(request, tags, RangerPolicy.POLICY_TYPE_AUDIT, accessTime);
 
 			if (CollectionUtils.isNotEmpty(evaluators)) {
 				for (PolicyEvaluatorForTag policyEvaluator : evaluators) {
@@ -943,7 +943,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 		}
 
 		boolean                     ret        = false;
-		List<RangerPolicyEvaluator> evaluators = policyRepository.getLikelyMatchAuditPolicyEvaluators(request.getResource());
+		List<RangerPolicyEvaluator> evaluators = policyRepository.getLikelyMatchAuditPolicyEvaluators(request);
 
 		if (CollectionUtils.isNotEmpty(evaluators)) {
 			for (RangerPolicyEvaluator evaluator : evaluators) {
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyRepository.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyRepository.java
index 2655103..b2f3438 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyRepository.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyRepository.java
@@ -676,7 +676,7 @@ public class RangerPolicyRepository {
 
     RangerPolicyEngineOptions getOptions() { return options; }
 
-    List<PolicyEvaluatorForTag> getLikelyMatchPolicyEvaluators(Set<RangerTagForEval> tags, int policyType, Date accessTime) {
+    List<PolicyEvaluatorForTag> getLikelyMatchPolicyEvaluators(RangerAccessRequest request, Set<RangerTagForEval> tags, int policyType, Date accessTime) {
         List<PolicyEvaluatorForTag> ret = Collections.EMPTY_LIST;
 
         if (CollectionUtils.isNotEmpty(tags) && getServiceDef() != null) {
@@ -685,8 +685,8 @@ public class RangerPolicyRepository {
 
             for (RangerTagForEval tag : tags) {
             	if (tag.isApplicable(accessTime)) {
-		            RangerAccessResource resource = new RangerTagResource(tag.getType(), getServiceDef());
-		            List<RangerPolicyEvaluator> evaluators = getLikelyMatchPolicyEvaluators(resource, policyType);
+		            RangerAccessRequest  tagRequest  = new RangerTagAccessRequest(tag, getServiceDef(), request);
+		            List<RangerPolicyEvaluator> evaluators = getLikelyMatchPolicyEvaluators(tagRequest, policyType);
 
 		            if (CollectionUtils.isNotEmpty(evaluators)) {
 			            for (RangerPolicyEvaluator evaluator : evaluators) {
@@ -723,11 +723,11 @@ public class RangerPolicyRepository {
         return ret;
     }
 
-    public List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(RangerAccessResource resource) {
+    public List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(RangerAccessRequest request) {
         List<RangerPolicyEvaluator> ret = new ArrayList<>();
 
         for (int policyType : RangerPolicy.POLICY_TYPES) {
-            List<RangerPolicyEvaluator> evaluators = getLikelyMatchPolicyEvaluators(resource, policyType);
+            List<RangerPolicyEvaluator> evaluators = getLikelyMatchPolicyEvaluators(request, policyType);
             if (CollectionUtils.isNotEmpty(evaluators)) {
                 ret.addAll(evaluators);
             }
@@ -735,16 +735,16 @@ public class RangerPolicyRepository {
         return ret;
     }
 
-    public List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(RangerAccessResource resource, int policyType) {
+    public List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(RangerAccessRequest request, int policyType) {
         switch (policyType) {
             case RangerPolicy.POLICY_TYPE_ACCESS:
-                return getLikelyMatchAccessPolicyEvaluators(resource);
+                return getLikelyMatchAccessPolicyEvaluators(request);
             case RangerPolicy.POLICY_TYPE_DATAMASK:
-                return getLikelyMatchDataMaskPolicyEvaluators(resource);
+                return getLikelyMatchDataMaskPolicyEvaluators(request);
             case RangerPolicy.POLICY_TYPE_ROWFILTER:
-                return getLikelyMatchRowFilterPolicyEvaluators(resource);
+                return getLikelyMatchRowFilterPolicyEvaluators(request);
             case RangerPolicy.POLICY_TYPE_AUDIT:
-                return getLikelyMatchAuditPolicyEvaluators(resource);
+                return getLikelyMatchAuditPolicyEvaluators(request);
             default:
                 return Collections.EMPTY_LIST;
         }
@@ -757,33 +757,39 @@ public class RangerPolicyRepository {
         return policyEvaluatorsMap.get(id);
     }
 
-    private List<RangerPolicyEvaluator> getLikelyMatchAccessPolicyEvaluators(RangerAccessResource resource) {
-       String resourceStr = resource == null ? null : resource.getAsString();
+    private List<RangerPolicyEvaluator> getLikelyMatchAccessPolicyEvaluators(RangerAccessRequest request) {
+        RangerAccessResource resource = request.getResource();
+        String resourceStr = resource == null ? null : resource.getAsString();
 
-       return policyResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getPolicyEvaluators() : getLikelyMatchPolicyEvaluators(policyResourceTrie, resource);
+        return policyResourceTrie == null || StringUtils.isEmpty(resourceStr) ? getPolicyEvaluators() : getLikelyMatchPolicyEvaluators(policyResourceTrie, request);
     }
 
-    private List<RangerPolicyEvaluator> getLikelyMatchDataMaskPolicyEvaluators(RangerAccessResource resource) {
+    private List<RangerPolicyEvaluator> getLikelyMatchDataMaskPolicyEvaluators(RangerAccessRequest request) {
+        RangerAccessResource resource = request.getResource();
         String resourceStr = resource == null ? null : resource.getAsString();
 
-        return dataMaskResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getDataMaskPolicyEvaluators() : getLikelyMatchPolicyEvaluators(dataMaskResourceTrie, resource);
+        return dataMaskResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getDataMaskPolicyEvaluators() : getLikelyMatchPolicyEvaluators(dataMaskResourceTrie, request);
     }
 
-    private List<RangerPolicyEvaluator> getLikelyMatchRowFilterPolicyEvaluators(RangerAccessResource resource) {
+    private List<RangerPolicyEvaluator> getLikelyMatchRowFilterPolicyEvaluators(RangerAccessRequest request) {
+        RangerAccessResource resource = request.getResource();
         String resourceStr = resource == null ? null : resource.getAsString();
 
-        return rowFilterResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getRowFilterPolicyEvaluators() : getLikelyMatchPolicyEvaluators(rowFilterResourceTrie, resource);
+        return rowFilterResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getRowFilterPolicyEvaluators() : getLikelyMatchPolicyEvaluators(rowFilterResourceTrie, request);
     }
 
-    List<RangerPolicyEvaluator> getLikelyMatchAuditPolicyEvaluators(RangerAccessResource resource) {
+    List<RangerPolicyEvaluator> getLikelyMatchAuditPolicyEvaluators(RangerAccessRequest request) {
+        RangerAccessResource resource = request.getResource();
         String resourceStr = resource == null ? null : resource.getAsString();
 
-        return auditFilterResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getAuditPolicyEvaluators() : getLikelyMatchPolicyEvaluators(auditFilterResourceTrie, resource);
+        return auditFilterResourceTrie == null || StringUtils.isEmpty(resourceStr)  ? getAuditPolicyEvaluators() : getLikelyMatchPolicyEvaluators(auditFilterResourceTrie, request);
     }
 
-    private List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(Map<String, RangerResourceTrie> resourceTrie, RangerAccessResource resource) {
+    private List<RangerPolicyEvaluator> getLikelyMatchPolicyEvaluators(Map<String, RangerResourceTrie> resourceTrie, RangerAccessRequest request) {
         List<RangerPolicyEvaluator> ret          = Collections.EMPTY_LIST;
 
+        RangerAccessResource              resource = request.getResource();
+
         RangerPerfTracer perf = null;
 
         if(RangerPerfTracer.isPerfTraceEnabled(PERF_TRIE_OP_LOG)) {
@@ -802,7 +808,7 @@ public class RangerPolicyRepository {
                     continue;
                 }
 
-                Set<RangerPolicyEvaluator> serviceResourceMatchersForResource = trie.getEvaluatorsForResource(resource.getValue(resourceName));
+                Set<RangerPolicyEvaluator> serviceResourceMatchersForResource = trie.getEvaluatorsForResource(resource.getValue(resourceName), request.getResourceMatchingScope());
                 Set<RangerPolicyEvaluator> inheritedResourceMatchers = trie.getInheritedEvaluators();
 
                 if (smallestList != null) {
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerResourceTrie.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerResourceTrie.java
index b41da47..6d784c2 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerResourceTrie.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerResourceTrie.java
@@ -26,11 +26,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyResource;
-import org.apache.ranger.plugin.model.RangerServiceDef;
+import org.apache.ranger.plugin.model.RangerServiceDef.RangerResourceDef;
 import org.apache.ranger.plugin.policyresourcematcher.RangerPolicyResourceEvaluator;
 import org.apache.ranger.plugin.resourcematcher.RangerAbstractResourceMatcher;
 import org.apache.ranger.plugin.resourcematcher.RangerResourceMatcher;
 import org.apache.ranger.plugin.util.RangerPerfTracer;
+import org.apache.ranger.plugin.util.ServiceDefUtil;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -42,24 +43,28 @@ import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import static org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher.DEFAULT_PATH_SEPARATOR_CHAR;
+import static org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher.OPTION_PATH_SEPARATOR;
+
 public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
-    private static final Log LOG = LogFactory.getLog(RangerResourceTrie.class);
-    private static final Log TRACE_LOG = RangerPerfTracer.getPerfLogger("resourcetrie.trace");
+    private static final Log LOG                = LogFactory.getLog(RangerResourceTrie.class);
+    private static final Log TRACE_LOG          = RangerPerfTracer.getPerfLogger("resourcetrie.trace");
     private static final Log PERF_TRIE_INIT_LOG = RangerPerfTracer.getPerfLogger("resourcetrie.init");
-    private static final Log PERF_TRIE_OP_LOG = RangerPerfTracer.getPerfLogger("resourcetrie.op");
+    private static final Log PERF_TRIE_OP_LOG   = RangerPerfTracer.getPerfLogger("resourcetrie.op");
 
-    private static final String DEFAULT_WILDCARD_CHARS = "*?";
+    private static final String DEFAULT_WILDCARD_CHARS    = "*?";
     private static final String TRIE_BUILDER_THREAD_COUNT = "ranger.policyengine.trie.builder.thread.count";
 
-    private final RangerServiceDef.RangerResourceDef resourceDef;
-    private final boolean optIgnoreCase;
-    private final boolean optWildcard;
-    private final String wildcardChars;
-    private final TrieNode<T> root;
-    private final boolean isOptimizedForRetrieval;
-    private       Set<T>  inheritedEvaluators;
+    private final RangerResourceDef resourceDef;
+    private final boolean           optIgnoreCase;
+    private final boolean           optWildcard;
+    private final String            wildcardChars;
+    private final boolean           isOptimizedForRetrieval;
+    private final Character         separatorChar;
+    private       Set<T>            inheritedEvaluators;
+    private final TrieNode<T>       root;
 
-    public RangerResourceTrie(RangerServiceDef.RangerResourceDef resourceDef, List<T> evaluators) {
+    public RangerResourceTrie(RangerResourceDef resourceDef, List<T> evaluators) {
         this(resourceDef, evaluators, true, null);
     }
 
@@ -70,19 +75,21 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             perf = RangerPerfTracer.getPerfTracer(PERF_TRIE_INIT_LOG, "RangerResourceTrie.copyTrie(name=" + other.resourceDef.getName() + ")");
         }
 
-        this.resourceDef = other.resourceDef;
-        this.optIgnoreCase = other.optIgnoreCase;
-        this.optWildcard = other.optWildcard;
-        this.wildcardChars = other.wildcardChars;
-        this.inheritedEvaluators = other.inheritedEvaluators != null ? new HashSet<>(other.inheritedEvaluators) : null;
+        this.resourceDef             = other.resourceDef;
+        this.optIgnoreCase           = other.optIgnoreCase;
+        this.optWildcard             = other.optWildcard;
+        this.wildcardChars           = other.wildcardChars;
         this.isOptimizedForRetrieval = false;
-        this.root = copyTrieSubtree(other.root, null);
+        this.separatorChar           = other.separatorChar;
+        this.inheritedEvaluators     = other.inheritedEvaluators != null ? new HashSet<>(other.inheritedEvaluators) : null;
+        this.root                    = copyTrieSubtree(other.root, null);
 
         RangerPerfTracer.logAlways(perf);
 
         if (PERF_TRIE_INIT_LOG.isDebugEnabled()) {
             PERF_TRIE_INIT_LOG.debug(toString());
         }
+
         if (TRACE_LOG.isTraceEnabled()) {
             StringBuilder sb = new StringBuilder();
             root.toString("", sb);
@@ -90,7 +97,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
     }
 
-    public RangerResourceTrie(RangerServiceDef.RangerResourceDef resourceDef, List<T> evaluators, boolean isOptimizedForRetrieval, RangerPluginContext pluginContext) {
+    public RangerResourceTrie(RangerResourceDef resourceDef, List<T> evaluators, boolean isOptimizedForRetrieval, RangerPluginContext pluginContext) {
         if(LOG.isDebugEnabled()) {
             LOG.debug("==> RangerResourceTrie(" + resourceDef.getName() + ", evaluatorCount=" + evaluators.size() + ", isOptimizedForRetrieval=" + isOptimizedForRetrieval + ")");
         }
@@ -112,11 +119,9 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             TRACE_LOG.trace("builderThreadCount is set to [" + builderThreadCount + "]");
         }
 
-        Map<String, String> matcherOptions = resourceDef.getMatcherOptions();
-
-        boolean optReplaceTokens = RangerAbstractResourceMatcher.getOptionReplaceTokens(matcherOptions);
-
-        String tokenReplaceSpecialChars = "";
+        Map<String, String> matcherOptions           = resourceDef.getMatcherOptions();
+        boolean             optReplaceTokens         = RangerAbstractResourceMatcher.getOptionReplaceTokens(matcherOptions);
+        String              tokenReplaceSpecialChars = "";
 
         if(optReplaceTokens) {
             char delimiterStart  = RangerAbstractResourceMatcher.getOptionDelimiterStart(matcherOptions);
@@ -128,11 +133,12 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             tokenReplaceSpecialChars += delimiterEscape;
         }
 
-        this.resourceDef   = resourceDef;
-        this.optIgnoreCase = RangerAbstractResourceMatcher.getOptionIgnoreCase(matcherOptions);
-        this.optWildcard   = RangerAbstractResourceMatcher.getOptionWildCard(matcherOptions);
-        this.wildcardChars = optWildcard ? DEFAULT_WILDCARD_CHARS + tokenReplaceSpecialChars : "" + tokenReplaceSpecialChars;
+        this.resourceDef             = resourceDef;
+        this.optIgnoreCase           = RangerAbstractResourceMatcher.getOptionIgnoreCase(matcherOptions);
+        this.optWildcard             = RangerAbstractResourceMatcher.getOptionWildCard(matcherOptions);
+        this.wildcardChars           = optWildcard ? DEFAULT_WILDCARD_CHARS + tokenReplaceSpecialChars : "" + tokenReplaceSpecialChars;
         this.isOptimizedForRetrieval = isOptimizedForRetrieval;
+        this.separatorChar           = ServiceDefUtil.getCharOption(matcherOptions, OPTION_PATH_SEPARATOR, DEFAULT_PATH_SEPARATOR_CHAR);
 
         TrieNode<T> tmpRoot = buildTrie(resourceDef, evaluators, builderThreadCount);
 
@@ -166,16 +172,20 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
     }
 
     public Set<T> getEvaluatorsForResource(Object resource) {
+        return getEvaluatorsForResource(resource, RangerAccessRequest.ResourceMatchingScope.SELF);
+    }
+
+    public Set<T> getEvaluatorsForResource(Object resource, RangerAccessRequest.ResourceMatchingScope scope) {
         if (resource instanceof String) {
-            return getEvaluatorsForResource((String) resource);
+            return getEvaluatorsForResource((String) resource, scope);
         } else if (resource instanceof Collection) {
             if (CollectionUtils.isEmpty((Collection) resource)) {  // treat empty collection same as empty-string
-                return getEvaluatorsForResource("");
+                return getEvaluatorsForResource("", scope);
             } else {
                 @SuppressWarnings("unchecked")
                 Collection<String> resources = (Collection<String>) resource;
 
-                return getEvaluatorsForResources(resources);
+                return getEvaluatorsForResources(resources, scope);
             }
         }
 
@@ -183,7 +193,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
     }
 
     public void add(RangerPolicyResource resource, T evaluator) {
-
         RangerPerfTracer perf = null;
 
         if(RangerPerfTracer.isPerfTraceEnabled(PERF_TRIE_INIT_LOG)) {
@@ -207,6 +216,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
 
         RangerPerfTracer.logAlways(perf);
+
         if (TRACE_LOG.isTraceEnabled()) {
             StringBuilder sb = new StringBuilder();
             root.toString("", sb);
@@ -215,7 +225,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
     }
 
     public void delete(RangerPolicyResource resource, T evaluator) {
-
         RangerPerfTracer perf = null;
 
         if(RangerPerfTracer.isPerfTraceEnabled(PERF_TRIE_INIT_LOG)) {
@@ -238,6 +247,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
 
         RangerPerfTracer.logAlways(perf);
+
         if (TRACE_LOG.isTraceEnabled()) {
             StringBuilder sb = new StringBuilder();
             root.toString("", sb);
@@ -278,13 +288,15 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             source.toString(sb);
             TRACE_LOG.trace("==> copyTrieSubtree(" + sb + ")");
         }
+
         TrieNode<T> dest = new TrieNode<>(source.str);
+
         if (parent != null) {
             parent.addChild(dest);
         }
 
         synchronized (source.children) {
-            dest.isSetup = source.isSetup;
+            dest.isSetup                           = source.isSetup;
             dest.isSharingParentWildcardEvaluators = source.isSharingParentWildcardEvaluators;
 
             if (source.isSharingParentWildcardEvaluators) {
@@ -300,6 +312,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                     dest.wildcardEvaluators = null;
                 }
             }
+
             if (source.evaluators != null) {
                 if (source.evaluators == source.wildcardEvaluators) {
                     dest.evaluators = dest.wildcardEvaluators;
@@ -312,6 +325,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
 
         Map<Character, TrieNode<T>> children = source.getChildren();
+
         for (Map.Entry<Character, TrieNode<T>> entry : children.entrySet()) {
             copyTrieSubtree(entry.getValue(), dest);
         }
@@ -323,10 +337,11 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
 
             TRACE_LOG.trace("<== copyTrieSubtree(" + sourceAsString + ") : " + destAsString);
         }
+
         return dest;
     }
 
-    private TrieNode<T> buildTrie(RangerServiceDef.RangerResourceDef resourceDef, List<T> evaluators, int builderThreadCount) {
+    private TrieNode<T> buildTrie(RangerResourceDef resourceDef, List<T> evaluators, int builderThreadCount) {
         if(LOG.isDebugEnabled()) {
             LOG.debug("==> buildTrie(" + resourceDef.getName() + ", evaluatorCount=" + evaluators.size() + ", isMultiThreaded=" + (builderThreadCount > 1) + ")");
         }
@@ -337,30 +352,33 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             perf = RangerPerfTracer.getPerfTracer(PERF_TRIE_INIT_LOG, "RangerResourceTrie.init(resourceDef=" + resourceDef.getName() + ")");
         }
 
-        TrieNode<T>                           ret                 = new TrieNode<>(null);
+        TrieNode<T>                           ret             = new TrieNode<>(null);
         final boolean                         isMultiThreaded = builderThreadCount > 1;
         final List<ResourceTrieBuilderThread> builderThreads;
         final Map<Character, Integer>         builderThreadMap;
-        final String                          resourceName = resourceDef.getName();
+        final String                          resourceName        = resourceDef.getName();
         int                                   lastUsedThreadIndex = 0;
 
         if (isMultiThreaded) {
             builderThreads = new ArrayList<>();
+
             for (int i = 0; i < builderThreadCount; i++) {
                 ResourceTrieBuilderThread t = new ResourceTrieBuilderThread();
+
                 t.setDaemon(true);
                 builderThreads.add(t);
                 t.start();
             }
+
             builderThreadMap = new HashMap<>();
         } else {
-            builderThreads = null;
+            builderThreads   = null;
             builderThreadMap = null;
         }
 
         for (T evaluator : evaluators) {
             Map<String, RangerPolicyResource> policyResources = evaluator.getPolicyResource();
-            RangerPolicyResource policyResource = policyResources != null ? policyResources.get(resourceName) : null;
+            RangerPolicyResource              policyResource  = policyResources != null ? policyResources.get(resourceName) : null;
 
             if (policyResource == null) {
                 if (evaluator.isAncestorOf(resourceDef)) {
@@ -395,6 +413,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                                 }
                             }
                         }
+
                         if (ret == null) {
                             break;
                         }
@@ -402,9 +421,9 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                 }
             }
         }
+
         if (ret != null) {
             if (isMultiThreaded) {
-
                 for (ResourceTrieBuilderThread t : builderThreads) {
                     try {
                         // Send termination signal to each thread
@@ -421,6 +440,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                         break;
                     }
                 }
+
                 cleanUpThreads(builderThreads);
             }
         }
@@ -453,6 +473,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         TrieData ret = new TrieData();
 
         root.populateTrieData(ret);
+
         ret.maxDepth = getMaxDepth();
 
         return ret;
@@ -492,7 +513,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
     }
 
     private void insert(TrieNode<T> currentRoot, String resource, boolean isRecursive, T evaluator) {
-
         TrieNode<T>   curr       = currentRoot;
         final String  prefix     = getNonWildcardPrefix(resource);
         final boolean isWildcard = prefix.length() != resource.length();
@@ -510,7 +530,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
     }
 
     private String getNonWildcardPrefix(String str) {
-
         int minIndex = str.length();
 
         for (int i = 0; i < wildcardChars.length(); i++) {
@@ -524,9 +543,9 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         return str.substring(0, minIndex);
     }
 
-    private Set<T> getEvaluatorsForResource(String resource) {
+    private Set<T> getEvaluatorsForResource(String resource, RangerAccessRequest.ResourceMatchingScope scope) {
         if(LOG.isDebugEnabled()) {
-            LOG.debug("==> RangerResourceTrie.getEvaluatorsForResource(" + resource + ")");
+            LOG.debug("==> RangerResourceTrie.getEvaluatorsForResource(" + resource + ", " + scope + ")");
         }
 
         RangerPerfTracer perf = null;
@@ -537,6 +556,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
 
         TrieNode<T> curr   = root;
         TrieNode<T> parent = null;
+        TrieNode<T> child  = null;
         final int   len    = resource.length();
         int         i      = 0;
 
@@ -545,7 +565,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                 curr.setupIfNeeded(parent);
             }
 
-            final TrieNode<T> child = curr.getChild(getLookupChar(resource, i));
+            child = curr.getChild(getLookupChar(resource, i));
 
             if (child == null) {
                 break;
@@ -558,20 +578,58 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             }
 
             parent = curr;
-            curr = child;
-            i += childStr.length();
+            curr   = child;
+            i      += childStr.length();
         }
 
         if (!isOptimizedForRetrieval) {
             curr.setupIfNeeded(parent);
         }
 
-        Set<T> ret = i == len ? curr.getEvaluators() : curr.getWildcardEvaluators();
+        boolean isSelfMatch                       = (i == len);
+        Set<T>  ret                               = isSelfMatch ? curr.getEvaluators() : curr.getWildcardEvaluators();
+        boolean includeEvaluatorsOfChildResources = scope == RangerAccessRequest.ResourceMatchingScope.SELF_OR_CHILD;
+
+        if (includeEvaluatorsOfChildResources) {
+            final Set<T>  childEvalautors     = new HashSet<>();
+            final boolean resourceEndsWithSep = resource.charAt(resource.length() - 1) == separatorChar;
+
+            if (isSelfMatch) { // resource == path(curr)
+                if (resourceEndsWithSep) { // ex: resource=/tmp/
+                    curr.getChildren().values().stream().forEach(c -> c.collectChildEvaluators(separatorChar, 0, childEvalautors));
+                } else { // ex: resource=/tmp
+                    curr = curr.getChild(separatorChar);
+
+                    if (curr != null) {
+                        curr.collectChildEvaluators(separatorChar, 1, childEvalautors);
+                    }
+                }
+            } else if (child != null) { // resource != path(child) ex: (resource=/tmp, path(child)=/tmp/test.txt or path(child)=/tmpdir)
+                int     remainingLen  = len - i;
+                boolean isPrefixMatch = child.getStr().regionMatches(optIgnoreCase, 0, resource, i, remainingLen);
+
+                if (isPrefixMatch) {
+                    if (resourceEndsWithSep) { // ex: resource=/tmp/
+                        child.collectChildEvaluators(separatorChar, remainingLen, childEvalautors);
+                    } else if (child.getStr().charAt(remainingLen) == separatorChar) { //  ex: resource=/tmp
+                        child.collectChildEvaluators(separatorChar, remainingLen + 1, childEvalautors);
+                    }
+                }
+            }
+
+            if (CollectionUtils.isNotEmpty(childEvalautors)) {
+                if (CollectionUtils.isNotEmpty(ret)) {
+                    childEvalautors.addAll(ret);
+                }
+
+                ret = childEvalautors;
+            }
+        }
 
         RangerPerfTracer.logAlways(perf);
 
         if(LOG.isDebugEnabled()) {
-            LOG.debug("<== RangerResourceTrie.getEvaluatorsForResource(" + resource + "): evaluatorCount=" + (ret == null ? 0 : ret.size()));
+            LOG.debug("<== RangerResourceTrie.getEvaluatorsForResource(" + resource + ", " + scope + "): evaluatorCount=" + (ret == null ? 0 : ret.size()));
         }
 
         return ret;
@@ -588,12 +646,11 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             perf = RangerPerfTracer.getPerfTracer(PERF_TRIE_OP_LOG, "RangerResourceTrie.getNodeForResource(resource=" + resource + ")");
         }
 
-        TrieNode<T> curr   = root;
-        final int   len    = resource.length();
-        int         i      = 0;
+        TrieNode<T> curr = root;
+        final int   len  = resource.length();
+        int         i    = 0;
 
         while (i < len) {
-
             final TrieNode<T> child = curr.getChild(getLookupChar(resource, i));
 
             if (child == null) {
@@ -607,7 +664,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             }
 
             curr = child;
-            i += childStr.length();
+            i    += childStr.length();
         }
 
         RangerPerfTracer.logAlways(perf);
@@ -619,16 +676,16 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         return curr;
     }
 
-    private Set<T> getEvaluatorsForResources(Collection<String> resources) {
+    private Set<T> getEvaluatorsForResources(Collection<String> resources, RangerAccessRequest.ResourceMatchingScope scope) {
         if(LOG.isDebugEnabled()) {
             LOG.debug("==> RangerResourceTrie.getEvaluatorsForResources(" + resources + ")");
         }
 
-        Set<T>      ret           = null;
+        Set<T>       ret           = null;
         Map<Long, T> evaluatorsMap = null;
 
         for (String resource : resources) {
-            Set<T> resourceEvaluators = getEvaluatorsForResource(resource);
+            Set<T> resourceEvaluators = getEvaluatorsForResource(resource, scope);
 
             if (CollectionUtils.isEmpty(resourceEvaluators)) {
                 continue;
@@ -696,10 +753,11 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             final T       evaluator;
 
             WorkItem(String resourceName, boolean isRecursive, T evaluator) {
-                this.resourceName   = resourceName;
-                this.isRecursive    = isRecursive;
-                this.evaluator      = evaluator;
+                this.resourceName = resourceName;
+                this.isRecursive  = isRecursive;
+                this.evaluator    = evaluator;
             }
+
             @Override
             public String toString() {
                 return
@@ -709,8 +767,8 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             }
         }
 
-        private final   TrieNode<T>             thisRoot  = new TrieNode<>(null);
-        private final   BlockingQueue<WorkItem> workQueue = new LinkedBlockingQueue<>();
+        private final TrieNode<T>             thisRoot  = new TrieNode<>(null);
+        private final BlockingQueue<WorkItem> workQueue = new LinkedBlockingQueue<>();
 
         ResourceTrieBuilderThread() {
         }
@@ -744,6 +802,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                     if (LOG.isDebugEnabled()) {
                         LOG.debug("Received termination signal. " + workItem);
                     }
+
                     break;
                 }
             }
@@ -861,35 +920,38 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
 
         TrieNode<U> getOrCreateChild(String str) {
-            int len = str.length();
-
+            int         len   = str.length();
             TrieNode<U> child = children.get(getLookupChar(str, 0));
 
             if (child == null) {
                 child = new TrieNode<>(str);
+
                 addChild(child);
             } else {
-                final String childStr = child.getStr();
-                final int childStrLen = childStr.length();
-
+                final String  childStr     = child.getStr();
+                final int     childStrLen  = childStr.length();
                 final boolean isExactMatch = optIgnoreCase ? StringUtils.equalsIgnoreCase(childStr, str) : StringUtils.equals(childStr, str);
 
                 if (!isExactMatch) {
                     final int numOfCharactersToMatch = Math.min(childStrLen, len);
-                    int index = 1;
+                    int       index                  = 1;
+
                     for (; index < numOfCharactersToMatch; index++) {
                         if (getLookupChar(childStr, index) != getLookupChar(str, index)) {
                             break;
                         }
                     }
+
                     if (index == numOfCharactersToMatch) {
                         // Matched all
                         if (childStrLen > len) {
                             // Existing node has longer string, need to break up this node
                             TrieNode<U> newChild = new TrieNode<>(str);
+
                             this.addChild(newChild);
                             child.setStr(childStr.substring(index));
                             newChild.addChild(child);
+
                             child = newChild;
                         } else {
                             // This is a longer string, build a child with leftover string
@@ -897,11 +959,13 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                         }
                     } else {
                         // Partial match for both; both have leftovers
-                        String matchedPart = str.substring(0, index);
-                        TrieNode<U> newChild = new TrieNode<>(matchedPart);
+                        String      matchedPart = str.substring(0, index);
+                        TrieNode<U> newChild    = new TrieNode<>(matchedPart);
+
                         this.addChild(newChild);
                         child.setStr(childStr.substring(index));
                         newChild.addChild(child);
+
                         child = newChild.getOrCreateChild(str.substring(index));
                     }
                 }
@@ -919,6 +983,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             if (evaluators == null) {
                 evaluators = new HashSet<>();
             }
+
             evaluators.add(evaluator);
         }
 
@@ -936,6 +1001,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         void removeEvaluator(U evaluator) {
             if (CollectionUtils.isNotEmpty(evaluators) && evaluators.contains(evaluator)) {
                 evaluators.remove(evaluator);
+
                 if (CollectionUtils.isEmpty(evaluators)) {
                     evaluators = null;
                 }
@@ -945,8 +1011,10 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         boolean removeWildcardEvaluator(U evaluator) {
             if (CollectionUtils.isNotEmpty(wildcardEvaluators) && wildcardEvaluators.contains(evaluator)) {
                 undoSetup();
+
                 if (CollectionUtils.isNotEmpty(wildcardEvaluators)) {
                     wildcardEvaluators.remove(evaluator);
+
                     if (CollectionUtils.isEmpty(wildcardEvaluators)) {
                         wildcardEvaluators = null;
                     }
@@ -962,6 +1030,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                 for (TrieNode<U> child : children.values()) {
                     child.undoSetup();
                 }
+
                 if (evaluators != null) {
                     if (evaluators == wildcardEvaluators) {
                         evaluators = null;
@@ -975,6 +1044,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                         }
                     }
                 }
+
                 if (wildcardEvaluators != null) {
                     if (isSharingParentWildcardEvaluators) {
                         wildcardEvaluators = null;
@@ -990,8 +1060,9 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                         }
                     }
                 }
+
                 isSharingParentWildcardEvaluators = false;
-                isSetup = false;
+                isSetup                           = false;
             }
         }
 
@@ -1010,7 +1081,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
         }
 
         void postSetup(Set<U> parentWildcardEvaluators) {
-
             setup(parentWildcardEvaluators);
 
             for (Map.Entry<Character, TrieNode<U>> entry : children.entrySet()) {
@@ -1018,11 +1088,9 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
 
                 child.postSetup(wildcardEvaluators);
             }
-
         }
 
         void setupIfNeeded(TrieNode<U> parent) {
-
             boolean setupNeeded = !isSetup;
 
             if (setupNeeded) {
@@ -1031,6 +1099,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
 
                     if (setupNeeded) {
                         setup(parent == null ? null : parent.getWildcardEvaluators());
+
                         if (TRACE_LOG.isTraceEnabled()) {
                             StringBuilder sb = new StringBuilder();
                             this.toString(sb);
@@ -1053,6 +1122,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                         }
                     }
                 }
+
                 this.isSharingParentWildcardEvaluators = wildcardEvaluators == parentWildcardEvaluators;
 
                 // finalize evaluators list by including wildcard evaluators
@@ -1070,6 +1140,20 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             }
         }
 
+        void collectChildEvaluators(Character sep, int startIdx, Set<U> childEvaluators) {
+            final int sepPos = startIdx < str.length() ? str.indexOf(sep, startIdx) : -1;
+
+            if (sepPos == -1) { // ex: startIdx=5, path(str)=/tmp/test, path(a child) could be: /tmp/test.txt, /tmp/test/, /tmp/test/a, /tmp/test/a/b
+                if (this.evaluators != null) {
+                    childEvaluators.addAll(this.evaluators);
+                }
+
+                children.values().stream().forEach(c -> c.collectChildEvaluators(sep, 0, childEvaluators));
+            } else if (sepPos == (str.length() - 1)) { // ex: str=/tmp/test/, startIdx=5
+                childEvaluators.addAll(this.evaluators);
+            }
+        }
+
         private void removeEvaluatorFromSubtree(U evaluator) {
             if (removeWildcardEvaluator(evaluator)) {
                 for (Map.Entry<Character, TrieNode<U>> entry : children.entrySet()) {
@@ -1078,7 +1162,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
             }
 
             removeEvaluator(evaluator);
-
         }
 
         void toString(StringBuilder sb) {
@@ -1102,6 +1185,7 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
                     sb.append(evaluator.getId()).append(" ");
                 }
             }
+            sb.append("]");
         }
 
         void toString(String prefix, StringBuilder sb) {
@@ -1116,7 +1200,6 @@ public class RangerResourceTrie<T extends RangerPolicyResourceEvaluator> {
 
                 child.toString(nodeValue, sb);
             }
-
         }
     }
 }
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerDefaultPolicyEvaluator.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerDefaultPolicyEvaluator.java
index 45774b9..a8ad632 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerDefaultPolicyEvaluator.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyevaluator/RangerDefaultPolicyEvaluator.java
@@ -239,13 +239,20 @@ public class RangerDefaultPolicyEvaluator extends RangerAbstractPolicyEvaluator
 						matchType = RangerPolicyResourceMatcher.MatchType.SELF;
 					}
 				} else {
+					if (request.getResourceMatchingScope() == RangerAccessRequest.ResourceMatchingScope.SELF_OR_CHILD) {
+						request.getContext().put("Scope", "SELF_OR_ONE_LEVEL");
+					}
 					matchType = resourceMatcher != null ? resourceMatcher.getMatchType(request.getResource(), request.getContext()) : RangerPolicyResourceMatcher.MatchType.NONE;
+					request.getContext().remove("Scope");
 				}
 
 				final boolean isMatched;
 
 				if (request.isAccessTypeAny()) {
 					isMatched = matchType != RangerPolicyResourceMatcher.MatchType.NONE;
+					if (request.getResourceMatchingScope() == RangerAccessRequest.ResourceMatchingScope.SELF_OR_CHILD) {
+						matchType = RangerPolicyResourceMatcher.MatchType.DESCENDANT;  // So that a deny policy does not take effect!
+					}
 				} else if (request.getResourceMatchingScope() == RangerAccessRequest.ResourceMatchingScope.SELF_OR_DESCENDANTS) {
 					isMatched = matchType != RangerPolicyResourceMatcher.MatchType.NONE;
 				} else {
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerAbstractResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerAbstractResourceMatcher.java
index 8f6facd..4ebffef 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerAbstractResourceMatcher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerAbstractResourceMatcher.java
@@ -47,6 +47,7 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 	public final static String OPTION_TOKEN_DELIMITER_END    = "tokenDelimiterEnd";
 	public final static String OPTION_TOKEN_DELIMITER_ESCAPE = "tokenDelimiterEscape";
 	public final static String OPTION_TOKEN_DELIMITER_PREFIX = "tokenDelimiterPrefix";
+	public final static String OPTION_SIMULATE_HIERARCHY     = "simulateHierarchy";
 
 	protected RangerResourceDef    resourceDef;
 	protected RangerPolicyResource policyResource;
@@ -64,6 +65,7 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 	protected char    endDelimiterChar   = '}';
 	protected char    escapeChar         = '\\';
 	protected String  tokenPrefix        = "";
+	protected boolean optSimulateHierarchy;
 
 	@Override
 	public void setResourceDef(RangerResourceDef resourceDef) {
@@ -124,6 +126,8 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 			}
 		}
 
+		optSimulateHierarchy = getOptionSimulateHierarchy(options);
+
 		resourceMatchers = buildResourceMatchers();
 		isMatchAny = resourceMatchers == null || CollectionUtils.isEmpty(resourceMatchers.getResourceMatchers());
 
@@ -166,6 +170,9 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 	public static String getOptionDelimiterPrefix(Map<String, String> options) {
 		return ServiceDefUtil.getOption(options, OPTION_TOKEN_DELIMITER_PREFIX, "");
 	}
+	public static boolean getOptionSimulateHierarchy(Map<String, String> options) {
+		return ServiceDefUtil.getBooleanOption(options, OPTION_SIMULATE_HIERARCHY, false);
+	}
 	protected ResourceMatcherWrapper buildResourceMatchers() {
 		List<ResourceMatcher> resourceMatchers = new ArrayList<>();
 		boolean needsDynamicEval = false;
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 9cf31a2..4a158a2 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
@@ -20,6 +20,7 @@
 package org.apache.ranger.plugin.resourcematcher;
 
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.io.IOCase;
 import org.apache.commons.lang.ArrayUtils;
@@ -32,6 +33,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiFunction;
 
 
 public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
@@ -40,8 +42,8 @@ public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 	public static final String OPTION_PATH_SEPARATOR       = "pathSeparatorChar";
 	public static final char   DEFAULT_PATH_SEPARATOR_CHAR = org.apache.hadoop.fs.Path.SEPARATOR_CHAR;
 
-	private boolean policyIsRecursive;
-	private char    pathSeparatorChar = '/';
+	private boolean   policyIsRecursive;
+	private Character pathSeparatorChar = '/';
 
 	@Override
 	public void init() {
@@ -96,8 +98,8 @@ public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 
 	@Override
 	ResourceMatcher getMatcher(String policyValue) {
-		if(! policyIsRecursive) {
-			return super.getMatcher(policyValue);
+		if (!policyIsRecursive) {
+			return getPathMatcher(policyValue);
 		}
 
 		final int len = policyValue != null ? policyValue.length() : 0;
@@ -127,10 +129,9 @@ public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 		final ResourceMatcher ret;
 
 		if (isWildcardPresent) {
-			ret = optIgnoreCase ? new CaseInsensitiveRecursiveWildcardMatcher(policyValue, pathSeparatorChar)
-								: new CaseSensitiveRecursiveWildcardMatcher(policyValue, pathSeparatorChar);
+			ret = new RecursiveWildcardResourceMatcher(policyValue, true, pathSeparatorChar, optIgnoreCase, RangerPathResourceMatcher::isRecursiveWildCardMatch, optIgnoreCase ? 8 : 7);
 		} else {
-			ret = optIgnoreCase ? new CaseInsensitiveRecursiveMatcher(policyValue, pathSeparatorChar) : new CaseSensitiveRecursiveMatcher(policyValue, pathSeparatorChar);
+			ret = new RecursivePathResourceMatcher(policyValue, true, pathSeparatorChar, optIgnoreCase ? StringUtils::equalsIgnoreCase : StringUtils::equals, optIgnoreCase ? StringUtils::startsWithIgnoreCase : StringUtils::startsWith, optIgnoreCase ? 8 : 7);
 		}
 
 		if (optReplaceTokens) {
@@ -140,7 +141,7 @@ public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 		return ret;
 	}
 
-	static boolean isRecursiveWildCardMatch(String pathToCheck, String wildcardPath, char pathSeparatorChar, IOCase caseSensitivity) {
+	static boolean isRecursiveWildCardMatch(String pathToCheck, String wildcardPath, Character pathSeparatorChar, IOCase caseSensitivity) {
 
 		boolean ret = false;
 
@@ -185,118 +186,235 @@ public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 
 		return sb;
 	}
-}
 
-final class CaseSensitiveRecursiveWildcardMatcher extends ResourceMatcher {
-	private final char levelSeparatorChar;
-	CaseSensitiveRecursiveWildcardMatcher(String value, char levelSeparatorChar) {
-		super(value);
-		this.levelSeparatorChar = levelSeparatorChar;
-	}
+	private ResourceMatcher getPathMatcher(String policyValue) {
+		final int len = policyValue != null ? policyValue.length() : 0;
 
-	@Override
-	boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
-		return RangerPathResourceMatcher.isRecursiveWildCardMatch(resourceValue, getExpandedValue(evalContext), levelSeparatorChar, IOCase.SENSITIVE);
-	}
-	int getPriority() { return 7 + (getNeedsDynamicEval() ? DYNAMIC_EVALUATION_PENALTY : 0);}
-}
+		if (len == 0) {
+			return null;
+		}
 
-final class CaseInsensitiveRecursiveWildcardMatcher extends ResourceMatcher {
-	private final char levelSeparatorChar;
-	CaseInsensitiveRecursiveWildcardMatcher(String value, char levelSeparatorChar) {
-		super(value);
-		this.levelSeparatorChar = levelSeparatorChar;
+		final ResourceMatcher ret;
+
+		int wildcardStartIdx = -1;
+		int wildcardEndIdx = -1;
+		boolean needWildcardMatch = false;
+
+		// If optWildcard is true
+		//   If ('?' found or non-contiguous '*'s found in policyValue)
+		//	   needWildcardMatch = true
+		// 	 End
+		//
+		// 	 wildcardStartIdx is set to index of first '*' in policyValue or -1 if '*' is not found in policyValue, and
+		// 	 wildcardEndIdx is set to index of last '*' in policyValue or -1 if '*' is not found in policyValue
+		// Else
+		// 	 needWildcardMatch is set to false
+		// End
+		if (optWildCard) {
+			for (int i = 0; i < len; i++) {
+				final char c = policyValue.charAt(i);
+
+				if (c == '?') {
+					needWildcardMatch = true;
+					break;
+				} else if (c == '*') {
+					if (wildcardEndIdx == -1 || wildcardEndIdx == (i - 1)) {
+						wildcardEndIdx = i;
+						if (wildcardStartIdx == -1) {
+							wildcardStartIdx = i;
+						}
+					} else {
+						needWildcardMatch = true;
+						break;
+					}
+				}
+			}
+		}
+
+		if (needWildcardMatch) { // test?, test*a*, test*a*b, *test*a
+			ret = new WildcardResourceMatcher(policyValue, true, pathSeparatorChar, optIgnoreCase, FilenameUtils::wildcardMatch, 6);
+		} else if (wildcardStartIdx == -1) { // test, testa, testab
+			ret = new StringResourceMatcher(policyValue, true, pathSeparatorChar, optIgnoreCase ? StringUtils::equalsIgnoreCase : StringUtils::equals, optIgnoreCase ? 2 : 1);
+		} else if (wildcardStartIdx == 0) { // *test, **test, *testa, *testab
+			String matchStr = policyValue.substring(wildcardEndIdx + 1);
+			ret = new StringResourceMatcher(matchStr, true, pathSeparatorChar, optIgnoreCase ? StringUtils::endsWithIgnoreCase : StringUtils::endsWith, optIgnoreCase ? 4 : 3);
+		} else if (wildcardEndIdx != (len - 1)) { // test*a, test*ab
+			ret = new WildcardResourceMatcher(policyValue, true, pathSeparatorChar, optIgnoreCase, FilenameUtils::wildcardMatch, 6);
+		} else { // test*, test**, testa*, testab*
+			String matchStr = policyValue.substring(0, wildcardStartIdx);
+			ret = new StringResourceMatcher(matchStr, true, pathSeparatorChar, optIgnoreCase ? StringUtils::startsWithIgnoreCase : StringUtils::startsWith, optIgnoreCase ? 4 : 3);
+		}
+
+		if (optReplaceTokens) {
+			ret.setDelimiters(startDelimiterChar, endDelimiterChar, escapeChar, tokenPrefix);
+		}
+
+		return ret;
 	}
 
-	@Override
-	boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
-		return RangerPathResourceMatcher.isRecursiveWildCardMatch(resourceValue, getExpandedValue(evalContext), levelSeparatorChar, IOCase.INSENSITIVE);
+	interface TriFunction<T, U, V, R> {
+		R apply(T t, U u, V v);
 	}
-	int getPriority() { return 8 + (getNeedsDynamicEval() ? DYNAMIC_EVALUATION_PENALTY : 0);}
 
-}
+	interface QuadFunction<T, U, V, W, R> {
+		R apply(T t, U u, V v, W w);
+	}
 
-abstract class RecursiveMatcher extends ResourceMatcher {
-	final char levelSeparatorChar;
-	String valueWithoutSeparator;
-	String valueWithSeparator;
+	static abstract class PathResourceMatcher extends ResourceMatcher {
+		final boolean optSimulateHierarchy;
+		final char    pathSeparatorChar;
+		final int     priority;
 
-	RecursiveMatcher(String value, char levelSeparatorChar) {
-		super(value);
-		this.levelSeparatorChar = levelSeparatorChar;
+		PathResourceMatcher(String value, boolean optSimulatedHierarchy, char pathSeparatorChar, int priority) {
+			super(value);
+			this.optSimulateHierarchy = optSimulatedHierarchy;
+			this.pathSeparatorChar    = pathSeparatorChar;
+			this.priority             = priority;
+		}
+		int getPriority() {
+			return priority + (getNeedsDynamicEval() ? DYNAMIC_EVALUATION_PENALTY : 0);
+		}
 	}
 
-	String getStringToCompare(String policyValue) {
-		if (StringUtils.isEmpty(policyValue)) {
-			return policyValue;
+	static class StringResourceMatcher extends PathResourceMatcher {
+		final BiFunction<String, String, Boolean> function;
+		StringResourceMatcher(String value, boolean optSimulatedHierarchy, char pathSeparatorChar, BiFunction<String, String, Boolean> function, int priority) {
+			super(value, optSimulatedHierarchy, pathSeparatorChar, priority);
+			this.function = function;
+		}
+		@Override
+		boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+			String expandedValue = getExpandedValue(evalContext);
+			boolean ret = function.apply(resourceValue, expandedValue);
+			if (!ret && optSimulateHierarchy) {
+				String scope = MapUtils.isNotEmpty(evalContext) ? (String) evalContext.get("Scope") : null;
+				if (StringUtils.equals(scope, "SELF_OR_ONE_LEVEL")) {
+					int lastLevelSeparatorIndex = expandedValue.lastIndexOf(pathSeparatorChar);
+					if (lastLevelSeparatorIndex != -1) {
+						String shorterExpandedValue = expandedValue.substring(0, lastLevelSeparatorIndex);
+						if (resourceValue.charAt(resourceValue.length()-1) == pathSeparatorChar) {
+							resourceValue = resourceValue.substring(0, resourceValue.length()-1);
+						}
+						ret = function.apply(resourceValue, shorterExpandedValue);
+					}
+				}
+			}
+			return ret;
 		}
-		return (policyValue.lastIndexOf(levelSeparatorChar) == policyValue.length()-1) ?
-			policyValue.substring(0, policyValue.length()-1) : policyValue;
-	}
-}
 
-final class CaseSensitiveRecursiveMatcher extends RecursiveMatcher {
-	CaseSensitiveRecursiveMatcher(String value, char levelSeparatorChar) {
-		super(value, levelSeparatorChar);
 	}
 
-	@Override
-	boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+	static class WildcardResourceMatcher extends PathResourceMatcher {
+		final TriFunction<String, String, IOCase, Boolean> function;
+		final IOCase ioCase;
 
-		final String noSeparator;
-		if (getNeedsDynamicEval()) {
-			String expandedPolicyValue = getExpandedValue(evalContext);
-			noSeparator = expandedPolicyValue != null ? getStringToCompare(expandedPolicyValue) : null;
-		} else {
-			if (valueWithoutSeparator == null && value != null) {
-				valueWithoutSeparator = getStringToCompare(value);
-				valueWithSeparator = valueWithoutSeparator + Character.toString(levelSeparatorChar);
+		WildcardResourceMatcher(String value, boolean optSimulatedHierarchy, char pathSeparatorChar, boolean optIgnoreCase, TriFunction<String, String, IOCase, Boolean> function, int priority) {
+			super(value, optSimulatedHierarchy, pathSeparatorChar, priority);
+			this.function = function;
+			this.ioCase   = optIgnoreCase ? IOCase.INSENSITIVE : IOCase.SENSITIVE;
+		}
+		@Override
+		boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+			String expandedValue = getExpandedValue(evalContext);
+			boolean ret = function.apply(resourceValue, expandedValue, ioCase);
+			if (!ret && optSimulateHierarchy) {
+				String scope = MapUtils.isNotEmpty(evalContext) ? (String) evalContext.get("Scope") : null;
+				if (StringUtils.equals(scope, "SELF_OR_ONE_LEVEL")) {
+					int lastLevelSeparatorIndex = expandedValue.lastIndexOf(pathSeparatorChar);
+					if (lastLevelSeparatorIndex != -1) {
+						String shorterExpandedValue = expandedValue.substring(0, lastLevelSeparatorIndex);
+						if (resourceValue.charAt(resourceValue.length()-1) == pathSeparatorChar) {
+							resourceValue = resourceValue.substring(0, resourceValue.length()-1);
+						}
+						ret = function.apply(resourceValue, shorterExpandedValue, ioCase);
+					}
+				}
 			}
-			noSeparator = valueWithoutSeparator;
+			return ret;
 		}
+	}
 
-		boolean ret = StringUtils.equals(resourceValue, noSeparator);
+	static class RecursiveWildcardResourceMatcher extends PathResourceMatcher {
+		final QuadFunction<String, String, Character, IOCase, Boolean> function;
+		final IOCase ioCase;
 
-		if (!ret && noSeparator != null) {
-			final String withSeparator = getNeedsDynamicEval() ? noSeparator + Character.toString(levelSeparatorChar) : valueWithSeparator;
-			ret = StringUtils.startsWith(resourceValue, withSeparator);
+		RecursiveWildcardResourceMatcher(String value, boolean optSimulatedHierarchy, char pathSeparatorChar, boolean optIgnoreCase, QuadFunction<String, String, Character, IOCase, Boolean> function, int priority) {
+			super(value, optSimulatedHierarchy, pathSeparatorChar, priority);
+			this.function = function;
+			this.ioCase   = optIgnoreCase ? IOCase.INSENSITIVE : IOCase.SENSITIVE;
+		}
+		@Override
+		boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+			String expandedValue = getExpandedValue(evalContext);
+			boolean ret = function.apply(resourceValue, expandedValue, pathSeparatorChar, ioCase);
+			if (!ret && optSimulateHierarchy) {
+				String scope = MapUtils.isNotEmpty(evalContext) ? (String) evalContext.get("Scope") : null;
+				if (StringUtils.equals(scope, "SELF_OR_ONE_LEVEL")) {
+					int lastLevelSeparatorIndex = expandedValue.lastIndexOf(pathSeparatorChar);
+					if (lastLevelSeparatorIndex != -1) {
+						String shorterExpandedValue = expandedValue.substring(0, lastLevelSeparatorIndex);
+						if (resourceValue.charAt(resourceValue.length()-1) == pathSeparatorChar) {
+							resourceValue = resourceValue.substring(0, resourceValue.length()-1);
+						}
+						ret = function.apply(resourceValue, shorterExpandedValue, pathSeparatorChar, ioCase);
+					}
+				}
+			}
+			return ret;
 		}
-
-		return ret;
 	}
-	int getPriority() { return 7 + (getNeedsDynamicEval() ? DYNAMIC_EVALUATION_PENALTY : 0);}
-}
 
-final class CaseInsensitiveRecursiveMatcher extends RecursiveMatcher {
-	CaseInsensitiveRecursiveMatcher(String value, char levelSeparatorChar) {
-		super(value, levelSeparatorChar);
-	}
+	static class RecursivePathResourceMatcher extends PathResourceMatcher {
+		String valueWithoutSeparator;
+		String valueWithSeparator;
 
-	@Override
-	boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+		final BiFunction<String, String, Boolean> primaryFunction;
+		final BiFunction<String, String, Boolean> fallbackFunction;
 
-		final String noSeparator;
-		if (getNeedsDynamicEval()) {
-			String expandedPolicyValue = getExpandedValue(evalContext);
-			noSeparator = expandedPolicyValue != null ? getStringToCompare(expandedPolicyValue) : null;
-		} else {
-			if (valueWithoutSeparator == null && value != null) {
-				valueWithoutSeparator = getStringToCompare(value);
-				valueWithSeparator = valueWithoutSeparator + Character.toString(levelSeparatorChar);
+		RecursivePathResourceMatcher(String value, boolean optSimulateHierarchy, char pathSeparatorChar, BiFunction<String, String, Boolean> primaryFunction, BiFunction<String, String, Boolean> fallbackFunction, int priority) {
+			super(value, optSimulateHierarchy, pathSeparatorChar, priority);
+			this.primaryFunction    = primaryFunction;
+			this.fallbackFunction   = fallbackFunction;
+		}
+
+		String getStringToCompare(String policyValue) {
+			if (StringUtils.isEmpty(policyValue)) {
+				return policyValue;
 			}
-			noSeparator = valueWithoutSeparator;
+			return (policyValue.lastIndexOf(pathSeparatorChar) == policyValue.length() - 1) ?
+					policyValue.substring(0, policyValue.length() - 1) : policyValue;
 		}
 
-		boolean ret = StringUtils.equalsIgnoreCase(resourceValue, noSeparator);
+		@Override
+		boolean isMatch(String resourceValue, Map<String, Object> evalContext) {
+
+			final String noSeparator;
+			if (getNeedsDynamicEval()) {
+				String expandedPolicyValue = getExpandedValue(evalContext);
+				noSeparator = expandedPolicyValue != null ? getStringToCompare(expandedPolicyValue) : null;
+			} else {
+				if (valueWithoutSeparator == null && value != null) {
+					valueWithoutSeparator = getStringToCompare(value);
+					valueWithSeparator = valueWithoutSeparator + pathSeparatorChar;
+				}
+				noSeparator = valueWithoutSeparator;
+			}
 
-		if (!ret && noSeparator != null) {
-			final String withSeparator = getNeedsDynamicEval() ? noSeparator + Character.toString(levelSeparatorChar) : valueWithSeparator;
-			ret = StringUtils.startsWithIgnoreCase(resourceValue, withSeparator);
-		}
+			boolean ret = primaryFunction.apply(resourceValue, noSeparator);
 
-		return ret;
-	}
+			if (!ret && noSeparator != null) {
+				final String withSeparator = getNeedsDynamicEval() ? noSeparator + pathSeparatorChar : valueWithSeparator;
+				String scope = MapUtils.isNotEmpty(evalContext) ? (String) evalContext.get("Scope") : null;
+
+				if (!optSimulateHierarchy || !StringUtils.equals(scope, "SELF_OR_ONE_LEVEL")) {
+					ret = fallbackFunction.apply(resourceValue, withSeparator);
+				} else {
+					ret = fallbackFunction.apply(withSeparator, resourceValue);
+				}
+			}
 
-	int getPriority() { return 8 + (getNeedsDynamicEval() ? DYNAMIC_EVALUATION_PENALTY : 0);}
+			return ret;
+		}
+	}
 
-}
\ No newline at end of file
+}
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerURLResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerURLResourceMatcher.java
index 40f3df9..880e485 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerURLResourceMatcher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerURLResourceMatcher.java
@@ -250,6 +250,24 @@ final class CaseInsensitiveURLRecursiveWildcardMatcher extends ResourceMatcher {
 
 }
 
+abstract class RecursiveMatcher extends ResourceMatcher {
+    final char levelSeparatorChar;
+    String valueWithoutSeparator;
+    String valueWithSeparator;
+
+    RecursiveMatcher(String value, char levelSeparatorChar) {
+        super(value);
+        this.levelSeparatorChar = levelSeparatorChar;
+    }
+
+    String getStringToCompare(String policyValue) {
+        if (StringUtils.isEmpty(policyValue)) {
+            return policyValue;
+        }
+        return (policyValue.lastIndexOf(levelSeparatorChar) == policyValue.length() - 1) ?
+                policyValue.substring(0, policyValue.length() - 1) : policyValue;
+    }
+}
 
 final class CaseSensitiveURLRecursiveMatcher extends RecursiveMatcher {
     CaseSensitiveURLRecursiveMatcher(String value, char levelSeparatorChar) {
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPathResourceTrie.java b/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPathResourceTrie.java
new file mode 100644
index 0000000..90e4575
--- /dev/null
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPathResourceTrie.java
@@ -0,0 +1,184 @@
+/*
+ * 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.plugin.policyengine;
+
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyResource;
+import org.apache.ranger.plugin.model.RangerServiceDef.RangerResourceDef;
+import org.apache.ranger.plugin.policyresourcematcher.RangerPolicyResourceEvaluator;
+import org.apache.ranger.plugin.policyresourcematcher.RangerPolicyResourceMatcher;
+import org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher;
+import org.apache.ranger.plugin.resourcematcher.RangerResourceMatcher;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestPathResourceTrie {
+	private static final RangerResourceDef   PATH_RESOURCE_DEF      = getPathResourceDef();
+
+	private static final RangerPolicyResourceEvaluator EVAL_             = getEvaluator("/");
+	private static final RangerPolicyResourceEvaluator EVAL_nr           = getEvaluator("/", false, false);
+	private static final RangerPolicyResourceEvaluator EVAL_HOME         = getEvaluator("/home");
+	private static final RangerPolicyResourceEvaluator EVAL_HOME_        = getEvaluator("/home/");
+	private static final RangerPolicyResourceEvaluator EVAL_TMPnr        = getEvaluator("/tmp", false, false);
+	private static final RangerPolicyResourceEvaluator EVAL_TMP_nr       = getEvaluator("/tmp/", false, false);
+	private static final RangerPolicyResourceEvaluator EVAL_TMP_AB       = getEvaluator("/tmp/ab");
+	private static final RangerPolicyResourceEvaluator EVAL_TMP_A_B      = getEvaluator("/tmp/a/b");
+	private static final RangerPolicyResourceEvaluator EVAL_TMP_AC_D_E_F = getEvaluator("/tmp/ac/d/e/f");
+	private static final RangerPolicyResourceEvaluator EVAL_TMPFILE      = getEvaluator("/tmpfile");
+	private static final RangerPolicyResourceEvaluator EVAL_TMPdTXT      = getEvaluator("/tmp.txt");
+	private static final RangerPolicyResourceEvaluator EVAL_TMPA_B       = getEvaluator("/tmpa/b");
+
+	private static final List<RangerPolicyResourceEvaluator> EVALUATORS = Arrays.asList(EVAL_,
+																						EVAL_nr,
+																						EVAL_HOME,
+																						EVAL_HOME_,
+																						EVAL_TMPnr,
+																						EVAL_TMP_nr,
+																						EVAL_TMP_AB,
+																						EVAL_TMP_A_B,
+																						EVAL_TMP_AC_D_E_F,
+																						EVAL_TMPFILE,
+																						EVAL_TMPdTXT,
+																						EVAL_TMPA_B
+																						);
+
+	private final RangerResourceTrie<RangerPolicyResourceEvaluator> trie = new RangerResourceTrie<>(PATH_RESOURCE_DEF, EVALUATORS);
+
+	@Test
+	public void testChildrenScope() {
+		final RangerAccessRequest.ResourceMatchingScope scope = RangerAccessRequest.ResourceMatchingScope.SELF_OR_CHILD;
+
+		verifyEvaluators("/", scope, EVAL_, EVAL_nr, EVAL_HOME, EVAL_HOME_, EVAL_TMPnr, EVAL_TMP_nr, EVAL_TMPFILE, EVAL_TMPdTXT);
+		verifyEvaluators("/tmp", scope, EVAL_, EVAL_TMPnr, EVAL_TMP_nr, EVAL_TMP_AB);
+		verifyEvaluators("/tmp/", scope, EVAL_, EVAL_TMP_nr, EVAL_TMP_AB);
+		verifyEvaluators("/tmp/a", scope, EVAL_, EVAL_TMP_A_B);
+		verifyEvaluators("/tmp/ac", scope, EVAL_);
+		verifyEvaluators("/tmp/ac/d", scope, EVAL_);
+		verifyEvaluators("/tmp/ac/d/e", scope, EVAL_, EVAL_TMP_AC_D_E_F);
+		verifyEvaluators("/unmatched", scope, EVAL_);
+		verifyEvaluators("invalid: does-not-begin-with-sep", scope);
+	}
+
+	@Test
+	public void testSelfScope() {
+		final RangerAccessRequest.ResourceMatchingScope scope = RangerAccessRequest.ResourceMatchingScope.SELF;
+
+		verifyEvaluators("/", scope, EVAL_, EVAL_nr);
+		verifyEvaluators("/tmp", scope, EVAL_, EVAL_TMPnr);
+		verifyEvaluators("/tmp/", scope, EVAL_, EVAL_TMP_nr);
+		verifyEvaluators("/tmp/a", scope, EVAL_);
+		verifyEvaluators("/tmp/ac", scope, EVAL_);
+		verifyEvaluators("/tmp/ac/d", scope, EVAL_);
+		verifyEvaluators("/tmp/ac/d/e", scope, EVAL_);
+		verifyEvaluators("/unmatched", scope, EVAL_);
+		verifyEvaluators("invalid: does-not-begin-with-sep", scope);
+	}
+
+	private void verifyEvaluators(String resource, RangerAccessRequest.ResourceMatchingScope scope, RangerPolicyResourceEvaluator... evaluators) {
+		Set<RangerPolicyResourceEvaluator> expected = evaluators.length == 0 ? null : new HashSet<>(Arrays.asList(evaluators));
+		Set<RangerPolicyResourceEvaluator> result   = trie.getEvaluatorsForResource(resource, scope);
+
+		assertEquals("incorrect evaluators for resource "  + resource, expected, result);
+	}
+
+	private static RangerResourceDef getPathResourceDef() {
+		RangerResourceDef ret = new RangerResourceDef();
+
+		ret.setItemId(1L);
+		ret.setName("path");
+		ret.setType("path");
+		ret.setLevel(10);
+		ret.setParent("");
+		ret.setMatcher("org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher");
+		ret.setMatcherOptions(new HashMap<String, String>() {{
+			put("wildCard", "true");
+			put("ignoreCase", "true");
+			put("pathSeparatorChar", "/");
+		}});
+
+		return ret;
+	}
+
+	private static RangerPolicyResourceEvaluator getEvaluator(String resource) {
+		return  new TestPolicyResourceEvaluator(new RangerPolicyResource(resource, false, true));
+	}
+
+	private static RangerPolicyResourceEvaluator getEvaluator(String resource, boolean  isExcludes, boolean isRecursive) {
+		return  new TestPolicyResourceEvaluator(new RangerPolicyResource(resource, isExcludes, isRecursive));
+	}
+
+	private static class TestPolicyResourceEvaluator implements RangerPolicyResourceEvaluator {
+		private static long nextId = 1;
+
+		private final long                  id;
+		private final RangerPolicyResource  policyResource;
+		private final RangerResourceMatcher resourceMatcher;
+
+		TestPolicyResourceEvaluator(RangerPolicyResource policyResource) {
+			this.id = nextId++;
+			this.policyResource = policyResource;
+			this.resourceMatcher = new RangerPathResourceMatcher();
+
+			resourceMatcher.setResourceDef(PATH_RESOURCE_DEF);
+			resourceMatcher.setPolicyResource(policyResource);
+
+			resourceMatcher.init();
+		}
+
+		@Override
+		public long getId() {
+			return id;
+		}
+
+		@Override
+		public RangerPolicyResourceMatcher getPolicyResourceMatcher() {
+			return null;
+		}
+
+		@Override
+		public Map<String, RangerPolicyResource> getPolicyResource() {
+			return Collections.singletonMap(PATH_RESOURCE_DEF.getName(), policyResource);
+		}
+
+		@Override
+		public RangerResourceMatcher getResourceMatcher(String resourceName) {
+			return resourceMatcher;
+		}
+
+		@Override
+		public boolean isAncestorOf(RangerResourceDef resourceDef) {
+			return false;
+		}
+
+		@Override
+		public String toString() {
+			return "id=" + id + ", resource=" + policyResource;
+		}
+	}
+}
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPolicyEngine.java b/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPolicyEngine.java
index dbb5b4a..5a06e1d 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPolicyEngine.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPolicyEngine.java
@@ -452,6 +452,13 @@ public class TestPolicyEngine {
 		runTestsFromResourceFiles(resourceFiles);
 	}
 
+	@Test
+	public void testPolicyEngine_aws() {
+		String[] awsTestResourceFiles = {"/policyengine/test_policyengine_aws.json"};
+
+		runTestsFromResourceFiles(awsTestResourceFiles);
+	}
+
 	private void runTestsFromResourceFiles(String[] resourceNames) {
 		for(String resourceName : resourceNames) {
 			InputStream inStream = this.getClass().getResourceAsStream(resourceName);
diff --git a/agents-common/src/test/resources/policyengine/test_policyengine_aws.json b/agents-common/src/test/resources/policyengine/test_policyengine_aws.json
new file mode 100644
index 0000000..3e77506
--- /dev/null
+++ b/agents-common/src/test/resources/policyengine/test_policyengine_aws.json
@@ -0,0 +1,134 @@
+{
+  "serviceName":"awsdev",
+
+  "serviceDef":{
+    "name":"hdfs",
+    "id":1,
+    "resources":[
+      {"name":"path","type":"path","level":1,"mandatory":true,"lookupSupported":true,"recursiveSupported": true,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"Resource Path","description":"HDFS file or directory path"}
+    ],
+    "accessTypes":[
+      {"name":"read","label":"Read"},
+      {"name":"write","label":"Write"},
+      {"name":"execute","label":"Execute"}
+    ],
+    "contextEnrichers": [],
+    "policyConditions": []
+  },
+
+  "policies":[
+    {"id":10,"name":"allow-all-to-user1 /","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":20,"name":"allow-all-to-user1 /home/","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/home/"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+    ,
+    {"id":30,"name":"allow-all-to-user1 /tmpa/b","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmpa/b"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":40,"name":"allow-all-to-user1 /tmp/ab","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmp/ab"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":50,"name":"allow-all-to-user1 /tmp/a/b","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmp/a/b"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":60,"name":"allow-all-to-user1 /tmp/ac/d/e/f","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmp/ac/d/e/f"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":70,"name":"allow-all-to-user1 /tmpfile","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmpfile"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ,
+    {"id":80,"name":"allow-all-to-user1 /tmp.txt","isEnabled":true,"isAuditEnabled":true,
+      "resources":{"path":{"values":["/tmp.txt"],"isRecursive":false}},
+      "policyItems":[
+        {"accesses":[{"type":"read","isAllowed":true}, {"type":"write","isAllowed":true}, {"type":"execute","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
+      ]
+    }
+  ],
+
+  "tests":[
+    {"name":"ALLOW 'ANY /' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":20}
+    }
+  ,
+    {"name":"ALLOW 'ANY /tmp' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":40}
+    }
+  ,
+    {"name":"ALLOW 'ANY /tmp/' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp/"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp/"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":40}
+    }
+  ,
+    {"name":"ALLOW 'ANY /tmp/a' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp/a"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp/a"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":50}
+    }
+  ,
+    {"name":"DENY 'ANY /tmp/ac' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp/ac"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp/ac"
+      },
+      "result":{"isAudited":false,"isAllowed":false,"policyId":-1}
+    }
+  ,
+    {"name":"DENY 'ANY /tmp/ac/d' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp/ac/d"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp/ac/d"
+      },
+      "result":{"isAudited":false,"isAllowed":false,"policyId":-1}
+    }
+  ,
+    {"name":"ALLOW 'ANY /tmp/ac/d/e' for u=user1",
+      "request":{
+        "resource":{"elements":{"path":"/tmp/ac/d/e"}}, "resourceMatchingScope": "SELF_OR_CHILD",
+        "accessType":"","user":"user1","userGroups":[],"requestData":"ANY /tmp/ac/d/e"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":10}
+    }
+  ]
+}
\ No newline at end of file
diff --git a/security-admin/src/main/java/org/apache/ranger/biz/RangerPolicyAdminImpl.java b/security-admin/src/main/java/org/apache/ranger/biz/RangerPolicyAdminImpl.java
index eb332ac..66835bd 100644
--- a/security-admin/src/main/java/org/apache/ranger/biz/RangerPolicyAdminImpl.java
+++ b/security-admin/src/main/java/org/apache/ranger/biz/RangerPolicyAdminImpl.java
@@ -36,6 +36,7 @@ import org.apache.ranger.plugin.policyengine.RangerAccessResource;
 import org.apache.ranger.plugin.policyengine.RangerPluginContext;
 import org.apache.ranger.plugin.policyengine.RangerPolicyEngine;
 import org.apache.ranger.plugin.policyengine.RangerPolicyRepository;
+import org.apache.ranger.plugin.policyengine.RangerTagAccessRequest;
 import org.apache.ranger.plugin.policyengine.RangerTagResource;
 import org.apache.ranger.plugin.policyevaluator.RangerPolicyEvaluator;
 import org.apache.ranger.plugin.policyresourcematcher.RangerPolicyResourceMatcher;
@@ -116,7 +117,10 @@ public class RangerPolicyAdminImpl implements RangerPolicyAdmin {
             Set<String> roles = getRolesFromUserAndGroups(user, userGroups);
             Set<String> requestedAccesses = new HashSet<>(accessTypes);
 
-            for (RangerPolicyEvaluator evaluator : matchedRepository.getLikelyMatchPolicyEvaluators(resource, RangerPolicy.POLICY_TYPE_ACCESS)) {
+            RangerAccessRequestImpl request = new RangerAccessRequestImpl();
+            request.setResource(resource);
+
+            for (RangerPolicyEvaluator evaluator : matchedRepository.getLikelyMatchPolicyEvaluators(request, RangerPolicy.POLICY_TYPE_ACCESS)) {
 
                 Set<String> allowedAccesses = evaluator.getAllowedAccesses(resource, user, userGroups, roles, requestedAccesses);
                 if (CollectionUtils.isNotEmpty(allowedAccesses)) {
@@ -415,7 +419,8 @@ public class RangerPolicyAdminImpl implements RangerPolicyAdmin {
 
                     for (RangerTagForEval tag : tags) {
                         RangerAccessResource        tagResource      = new RangerTagResource(tag.getType(), policyEngine.getTagPolicyRepository().getServiceDef());
-                        List<RangerPolicyEvaluator> likelyEvaluators = policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(tagResource);
+                        RangerAccessRequest         tagRequest       = new RangerTagAccessRequest(tag, policyEngine.getTagPolicyRepository().getServiceDef(), request);
+                        List<RangerPolicyEvaluator> likelyEvaluators = policyEngine.getTagPolicyRepository().getLikelyMatchPolicyEvaluators(tagRequest);
 
                         for (RangerPolicyEvaluator evaluator : likelyEvaluators) {
                             String policyZoneName = evaluator.getPolicy().getZoneName();
@@ -452,7 +457,7 @@ public class RangerPolicyAdminImpl implements RangerPolicyAdmin {
 
 
             if (policyEngine.hasResourcePolicies(matchedRepository)) {
-                List<RangerPolicyEvaluator> likelyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request.getResource());
+                List<RangerPolicyEvaluator> likelyEvaluators = matchedRepository.getLikelyMatchPolicyEvaluators(request);
 
                 for (RangerPolicyEvaluator evaluator : likelyEvaluators) {
                     RangerPolicyResourceMatcher matcher = evaluator.getPolicyResourceMatcher();