You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ma...@apache.org on 2016/11/06 22:24:43 UTC

incubator-ranger git commit: RANGER-1208: Optimize tag enricher for requests containing empty resource

Repository: incubator-ranger
Updated Branches:
  refs/heads/master f4a20e0b9 -> ad0fe4b89


RANGER-1208: Optimize tag enricher for requests containing empty resource

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


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

Branch: refs/heads/master
Commit: ad0fe4b8976aa3fcee85d722bf8fa7bb32d16245
Parents: f4a20e0
Author: Abhay Kulkarni <ak...@hortonworks.com>
Authored: Thu Nov 3 16:35:45 2016 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Sun Nov 6 13:57:16 2016 -0800

----------------------------------------------------------------------
 .../plugin/audit/RangerDefaultAuditHandler.java |   2 +-
 .../RangerScriptExecutionContext.java           |   9 +-
 .../contextenricher/RangerTagEnricher.java      | 114 ++++++++++++-------
 .../contextenricher/RangerTagForEval.java       |  55 +++++++++
 .../apache/ranger/plugin/model/RangerTag.java   |  40 +++++++
 .../policyengine/RangerPolicyEngineImpl.java    |   5 +-
 .../plugin/util/RangerAccessRequestUtil.java    |  12 +-
 .../plugin/contextenricher/TestTagEnricher.java |   4 +-
 .../plugin/policyengine/TestPolicyEngine.java   |   7 +-
 .../contextenricher/test_tagenricher_hive.json  |  20 ++--
 .../resources/policyengine/descendant_tags.json |  16 ++-
 .../test_policyengine_descendant_tags.json      |  22 +++-
 .../resources/META-INF/jpa_named_queries.xml    |  16 +--
 13 files changed, 243 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
