You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by ma...@apache.org on 2015/04/02 22:50:32 UTC

incubator-ranger git commit: RANGER-365 Policy validation: Only users with admin role can create policies having excludes resource

Repository: incubator-ranger
Updated Branches:
  refs/heads/master be34cc292 -> 0d9a31a93


RANGER-365 Policy validation: Only users with admin role can create policies having excludes 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/0d9a31a9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ranger/tree/0d9a31a9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ranger/diff/0d9a31a9

Branch: refs/heads/master
Commit: 0d9a31a93af7b5540444ebe45d53073adc0c746b
Parents: be34cc2
Author: Alok Lal <al...@hortonworks.com>
Authored: Wed Apr 1 20:34:26 2015 -0700
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Apr 2 13:46:57 2015 -0700

----------------------------------------------------------------------
 .../RangerPolicyResourceSignature.java          |  19 ++
 .../model/validation/RangerPolicyValidator.java | 112 +++++----
 .../validation/RangerServiceDefValidator.java   |  19 ++
 .../ranger/plugin/util/RangerObjectFactory.java |  19 ++
 .../TestRangerPolicyResourceSignature.java      |  19 ++
 .../validation/TestRangerPolicyValidator.java   | 228 +++++++++++--------
 .../TestRangerServiceDefValidator.java          |  19 ++
 .../org/apache/ranger/rest/ServiceREST.java     |  10 +-
 8 files changed, 306 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyResourceSignature.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyResourceSignature.java b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyResourceSignature.java
index 0952ae8..e95d1e3 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyResourceSignature.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyResourceSignature.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyValidator.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyValidator.java b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyValidator.java
index b7500bd..36fc550 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyValidator.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerPolicyValidator.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 import java.util.ArrayList;
@@ -29,13 +48,13 @@ public class RangerPolicyValidator extends RangerValidator {
 		super(store);
 	}
 
-	public void validate(RangerPolicy policy, Action action) throws Exception {
+	public void validate(RangerPolicy policy, Action action, boolean isAdmin) throws Exception {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("==> RangerPolicyValidator.validate(%s, %s)", policy, action));
+			LOG.debug(String.format("==> RangerPolicyValidator.validate(%s, %s, %s)", policy, action, isAdmin));
 		}
 
 		List<ValidationFailureDetails> failures = new ArrayList<ValidationFailureDetails>();
-		boolean valid = isValid(policy, action, failures);
+		boolean valid = isValid(policy, action, isAdmin, failures);
 		String message = "";
 		try {
 			if (!valid) {
@@ -44,7 +63,7 @@ public class RangerPolicyValidator extends RangerValidator {
 			}
 		} finally {
 			if(LOG.isDebugEnabled()) {
-				LOG.debug(String.format("<== RangerPolicyValidator.validate(%s, %s): %s, reason[%s]", policy, action, valid, message));
+				LOG.debug(String.format("<== RangerPolicyValidator.validate(%s, %s, %s): %s, reason[%s]", policy, action, isAdmin, valid, message));
 			}
 		}
 	}
@@ -83,9 +102,9 @@ public class RangerPolicyValidator extends RangerValidator {
 		return valid;
 	}
 
-	boolean isValid(RangerPolicy policy, Action action, List<ValidationFailureDetails> failures) {
+	boolean isValid(RangerPolicy policy, Action action, boolean isAdmin, List<ValidationFailureDetails> failures) {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("==> RangerPolicyValidator.isValid(%s, %s, %s)", policy, action, failures));
+			LOG.debug(String.format("==> RangerPolicyValidator.isValid(%s, %s, %s, %s)", policy, action, isAdmin, failures));
 		}
 
 		if (!(action == Action.CREATE || action == Action.UPDATE)) {
@@ -203,34 +222,36 @@ public class RangerPolicyValidator extends RangerValidator {
 					valid = isValidPolicyItems(policyItems, failures, serviceDef) && valid;
 				}
 			}
-			valid = isValidResources(policy, failures, action, serviceDef, serviceName) && valid;
+			valid = isValidResources(policy, failures, action, isAdmin, serviceDef, serviceName) && valid;
 		}
 		
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("<== RangerPolicyValidator.isValid(%s, %s, %s): %s", policy, action, failures, valid));
+			LOG.debug(String.format("<== RangerPolicyValidator.isValid(%s, %s, %s, %s): %s", policy, action, isAdmin, failures, valid));
 		}
 		return valid;
 	}
 	
