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/08/16 08:11:17 UTC

incubator-ranger git commit: RANGER-1145: Policy engine optimization: convert wildcard matches into prefix and suffix match

Repository: incubator-ranger
Updated Branches:
  refs/heads/master 3a36875e9 -> 818e1d354


RANGER-1145: Policy engine optimization: convert wildcard matches into prefix and suffix match

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/818e1d35
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ranger/tree/818e1d35
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ranger/diff/818e1d35

Branch: refs/heads/master
Commit: 818e1d354312443fd8e29d49861b91d13e239be2
Parents: 3a36875
Author: Abhay Kulkarni <ak...@hortonworks.com>
Authored: Thu Aug 4 20:53:58 2016 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Tue Aug 16 01:02:00 2016 -0700

----------------------------------------------------------------------
 .../RangerAbstractResourceMatcher.java          | 199 ++++++++++++++++---
 .../RangerDefaultResourceMatcher.java           |  21 +-
 .../RangerPathResourceMatcher.java              | 144 ++++++++------
 .../resourcematcher/RangerResourceMatcher.java  |   1 +
 .../plugin/resourcematcher/ResourceMatcher.java |  39 ++++
 .../plugin/policyengine/TestPolicyEngine.java   |   7 +
 .../RangerDefaultResourceMatcherTest.java       |  13 +-
 .../RangerPathResourceMatcherTest.java          |  81 ++++++++
 .../test_policyengine_hdfs_resourcespec.json    |  38 ++++
 9 files changed, 443 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerAbstractResourceMatcher.java
----------------------------------------------------------------------
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 cd725c9..329b8e8 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
@@ -20,10 +20,13 @@
 package org.apache.ranger.plugin.resourcematcher;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.commons.io.IOCase;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -51,6 +54,7 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 	protected List<String> policyValues     = null;
 	protected boolean      policyIsExcludes = false;
 	protected boolean      isMatchAny       = false;