index 2a0896d..1cb2295 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/audit/RangerDefaultAuditHandler.java
@@ -219,7 +219,7 @@ public class RangerDefaultAuditHandler implements RangerAccessResultProcessor {
 
 	protected final Set<String> getTags(RangerAccessRequest request) {
 		Set<String>     ret  = null;
-		List<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
+		Set<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
 
 		if (CollectionUtils.isNotEmpty(tags)) {
 			ret = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/conditionevaluator/RangerScriptExecutionContext.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/conditionevaluator/RangerScriptExecutionContext.java b/agents-common/src/main/java/org/apache/ranger/plugin/conditionevaluator/RangerScriptExecutionContext.java
index 1428454..f91e01e 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/conditionevaluator/RangerScriptExecutionContext.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/conditionevaluator/RangerScriptExecutionContext.java
@@ -117,7 +117,7 @@ public final class RangerScriptExecutionContext {
 
 	public Set<String> getAllTagTypes() {
 		Set<String>     allTagTypes   = null;
-		List<RangerTagForEval> tagObjectList = getAllTags();
+		Set<RangerTagForEval> tagObjectList = getAllTags();
 
 		if (CollectionUtils.isNotEmpty(tagObjectList)) {
 			for (RangerTagForEval tag : tagObjectList) {
@@ -136,7 +136,7 @@ public final class RangerScriptExecutionContext {
 		Map<String, String> ret = null;
 
 		if (StringUtils.isNotBlank(tagType)) {
-			List<RangerTagForEval> tagObjectList = getAllTags();
+			Set<RangerTagForEval> tagObjectList = getAllTags();
 
 			// Assumption: There is exactly one tag with given tagType in the list of tags - may not be true ***TODO***
 			// This will get attributes of the first tagType that matches
@@ -279,9 +279,8 @@ public final class RangerScriptExecutionContext {
 		return ret;
 	}
 
-	private List<RangerTagForEval> getAllTags() {
-		List<RangerTagForEval> ret = RangerAccessRequestUtil.getRequestTagsFromContext(accessRequest.getContext());
-		
+	private Set<RangerTagForEval> getAllTags() {
+		Set<RangerTagForEval> ret = RangerAccessRequestUtil.getRequestTagsFromContext(accessRequest.getContext());
 		if(ret == null) {
 			if (LOG.isDebugEnabled()) {
 				String resource = accessRequest.getResource().getAsString();

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagEnricher.java
----------------------------------------------------------------------
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 62e7c99..274d6be 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
@@ -43,6 +43,7 @@ import java.io.*;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -58,10 +59,8 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 
 	private RangerTagRefresher                 tagRefresher               = null;
 	private RangerTagRetriever                 tagRetriever               = null;
-	private ServiceTags                        serviceTags                = null;
-	private List<RangerServiceResourceMatcher> serviceResourceMatchers    = null;
-	private Map<String, RangerResourceTrie>    serviceResourceTrie        = null;
 	private boolean                            disableTrieLookupPrefilter = false;
+	private EnrichedServiceTags                enrichedServiceTags;
 
 	@Override
 	public void init() {
@@ -90,7 +89,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 			} catch (ClassCastException exception) {
 				LOG.error("Class " + tagRetrieverClassName + " is not a type of RangerTagRetriever, exception=" + exception);
 			} catch (IllegalAccessException exception) {
-				LOG.error("Class " + tagRetrieverClassName + " could not be instantiated, exception=" + exception);
+				LOG.error("Class " + tagRetrieverClassName + " illegally accessed, exception=" + exception);
 			} catch (InstantiationException exception) {
 				LOG.error("Class " + tagRetrieverClassName + " could not be instantiated, exception=" + exception);
 			}
@@ -133,9 +132,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 			LOG.debug("==> RangerTagEnricher.enrich(" + request + ")");
 		}
 
-		final List<RangerTagForEval> matchedTags;
-
-		matchedTags = findMatchingTags(request);
+		final Set<RangerTagForEval> matchedTags = findMatchingTags(request);
 
 		RangerAccessRequestUtil.setRequestTagsInContext(request.getContext(), matchedTags);
 
@@ -171,19 +168,22 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 
 		}
 
-		Map<String, RangerResourceTrie> serviceResourceTrie = null;
+		Map<String, RangerResourceTrie<RangerServiceResourceMatcher>> serviceResourceTrie = null;
 
 		if(!disableTrieLookupPrefilter) {
-			serviceResourceTrie = new HashMap<String, RangerResourceTrie>();
+			serviceResourceTrie = new HashMap<String, RangerResourceTrie<RangerServiceResourceMatcher>>();
 
 			for (RangerServiceDef.RangerResourceDef resourceDef : serviceDef.getResources()) {
-				serviceResourceTrie.put(resourceDef.getName(), new RangerResourceTrie(resourceDef, resourceMatchers));
+				serviceResourceTrie.put(resourceDef.getName(), new RangerResourceTrie<RangerServiceResourceMatcher>(resourceDef, resourceMatchers));
 			}
 		}
 
-		this.serviceResourceMatchers = resourceMatchers;
-		this.serviceResourceTrie     = serviceResourceTrie;
-		this.serviceTags             = serviceTags;
+		Set<RangerTagForEval> allTagsForEval = new HashSet<RangerTagForEval>();
+		for (Map.Entry<Long, RangerTag> entry : serviceTags.getTags().entrySet()) {
+			allTagsForEval.add(new RangerTagForEval(entry.getValue(), RangerPolicyResourceMatcher.MatchType.ANCESTOR));
+		}
+
+		enrichedServiceTags = new EnrichedServiceTags(serviceTags, resourceMatchers, serviceResourceTrie, allTagsForEval);
 	}
 
 	@Override
@@ -205,37 +205,45 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		return ret;
 	}
 
-	private List<RangerTagForEval> findMatchingTags(final RangerAccessRequest request) {
+	private Set<RangerTagForEval> findMatchingTags(final RangerAccessRequest request) {
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("==> RangerTagEnricher.findMatchingTags(" + request + ")");
 		}
 
+		// To minimize chance for race condition between Tag-Refresher thread and access-evaluation thread
+		final EnrichedServiceTags enrichedServiceTags = this.enrichedServiceTags;
+
+		Set<RangerTagForEval> ret = null;
+
 		RangerAccessResource resource = request.getResource();
-		List<RangerTagForEval> ret = null;
-		final List<RangerServiceResourceMatcher> serviceResourceMatchers = getEvaluators(resource);
 
-		if (CollectionUtils.isNotEmpty(serviceResourceMatchers)) {
+		if ((resource == null || resource.getKeys() == null || resource.getKeys().size() == 0) && request.isAccessTypeAny()) {
+			ret = enrichedServiceTags.getAllTagsForEval();
+		} else {
 
-			final ServiceTags serviceTags = this.serviceTags;
+			final List<RangerServiceResourceMatcher> serviceResourceMatchers = getEvaluators(resource, enrichedServiceTags);
 
-			for (RangerServiceResourceMatcher resourceMatcher : serviceResourceMatchers) {
+			if (CollectionUtils.isNotEmpty(serviceResourceMatchers)) {
 
-				final RangerPolicyResourceMatcher.MatchType matchType = resourceMatcher.getMatchType(resource, request.getContext());
+				for (RangerServiceResourceMatcher resourceMatcher : serviceResourceMatchers) {
 
-				final boolean isMatched;
+					final RangerPolicyResourceMatcher.MatchType matchType = resourceMatcher.getMatchType(resource, request.getContext());
 
-				if(request.isAccessTypeAny()) {
-					isMatched = matchType != RangerPolicyResourceMatcher.MatchType.NONE;
-				}  else if (request.getResourceMatchingScope() == RangerAccessRequest.ResourceMatchingScope.SELF_OR_DESCENDANTS) {
-					isMatched = matchType == RangerPolicyResourceMatcher.MatchType.SELF || matchType == RangerPolicyResourceMatcher.MatchType.DESCENDANT;
-				} else {
-					isMatched = matchType == RangerPolicyResourceMatcher.MatchType.SELF || matchType == RangerPolicyResourceMatcher.MatchType.ANCESTOR;
-				}
-				if (isMatched) {
-					if (ret == null) {
-						ret = new ArrayList<RangerTagForEval>();
+					final boolean isMatched;
+
+					if (request.isAccessTypeAny()) {
+						isMatched = matchType != RangerPolicyResourceMatcher.MatchType.NONE;
+					} else if (request.getResourceMatchingScope() == RangerAccessRequest.ResourceMatchingScope.SELF_OR_DESCENDANTS) {
+						isMatched = matchType == RangerPolicyResourceMatcher.MatchType.SELF || matchType == RangerPolicyResourceMatcher.MatchType.DESCENDANT;
+					} else {
+						isMatched = matchType == RangerPolicyResourceMatcher.MatchType.SELF || matchType == RangerPolicyResourceMatcher.MatchType.ANCESTOR;
+					}
+					if (isMatched) {
+						if (ret == null) {
+							ret = new HashSet<RangerTagForEval>();
+						}
+						ret.addAll(getTagsForServiceResource(enrichedServiceTags.getServiceTags(), resourceMatcher.getServiceResource(), matchType));
 					}
-					ret.addAll(getTagsForServiceResource(serviceTags, resourceMatcher.getServiceResource(), matchType));
 				}
 			}
 		}
@@ -255,12 +263,17 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		return ret;
 	}
 
-	private List<RangerServiceResourceMatcher> getEvaluators(RangerAccessResource resource) {
+	private List<RangerServiceResourceMatcher> getEvaluators(RangerAccessResource resource, EnrichedServiceTags enrichedServiceTags) {
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("==> RangerTagEnricher.getEvaluators(" + (resource != null ? resource.getAsString() : null) + ")");
+		}
+
 		List<RangerServiceResourceMatcher> ret = null;
 
-		String resourceStr = resource != null ? resource.getAsString() : null;
+		final List<RangerServiceResourceMatcher> serviceResourceMatchers = enrichedServiceTags.getServiceResourceMatchers();
+		final Map<String, RangerResourceTrie<RangerServiceResourceMatcher>> serviceResourceTrie = enrichedServiceTags.getServiceResourceTrie();
 
-		if (serviceResourceTrie == null || StringUtils.isEmpty(resourceStr)) {
+		if (resource == null || resource.getKeys() == null || resource.getKeys().size() == 0 || serviceResourceTrie == null) {
 			ret = serviceResourceMatchers;
 		} else {
 			Set<String> resourceKeys = resource.getKeys();
@@ -270,7 +283,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 				boolean isRetModifiable = false;
 
 				for (String resourceName : resourceKeys) {
-					RangerResourceTrie trie = serviceResourceTrie.get(resourceName);
+					RangerResourceTrie<RangerServiceResourceMatcher> trie = serviceResourceTrie.get(resourceName);
 
 					if (trie == null) { // if no trie exists for this resource level, ignore and continue to next level
 						continue;
@@ -282,7 +295,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 						ret = null;
 					} else if (ret == null) { // initialize ret with policies found for this resource
 						ret = resourceEvaluators;
-					} else { // remove policies from ret that are not in resourceEvaluators
+					} else { // remove matchers from ret that are not in resourceEvaluators
 						if (isRetModifiable) {
 							ret.retainAll(resourceEvaluators);
 						} else {
@@ -303,7 +316,7 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 						}
 					}
 
-					if (CollectionUtils.isEmpty(ret)) { // if no policy exists, bail out and return empty list
+					if (CollectionUtils.isEmpty(ret)) { // if no matcher exists, bail out and return empty list
 						ret = null;
 						break;
 					}
@@ -316,15 +329,15 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerTagEnricher.getEvaluators(" + resource.getAsString() + "): evaluatorCount=" + ret.size());
+			LOG.debug("<== RangerTagEnricher.getEvaluators(" + (resource != null ? resource.getAsString() : null) + "): evaluatorCount=" + ret.size());
 		}
 
 		return ret;
 	}
 
-	static private List<RangerTagForEval> getTagsForServiceResource(final ServiceTags serviceTags, final RangerServiceResource serviceResource, final RangerPolicyResourceMatcher.MatchType matchType) {
+	static private Set<RangerTagForEval> getTagsForServiceResource(final ServiceTags serviceTags, final RangerServiceResource serviceResource, final RangerPolicyResourceMatcher.MatchType matchType) {
 
-		List<RangerTagForEval> ret = new ArrayList<RangerTagForEval>();
+		Set<RangerTagForEval> ret = new HashSet<RangerTagForEval>();
 
 		final Long resourceId = serviceResource.getId();
 
@@ -351,6 +364,25 @@ public class RangerTagEnricher extends RangerAbstractContextEnricher {
 		return ret;
 	}
 
+	static private final class EnrichedServiceTags {
+		final private ServiceTags                        serviceTags;
+		final private List<RangerServiceResourceMatcher> serviceResourceMatchers;
+		final private Map<String, RangerResourceTrie<RangerServiceResourceMatcher>>    serviceResourceTrie;
+		final private Set<RangerTagForEval>              allTagsForEval;
+
+		EnrichedServiceTags(ServiceTags serviceTags, List<RangerServiceResourceMatcher> serviceResourceMatchers,
+							Map<String, RangerResourceTrie<RangerServiceResourceMatcher>> serviceResourceTrie, Set<RangerTagForEval> allTagsForEval) {
+			this.serviceTags             = serviceTags;
+			this.serviceResourceMatchers = serviceResourceMatchers;
+			this.serviceResourceTrie     = serviceResourceTrie;
+			this.allTagsForEval          = allTagsForEval;
+		}
+		ServiceTags getServiceTags() {return serviceTags;}
+		List<RangerServiceResourceMatcher> getServiceResourceMatchers() { return serviceResourceMatchers;}
+		Map<String, RangerResourceTrie<RangerServiceResourceMatcher>> getServiceResourceTrie() { return serviceResourceTrie;}
+		Set<RangerTagForEval> getAllTagsForEval() { return allTagsForEval;}
+	}
+
 	static class RangerTagRefresher extends Thread {
 		private static final Log LOG = LogFactory.getLog(RangerTagRefresher.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagForEval.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagForEval.java b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagForEval.java
index 014686d..42ae6f3 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagForEval.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/contextenricher/RangerTagForEval.java
@@ -69,4 +69,59 @@ public class RangerTagForEval {
     public Long getVersion() {
         return tag.getVersion();
     }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+
+        toString(sb);
+
+        return sb.toString();
+    }
+
+    public StringBuilder toString(StringBuilder sb) {
+        sb.append("RangerTagForEval={");
+        sb.append("tag={");
+        if (tag != null) {
+            tag.toString(sb);
+        }
+        sb.append("}");
+        sb.append(", {matchType={").append(matchType).append("}");
+        sb.append("}");
+        return sb;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result
+                + ((tag == null) ? 0 : tag.hashCode());
+        result = prime * result
+                + ((matchType == null) ? 0 : matchType.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        RangerTagForEval other = (RangerTagForEval) obj;
+        if (tag == null) {
+            if (other.tag != null)
+                return false;
+        } else if (!tag.equals(other.tag))
+            return false;
+        if (matchType == null) {
+            if (other.matchType != null)
+                return false;
+        } else if (!matchType.equals(other.matchType))
+            return false;
+
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerTag.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerTag.java b/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerTag.java
index f19cc60..743b028 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerTag.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/model/RangerTag.java
@@ -118,5 +118,45 @@ public class RangerTag extends RangerBaseModelObject {
 
 		return sb;
 	}
+
+	@Override
+	public int hashCode() {
+		final int prime = 31;
+		int result = 1;
+		result = prime * result
+				+ ((type == null) ? 0 : type.hashCode());
+		result = prime * result
+				+ ((owner == null) ? 0 : owner.hashCode());
+		result = prime * result
+				+ ((attributes == null) ? 0 : attributes.hashCode());
+		return result;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (this == obj)
+			return true;
+		if (obj == null)
+			return false;
+		if (getClass() != obj.getClass())
+			return false;
+		RangerTag other = (RangerTag) obj;
+		if (type == null) {
+			if (other.type != null)
+				return false;
+		} else if (!type.equals(other.type))
+			return false;
+		if (owner == null) {
+			if (other.owner != null)
+				return false;
+		} else if (!owner.equals(other.owner))
+			return false;
+		if (attributes == null) {
+			if (other.attributes != null)
+				return false;
+		} else if (!attributes.equals(other.attributes))
+			return false;
+		return true;
+	}
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngineImpl.java
----------------------------------------------------------------------
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 6a9fc18..0ce7a17 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
@@ -549,8 +549,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 		List<RangerPolicyEvaluator> tagPolicyEvaluators = tagPolicyRepository == null ? null : tagPolicyRepository.getPolicyEvaluators();
 
 		if (CollectionUtils.isNotEmpty(tagPolicyEvaluators)) {
-			List<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
-
+			Set<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
 			if(CollectionUtils.isNotEmpty(tags)) {
 				for (RangerTagForEval tag : tags) {
 					RangerAccessRequest tagEvalRequest = new RangerTagAccessRequest(tag, tagPolicyRepository.getServiceDef(), request);
@@ -651,7 +650,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 		List<RangerPolicyEvaluator> tagEvaluators = tagPolicyRepository == null ? null : tagPolicyRepository.getPolicyEvaluators();
 
 		if (CollectionUtils.isNotEmpty(tagEvaluators)) {
-			List<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
+			Set<RangerTagForEval> tags = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
 
 			if (CollectionUtils.isNotEmpty(tags)) {
 				for (RangerTagForEval tag : tags) {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
index a46d975..c2f0b67 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerAccessRequestUtil.java
@@ -20,8 +20,8 @@
 package org.apache.ranger.plugin.util;
 
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
@@ -40,7 +40,7 @@ public class RangerAccessRequestUtil {
 	public static final String KEY_TOKEN_NAMESPACE = "token:";
 	public static final String KEY_USER = "USER";
 
-	public static void setRequestTagsInContext(Map<String, Object> context, List<RangerTagForEval> tags) {
+	public static void setRequestTagsInContext(Map<String, Object> context, Set<RangerTagForEval> tags) {
 		if(CollectionUtils.isEmpty(tags)) {
 			context.remove(KEY_CONTEXT_TAGS);
 		} else {
@@ -48,14 +48,14 @@ public class RangerAccessRequestUtil {
 		}
 	}
 
-	public static List<RangerTagForEval> getRequestTagsFromContext(Map<String, Object> context) {
-		List<RangerTagForEval> ret = null;
+	public static Set<RangerTagForEval> getRequestTagsFromContext(Map<String, Object> context) {
+		Set<RangerTagForEval> ret = null;
 		Object          val = context.get(RangerAccessRequestUtil.KEY_CONTEXT_TAGS);
 
-		if (val != null && val instanceof List<?>) {
+		if (val != null && val instanceof Set<?>) {
 			try {
 				@SuppressWarnings("unchecked")
-				List<RangerTagForEval> tags = (List<RangerTagForEval>) val;
+				Set<RangerTagForEval> tags = (Set<RangerTagForEval>) val;
 
 				ret = tags;
 			} catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/test/java/org/apache/ranger/plugin/contextenricher/TestTagEnricher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/contextenricher/TestTagEnricher.java b/agents-common/src/test/java/org/apache/ranger/plugin/contextenricher/TestTagEnricher.java
index 3454c30..a399fd3 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/contextenricher/TestTagEnricher.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/contextenricher/TestTagEnricher.java
@@ -44,6 +44,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -107,7 +108,8 @@ public class TestTagEnricher {
             tagEnricher.enrich(request);
 
             List<RangerTag> expected = test.result;
-            List<RangerTagForEval> result   = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
+
+            Set<RangerTagForEval> result   = RangerAccessRequestUtil.getRequestTagsFromContext(request.getContext());
 
             expectedTags.clear();
             if(expected != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/test/java/org/apache/ranger/plugin/policyengine/TestPolicyEngine.java
----------------------------------------------------------------------
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 71e2eb6..c8f67d2 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
@@ -53,6 +53,7 @@ import java.lang.reflect.Type;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 import static org.junit.Assert.*;
 
@@ -350,11 +351,11 @@ public class TestPolicyEngine {
 
 				if(!StringUtils.isEmpty(tagsJsonString)) {
 					try {
-						Type listType = new TypeToken<List<RangerTagForEval>>() {
+						Type setType = new TypeToken<Set<RangerTagForEval>>() {
 						}.getType();
-						List<RangerTagForEval> tagList = gsonBuilder.fromJson(tagsJsonString, listType);
+						Set<RangerTagForEval> tags = gsonBuilder.fromJson(tagsJsonString, setType);
 
-						context.put(RangerAccessRequestUtil.KEY_CONTEXT_TAGS, tagList);
+						context.put(RangerAccessRequestUtil.KEY_CONTEXT_TAGS, tags);
 					} catch (Exception e) {
 						System.err.println("TestPolicyEngine.runTests(): error parsing TAGS JSON string in file " + testName + ", tagsJsonString=" +
 								tagsJsonString + ", exception=" + e);

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/test/resources/contextenricher/test_tagenricher_hive.json
----------------------------------------------------------------------
diff --git a/agents-common/src/test/resources/contextenricher/test_tagenricher_hive.json b/agents-common/src/test/resources/contextenricher/test_tagenricher_hive.json
index 9fba96b..0b8b41c 100644
--- a/agents-common/src/test/resources/contextenricher/test_tagenricher_hive.json
+++ b/agents-common/src/test/resources/contextenricher/test_tagenricher_hive.json
@@ -26,13 +26,16 @@
   "tagDefinitions": {
     "1":{"name":"PII"},
     "2":{"name":"EXPIRES_ON","attributeDefs":[{"name":"expiry_date","type":"date"}]},
-    "3":{"name":"FINANCE"}
+    "3":{"name":"FINANCE"},
+    "4":{"name":"PCI"}
   },
 
   "tags": {
     "1":{"type":"PII"},
     "2":{"type":"EXPIRES_ON","attributes":{"expiry_date":"2015/12/31"}},
-    "3":{"type":"FINANCE"}
+    "3":{"type":"FINANCE"},
+    "4":{"type":"EXPIRES_ON","attributes":{"expiry_date":"2025/12/31"}},
+    "5":{"type":"PCI"}
   },
 
   "serviceResources": [
@@ -48,8 +51,9 @@
     "1":[1],
 	"2":[3],
     "3":[2],
-    "4":[1],
-    "5":[1]
+    "4":[1, 4],
+    "5":[1],
+    "6":[5]
   },
 
   "tests":[
@@ -62,13 +66,13 @@
     {"name":"finance-any",
       "resource":{"elements":{"database":"finance"}},
       "accessType": "",
-      "result":[{"type":"EXPIRES_ON"}, {"type":"FINANCE"}, {"type":"PII"}]
+      "result":[{"type":"EXPIRES_ON"}, {"type":"FINANCE"}, {"type":"PII"}, {"type":"EXPIRES_ON"}]
     }
     ,
     {"name":"finance-and-tax2010any",
       "resource":{"elements":{"database":"finance", "table":"tax_2010"}},
       "accessType": "",
-      "result":[{"type":"FINANCE"},{"type":"EXPIRES_ON"}, {"type":"PII"}]
+      "result":[{"type":"FINANCE"},{"type":"EXPIRES_ON"}, {"type":"PII"}, {"type":"EXPIRES_ON"}]
     }
     ,
     {"name":"hr-any",
@@ -80,7 +84,7 @@
     {"name":"none-any",
       "resource":{"elements":{}},
       "accessType": "",
-      "result":[{"type":"EXPIRES_ON"}, {"type":"FINANCE"}, {"type":"PII"}, {"type":"PII"}, {"type":"PII"}]
+      "result":[{"type":"EXPIRES_ON"}, {"type":"FINANCE"}, {"type":"PII"}, {"type":"EXPIRES_ON"}, {"type":"PCI"}]
     }
     ,
     {"name":"hr.employee.ssn",
@@ -104,7 +108,7 @@
     {"name":"finance.tax_2010.ssn",
       "resource":{"elements":{"database":"finance","table":"tax_2010","column":"ssn"}},
       "accessType": "read",
-      "result":[{"type":"EXPIRES_ON"},{"type":"PII"},{"type":"FINANCE"}]
+      "result":[{"type":"EXPIRES_ON"},{"type":"PII"},{"type":"FINANCE"}, {"type":"EXPIRES_ON"}]
     }
     ,
     {"name":"finance.tax_2010.id",

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/test/resources/policyengine/descendant_tags.json
----------------------------------------------------------------------
diff --git a/agents-common/src/test/resources/policyengine/descendant_tags.json b/agents-common/src/test/resources/policyengine/descendant_tags.json
index 101ae26..8007c53 100644
--- a/agents-common/src/test/resources/policyengine/descendant_tags.json
+++ b/agents-common/src/test/resources/policyengine/descendant_tags.json
@@ -43,6 +43,20 @@
         "id": 3,
         "guid": "tag-expires_on-3-guid"
       }
+      ,
+      "4": {
+        "type": "EXPIRES_ON",
+        "attributes": { "expiry_date": "2025/08/10" },
+        "id": 4,
+        "guid": "tag-expires_on-4-guid"
+      }
+      ,
+      "5": {
+        "type": "EXPIRES_ON",
+        "attributes": { "expiry_date": "2050/08/10" },
+        "id": 5,
+        "guid": "tag-expires_on-5-guid"
+      }
     },
     "serviceResources": [
       {
@@ -102,7 +116,7 @@
       "2": [ 2 ],
       "3": [1],
       "4": [2],
-      "5": [3]
+      "5": [3, 4, 5]
     }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/agents-common/src/test/resources/policyengine/test_policyengine_descendant_tags.json
----------------------------------------------------------------------
diff --git a/agents-common/src/test/resources/policyengine/test_policyengine_descendant_tags.json b/agents-common/src/test/resources/policyengine/test_policyengine_descendant_tags.json
index 831342f..757b37d 100644
--- a/agents-common/src/test/resources/policyengine/test_policyengine_descendant_tags.json
+++ b/agents-common/src/test/resources/policyengine/test_policyengine_descendant_tags.json
@@ -142,12 +142,12 @@
         "resources":{"tag":{"values":["RESTRICTED"],"isRecursive":false}},
         "policyItems":[
           {
-            "accesses":[{"type":"hive:all","isAllowed":true}],"users":["hive", "user1"],"groups":[],"delegateAdmin":false,
+            "accesses":[{"type":"hive:all","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false,
             "conditions":[]
           }
         ],
         "denyPolicyItems":[
-          {"accesses":[{"type":"hive:all","isAllowed":true}],"users":["user1","hive"],"groups":[],"delegateAdmin":false}
+          {"accesses":[{"type":"hive:all","isAllowed":true}],"users":["user1"],"groups":[],"delegateAdmin":false}
         ],
         "denyExceptions":[
           {"accesses":[{"type":"hive:select","isAllowed":true}],"users":["user3"],"groups":[],"delegateAdmin":false}
@@ -273,6 +273,24 @@
       },
       "result":{"isAudited":true,"isAllowed":true,"policyId":3}
     }
+    ,
+    {
+      "name":"ALLOW 'show databases;' for user",
+      "request":{
+        "resource":{"elements":{}},
+        "accessType":"","user":"user","userGroups":[],"requestData":"'show databases;' for user"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":3}
+    }
+    ,
+    {
+      "name":"DENY 'show databases;' for hive",
+      "request":{
+        "resource":{"elements":{}},
+        "accessType":"","user":"hive","userGroups":[],"requestData":"'show databases;' for hive"
+      },
+      "result":{"isAudited":true,"isAllowed":false,"policyId":3}
+    }
   ]
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/ad0fe4b8/security-admin/src/main/resources/META-INF/jpa_named_queries.xml
----------------------------------------------------------------------
diff --git a/security-admin/src/main/resources/META-INF/jpa_named_queries.xml b/security-admin/src/main/resources/META-INF/jpa_named_queries.xml
index fe9d91a..e54bfe2 100644
--- a/security-admin/src/main/resources/META-INF/jpa_named_queries.xml
+++ b/security-admin/src/main/resources/META-INF/jpa_named_queries.xml
@@ -788,7 +788,7 @@
 			(select tagResMap.resourceId from XXTagResourceMap tagResMap, XXTag tag, XXTagDef tagDef
 			where tagResMap.tagId = tag.id and tag.type = tagDef.id and tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			)
 			)
 			order by obj.resourceId, obj.id
@@ -827,7 +827,7 @@
 			(select tagResMap.resourceId from XXTagResourceMap tagResMap, XXTag tag, XXTagDef tagDef
 			where tagResMap.tagId = tag.id and tag.type = tagDef.id and tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			)
 			)
 			order by serviceResElem.resourceId, serviceResElem.id
@@ -885,7 +885,7 @@
 			select obj from XXTagResourceMap obj, XXService service where service.id = :serviceId and service.tagService is not null and obj.tagId in
 			(select tag.id from XXTag tag, XXTagDef tagDef where tag.type = tagDef.id and tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			)
 			order by obj.resourceId
 		</query>
@@ -916,7 +916,7 @@
 		<query>select obj from XXTagDef obj, XXService service where service.id = :serviceId and service.tagService is not null and
 			obj.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policyRes.policyId = policy.id and policy.isEnabled = TRUE and policyResMap.resourceId = policyRes.id)
 			order by obj.id
 		</query>
 	</named-query>
@@ -946,7 +946,7 @@
 			(select tagDef.id from XXTagDef tagDef, XXService service where service.id = :serviceId and service.tagService is not null and
 			tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			)
 			order by obj.tagDefId
 		</query>
@@ -991,7 +991,7 @@
 			select obj from XXTag obj, XXTagDef tagDef, XXService service where service.id = :serviceId and service.tagService is not null and
 			obj.type = tagDef.id and tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			order by obj.id
 		</query>
 	</named-query>
@@ -1036,7 +1036,7 @@
 			(select tag.id from XXTag tag, XXTagDef tagDef, XXService service where service.id = :serviceId and service.tagService is not null and
 			tag.type = tagDef.id and tagDef.name in
 			(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-			where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+			where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 			)
 			order by obj.tagId
 		</query>
@@ -1065,7 +1065,7 @@
 				(select tagResMap.resourceId from XXTagResourceMap tagResMap, XXTag tag, XXTagDef tagDef
 					where tagResMap.tagId = tag.id and tag.type = tagDef.id and tagDef.name in
 						(select policyResMap.value from XXPolicyResourceMap policyResMap, XXPolicyResource policyRes, XXPolicy policy
-							where policy.service = service.tagService and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
+							where policy.service = service.tagService and policy.isEnabled = TRUE and policyRes.policyId = policy.id and policyResMap.resourceId = policyRes.id)
 				)
 			order by obj.id
 		</query>