-	boolean isValidResources(RangerPolicy policy, final List<ValidationFailureDetails> failures, Action action, final RangerServiceDef serviceDef, final String serviceName) {
+	boolean isValidResources(RangerPolicy policy, final List<ValidationFailureDetails> failures, Action action, boolean isAdmin, final RangerServiceDef serviceDef, final String serviceName) {
 		
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("==> RangerPolicyValidator.isValidResources(%s, %s, %s, %s, %s)", policy, failures, action, serviceDef, serviceName));
+			LOG.debug(String.format("==> RangerPolicyValidator.isValidResources(%s, %s, %s, %s, %s, %s)", policy, failures, action, isAdmin, serviceDef, serviceName));
 		}
 		
 		boolean valid = true;
 		if (serviceDef != null) { // following checks can't be done meaningfully otherwise
 			valid = isValidResourceNames(policy, failures, serviceDef);
 			Map<String, RangerPolicyResource> resourceMap = policy.getResources();
-			valid = isValidResourceValues(resourceMap, failures, serviceDef) && valid;
-			valid = isValidResourceFlags(resourceMap, failures, serviceDef.getResources(), serviceDef.getName(), policy.getName()) && valid;
+			if (resourceMap != null) { // following checks can't be done meaningfully otherwise
+				valid = isValidResourceValues(resourceMap, failures, serviceDef) && valid;
+				valid = isValidResourceFlags(resourceMap, failures, serviceDef.getResources(), serviceDef.getName(), policy.getName(), isAdmin) && valid;
+			}
 		}
 		if (StringUtils.isNotBlank(serviceName)) { // resource uniqueness check cannot be done meaningfully otherwise
 			valid = isPolicyResourceUnique(policy, failures, action, serviceName) && valid;
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("<== RangerPolicyValidator.isValidResources(%s, %s, %s, %s, %s): %s", policy, failures, action, serviceDef, serviceName, valid));
+			LOG.debug(String.format("<== RangerPolicyValidator.isValidResources(%s, %s, %s, %s, %s, %s): %s", policy, failures, action, isAdmin, serviceDef, serviceName, valid));
 		}
 		return valid;
 	}
@@ -309,15 +330,13 @@ public class RangerPolicyValidator extends RangerValidator {
 	}
 	
 	boolean isValidResourceFlags(final Map<String, RangerPolicyResource> inputPolicyResources, final List<ValidationFailureDetails> failures,
-			final List<RangerResourceDef> resourceDefs, final String serviceDefName, final String policyName) {
+			final List<RangerResourceDef> resourceDefs, final String serviceDefName, final String policyName, boolean isAdmin) {
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("==> RangerPolicyValidator.isValidResourceFlags(%s, %s, %s, %s, %s)", inputPolicyResources, failures, resourceDefs, serviceDefName, policyName));
+			LOG.debug(String.format("==> RangerPolicyValidator.isValidResourceFlags(%s, %s, %s, %s, %s, %s)", inputPolicyResources, failures, resourceDefs, serviceDefName, policyName, isAdmin));
 		}
 
 		boolean valid = true;