+	protected List<ResourceMatcher> resourceMatchers = null;
 
 	@Override
 	public void setResourceDef(RangerResourceDef resourceDef) {
@@ -75,33 +79,42 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 		policyIsExcludes = policyResource == null ? false : policyResource.getIsExcludes();
 
 		if(policyResource != null && policyResource.getValues() != null) {
-			boolean isWildCardPresent = false;
-			for(String policyValue : policyResource.getValues()) {
-				if(StringUtils.isEmpty(policyValue)) {
+			for (String policyValue : policyResource.getValues()) {
+				if (StringUtils.isEmpty(policyValue)) {
 					continue;
 				}
-
-				if(optWildCard) {
-					if (StringUtils.containsOnly(policyValue, WILDCARD_ASTERISK)) {
-						isMatchAny = true;
-					} else if (!isWildCardPresent && StringUtils.containsAny(policyValue, WILDCARDS)) {
-						isWildCardPresent = true;
-					}
-				}
 				policyValues.add(policyValue);
 			}
-			optWildCard = optWildCard && isWildCardPresent;
-		}
-
-		if(policyValues.isEmpty()) {
-			isMatchAny = true;
 		}
+		resourceMatchers = buildResourceMatchers();
+		isMatchAny = CollectionUtils.isEmpty(resourceMatchers);
 
 		if(LOG.isDebugEnabled()) {
 			LOG.debug("<== RangerAbstractResourceMatcher.init()");
 		}
 	}
 
+	protected List<ResourceMatcher> buildResourceMatchers() {
+		List<ResourceMatcher> ret = new ArrayList<ResourceMatcher> ();
+
+		for (String policyValue : policyValues) {
+			ResourceMatcher matcher = getMatcher(policyValue);
+
+			if (matcher != null) {
+				if (matcher.isMatchAny()) {
+					ret.clear();
+					break;
+				} else {
+					ret.add(matcher);
+				}
+			}
+		}
+
+		Collections.sort(ret);
+
+		return ret;
+	}
+
 	@Override
 	public boolean isCompleteMatch(String resource) {
 		if(LOG.isDebugEnabled()) {
@@ -228,11 +241,6 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 		return sb;
 	}
 
-	/**
-	 * Is resource asking to authorize all possible values at this level?
-	 * @param resource
-	 * @return
-	 */
 	boolean isAllValuesRequested(String resource) {
 		boolean result = StringUtils.isEmpty(resource) || WILDCARD_ASTERISK.equals(resource);
 		if (LOG.isDebugEnabled()) {
@@ -246,13 +254,156 @@ public abstract class RangerAbstractResourceMatcher implements RangerResourceMat
 	 * - Resource denotes all possible values (i.e. resource in (null, "", "*")
 	 * - where as policy does not allow all possible values (i.e. policy.values().contains("*")
 	 *
-	 * @param allValuesRequested
-	 * @param resultWithoutExcludes
-     * @return
      */
 	public boolean applyExcludes(boolean allValuesRequested, boolean resultWithoutExcludes) {
 		if (!policyIsExcludes) return resultWithoutExcludes;                   // not an excludes policy!
 		if (allValuesRequested && !isMatchAny)  return resultWithoutExcludes;  // one case where excludes has no effect
 		return !resultWithoutExcludes;                                         // all other cases flip it
 	}
+
+	ResourceMatcher getMatcher(String policyValue) {
+		final int len = policyValue != null ? policyValue.length() : 0;
+
+		if (len == 0) {
+			return null;
+		}
+
+		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) {
+			ret = optIgnoreCase ? new CaseInsensitiveWildcardMatcher(policyValue) : new CaseSensitiveWildcardMatcher(policyValue);
+		} else if (wildcardStartIdx == -1) {
+			ret = optIgnoreCase ? new CaseInsensitiveStringMatcher(policyValue) : new CaseSensitiveStringMatcher(policyValue);
+		} else if (wildcardStartIdx == 0) {
+			String matchStr = policyValue.substring(wildcardEndIdx + 1);
+			ret = optIgnoreCase ? new CaseInsensitiveEndsWithMatcher(matchStr) : new CaseSensitiveEndsWithMatcher(matchStr);
+		} else {
+			String matchStr = policyValue.substring(0, wildcardStartIdx);
+			ret = optIgnoreCase ? new CaseInsensitiveStartsWithMatcher(matchStr) : new CaseSensitiveStartsWithMatcher(matchStr);
+		}
+
+		return ret;
+	}
+}
+
+final class CaseSensitiveStringMatcher extends ResourceMatcher {
+	CaseSensitiveStringMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return StringUtils.equals(str, value);
+	}
+	int getPriority() { return 1;}
+}
+
+final class CaseInsensitiveStringMatcher extends ResourceMatcher {
+	CaseInsensitiveStringMatcher(String value) { super(value); }
+
+	boolean isMatch(String str) {
+		return StringUtils.equalsIgnoreCase(str, value);
+	}
+	int getPriority() {return 2; }
+}
+
+final class CaseSensitiveStartsWithMatcher extends ResourceMatcher {
+	CaseSensitiveStartsWithMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return StringUtils.startsWith(str, value);
+	}
+	int getPriority() { return 3;}
+}
+
+final class CaseInsensitiveStartsWithMatcher extends ResourceMatcher {
+	CaseInsensitiveStartsWithMatcher(String value) { super(value); }
+
+	boolean isMatch(String str) {
+		return StringUtils.startsWithIgnoreCase(str, value);
+	}
+	int getPriority() { return 4; }
 }
+
+final class CaseSensitiveEndsWithMatcher extends ResourceMatcher {
+	CaseSensitiveEndsWithMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return StringUtils.endsWith(str, value);
+	}
+	int getPriority() { return 3; }
+}
+
+final class CaseInsensitiveEndsWithMatcher extends ResourceMatcher {
+	CaseInsensitiveEndsWithMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return StringUtils.endsWithIgnoreCase(str, value);
+	}
+	int getPriority() { return 4; }
+}
+
+final class CaseSensitiveWildcardMatcher extends ResourceMatcher {
+	CaseSensitiveWildcardMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return FilenameUtils.wildcardMatch(str, value, IOCase.SENSITIVE);
+	}
+	int getPriority() { return 5; }
+}
+
+
+final class CaseInsensitiveWildcardMatcher extends ResourceMatcher {
+	CaseInsensitiveWildcardMatcher(String value) {
+		super(value);
+	}
+
+	boolean isMatch(String str) {
+		return FilenameUtils.wildcardMatch(str, value, IOCase.INSENSITIVE);
+	}
+	int getPriority() {return 6; }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcher.java
index 669cf0a..0a11be0 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcher.java
@@ -20,9 +20,7 @@
 package org.apache.ranger.plugin.resourcematcher;
 
 
-import org.apache.commons.io.FilenameUtils;
-import org.apache.commons.io.IOCase;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -42,17 +40,12 @@ public class RangerDefaultResourceMatcher extends RangerAbstractResourceMatcher
 		if(allValuesRequested || isMatchAny) {
 			ret = isMatchAny;
 		} else {
-			for(String policyValue : policyValues) {
-				if(optWildCard) {
-					ret = optIgnoreCase ? FilenameUtils.wildcardMatch(resource, policyValue, IOCase.INSENSITIVE)
-										: FilenameUtils.wildcardMatch(resource, policyValue, IOCase.SENSITIVE);
-				} else {
-					ret = optIgnoreCase ? StringUtils.equalsIgnoreCase(resource, policyValue)
-										: StringUtils.equals(resource, policyValue);
-				}
-
-				if(ret) {
-					break;
+			if (CollectionUtils.isNotEmpty(resourceMatchers)) {
+				for (ResourceMatcher resourceMatcher : resourceMatchers) {
+					ret = resourceMatcher.isMatch(resource);
+					if (ret) {
+						break;
+					}
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcher.java
index 5c555eb..d508f3f 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
@@ -19,10 +19,6 @@
 
 package org.apache.ranger.plugin.resourcematcher;
 
-
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.io.IOCase;
 import org.apache.commons.lang.ArrayUtils;
@@ -30,16 +26,19 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
 
-public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
+public class RangerPathResourceMatcher extends RangerDefaultResourceMatcher {
 	private static final Log LOG = LogFactory.getLog(RangerPathResourceMatcher.class);
 
-	public static final String OPTION_PATH_SEPERATOR       = "pathSeparatorChar";
-	public static final char   DEFAULT_PATH_SEPERATOR_CHAR = org.apache.hadoop.fs.Path.SEPARATOR_CHAR;
+	private static final String OPTION_PATH_SEPARATOR       = "pathSeparatorChar";
+	private static final char   DEFAULT_PATH_SEPARATOR_CHAR = org.apache.hadoop.fs.Path.SEPARATOR_CHAR;
 
-	private boolean      policyIsRecursive    = false;
-	private char         pathSeparatorChar    = DEFAULT_PATH_SEPERATOR_CHAR;
-	private List<String> policyValuesForMatch = null;
+	private boolean policyIsRecursive    = false;
+	private char    pathSeparatorChar = '/';
 
 	@Override
 	public void init() {
@@ -47,76 +46,82 @@ public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
 			LOG.debug("==> RangerPathResourceMatcher.init()");
 		}
 
+		policyIsRecursive = policyResource == null ? false : policyResource.getIsRecursive();
+		pathSeparatorChar = getCharOption(OPTION_PATH_SEPARATOR, DEFAULT_PATH_SEPARATOR_CHAR);
+
 		super.init();
 
-		policyIsRecursive = policyResource == null ? false : policyResource.getIsRecursive();
-		pathSeparatorChar = getCharOption(OPTION_PATH_SEPERATOR, DEFAULT_PATH_SEPERATOR_CHAR);
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("<== RangerPathResourceMatcher.init()");
+		}
+	}
 
-		if(policyIsRecursive && optWildCard && !isMatchAny) {
-			policyValuesForMatch = new ArrayList<String>();
+	@Override
+	protected List<ResourceMatcher> buildResourceMatchers() {
+		List<ResourceMatcher> ret = new ArrayList<ResourceMatcher>();
+
+		for (String policyValue : policyValues) {
+			if (optWildCard && policyIsRecursive) {
+				if (policyValue.charAt(policyValue.length() - 1) == pathSeparatorChar) {
+					policyValue += WILDCARD_ASTERISK;
+				}
+			}
 
-			for(String policyValue : policyValues) {
-				if(policyValue.charAt(policyValue.length() - 1) == pathSeparatorChar) {
-					policyValuesForMatch.add(policyValue + WILDCARD_ASTERISK);
+			ResourceMatcher matcher = getMatcher(policyValue);
+
+			if (matcher != null) {
+				if (matcher.isMatchAny()) {
+					ret.clear();
+					break;
 				} else {
-					policyValuesForMatch.add(policyValue);
+					ret.add(matcher);
 				}
 			}
-		} else {
-			policyValuesForMatch = policyValues;
 		}
 
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerPathResourceMatcher.init()");
-		}
+		Collections.sort(ret);
+
+		return ret;
 	}
 
 	@Override
-	public boolean isMatch(String resource) {
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerPathResourceMatcher.isMatch(" + resource + ")");
+	ResourceMatcher getMatcher(String policyValue) {
+		if(! policyIsRecursive) {
+			return super.getMatcher(policyValue);
 		}
 
-		boolean ret = false;
-		boolean allValuesRequested = isAllValuesRequested(resource);
+		final int len = policyValue != null ? policyValue.length() : 0;
 
-		if(allValuesRequested || isMatchAny) {
-			ret = isMatchAny;
-		} else {
-			IOCase caseSensitivity = optIgnoreCase ? IOCase.INSENSITIVE : IOCase.SENSITIVE;
-
-			for(String policyValue : policyValuesForMatch) {
-				if(policyIsRecursive && optWildCard) {
-					ret = isRecursiveWildCardMatch(resource, policyValue, pathSeparatorChar, caseSensitivity);
-				} else if(policyIsRecursive) {
-					ret = optIgnoreCase ? StringUtils.startsWithIgnoreCase(resource, policyValue)
-										: StringUtils.startsWith(resource, policyValue);
-				} else if(optWildCard) {
-					ret = FilenameUtils.wildcardMatch(resource, policyValue, caseSensitivity);
-				} else {
-					ret = optIgnoreCase ? StringUtils.equalsIgnoreCase(resource, policyValue)
-										: StringUtils.equals(resource, policyValue);
-				}
+		if (len == 0) {
+			return null;
+		}
+
+		boolean isWildcardPresent = false;
+
+		if (optWildCard) {
+			for (int i = 0; i < len; i++) {
+				final char c = policyValue.charAt(i);
 
-				if(ret) {
+				if (c == '?' || c == '*') {
+					isWildcardPresent = true;
 					break;
 				}
 			}
 		}
 
-		ret = applyExcludes(allValuesRequested, ret);
+		final ResourceMatcher ret;
 
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerPathResourceMatcher.isMatch(" + resource + "): " + ret);
+		if (isWildcardPresent) {
+			ret = optIgnoreCase ? new CaseInsensitiveRecursiveWildcardMatcher(policyValue, pathSeparatorChar)
+								: new CaseSensitiveRecursiveWildcardMatcher(policyValue, pathSeparatorChar);
+		} else {
+			ret = optIgnoreCase ? new CaseInsensitiveStartsWithMatcher(policyValue) : new CaseSensitiveStartsWithMatcher(policyValue);
 		}
 
 		return ret;
 	}
 
-	private boolean isRecursiveWildCardMatch(String pathToCheck, String wildcardPath, char pathSeparatorChar, IOCase caseSensitivity) {
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("==> RangerPathResourceMatcher.isRecursiveWildCardMatch(" + pathToCheck + ", " + wildcardPath + ", " + pathSeparatorChar + ")");
-		}
+	static boolean isRecursiveWildCardMatch(String pathToCheck, String wildcardPath, char pathSeparatorChar, IOCase caseSensitivity) {
 
 		boolean ret = false;
 
@@ -148,13 +153,10 @@ public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
 			}
 		}
 
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("<== RangerPathResourceMatcher.isRecursiveWildCardMatch(" + pathToCheck + ", " + wildcardPath + ", " + pathSeparatorChar + "): " + ret);
-		}
-
 		return ret;
 	}
 
+
 	public StringBuilder toString(StringBuilder sb) {
 		sb.append("RangerPathResourceMatcher={");
 
@@ -167,3 +169,31 @@ public class RangerPathResourceMatcher extends RangerAbstractResourceMatcher {
 		return sb;
 	}
 }
+
+final class CaseSensitiveRecursiveWildcardMatcher extends ResourceMatcher {
+	private final char levelSeparatorChar;
+	CaseSensitiveRecursiveWildcardMatcher(String value, char levelSeparatorChar) {
+		super(value);
+		this.levelSeparatorChar = levelSeparatorChar;
+	}
+
+	boolean isMatch(String str) {
+		return RangerPathResourceMatcher.isRecursiveWildCardMatch(str, value, levelSeparatorChar, IOCase.SENSITIVE);
+	}
+	int getPriority() { return 7;}
+}
+
+final class CaseInsensitiveRecursiveWildcardMatcher extends ResourceMatcher {
+	private final char levelSeparatorChar;
+	CaseInsensitiveRecursiveWildcardMatcher(String value, char levelSeparatorChar) {
+		super(value);
+		this.levelSeparatorChar = levelSeparatorChar;
+	}
+
+	boolean isMatch(String str) {
+		return RangerPathResourceMatcher.isRecursiveWildCardMatch(str, value, levelSeparatorChar, IOCase.INSENSITIVE);
+	}
+	int getPriority() { return 8;}
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerResourceMatcher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerResourceMatcher.java
index e4d3ce5..c1b3404 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerResourceMatcher.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/RangerResourceMatcher.java
@@ -32,4 +32,5 @@ public interface RangerResourceMatcher {
 	boolean isMatch(String resource);
 
 	boolean isCompleteMatch(String resource);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/ResourceMatcher.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/ResourceMatcher.java b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/ResourceMatcher.java
new file mode 100644
index 0000000..39eb339
--- /dev/null
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/resourcematcher/ResourceMatcher.java
@@ -0,0 +1,39 @@
+/*
+ * 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.resourcematcher;
+
+abstract class ResourceMatcher implements Comparable<ResourceMatcher> {
+    protected final String value;
+
+    ResourceMatcher(String value) { this.value = value; }
+
+    abstract boolean isMatch(String str);
+    abstract int getPriority();
+
+    boolean isMatchAny() { return value != null && value.length() == 0; }
+
+    @Override
+    public int compareTo(ResourceMatcher other) { return Integer.compare(getPriority(), other.getPriority()); }
+
+    @Override
+    public String toString() {
+        return this.getClass().getName() + "(" + this.value + ")";
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/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 ab5626e..f825f40 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
@@ -160,6 +160,13 @@ public class TestPolicyEngine {
 	}
 
 	@Test
+	public void testPolicyEngine_hdfs_resourcespec() {
+		String[] hdfsTestResourceFiles = { "/policyengine/test_policyengine_hdfs_resourcespec.json" };
+
+		runTestsFromResourceFiles(hdfsTestResourceFiles);
+	}
+
+	@Test
 	public void testPolicyEngine_hdfs() {
 		String[] hdfsTestResourceFiles = { "/policyengine/test_policyengine_hdfs.json" };
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcherTest.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcherTest.java b/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcherTest.java
index 5576a09..d2fb62c 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcherTest.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerDefaultResourceMatcherTest.java
@@ -20,6 +20,7 @@
 package org.apache.ranger.plugin.resourcematcher;
 
 import com.google.common.collect.Lists;
+import org.apache.ranger.plugin.model.RangerPolicy;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -60,14 +61,16 @@ public class RangerDefaultResourceMatcherTest {
 
     static class MatcherWrapper extends RangerDefaultResourceMatcher {
         MatcherWrapper(String policyValue, boolean exclude) {
-            this.policyValues = Lists.newArrayList(policyValue);
-            if (WILDCARD_ASTERISK.equals(policyValue)) {
-                this.isMatchAny = true;
-            }
+            RangerPolicy.RangerPolicyResource policyResource = new RangerPolicy.RangerPolicyResource();
+            policyResource.setIsExcludes(exclude);
+            policyResource.setValues(Lists.newArrayList(policyValue));
+            setPolicyResource(policyResource);
+
             if (policyValue.contains(WILDCARD_ASTERISK)) {
                 this.optWildCard = true;
             }
-            this.policyIsExcludes = exclude;
+            this.optIgnoreCase = false;
+            init();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcherTest.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcherTest.java b/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcherTest.java
new file mode 100644
index 0000000..c9d207f
--- /dev/null
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/resourcematcher/RangerPathResourceMatcherTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.resourcematcher;
+
+import com.google.common.collect.Lists;
+import org.apache.ranger.plugin.model.RangerPolicy;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class RangerPathResourceMatcherTest {
+
+    Object[][] data = {
+            // { resource, policy, optWildcard, recursive, result
+            { "/app/hive/test.db",    "/",                 true, false, false },
+            { "/app/hive/test.db",    "/",                 true, true,  true },
+            { "/app/hive/test.db",    "/*",                true, false, true },
+            { "/app/hbase/test.tbl",  "/*",                true, false, true },
+            { "/app/hive/test.db",    "/app",              true, false, false },
+            { "/app/hive/test.db",    "/app/",             true, false, false },
+            { "/app/hive/test.db",    "/app/",             true, true,  true },
+            { "/app/hive/test.db",    "/app/*",            true, false, true },
+            { "/app/hbase/test.tbl",  "/app/*",            true, false, true },
+            { "/app/hive/test.db",    "/app/hive/*",       true, false, true },
+            { "/app/hbase/test.tbl",  "/app/hive/*",       true, false, false },
+            { "/app/hive/test.db",    "/app/hive/test*",   true, false, true },
+            { "/app/hbase/test.tbl",  "/app/hive/test*",   true, false, false },
+            { "/app/hive/test.db",    "/app/hive/test.db", true, false, true },
+            { "/app/hbase/test.tbl",  "/app/hive/test.db", true, false, false },
+    };
+
+    @Test
+    public void testIsMatch() throws Exception {
+        for (Object[] row : data) {
+            String resource = (String)row[0];
+            String policyValue = (String)row[1];
+            boolean optWildcard = (boolean)row[2];
+            boolean isRecursive = (boolean)row[3];
+            boolean result = (boolean)row[4];
+
+            MatcherWrapper matcher = new MatcherWrapper(policyValue, optWildcard, isRecursive);
+            assertEquals(getMessage(row), result, matcher.isMatch(resource));
+        }
+    }
+
+    String getMessage(Object[] row) {
+        return String.format("Resource=%s, Policy=%s, optWildcard=%s, recursive=%s, result=%s",
+                (String)row[0], (String)row[1], (boolean)row[2], (boolean)row[3], (boolean)row[4]);
+    }
+
+    static class MatcherWrapper extends RangerPathResourceMatcher {
+        MatcherWrapper(String policyValue, boolean optWildcard, boolean isRecursive) {
+            super.optWildCard = optWildcard;
+
+            RangerPolicy.RangerPolicyResource policyResource = new RangerPolicy.RangerPolicyResource();
+            policyResource.setIsRecursive(isRecursive);
+            policyResource.setValues(Lists.newArrayList(policyValue));
+            setPolicyResource(policyResource);
+
+            init();
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/818e1d35/agents-common/src/test/resources/policyengine/test_policyengine_hdfs_resourcespec.json
----------------------------------------------------------------------
diff --git a/agents-common/src/test/resources/policyengine/test_policyengine_hdfs_resourcespec.json b/agents-common/src/test/resources/policyengine/test_policyengine_hdfs_resourcespec.json
new file mode 100644
index 0000000..da0a629
--- /dev/null
+++ b/agents-common/src/test/resources/policyengine/test_policyengine_hdfs_resourcespec.json
@@ -0,0 +1,38 @@
+{
+  "serviceName":"hdfsdev",
+
+  "serviceDef":{
+    "name":"hdfs",
+    "id":1,
+    "resources":[
+    {"name":"path","type":"path","level":1,"mandatory":true,"lookupSupported":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":1,"name":"allow-read-to-finance under /finance/rest*ricted/","isEnabled":true,"isAuditEnabled":true,
+     "resources":{"path":{"values":["/finance/rest*ricted/"],"isRecursive":true}},
+     "policyItems":[
+       {"accesses":[{"type":"read","isAllowed":true}],"users":[],"groups":["finance"],"delegateAdmin":false, "conditions":[ ] }
+     ]
+    }
+  ],
+
+  "tests":[
+    {"name":"ALLOW 'read /finance/restricted/tmp/sales.db' for g=finance",
+     "request":{
+      "resource":{"elements":{"path":"/finance/restricted/tmp/sales.db"}},
+      "accessType":"read","user":"user1","userGroups":["finance"],"requestData":"read /finance/restricted/tmp/sales.db"
+     },
+     "result":{"isAudited":true,"isAllowed":true,"policyId":1}
+    }
+  ]
+}
+