-		if (inputPolicyResources == null) {
-			LOG.debug("isValidResourceFlags: resourceMap is null");
-		} else if (resourceDefs == null) {
+		if (resourceDefs == null) {
 			LOG.debug("isValidResourceFlags: service Def is null");
 		} else {
 			Map<String, RangerPolicyResource> policyResources = getPolicyResourceWithLowerCaseKeys(inputPolicyResources);
@@ -345,13 +364,22 @@ public class RangerPolicyValidator extends RangerValidator {
 						}
 					} else {
 						boolean excludesSupported = Boolean.TRUE.equals(resourceDef.getExcludesSupported()); // could be null
-						boolean policyIsExcludes = Boolean.TRUE.equals(policyResource.getIsExcludes()); // could be null
-						if (policyIsExcludes && !excludesSupported) {
+						boolean policyResourceIsExcludes = Boolean.TRUE.equals(policyResource.getIsExcludes()); // could be null
+						if (policyResourceIsExcludes && !excludesSupported) {
 							failures.add(new ValidationFailureDetailsBuilder()
 								.field("isExcludes")
 								.subField(resourceName)
 								.isSemanticallyIncorrect()
-								.becauseOf("isExcludes specified as [" + policyIsExcludes + "] for resource [" + resourceName + "] which doesn't support isExcludes")
+								.becauseOf("isExcludes specified as [" + policyResourceIsExcludes + "] for resource [" + resourceName + "] which doesn't support isExcludes")
+								.build());
+							valid = false;
+						}
+						if (policyResourceIsExcludes && !isAdmin) {
+							failures.add(new ValidationFailureDetailsBuilder()
+								.field("isExcludes")
+								.subField("isAdmin")
+								.isSemanticallyIncorrect()
+								.becauseOf("isExcludes specified as [" + policyResourceIsExcludes + "] for resource [" + resourceName + "].  Insufficient permissions to create excludes policy.")
 								.build());
 							valid = false;
 						}
@@ -372,7 +400,7 @@ public class RangerPolicyValidator extends RangerValidator {
 		}
 
 		if(LOG.isDebugEnabled()) {
-			LOG.debug(String.format("<== RangerPolicyValidator.isValidResourceFlags(%s, %s, %s, %s, %s): %s", inputPolicyResources, failures, resourceDefs, serviceDefName, policyName, valid));
+			LOG.debug(String.format("<== RangerPolicyValidator.isValidResourceFlags(%s, %s, %s, %s, %s, %s): %s", inputPolicyResources, failures, resourceDefs, serviceDefName, policyName, isAdmin, valid));
 		}
 		return valid;
 	}
@@ -383,29 +411,23 @@ public class RangerPolicyValidator extends RangerValidator {
 		}
 
 		boolean valid = true;
-		if (resourceMap == null) {
-			LOG.debug("isValidResourceValues: resourceMap is null");
-		} else if (serviceDef == null) {
-			LOG.debug("isValidResourceValues: service Def is null");
-		} else {
-			Map<String, String> validationRegExMap = getValidationRegExes(serviceDef);
-			for (Map.Entry<String, RangerPolicyResource> entry : resourceMap.entrySet()) {
-				String name = entry.getKey();
-				RangerPolicyResource policyResource = entry.getValue();
-				if (validationRegExMap.containsKey(name) && policyResource != null && CollectionUtils.isNotEmpty(policyResource.getValues())) {
-					String regEx = validationRegExMap.get(name);
-					for (String aValue : policyResource.getValues()) {
-						if (StringUtils.isBlank(aValue)) {
-							LOG.debug("resource value was blank");
-						} else if (!aValue.matches(regEx)) {
-							failures.add(new ValidationFailureDetailsBuilder()
-								.field("resource-values")
-								.subField(name)
-								.isSemanticallyIncorrect()
-								.becauseOf("resources value[" + aValue + "] does not match validation regex[" + regEx + "] defined on service-def[" + serviceDef.getName() + "]")
-								.build());
-							valid = false;
-						}
+		Map<String, String> validationRegExMap = getValidationRegExes(serviceDef);
+		for (Map.Entry<String, RangerPolicyResource> entry : resourceMap.entrySet()) {
+			String name = entry.getKey();
+			RangerPolicyResource policyResource = entry.getValue();
+			if (validationRegExMap.containsKey(name) && policyResource != null && CollectionUtils.isNotEmpty(policyResource.getValues())) {
+				String regEx = validationRegExMap.get(name);
+				for (String aValue : policyResource.getValues()) {
+					if (StringUtils.isBlank(aValue)) {
+						LOG.debug("resource value was blank");
+					} else if (!aValue.matches(regEx)) {
+						failures.add(new ValidationFailureDetailsBuilder()
+							.field("resource-values")
+							.subField(name)
+							.isSemanticallyIncorrect()
+							.becauseOf("resources value[" + aValue + "] does not match validation regex[" + regEx + "] defined on service-def[" + serviceDef.getName() + "]")
+							.build());
+						valid = false;
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerServiceDefValidator.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerServiceDefValidator.java b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerServiceDefValidator.java
index a870e28..0ef7ff9 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerServiceDefValidator.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/model/validation/RangerServiceDefValidator.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerObjectFactory.java
----------------------------------------------------------------------
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerObjectFactory.java b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerObjectFactory.java
index e02c968..0faa20d 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerObjectFactory.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/util/RangerObjectFactory.java
@@ -1,3 +1,22 @@
+/*
+ * 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.util;
 
 import org.apache.ranger.plugin.model.RangerPolicy;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyResourceSignature.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyResourceSignature.java b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyResourceSignature.java
index 7d34d96..a8b03ce 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyResourceSignature.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyResourceSignature.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyValidator.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyValidator.java b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyValidator.java
index edf19d5..55404b2 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyValidator.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerPolicyValidator.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 import static org.junit.Assert.assertFalse;
@@ -142,22 +161,24 @@ public class TestRangerPolicyValidator {
 		// null value for audit is treated as audit on.
 		for (Action action : cu) {
 			for (Boolean auditEnabled : new Boolean[] { null, true } ) {
-				when(_policy.getIsAuditEnabled()).thenReturn(auditEnabled);
-				if (action == Action.CREATE) {
-					when(_policy.getId()).thenReturn(7L);
-					when(_policy.getName()).thenReturn("policy-name-1");
-					assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, _failures));
-					assertTrue(_failures.isEmpty());
-				} else {
-					// update should work both when by-name is found or not, since nothing found by-name means name is being updated.
-					when(_policy.getId()).thenReturn(8L);
-					when(_policy.getName()).thenReturn("policy-name-1");
-					assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, _failures));
-					assertTrue(_failures.isEmpty());
-
-					when(_policy.getName()).thenReturn("policy-name-2");
-					assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, _failures));
-					assertTrue(_failures.isEmpty());
+				for (boolean isAdmin : new boolean[] { true, false }) {
+					when(_policy.getIsAuditEnabled()).thenReturn(auditEnabled);
+					if (action == Action.CREATE) {
+						when(_policy.getId()).thenReturn(7L);
+						when(_policy.getName()).thenReturn("policy-name-1");
+						assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, isAdmin, _failures));
+						assertTrue(_failures.isEmpty());
+					} else {
+						// update should work both when by-name is found or not, since nothing found by-name means name is being updated.
+						when(_policy.getId()).thenReturn(8L);
+						when(_policy.getName()).thenReturn("policy-name-1");
+						assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, isAdmin, _failures));
+						assertTrue(_failures.isEmpty());
+	
+						when(_policy.getName()).thenReturn("policy-name-2");
+						assertTrue("" + action + ", " + auditEnabled, _validator.isValid(_policy, action, isAdmin, _failures));
+						assertTrue(_failures.isEmpty());
+					}
 				}
 			}
 		}
@@ -166,15 +187,17 @@ public class TestRangerPolicyValidator {
 		when(_policy.getPolicyItems()).thenReturn(policyItems);
 		when(_policy.getIsAuditEnabled()).thenReturn(false);
 		for (Action action : cu) {
-			if (action == Action.CREATE) {
-				when(_policy.getId()).thenReturn(7L);
-				when(_policy.getName()).thenReturn("policy-name-1");
-			} else {
-				when(_policy.getId()).thenReturn(8L);
-				when(_policy.getName()).thenReturn("policy-name-2");
+			for (boolean isAdmin : new boolean[] { true, false}) {
+				if (action == Action.CREATE) {
+					when(_policy.getId()).thenReturn(7L);
+					when(_policy.getName()).thenReturn("policy-name-1");
+				} else {
+					when(_policy.getId()).thenReturn(8L);
+					when(_policy.getName()).thenReturn("policy-name-2");
+				}
+				assertTrue("" + action , _validator.isValid(_policy, action, isAdmin, _failures));
+				assertTrue(_failures.isEmpty());
 			}
-			assertTrue("" + action , _validator.isValid(_policy, action, _failures));
-			assertTrue(_failures.isEmpty());
 		}
 		
 		// above succeeded as service def did not have any resources on it, mandatory or otherwise.
@@ -198,7 +221,7 @@ public class TestRangerPolicyValidator {
 				when(_policy.getId()).thenReturn(8L);
 				when(_policy.getName()).thenReturn("policy-name-2");
 			}
-			assertTrue("" + action , _validator.isValid(_policy, action, _failures));
+			assertTrue("" + action , _validator.isValid(_policy, action, true, _failures)); // since policy resource has excludes admin privilages would be required
 			assertTrue(_failures.isEmpty());
 		}
 	}
@@ -208,21 +231,24 @@ public class TestRangerPolicyValidator {
 	}
 	
 	void checkFailure_isValid(Action action, String errorType, String field, String subField) {
-		_failures.clear();
-		assertFalse(_validator.isValid(_policy, action, _failures));
-		switch (errorType) {
-		case "missing":
-			_utils.checkFailureForMissingValue(_failures, field, subField);
-			break;
-		case "semantic":
-			_utils.checkFailureForSemanticError(_failures, field, subField);
-			break;
-		case "internal error":
-			_utils.checkFailureForInternalError(_failures);
-			break;
-		default:
-			fail("Unsupported errorType[" + errorType + "]");
-			break;
+		
+		for (boolean isAdmin : new boolean[] { true, false}) {
+			_failures.clear();
+			assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+			switch (errorType) {
+			case "missing":
+				_utils.checkFailureForMissingValue(_failures, field, subField);
+				break;
+			case "semantic":
+				_utils.checkFailureForSemanticError(_failures, field, subField);
+				break;
+			case "internal error":
+				_utils.checkFailureForInternalError(_failures);
+				break;
+			default:
+				fail("Unsupported errorType[" + errorType + "]");
+				break;
+			}
 		}
 	}
 	
@@ -282,40 +308,46 @@ public class TestRangerPolicyValidator {
 		existingPolicies.add(existingPolicy);
 		existingPolicy = mock(RangerPolicy.class);
 		existingPolicies.add(existingPolicy);
-		_failures.clear(); assertFalse(_validator.isValid(_policy, Action.UPDATE, _failures));
-		_utils.checkFailureForInternalError(_failures);
+		for (boolean isAdmin : new boolean[] { true, false }) {
+			_failures.clear(); assertFalse(_validator.isValid(_policy, Action.UPDATE, isAdmin, _failures));
+			_utils.checkFailureForInternalError(_failures);
+		}
 		
 		// policy must have service name on it and it should be valid
 		when(_policy.getName()).thenReturn("policy-name");
 		for (Action action : cu) {
-			when(_policy.getService()).thenReturn(null);
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "service");
-
-			when(_policy.getService()).thenReturn("");
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "service");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				when(_policy.getService()).thenReturn(null);
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "service");
+	
+				when(_policy.getService()).thenReturn("");
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "service");
+			}
 		}
 		
 		// service name should be valid
 		when(_store.getServiceByName("service-name")).thenReturn(null);
 		when(_store.getServiceByName("another-service-name")).thenThrow(new Exception());
 		for (Action action : cu) {
-			when(_policy.getService()).thenReturn(null);
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "service");
-
-			when(_policy.getService()).thenReturn(null);
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "service");
-
-			when(_policy.getService()).thenReturn("service-name");
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForSemanticError(_failures, "service");
-
-			when(_policy.getService()).thenReturn("another-service-name");
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForSemanticError(_failures, "service");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				when(_policy.getService()).thenReturn(null);
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "service");
+	
+				when(_policy.getService()).thenReturn(null);
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "service");
+	
+				when(_policy.getService()).thenReturn("service-name");
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForSemanticError(_failures, "service");
+	
+				when(_policy.getService()).thenReturn("another-service-name");
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForSemanticError(_failures, "service");
+			}
 		}
 		
 		// policy must contain at least one policy item
@@ -324,14 +356,16 @@ public class TestRangerPolicyValidator {
 		RangerService service = mock(RangerService.class);
 		when(_store.getServiceByName("service-name")).thenReturn(service);
 		for (Action action : cu) {
-			// when it is null
-			when(_policy.getPolicyItems()).thenReturn(null);
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "policy items");
-			// or when it is not null but empty.
-			when(_policy.getPolicyItems()).thenReturn(policyItems);
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "policy items");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				// when it is null
+				when(_policy.getPolicyItems()).thenReturn(null);
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "policy items");
+				// or when it is not null but empty.
+				when(_policy.getPolicyItems()).thenReturn(policyItems);
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "policy items");
+			}
 		}
 		
 		// these are known good policy items -- same as used above in happypath
@@ -341,18 +375,22 @@ public class TestRangerPolicyValidator {
 		when(service.getType()).thenReturn("service-type");
 		when(_store.getServiceDefByName("service-type")).thenReturn(null);
 		for (Action action : cu) {
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForInternalError(_failures, "policy service def");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForInternalError(_failures, "policy service def");
+			}
 		}
 		
 		// service-def should contain the right access types on it.
 		_serviceDef = _utils.createServiceDefWithAccessTypes(accessTypes_bad);
 		when(_store.getServiceDefByName("service-type")).thenReturn(_serviceDef);
 		for (Action action : cu) {
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForSemanticError(_failures, "policy item access type");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForSemanticError(_failures, "policy item access type");
+			}
 		}
-
+		
 		// create the right service def with right resource defs - this is the same as in the happypath test above.
 		_serviceDef = _utils.createServiceDefWithAccessTypes(accessTypes);
 		when(_store.getPolicies(filter)).thenReturn(null);
@@ -364,12 +402,14 @@ public class TestRangerPolicyValidator {
 		Map<String, RangerPolicyResource> policyResources = _utils.createPolicyResourceMap(policyResourceMap_bad);
 		when(_policy.getResources()).thenReturn(policyResources);
 		for (Action action : cu) {
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForMissingValue(_failures, "resources", "tbl"); // for missing resource: tbl
-			_utils.checkFailureForSemanticError(_failures, "resources", "extra"); // for spurious resource: "extra"
-			_utils.checkFailureForSemanticError(_failures, "resource-values", "col"); // for spurious resource: "extra"
-			_utils.checkFailureForSemanticError(_failures, "isRecursive", "db"); // for specifying it as true when def did not allow it
-			_utils.checkFailureForSemanticError(_failures, "isExcludes", "col"); // for specifying it as true when def did not allow it
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForMissingValue(_failures, "resources", "tbl"); // for missing resource: tbl
+				_utils.checkFailureForSemanticError(_failures, "resources", "extra"); // for spurious resource: "extra"
+				_utils.checkFailureForSemanticError(_failures, "resource-values", "col"); // for spurious resource: "extra"
+				_utils.checkFailureForSemanticError(_failures, "isRecursive", "db"); // for specifying it as true when def did not allow it
+				_utils.checkFailureForSemanticError(_failures, "isExcludes", "col"); // for specifying it as true when def did not allow it
+			}
 		}
 		
 		// create the right resource def but let it clash with another policy with matching resource-def
@@ -380,8 +420,10 @@ public class TestRangerPolicyValidator {
 		// we are doctoring the factory to always return the same signature
 		when(_factory.createPolicyResourceSignature(anyPolicy())).thenReturn(new RangerPolicyResourceSignature("blah"));
 		for (Action action : cu) {
-			_failures.clear(); assertFalse(_validator.isValid(_policy, action, _failures));
-			_utils.checkFailureForSemanticError(_failures, "resources");
+			for (boolean isAdmin : new boolean[] { true, false }) {
+				_failures.clear(); assertFalse(_validator.isValid(_policy, action, isAdmin, _failures));
+				_utils.checkFailureForSemanticError(_failures, "resources");
+			}
 		}
 	}
 	
@@ -540,20 +582,22 @@ public class TestRangerPolicyValidator {
 	
 	@Test
 	public final void test_isValidResourceFlags_happyPath() {
-		// passing null values effectively bypasses the filter
-		assertTrue(_validator.isValidResourceFlags(null, _failures, null, "a-service-def", "a-policy"));
-		// so does passing in empty collections
+
 		Map<String, RangerPolicyResource> resourceMap = _utils.createPolicyResourceMap(policyResourceMap_happyPath);
 		List<RangerResourceDef> resourceDefs = _utils.createResourceDefs2(resourceDef_happyPath);
 		when(_serviceDef.getResources()).thenReturn(resourceDefs);
-		assertTrue(_validator.isValidResourceFlags(resourceMap, _failures, resourceDefs, "a-service-def", "a-policy"));
+		assertTrue(_validator.isValidResourceFlags(resourceMap, _failures, resourceDefs, "a-service-def", "a-policy", true));
+
+		// Since one of the resource has excludes set to true, without admin privilages it should fail and contain appropriate error messages
+		assertFalse(_validator.isValidResourceFlags(resourceMap, _failures, resourceDefs, "a-service-def", "a-policy", false));
+		_utils.checkFailureForSemanticError(_failures, "isExcludes", "isAdmin");
 	}
 
 	private Object[][] policyResourceMap_failures = new Object[][] {
 			// { "resource-name", "values" "isExcludes", "isRecursive" }
 			// values collection is null as it isn't relevant to the part being tested with this data
 			{ "db", null, true, true },    // ok: def has true for both  
-			{ "tbl", null, true, null },   // excludes: def==false, policy==true  
+			{ "tbl", null, true, null },   // excludes: definition does not allow excludes by resource has it set to true  
 			{ "col", null, false, true }    // recursive: def==null (i.e. false), policy==true
 	};
 	
@@ -563,9 +607,11 @@ public class TestRangerPolicyValidator {
 		List<RangerResourceDef> resourceDefs = _utils.createResourceDefs2(resourceDef_happyPath);
 		Map<String, RangerPolicyResource> resourceMap = _utils.createPolicyResourceMap(policyResourceMap_failures);
 		when(_serviceDef.getResources()).thenReturn(resourceDefs);
-		assertFalse(_validator.isValidResourceFlags(resourceMap, _failures, resourceDefs, "a-service-def", "a-policy"));
+		// should not error out on 
+		assertFalse(_validator.isValidResourceFlags(resourceMap, _failures, resourceDefs, "a-service-def", "a-policy", false));
 		_utils.checkFailureForSemanticError(_failures, "isExcludes", "tbl");
 		_utils.checkFailureForSemanticError(_failures, "isRecursive", "col");
+		_utils.checkFailureForSemanticError(_failures, "isExcludes", "isAdmin");
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerServiceDefValidator.java
----------------------------------------------------------------------
diff --git a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerServiceDefValidator.java b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerServiceDefValidator.java
index 1019aa1..1409d2c 100644
--- a/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerServiceDefValidator.java
+++ b/agents-common/src/test/java/org/apache/ranger/plugin/model/validation/TestRangerServiceDefValidator.java
@@ -1,3 +1,22 @@
+/*
+ * 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.model.validation;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-ranger/blob/0d9a31a9/security-admin/src/main/java/org/apache/ranger/rest/ServiceREST.java
----------------------------------------------------------------------
diff --git a/security-admin/src/main/java/org/apache/ranger/rest/ServiceREST.java b/security-admin/src/main/java/org/apache/ranger/rest/ServiceREST.java
index ce175f1..d7182a3 100644
--- a/security-admin/src/main/java/org/apache/ranger/rest/ServiceREST.java
+++ b/security-admin/src/main/java/org/apache/ranger/rest/ServiceREST.java
@@ -45,6 +45,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ranger.admin.client.datatype.RESTResponse;
 import org.apache.ranger.biz.AssetMgr;
+import org.apache.ranger.biz.RangerBizUtil;
 import org.apache.ranger.biz.ServiceDBStore;
 import org.apache.ranger.biz.ServiceMgr;
 import org.apache.ranger.biz.XUserMgr;
@@ -129,6 +130,9 @@ public class ServiceREST {
 	
 	@Autowired
 	RangerSearchUtil searchUtil;
+	
+    @Autowired
+    RangerBizUtil bizUtil;
 
 	// this indirection for validation via a factory exists only for testability
 	// TODO move the instantiation to DI framework?
@@ -830,10 +834,10 @@ public class ServiceREST {
 		}
 
 		RangerPolicy ret = null;
-
+		
 		try {
 //			RangerPolicyValidator validator = validatorFactory.getPolicyValidator(svcStore);
-//			validator.validate(policy, Action.CREATE);
+//			validator.validate(policy, Action.CREATE, bizUtil.isAdmin());
 			ret = svcStore.createPolicy(policy);
 		} catch(Exception excp) {
 			LOG.error("createPolicy(" + policy + ") failed", excp);
@@ -860,7 +864,7 @@ public class ServiceREST {
 
 		try {
 //			RangerPolicyValidator validator = validatorFactory.getPolicyValidator(svcStore);
-//			validator.validate(policy, Action.UPDATE);
+//			validator.validate(policy, Action.UPDATE, bizUtil.isAdmin());
 			ret = svcStore.updatePolicy(policy);
 		} catch(Exception excp) {
 			LOG.error("updatePolicy(" + policy + ") failed", excp);