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 2020/04/06 17:03:18 UTC

[ranger] branch master updated (7b66386 -> 8b963d3)

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

madhan pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/ranger.git.


    from 7b66386  RANGER-2777:Ranger Knox Plugin failed to download Roles
     new f079713  RANGER-2779: updated tag-sync to process Atlas notifications for ADLS-Gen2 entities
     new 8b963d3  RANGER-2780: updated RangerBasePlugin to support audit-excluded-users

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../hadoop/config/RangerPluginConfig.java          |  39 +++++
 .../ranger/plugin/policyengine/PolicyEngine.java   |  63 --------
 .../plugin/policyengine/RangerPolicyEngine.java    |   4 -
 .../policyengine/RangerPolicyEngineImpl.java       | 175 +++++++++++++++------
 .../ranger/plugin/service/RangerBasePlugin.java    |  20 +--
 .../plugin/policyengine/TestPolicyEngine.java      |  36 ++++-
 ...cyengine_audit_exclude_users_groups_roles.json} | 175 ++++++++++++++++-----
 .../source/atlas/AtlasAdlsResourceMapper.java      | 166 +++++++++++++++++++
 .../source/atlas/AtlasResourceMapperUtil.java      |   2 +
 .../tagsync/process/TestAdlsResourceMapper.java    | 140 +++++++++++++++++
 10 files changed, 649 insertions(+), 171 deletions(-)
 copy agents-common/src/test/resources/policyengine/{test_policyengine_super_user_groups.json => test_policyengine_audit_exclude_users_groups_roles.json} (54%)
 create mode 100644 tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasAdlsResourceMapper.java
 create mode 100644 tagsync/src/test/java/org/apache/ranger/tagsync/process/TestAdlsResourceMapper.java


[ranger] 01/02: RANGER-2779: updated tag-sync to process Atlas notifications for ADLS-Gen2 entities

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

madhan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ranger.git

commit f0797138f6bdf9432400d1c2743b5ea7351a0c8d
Author: Madhan Neethiraj <ma...@apache.org>
AuthorDate: Sat Apr 4 17:31:31 2020 -0700

    RANGER-2779: updated tag-sync to process Atlas notifications for ADLS-Gen2 entities
---
 .../source/atlas/AtlasAdlsResourceMapper.java      | 166 +++++++++++++++++++++
 .../source/atlas/AtlasResourceMapperUtil.java      |   2 +
 .../tagsync/process/TestAdlsResourceMapper.java    | 140 +++++++++++++++++
 3 files changed, 308 insertions(+)

diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasAdlsResourceMapper.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasAdlsResourceMapper.java
new file mode 100644
index 0000000..e38f5fa
--- /dev/null
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasAdlsResourceMapper.java
@@ -0,0 +1,166 @@
+/*
+ * 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.tagsync.source.atlas;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyResource;
+import org.apache.ranger.plugin.model.RangerServiceResource;
+import org.apache.ranger.tagsync.source.atlasrest.RangerAtlasEntity;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class AtlasAdlsResourceMapper extends AtlasResourceMapper {
+	public static final String ENTITY_TYPE_ADLS_GEN2_ACCOUNT   = "adls_gen2_account";
+	public static final String ENTITY_TYPE_ADLS_GEN2_CONTAINER = "adls_gen2_container";
+	public static final String ENTITY_TYPE_ADLS_GEN2_DIRECTORY = "adls_gen2_directory";
+
+	public static final String RANGER_TYPE_ADLS_GEN2_ACCOUNT       = "storageaccount";
+	public static final String RANGER_TYPE_ADLS_GEN2_CONTAINER     = "container";
+	public static final String RANGER_TYPE_ADLS_GEN2_RELATIVE_PATH = "relativepath";
+
+	public static final String[] SUPPORTED_ENTITY_TYPES = { ENTITY_TYPE_ADLS_GEN2_ACCOUNT, ENTITY_TYPE_ADLS_GEN2_CONTAINER, ENTITY_TYPE_ADLS_GEN2_DIRECTORY };
+
+	private static final String SEP_PROTOCOL               = "://";
+	private static final String SEP_CONTAINER              = "@";
+	private static final String SEP_ACCOUNT                = ".";
+	private static final String SEP_RELATIVE_PATH          = "/";
+	private static final int    IDX_RESOURCE_ACCOUNT       = 0;
+	private static final int    IDX_RESOURCE_CONTAINER     = 1;
+	private static final int    IDX_RESOURCE_RELATIVE_PATH = 2;
+	private static final int    IDX_CLUSTER_NAME           = 3;
+	private static final int    RESOURCE_COUNT             = 4;
+
+
+	public AtlasAdlsResourceMapper() {
+		super("adls", SUPPORTED_ENTITY_TYPES);
+	}
+
+	@Override
+	public RangerServiceResource buildResource(final RangerAtlasEntity entity) throws Exception {
+		String qualifiedName = (String)entity.getAttributes().get(AtlasResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME);
+
+		if (StringUtils.isEmpty(qualifiedName)) {
+			throw new Exception("attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "' not found in entity");
+		}
+
+		String[] resources   = parseQualifiedName(qualifiedName);
+		String   clusterName = resources[IDX_CLUSTER_NAME];
+		String   accountName = resources[IDX_RESOURCE_ACCOUNT];
+
+		if (StringUtils.isEmpty(clusterName)) {
+			throwExceptionWithMessage("cluster-name not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
+		}
+
+		if (StringUtils.isEmpty(accountName)) {
+			throwExceptionWithMessage("account-name not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
+		}
+
+		String entityType  = entity.getTypeName();
+		String entityGuid  = entity.getGuid();
+		String serviceName = getRangerServiceName(clusterName);
+
+		Map<String, RangerPolicyResource> elements = new HashMap<String, RangerPolicyResource>();
+
+		if (StringUtils.equals(entityType, ENTITY_TYPE_ADLS_GEN2_ACCOUNT)) {
+			elements.put(RANGER_TYPE_ADLS_GEN2_ACCOUNT, new RangerPolicyResource(accountName));
+		} else if (StringUtils.equals(entityType, ENTITY_TYPE_ADLS_GEN2_CONTAINER)) {
+			String containerName = resources[IDX_RESOURCE_CONTAINER];
+
+			if (StringUtils.isEmpty(containerName)) {
+				throwExceptionWithMessage("container-name not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
+			}
+
+			elements.put(RANGER_TYPE_ADLS_GEN2_ACCOUNT, new RangerPolicyResource(accountName));
+			elements.put(RANGER_TYPE_ADLS_GEN2_CONTAINER, new RangerPolicyResource(containerName));
+		} else if (StringUtils.equals(entityType, ENTITY_TYPE_ADLS_GEN2_DIRECTORY)) {
+			String containerName = resources[IDX_RESOURCE_CONTAINER];
+			String relativePath  = resources[IDX_RESOURCE_RELATIVE_PATH];
+
+			if (StringUtils.isEmpty(containerName)) {
+				throwExceptionWithMessage("container-name not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
+			}
+
+			if (StringUtils.isEmpty(relativePath)) {
+				throwExceptionWithMessage("relative-path not found in attribute '" +  ENTITY_ATTRIBUTE_QUALIFIED_NAME + "': " + qualifiedName);
+			}
+
+			elements.put(RANGER_TYPE_ADLS_GEN2_ACCOUNT, new RangerPolicyResource(accountName));
+			elements.put(RANGER_TYPE_ADLS_GEN2_CONTAINER, new RangerPolicyResource(containerName));
+			elements.put(RANGER_TYPE_ADLS_GEN2_RELATIVE_PATH, new RangerPolicyResource(relativePath));
+		} else {
+			throwExceptionWithMessage("unrecognized entity-type: " + entityType);
+		}
+
+		RangerServiceResource ret = new RangerServiceResource(entityGuid, serviceName, elements);
+
+		return ret;
+	}
+
+	/* qualifiedName can be of format, depending upon the entity-type:
+	    adls_gen2_account:   abfs://<accountName>@<clusterName>
+	    adls_gen2_container: abfs://<containerName>@<accountName>.dfs.core.windows.net@<clusterName>
+	    adls_gen2_directory: abfs://<containerName>@<accountName>.dfs.core.windows.net/<relativePath>@<clusterName>
+	 */
+	private String[] parseQualifiedName(String qualifiedName) {
+		String[] ret = new String[RESOURCE_COUNT];
+
+		if(StringUtils.isNotBlank(qualifiedName)) {
+			int idxClusterNameSep = qualifiedName.lastIndexOf(CLUSTER_DELIMITER);
+
+			if (idxClusterNameSep != -1) {
+				ret[IDX_CLUSTER_NAME] = qualifiedName.substring(idxClusterNameSep + CLUSTER_DELIMITER.length());
+			}
+
+			int idxProtocolStart = qualifiedName.indexOf(SEP_PROTOCOL);
+
+			if (idxProtocolStart != -1) {
+				int idxResourceStart = idxProtocolStart + SEP_PROTOCOL.length();
+				int idxContainerSep  = qualifiedName.indexOf(SEP_CONTAINER, idxResourceStart);
+
+				if (idxContainerSep != -1) {
+					if (idxContainerSep == idxClusterNameSep) { // this is adls_gen2_account, so no containerName
+						ret[IDX_RESOURCE_ACCOUNT] = qualifiedName.substring(idxResourceStart, idxContainerSep);
+					} else {
+						ret[IDX_RESOURCE_CONTAINER] = qualifiedName.substring(idxResourceStart, idxContainerSep);
+
+						int idxAccountSep = qualifiedName.indexOf(SEP_ACCOUNT, idxContainerSep + SEP_CONTAINER.length());
+
+						if (idxAccountSep != -1) {
+							ret[IDX_RESOURCE_ACCOUNT] = qualifiedName.substring(idxContainerSep + SEP_CONTAINER.length(), idxAccountSep);
+
+							int idxRelativePath = qualifiedName.indexOf(SEP_RELATIVE_PATH, idxAccountSep + SEP_ACCOUNT.length());
+
+							if (idxRelativePath != -1) {
+								if (idxClusterNameSep == -1) {
+									ret[IDX_RESOURCE_RELATIVE_PATH] = qualifiedName.substring(idxRelativePath);
+								} else {
+									ret[IDX_RESOURCE_RELATIVE_PATH] = qualifiedName.substring(idxRelativePath, idxClusterNameSep);
+								}
+							}
+						}
+					}
+				}
+			}
+		}
+
+		return ret;
+	}
+}
diff --git a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
index cd2cb63..999c206 100644
--- a/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
+++ b/tagsync/src/main/java/org/apache/ranger/tagsync/source/atlas/AtlasResourceMapperUtil.java
@@ -90,6 +90,8 @@ public class AtlasResourceMapperUtil {
 		mapperNames.add("org.apache.ranger.tagsync.source.atlas.AtlasHbaseResourceMapper");
 		mapperNames.add("org.apache.ranger.tagsync.source.atlas.AtlasKafkaResourceMapper");
 
+		mapperNames.add(AtlasAdlsResourceMapper.class.getName());
+
 		if (StringUtils.isNotBlank(customMapperNames)) {
 			for (String customMapperName : customMapperNames.split(MAPPER_NAME_DELIMITER)) {
 				mapperNames.add(customMapperName.trim());
diff --git a/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestAdlsResourceMapper.java b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestAdlsResourceMapper.java
new file mode 100644
index 0000000..d0b4339
--- /dev/null
+++ b/tagsync/src/test/java/org/apache/ranger/tagsync/process/TestAdlsResourceMapper.java
@@ -0,0 +1,140 @@
+/*
+ * 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.tagsync.process;
+
+import org.apache.ranger.plugin.model.RangerServiceResource;
+import org.apache.ranger.tagsync.source.atlas.AtlasAdlsResourceMapper;
+import org.apache.ranger.tagsync.source.atlasrest.RangerAtlasEntity;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.apache.ranger.tagsync.source.atlas.AtlasAdlsResourceMapper.*;
+import static org.apache.ranger.tagsync.source.atlas.AtlasResourceMapper.ENTITY_ATTRIBUTE_QUALIFIED_NAME;
+
+
+public class TestAdlsResourceMapper {
+    private static final String ACCOUNT_QUALIFIED_NAME       = "abfs://myaccount@cl1";
+    private static final String CONTAINER_QUALIFIED_NAME     = "abfs://mycontainer@myaccount.dfs.core.windows.net@cl1";
+    private static final String RELATIVE_PATH_QUALIFIED_NAME = "abfs://mycontainer@myaccount.dfs.core.windows.net/tmp@cl1";
+
+    private static final String SERVICE_NAME                 = "cl1_adls";
+    private static final String ACCOUNT_NAME                 = "myaccount";
+    private static final String CONTAINER_NAME               = "mycontainer";
+    private static final String RELATIVE_PATH_NAME           = "/tmp";
+
+    AtlasAdlsResourceMapper resourceMapper = new AtlasAdlsResourceMapper();
+
+    @Test
+    public void testAccountEntity() throws Exception {
+        RangerAtlasEntity     entity   = getEntity(ENTITY_TYPE_ADLS_GEN2_ACCOUNT, ACCOUNT_QUALIFIED_NAME);
+        RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+        Assert.assertEquals(SERVICE_NAME, resource.getServiceName());
+        assertResourceElementCount(resource, 1);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_ACCOUNT, ACCOUNT_NAME);
+    }
+
+    @Test
+    public void testContainerEntity() throws Exception {
+        RangerAtlasEntity     entity   = getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, CONTAINER_QUALIFIED_NAME);
+        RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+        Assert.assertEquals(SERVICE_NAME, resource.getServiceName());
+        assertResourceElementCount(resource, 2);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_ACCOUNT, ACCOUNT_NAME);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_CONTAINER, CONTAINER_NAME);
+    }
+
+    @Test
+    public void testDirectoryEntity() throws Exception {
+        RangerAtlasEntity     entity   = getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, RELATIVE_PATH_QUALIFIED_NAME);
+        RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+        Assert.assertEquals(SERVICE_NAME, resource.getServiceName());
+        assertResourceElementCount(resource, 3);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_ACCOUNT, ACCOUNT_NAME);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_CONTAINER, CONTAINER_NAME);
+        assertResourceElementValue(resource, RANGER_TYPE_ADLS_GEN2_RELATIVE_PATH, RELATIVE_PATH_NAME);
+    }
+
+    @Test
+    public void testInvalidEntityType() {
+        assertException(getEntity("Unknown", RELATIVE_PATH_QUALIFIED_NAME), "unrecognized entity-type");
+    }
+
+    @Test
+    public void testInvalidAccountEntity() {
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_ACCOUNT, null), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_ACCOUNT, ""), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_ACCOUNT, "test"), "cluster-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_ACCOUNT, "test@cl1"), "account-name not found");
+    }
+
+    @Test
+    public void testInvalidContainerEntity() {
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, null), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, ""), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, "test"), "cluster-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, "test@cl1"), "account-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, "abfs://test@cl1"), "container-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_CONTAINER, "abfs://a@test@cl1"), "account-name not found");
+    }
+
+    @Test
+    public void testInvalidDirectoryEntity() {
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, null), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, ""), "attribute 'qualifiedName' not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, "test"), "cluster-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, "test@cl1"), "account-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, "abfs://test@cl1"), "container-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, "abfs://a@test@cl1"), "account-name not found");
+        assertException(getEntity(ENTITY_TYPE_ADLS_GEN2_DIRECTORY, "abfs://a@test.dfs.core.windows.net@cl1"), "relative-path not found");
+    }
+
+    private RangerAtlasEntity getEntity(String entityType, String qualifiedName) {
+        return new RangerAtlasEntity(entityType, "guid-" + entityType, Collections.singletonMap(ENTITY_ATTRIBUTE_QUALIFIED_NAME, qualifiedName));
+    }
+
+    private void assertResourceElementCount(RangerServiceResource resource, int count) {
+        Assert.assertNotNull(resource);
+        Assert.assertNotNull(resource.getResourceElements());
+        Assert.assertEquals(count, resource.getResourceElements().size());
+    }
+
+    private void assertResourceElementValue(RangerServiceResource resource, String resourceName, String value) {
+        Assert.assertTrue(resource.getResourceElements().containsKey(resourceName));
+        Assert.assertNotNull(resource.getResourceElements().get(resourceName).getValues());
+        Assert.assertEquals(1, resource.getResourceElements().get(resourceName).getValues().size());
+        Assert.assertEquals(value, resource.getResourceElements().get(resourceName).getValues().get(0));
+    }
+
+    private void assertException(RangerAtlasEntity entity, String exceptionMessage) {
+        try {
+            RangerServiceResource resource = resourceMapper.buildResource(entity);
+
+            Assert.assertFalse("Expected buildResource() to fail. But it returned " + resource, true);
+        } catch (Exception excp) {
+            Assert.assertTrue("Unexpected exception message: expected=" + exceptionMessage + "; found " + excp.getMessage(),
+                    excp.getMessage().startsWith(exceptionMessage));
+        }
+    }
+}


[ranger] 02/02: RANGER-2780: updated RangerBasePlugin to support audit-excluded-users

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

madhan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ranger.git

commit 8b963d304c77f7daf86131a514f1744a44b485b1
Author: Madhan Neethiraj <ma...@apache.org>
AuthorDate: Sun Apr 5 17:09:47 2020 -0700

    RANGER-2780: updated RangerBasePlugin to support audit-excluded-users
---
 .../hadoop/config/RangerPluginConfig.java          |  39 +++
 .../ranger/plugin/policyengine/PolicyEngine.java   |  63 -----
 .../plugin/policyengine/RangerPolicyEngine.java    |   4 -
 .../policyengine/RangerPolicyEngineImpl.java       | 175 +++++++++----
 .../ranger/plugin/service/RangerBasePlugin.java    |  20 +-
 .../plugin/policyengine/TestPolicyEngine.java      |  36 ++-
 ...icyengine_audit_exclude_users_groups_roles.json | 283 +++++++++++++++++++++
 7 files changed, 486 insertions(+), 134 deletions(-)

diff --git a/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/config/RangerPluginConfig.java b/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/config/RangerPluginConfig.java
index cea6842..89a31cc 100644
--- a/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/config/RangerPluginConfig.java
+++ b/agents-common/src/main/java/org/apache/ranger/authorization/hadoop/config/RangerPluginConfig.java
@@ -19,6 +19,7 @@
 
 package org.apache.ranger.authorization.hadoop.config;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
@@ -27,6 +28,9 @@ import org.apache.ranger.plugin.policyengine.RangerPolicyEngineOptions;
 
 import java.io.File;
 import java.net.URL;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
 
 
 public class RangerPluginConfig extends RangerConfiguration {
@@ -43,6 +47,11 @@ public class RangerPluginConfig extends RangerConfiguration {
     private final boolean                   useForwardedIPAddress;
     private final String[]                  trustedProxyAddresses;
     private final String                    propertyPrefix;
+    private       Set<String>               auditExcludedUsers  = Collections.emptySet();
+    private       Set<String>               auditExcludedGroups = Collections.emptySet();
+    private       Set<String>               auditExcludedRoles  = Collections.emptySet();
+    private       Set<String>               superUsers          = Collections.emptySet();
+    private       Set<String>               superGroups         = Collections.emptySet();
 
 
     public RangerPluginConfig(String serviceType, String serviceName, String appId, String clusterName, String clusterType, RangerPolicyEngineOptions policyEngineOptions) {
@@ -143,6 +152,36 @@ public class RangerPluginConfig extends RangerConfiguration {
         return policyEngineOptions;
     }
 
+    public void setAuditExcludedUsersGroupsRoles(Set<String> users, Set<String> groups, Set<String> roles) {
+        auditExcludedUsers  = CollectionUtils.isEmpty(users) ? Collections.emptySet() : new HashSet<>(users);
+        auditExcludedGroups = CollectionUtils.isEmpty(groups) ? Collections.emptySet() : new HashSet<>(groups);
+        auditExcludedRoles  = CollectionUtils.isEmpty(groups) ? Collections.emptySet() : new HashSet<>(roles);
+    }
+
+    public void setSuperUsersGroups(Set<String> users, Set<String> groups) {
+        superUsers  = CollectionUtils.isEmpty(users) ? Collections.emptySet() : new HashSet<>(users);
+        superGroups = CollectionUtils.isEmpty(groups) ? Collections.emptySet() : new HashSet<>(groups);
+    }
+
+    public boolean isAuditExcludedUser(String userName) {
+        return auditExcludedUsers.contains(userName);
+    }
+
+    public boolean hasAuditExcludedGroup(Set<String> userGroups) {
+        return userGroups != null && userGroups.size() > 0 && auditExcludedGroups.size() > 0 && CollectionUtils.containsAny(userGroups, auditExcludedGroups);
+    }
+
+    public boolean hasAuditExcludedRole(Set<String> userRoles) {
+        return userRoles != null && userRoles.size() > 0 && auditExcludedRoles.size() > 0 && CollectionUtils.containsAny(userRoles, auditExcludedRoles);
+    }
+
+    public boolean isSuperUser(String userName) {
+        return superUsers.contains(userName);
+    }
+
+    public boolean hasSuperGroup(Set<String> userGroups) {
+        return userGroups != null && userGroups.size() > 0 && superGroups.size() > 0 && CollectionUtils.containsAny(userGroups, superGroups);
+    }
 
     private void addResourcesForServiceType(String serviceType) {
         String auditCfg    = "ranger-" + serviceType + "-audit.xml";
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/PolicyEngine.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/PolicyEngine.java
index bc80677..297d02f 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/PolicyEngine.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/PolicyEngine.java
@@ -33,7 +33,6 @@ import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.ranger.authorization.utils.StringUtil;
 import org.apache.ranger.plugin.contextenricher.RangerContextEnricher;
 import org.apache.ranger.plugin.model.RangerPolicy;
 import org.apache.ranger.plugin.model.RangerPolicyDelta;
@@ -64,9 +63,6 @@ public class PolicyEngine {
     private       boolean                             useForwardedIPAddress;
     private       String[]                            trustedProxyAddresses;
     private       boolean                             isPreCleaned = false;
-    private final Set<String>                         auditExcludeUsers;
-    private final Set<String>                         auditExcludeGroups;
-    private final Set<String>                         auditExcludeRoles;
 
 
     public boolean getUseForwardedIPAddress() {
@@ -115,18 +111,6 @@ public class PolicyEngine {
 
     public RangerPluginContext getPluginContext() { return pluginContext; }
 
-	public Set<String> getAuditExcludeUsers() {
-		return auditExcludeUsers;
-	}
-
-	public Set<String> getAuditExcludeGroups() {
-		return auditExcludeGroups;
-	}
-
-	public Set<String> getAuditExcludeRoles() {
-		return auditExcludeRoles;
-	}
-
     @Override
     public String toString() {
         return toString(new StringBuilder()).toString();
@@ -177,45 +161,6 @@ public class PolicyEngine {
         return resourceZoneTrie;
     }
 
-    public RangerAccessResult createAccessResult(RangerAccessRequest request, int policyType) {
-        RangerAccessResult ret = new RangerAccessResult(policyType, getServiceName(), getPolicyRepository().getServiceDef(), request);
-
-        switch (getPolicyRepository().getAuditModeEnum()) {
-            case AUDIT_ALL:
-                ret.setIsAudited(true);
-                break;
-
-            case AUDIT_NONE:
-                ret.setIsAudited(false);
-                break;
-
-            default:
-                if (CollectionUtils.isEmpty(getPolicyRepository().getPolicies()) && getTagPolicyRepository() == null) {
-                    ret.setIsAudited(true);
-                }
-
-                break;
-        }
-        if (isAuditExcludedRequest(request)) {
-            ret.setIsAudited(false);
-        }
-        return ret;
-    }
-
-	private boolean isAuditExcludedRequest(RangerAccessRequest request) {
-		boolean ret = getAuditExcludeUsers().contains(request.getUser());
-
-		if (!ret && CollectionUtils.isNotEmpty(getAuditExcludeGroups())) {
-			ret = CollectionUtils.containsAny(getAuditExcludeGroups(), request.getUserGroups());
-		}
-
-		if (!ret && CollectionUtils.isNotEmpty(getAuditExcludeRoles())) {
-			Set<String> roles = this.pluginContext.getAuthContext().getRolesForUserAndGroups(request.getUser(), request.getUserGroups());
-			ret = CollectionUtils.containsAny(getAuditExcludeRoles(), roles);
-		}
-		return ret;
-	}
-
     public PolicyEngine(ServicePolicies servicePolicies, RangerPluginContext pluginContext, RangerRoles roles) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> PolicyEngine(" + ", " + servicePolicies + ", " + pluginContext + ")");
@@ -290,10 +235,6 @@ public class PolicyEngine {
             }
         }
 
-        this.auditExcludeUsers  = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_USERS)) : new HashSet<String>();
-        this.auditExcludeGroups = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_GROUPS)) : new HashSet<String>();
-        this.auditExcludeRoles  = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_ROLES)) : new HashSet<String>();
-
         RangerPerfTracer.log(perf);
 
         if (PERF_POLICYENGINE_INIT_LOG.isDebugEnabled()) {
@@ -622,10 +563,6 @@ public class PolicyEngine {
         this.trustedProxyAddresses = other.trustedProxyAddresses;
         this.pluginContext         = other.pluginContext;
 
-        this.auditExcludeUsers  = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_USERS)) : new HashSet<String>();
-        this.auditExcludeGroups = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_GROUPS)) : new HashSet<String>();
-        this.auditExcludeRoles  = servicePolicies.getServiceConfig() !=null ? StringUtil.toSet(servicePolicies.getServiceConfig().get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_ROLES)) : new HashSet<String>();
-
         long                    policyVersion                   = servicePolicies.getPolicyVersion() != null ? servicePolicies.getPolicyVersion() : -1L;
         List<RangerPolicyDelta> defaultZoneDeltas               = new ArrayList<>();
         List<RangerPolicyDelta> defaultZoneDeltasForTagPolicies = new ArrayList<>();
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngine.java b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngine.java
index c673169..fdec9ca 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngine.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/policyengine/RangerPolicyEngine.java
@@ -51,10 +51,6 @@ public interface RangerPolicyEngine {
 
 	void setTrustedProxyAddresses(String[] trustedProxyAddresses);
 
-	void setSuperUsersAndGroups(Set<String> users, Set<String> groups);
-
-	boolean isSuperUser(String userName, Set<String> userGroups);
-
 	RangerServiceDef getServiceDef();
 
 	long getPolicyVersion();
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 c68e22d..6140549 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
@@ -24,6 +24,7 @@ import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.ranger.authorization.hadoop.config.RangerPluginConfig;
 import org.apache.ranger.authorization.utils.StringUtil;
 import org.apache.ranger.plugin.contextenricher.RangerTagForEval;
 import org.apache.ranger.plugin.model.RangerPolicy;
@@ -59,11 +60,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 
 	private final PolicyEngine                 policyEngine;
 	private final RangerAccessRequestProcessor requestProcessor;
-	private final Set<String>                  svcCfgSuperUsers;
-	private final Set<String>                  svcCfgSuperGroups;
-	private       Set<String>                  superUsers;
-	private       Set<String>                  superGroups;
-	private       boolean                      isEmptySupers = true;
+	private final ServiceConfig                serviceConfig;
 
 
 	static public RangerPolicyEngine getPolicyEngine(final RangerPolicyEngineImpl other, final ServicePolicies servicePolicies) {
@@ -81,29 +78,9 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 	}
 
 	public RangerPolicyEngineImpl(ServicePolicies servicePolicies, RangerPluginContext pluginContext, RangerRoles roles) {
-		this(servicePolicies, pluginContext, roles, null, null);
-	}
-
-	public RangerPolicyEngineImpl(ServicePolicies servicePolicies, RangerPluginContext pluginContext, RangerRoles roles, Set<String> superUsers, Set<String> superGroups) {
-		policyEngine = new PolicyEngine(servicePolicies, pluginContext, roles);
-
-		policyEngine.getPluginContext().getAuthContext().setRoles(roles);
-
+		policyEngine     = new PolicyEngine(servicePolicies, pluginContext, roles);
+		serviceConfig    = new ServiceConfig(servicePolicies.getServiceConfig());
 		requestProcessor = new RangerDefaultRequestProcessor(policyEngine);
-
-		Map<String, String> svcConfig      = servicePolicies.getServiceConfig();
-		String              cfgSuperUsers  = null;
-		String              cfgSuperGroups = null;
-
-		if (svcConfig != null) {
-			cfgSuperUsers  = svcConfig.get(RangerPolicyEngine.PLUGIN_SUPER_USERS);
-			cfgSuperGroups = svcConfig.get(RangerPolicyEngine.PLUGIN_SUPER_GROUPS);
-		}
-
-		svcCfgSuperUsers  = StringUtils.isNotBlank(cfgSuperUsers) ? StringUtil.toSet(cfgSuperUsers) : Collections.emptySet();
-		svcCfgSuperGroups = StringUtils.isNotBlank(cfgSuperGroups) ? StringUtil.toSet(cfgSuperGroups) : Collections.emptySet();
-
-		setSuperUsersAndGroups(superUsers, superGroups);
 	}
 
 	@Override
@@ -497,21 +474,6 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 	}
 
 	@Override
-	public void setSuperUsersAndGroups(Set<String> users, Set<String> groups) {
-		this.superUsers    = users == null ? Collections.emptySet() : users;
-		this.superGroups   = groups == null ? Collections.emptySet() : groups;
-		this.isEmptySupers = CollectionUtils.isEmpty(superUsers) && CollectionUtils.isEmpty(svcCfgSuperUsers) &&
-		                     CollectionUtils.isEmpty(superGroups) && CollectionUtils.isEmpty(svcCfgSuperGroups);
-	}
-
-	@Override
-	public boolean isSuperUser(String userName, Set<String> userGroups) {
-		return !isEmptySupers && (superUsers.contains(userName) || svcCfgSuperUsers.contains(userName) ||
-		                          CollectionUtils.containsAny(superGroups, userGroups) ||
-		                          CollectionUtils.containsAny(svcCfgSuperGroups, userGroups));
-	}
-
-	@Override
 	public RangerServiceDef getServiceDef() {
 		return policyEngine.getServiceDef();
 	}
@@ -574,12 +536,9 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 	}
 
 	private RangerPolicyEngineImpl(final PolicyEngine policyEngine, RangerPolicyEngineImpl other) {
-		this.policyEngine      = policyEngine;
-		this.requestProcessor  = new RangerDefaultRequestProcessor(policyEngine);
-		this.svcCfgSuperUsers  = new HashSet<>(other.svcCfgSuperUsers);
-		this.svcCfgSuperGroups = new HashSet<>(other.svcCfgSuperGroups);
-		this.superUsers        = new HashSet<>(other.superUsers);
-		this.superGroups       = new HashSet<>(other.superGroups);
+		this.policyEngine     = policyEngine;
+		this.requestProcessor = new RangerDefaultRequestProcessor(policyEngine);
+		this.serviceConfig    = new ServiceConfig(other.serviceConfig);
 	}
 
 	private RangerAccessResult zoneAwareAccessEvaluationWithNoAudit(RangerAccessRequest request, int policyType) {
@@ -623,7 +582,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 		}
 
 		final Date               accessTime  = request.getAccessTime() != null ? request.getAccessTime() : new Date();
-		final RangerAccessResult ret         = policyEngine.createAccessResult(request, policyType);
+		final RangerAccessResult ret         = createAccessResult(request, policyType);
 		final boolean            isSuperUser = isSuperUser(request.getUser(), request.getUserGroups());
 
 		// for superusers, set access as allowed
@@ -751,7 +710,7 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 
 				RangerTagForEval    tag            = policyEvaluator.getTag();
 				RangerAccessRequest tagEvalRequest = new RangerTagAccessRequest(tag, tagPolicyRepository.getServiceDef(), request);
-				RangerAccessResult  tagEvalResult  = policyEngine.createAccessResult(tagEvalRequest, policyType);
+				RangerAccessResult  tagEvalResult  = createAccessResult(tagEvalRequest, policyType);
 
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("RangerPolicyEngineImpl.evaluateTagPolicies: Evaluating policies for tag (" + tag.getType() + ")");
@@ -798,4 +757,120 @@ public class RangerPolicyEngineImpl implements RangerPolicyEngine {
 			LOG.debug("<== RangerPolicyEngineImpl.evaluateTagPolicies(" + request + ", policyType =" + policyType + ", zoneName=" + zoneName + ", " + result + ")");
 		}
 	}
+
+	private RangerAccessResult createAccessResult(RangerAccessRequest request, int policyType) {
+		RangerPolicyRepository repository = policyEngine.getPolicyRepository();
+		RangerAccessResult     ret        = new RangerAccessResult(policyType, repository.getServiceName(), repository.getServiceDef(), request);
+
+		switch (repository.getAuditModeEnum()) {
+			case AUDIT_ALL:
+				ret.setIsAudited(true);
+				break;
+
+			case AUDIT_NONE:
+				ret.setIsAudited(false);
+				break;
+
+			default:
+				if (CollectionUtils.isEmpty(repository.getPolicies()) && policyEngine.getTagPolicyRepository() == null) {
+					ret.setIsAudited(true);
+				}
+
+				break;
+		}
+
+		if (isAuditExcludedUser(request.getUser(), request.getUserGroups(), RangerAccessRequestUtil.getCurrentUserRolesFromContext(request.getContext()))) {
+			ret.setIsAudited(false);
+		}
+
+		return ret;
+	}
+
+	private boolean isAuditExcludedUser(String userName, Set<String> userGroups, Set<String> userRoles) {
+		boolean ret = serviceConfig.isAuditExcludedUser(userName);
+
+		if (!ret) {
+			RangerPluginConfig pluginConfig = policyEngine.getPluginContext().getConfig();
+
+			ret = pluginConfig.isAuditExcludedUser(userName);
+
+			if (!ret && userGroups != null && userGroups.size() > 0) {
+				ret = serviceConfig.hasAuditExcludedGroup(userGroups) || pluginConfig.hasAuditExcludedGroup(userGroups);
+			}
+
+			if (!ret && userRoles != null && userRoles.size() > 0) {
+				ret = serviceConfig.hasAuditExcludedRole(userRoles) || pluginConfig.hasAuditExcludedRole(userRoles);
+			}
+		}
+
+		return ret;
+	}
+
+	private boolean isSuperUser(String userName, Set<String> userGroups) {
+		boolean ret = serviceConfig.isSuperUser(userName);
+
+		if (!ret) {
+			RangerPluginConfig pluginConfig = policyEngine.getPluginContext().getConfig();
+
+			ret = pluginConfig.isSuperUser(userName);
+
+			if (!ret && userGroups != null && userGroups.size() > 0) {
+				ret = serviceConfig.hasSuperGroup(userGroups) || pluginConfig.hasSuperGroup(userGroups);
+			}
+		}
+
+		return ret;
+	}
+
+	private static class ServiceConfig {
+		private final Set<String> auditExcludedUsers;
+		private final Set<String> auditExcludedGroups;
+		private final Set<String> auditExcludedRoles;
+		private final Set<String> superUsers;
+		private final Set<String> superGroups;
+
+		public ServiceConfig(Map<String, String> svcConfig) {
+			if (svcConfig != null) {
+				auditExcludedUsers  = StringUtil.toSet(svcConfig.get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_USERS));
+				auditExcludedGroups = StringUtil.toSet(svcConfig.get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_GROUPS));
+				auditExcludedRoles  = StringUtil.toSet(svcConfig.get(RangerPolicyEngine.PLUGIN_AUDIT_EXCLUDE_ROLES));
+				superUsers          = StringUtil.toSet(svcConfig.get(RangerPolicyEngine.PLUGIN_SUPER_USERS));
+				superGroups         = StringUtil.toSet(svcConfig.get(RangerPolicyEngine.PLUGIN_SUPER_GROUPS));
+			} else {
+				auditExcludedUsers  = Collections.emptySet();
+				auditExcludedGroups = Collections.emptySet();
+				auditExcludedRoles  = Collections.emptySet();
+				superUsers          = Collections.emptySet();
+				superGroups         = Collections.emptySet();
+			}
+		}
+
+		public ServiceConfig(ServiceConfig other) {
+			auditExcludedUsers  = other == null || CollectionUtils.isEmpty(other.auditExcludedUsers) ? Collections.emptySet() : new HashSet<>(other.auditExcludedUsers);
+			auditExcludedGroups = other == null || CollectionUtils.isEmpty(other.auditExcludedGroups) ? Collections.emptySet() : new HashSet<>(other.auditExcludedGroups);
+			auditExcludedRoles  = other == null || CollectionUtils.isEmpty(other.auditExcludedRoles) ? Collections.emptySet() : new HashSet<>(other.auditExcludedRoles);
+			superUsers          = other == null || CollectionUtils.isEmpty(other.superUsers) ? Collections.emptySet() : new HashSet<>(other.superUsers);
+			superGroups         = other == null || CollectionUtils.isEmpty(other.superGroups) ? Collections.emptySet() : new HashSet<>(other.superGroups);
+		}
+
+		public boolean isAuditExcludedUser(String userName) {
+			return auditExcludedUsers.contains(userName);
+		}
+
+		public boolean hasAuditExcludedGroup(Set<String> userGroups) {
+			return userGroups != null && userGroups.size() > 0 && auditExcludedGroups.size() > 0 && CollectionUtils.containsAny(userGroups, auditExcludedGroups);
+		}
+
+		public boolean hasAuditExcludedRole(Set<String> userRoles) {
+			return userRoles != null && userRoles.size() > 0 && auditExcludedRoles.size() > 0 && CollectionUtils.containsAny(userRoles, auditExcludedRoles);
+		}
+
+		public boolean isSuperUser(String userName) {
+			return superUsers.contains(userName);
+		}
+
+		public boolean hasSuperGroup(Set<String> userGroups) {
+			return userGroups != null && userGroups.size() > 0 && superGroups.size() > 0 && CollectionUtils.containsAny(userGroups, superGroups);
+		}
+	}
 }
diff --git a/agents-common/src/main/java/org/apache/ranger/plugin/service/RangerBasePlugin.java b/agents-common/src/main/java/org/apache/ranger/plugin/service/RangerBasePlugin.java
index b5e18ba..41b2492 100644
--- a/agents-common/src/main/java/org/apache/ranger/plugin/service/RangerBasePlugin.java
+++ b/agents-common/src/main/java/org/apache/ranger/plugin/service/RangerBasePlugin.java
@@ -56,7 +56,6 @@ import org.apache.ranger.plugin.util.*;
 public class RangerBasePlugin {
 	private static final Log LOG = LogFactory.getLog(RangerBasePlugin.class);
 
-
 	private final RangerPluginConfig          pluginConfig;
 	private final RangerPluginContext         pluginContext;
 	private final Map<String, LogHistory>     logHistoryList = new Hashtable<>();
@@ -67,8 +66,6 @@ public class RangerBasePlugin {
 	private       RangerAuthContext           currentAuthContext;
 	private       RangerAccessResultProcessor resultProcessor;
 	private       RangerRoles                 roles;
-	private       Set<String>                 superUsers  = Collections.emptySet();
-	private       Set<String>                 superGroups = Collections.emptySet();
 
 
 	public RangerBasePlugin(String serviceType, String appId) {
@@ -130,15 +127,12 @@ public class RangerBasePlugin {
 		pluginContext.notifyAuthContextChanged();
 	}
 
-	public void setSuperUsersAndGroups(Set<String> users, Set<String> groups) {
-		this.superUsers  = users == null ? Collections.emptySet() : users;
-		this.superGroups = groups == null ? Collections.emptySet() : groups;
-
-		RangerPolicyEngine policyEngine = this.policyEngine;
+	public void setAuditExcludedUsersGroupsRoles(Set<String> users, Set<String> groups, Set<String> roles) {
+		pluginConfig.setAuditExcludedUsersGroupsRoles(users, groups, roles);
+	}
 
-		if (policyEngine != null) {
-			policyEngine.setSuperUsersAndGroups(this.superUsers, this.superGroups);
-		}
+	public void setSuperUsersAndGroups(Set<String> users, Set<String> groups) {
+		pluginConfig.setSuperUsersGroups(users, groups);
 	}
 
 	public RangerServiceDef getServiceDef() {
@@ -230,7 +224,7 @@ public class RangerBasePlugin {
 						LOG.debug("Creating engine from policies");
 					}
 
-					newPolicyEngine = new RangerPolicyEngineImpl(policies, pluginContext, roles, superUsers, superGroups);
+					newPolicyEngine = new RangerPolicyEngineImpl(policies, pluginContext, roles);
 				} else {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("policy-deltas are not null");
@@ -259,7 +253,7 @@ public class RangerBasePlugin {
 								LOG.debug("Creating new engine from servicePolicies:[" + servicePolicies + "]");
 							}
 
-							newPolicyEngine = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles, superUsers, superGroups);
+							newPolicyEngine = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles);
 						}
 					} else {
 						if (LOG.isDebugEnabled()) {
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 8811f2a..26c7dfb 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
@@ -408,6 +408,13 @@ public class TestPolicyEngine {
 		runTestsFromResourceFiles(resourceFiles);
 	}
 
+	@Test
+	public void testPolicyEngine_auditExcludeUsersGroupsRoles() {
+		String[] resourceFiles = {"/policyengine/test_policyengine_audit_exclude_users_groups_roles.json"};
+
+		runTestsFromResourceFiles(resourceFiles);
+	}
+
 	private void runTestsFromResourceFiles(String[] resourceNames) {
 		for(String resourceName : resourceNames) {
 			InputStream inStream = this.getClass().getResourceAsStream(resourceName);
@@ -462,6 +469,7 @@ public class TestPolicyEngine {
 
 		Map<String, Set<String>> userRoleMapping = testCase.userRoles;
 		Map<String, Set<String>> groupRoleMapping = testCase.groupRoles;
+		Map<String, Set<String>> roleRoleMapping = testCase.roleRoles;
 		if (userRoleMapping != null) {
 			for (Map.Entry<String, Set<String>> userRole : userRoleMapping.entrySet()) {
 				String user = userRole.getKey();
@@ -482,26 +490,42 @@ public class TestPolicyEngine {
 				RangerRole.RoleMember groupRoleMember = new RangerRole.RoleMember(group, true);
 				List<RangerRole.RoleMember> groupRoleMembers = Arrays.asList(groupRoleMember);
 				for (String grpRole : groupRoles) {
-					RangerRole rangerGroupRole = new RangerRole(grpRole, grpRole, null, groupRoleMembers, null);
+					RangerRole rangerGroupRole = new RangerRole(grpRole, grpRole, null, null, groupRoleMembers);
 					rolesSet.add(rangerGroupRole);
 				}
 			}
 		}
 
+		if (roleRoleMapping != null) {
+			for (Map.Entry<String, Set<String>> roleRole : roleRoleMapping.entrySet()) {
+				String role = roleRole.getKey();
+				Set<String> roleRoles = roleRole.getValue();
+				RangerRole.RoleMember roleRoleMember = new RangerRole.RoleMember(role, true);
+				List<RangerRole.RoleMember> roleRoleMembers = Arrays.asList(roleRoleMember);
+				for (String rleRole : roleRoles) {
+					RangerRole rangerRoleRole = new RangerRole(rleRole, rleRole, null, null, null, roleRoleMembers);
+					rolesSet.add(rangerRoleRole);
+				}
+			}
+		}
+
 		roles.setRangerRoles(rolesSet);
 
         RangerPolicyEngineOptions policyEngineOptions = pluginContext.getConfig().getPolicyEngineOptions();
 
         policyEngineOptions.disableAccessEvaluationWithPolicyACLSummary = true;
 
-        RangerPolicyEngineImpl policyEngine = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles, testCase.superUsers, testCase.superGroups);
+        pluginContext.getConfig().setSuperUsersGroups(testCase.superUsers,  testCase.superGroups);
+		pluginContext.getConfig().setAuditExcludedUsersGroupsRoles(testCase.auditExcludedUsers,  testCase.auditExcludedGroups, testCase.auditExcludedRoles);
+
+        RangerPolicyEngineImpl policyEngine = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles);
 
         policyEngine.setUseForwardedIPAddress(useForwardedIPAddress);
         policyEngine.setTrustedProxyAddresses(trustedProxyAddresses);
 
         policyEngineOptions.disableAccessEvaluationWithPolicyACLSummary = false;
 
-		RangerPolicyEngineImpl policyEngineForEvaluatingWithACLs = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles, testCase.superUsers, testCase.superGroups);
+		RangerPolicyEngineImpl policyEngineForEvaluatingWithACLs = new RangerPolicyEngineImpl(servicePolicies, pluginContext, roles);
 
 		policyEngineForEvaluatingWithACLs.setUseForwardedIPAddress(useForwardedIPAddress);
 		policyEngineForEvaluatingWithACLs.setTrustedProxyAddresses(trustedProxyAddresses);
@@ -672,6 +696,7 @@ public class TestPolicyEngine {
 		public Map<String, ServicePolicies.SecurityZoneInfo> securityZones;
 		public Map<String, Set<String>> userRoles;
 		public Map<String, Set<String>> groupRoles;
+		public Map<String, Set<String>> roleRoles;
 		public String             auditMode;
 		public List<TestData>     tests;
 		public Map<String, String> serviceConfig;
@@ -679,7 +704,10 @@ public class TestPolicyEngine {
 		public List<TestData>     updatedTests;
 		public Set<String>        superUsers;
 		public Set<String>        superGroups;
-		
+		public Set<String>        auditExcludedUsers;
+		public Set<String>        auditExcludedGroups;
+		public Set<String>        auditExcludedRoles;
+
 		class TestData {
 			public String              name;
 			public RangerAccessRequest request;
diff --git a/agents-common/src/test/resources/policyengine/test_policyengine_audit_exclude_users_groups_roles.json b/agents-common/src/test/resources/policyengine/test_policyengine_audit_exclude_users_groups_roles.json
new file mode 100644
index 0000000..191a7d0
--- /dev/null
+++ b/agents-common/src/test/resources/policyengine/test_policyengine_audit_exclude_users_groups_roles.json
@@ -0,0 +1,283 @@
+{
+  "serviceName":"hivedev",
+
+  "original-serviceDef":{
+    "name":"hive",
+    "id":3,
+    "resources":[
+      {"name":"database","level":1,"mandatory":true,"lookupSupported":true,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"Hive Database","description":"Hive Database"},
+      {"name":"url","level":1,"mandatory":true,"lookupSupported":false,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"URL","description":"URL"},
+      {"name":"hiveservice","level":1,"mandatory":true,"lookupSupported":false,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"HiveService","description":"HiveService"},
+      {"name":"table","level":2,"parent":"database","mandatory":true,"lookupSupported":true,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"Hive Table","description":"Hive Table"},
+      {"name":"udf","level":2,"parent":"database","mandatory":true,"lookupSupported":true,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"Hive UDF","description":"Hive UDF"},
+      {"name":"column","level":3,"parent":"table","mandatory":true,"lookupSupported":true,"matcher":"org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher","matcherOptions":{"wildCard":true, "ignoreCase":true},"label":"Hive Column","description":"Hive Column"}
+    ],
+    "accessTypes":[
+      {"name":"select","label":"Select"},
+      {"name":"update","label":"Update"},
+      {"name":"create","label":"Create"},
+      {"name":"drop","label":"Drop"},
+      {"name":"alter","label":"Alter"},
+      {"name":"index","label":"Index"},
+      {"name":"lock","label":"Lock"},
+      {"name":"read","label":"Read"},
+      {"name":"write","label":"Write"},
+      {"name":"repladmin","label":"ReplAdmin"},
+      {"name":"serviceadmin","label":"ServiceAdmin"},
+      {"name":"all","label":"All",
+        "impliedGrants": ["select", "update", "create", "drop", "alter", "index", "lock", "read", "write", "repladmin", "serviceadmin"]
+      }
+    ]
+  },
+
+  "serviceDef": {
+    "id":3,
+    "name": "hive",
+    "implClass": "org.apache.ranger.services.hive.RangerServiceHive",
+    "label": "Hive Server2",
+    "description": "Hive Server2",
+    "guid": "3e1afb5a-184a-4e82-9d9c-87a5cacc243c",
+
+    "resources": [
+      {"itemId": 1, "name": "database", "type": "string", "level": 10, "parent": "", "mandatory": true, "lookupSupported": true, "recursiveSupported": false, "excludesSupported": true, "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", "matcherOptions": { "wildCard":true, "ignoreCase":true }, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Hive Database", "description": "Hive Database", "isValidLeaf": true},
+      {"itemId": 2, "name": "table", "type": "string", "level": 20, "parent": "database", "mandatory": true, "lookupSupported": true, "recursiveSupported": false, "excludesSupported": true, "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", "matcherOptions": { "wildCard":true, "ignoreCase":true }, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Hive Table", "description": "Hive Table", "isValidLeaf": true},
+      {"itemId": 3, "name": "udf", "type": "string", "level": 20, "parent": "database", "mandatory": true, "lookupSupported": true, "recursiveSupported": false, "excludesSupported": true, "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", "matcherOptions": { "wildCard":true, "ignoreCase":true }, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Hive UDF", "description": "Hive UDF", "isValidLeaf": true},
+      {"itemId": 4, "name": "column", "type": "string", "level": 30, "parent": "table", "mandatory": true, "lookupSupported": true, "recursiveSupported": false, "excludesSupported": true, "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", "matcherOptions": { "wildCard":true, "ignoreCase":true }, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Hive Column", "description": "Hive Column", "isValidLeaf": true},
+      {"itemId": 5, "name": "url", "type": "string", "level": 10, "parent": "", "mandatory": true, "lookupSupported": false, "recursiveSupported": true, "excludesSupported": false, "matcher": "org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher", "matcherOptions": { "wildCard":true, "ignoreCase":false }, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "URL", "description": "URL", "isValidLeaf": true}
+    ],
+
+    "accessTypes": [
+      {"itemId": 1, "name": "select", "label": "select"},
+      {"itemId": 2, "name": "update", "label": "update"},
+      {"itemId": 3, "name": "create", "label": "Create"},
+      {"itemId": 4, "name": "drop", "label": "Drop"},
+      {"itemId": 5, "name": "alter", "label": "Alter"},
+      {"itemId": 6, "name": "index", "label": "Index"},
+      {"itemId": 7, "name": "lock", "label": "Lock"},
+      {"itemId": 8, "name": "all", "label": "All",
+        "impliedGrants": ["select", "update", "create", "drop", "alter", "index", "lock", "read", "write"]},
+      {"itemId": 9, "name": "read", "label": "Read"},
+      {"itemId": 10, "name": "write", "label": "Write"}
+    ],
+
+    "configs": [
+      {"itemId": 1, "name": "username", "type": "string", "mandatory": true, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Username"},
+      {"itemId": 2, "name": "password", "type": "password", "mandatory": true, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Password"},
+      {"itemId": 3, "name": "jdbc.driverClassName", "type": "string", "mandatory": true, "validationRegEx":"", "validationMessage": "", "uiHint":"", "defaultValue": "org.apache.hive.jdbc.HiveDriver"},
+      {"itemId": 4, "name": "jdbc.url", "type": "string", "mandatory": true, "defaultValue": "", "validationRegEx":"", "validationMessage": "", "uiHint":""},
+      {"itemId": 5, "name": "commonNameForCertificate", "type": "string", "mandatory": false, "validationRegEx":"", "validationMessage": "", "uiHint":"", "label": "Common Name for Certificate"}
+    ],
+
+    "enums": [
+    ],
+
+    "contextEnrichers": [
+    ],
+
+    "policyConditions": [
+    ]
+  },
+
+  "userRoles": {
+    "user-in-aer1": [ "aer1" ],
+    "user-in-aer2": [ "aer2" ],
+    "user-in-svc-cfg-aer1": [ "svc-cfg-aer1" ],
+    "user-in-svc-cfg-aer2": [ "svc-cfg-aer2" ]
+  },
+
+  "groupRoles": {
+    "group-in-aer1": [ "aer1" ],
+    "group-in-aer2": [ "aer2" ],
+    "group-in-svc-cfg-aer1": [ "svc-cfg-aer1" ],
+    "group-in-svc-cfg-aer2": [ "svc-cfg-aer2" ]
+  },
+
+  "roleRoles": {
+    "role-in-aer1": [ "aer1" ],
+    "role-in-aer2": [ "aer2" ],
+    "role-in-svc-cfg-aer1": [ "svc-cfg-aer1" ],
+    "role-in-svc-cfg-aer2": [ "svc-cfg-aer2" ]
+  },
+
+  "serviceConfig": {
+    "ranger.plugin.audit.exclude.users":  "svc-cfg-aeu1, svc-cfg-aeu2",
+    "ranger.plugin.audit.exclude.groups": "svc-cfg-aeg1, svc-cfg-aeg2",
+    "ranger.plugin.audit.exclude.roles":  "svc-cfg-aer1, svc-cfg-aer2"
+  },
+
+  "policies":[
+    {"id":1,"name":"database=db-*,table=*,column=* - allow create to everyone, audit accesses","isEnabled":true,"isAuditEnabled":true,
+     "resources":{"database":{"values":["db-*"]},"table":{"values":["*"]},"column":{"values":["*"]}},
+     "policyItems":[
+       {"accesses":[{"type":"create","isAllowed":true}],"users":[],"groups":["public"],"delegateAdmin":false}
+     ]
+    }
+  ],
+
+  "auditExcludedUsers":  [ "aeu1", "aeu2" ],
+  "auditExcludedGroups": [ "aeg1", "aeg2" ],
+  "auditExcludedRoles":  [ "aer1", "aer2" ],
+
+  "tests":[
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for group-in-aer1",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user3","userGroups":["group-in-aer1"],"requestData":"create db-1.tbl-1;' for group-in-aer1"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1;' for aeu1",
+      "request":{
+        "resource":{"elements":{"database":"db-1"}},
+        "accessType":"create","user":"aeu1","requestData":"create db-1 for aeu1"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+    ,
+    {"name":"NO AUDIT 'create db-2;' for aeu2",
+      "request":{
+        "resource":{"elements":{"database":"db-2"}},
+        "accessType":"create","user":"aeu2","requestData":"create db-2 for aeu2"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1;' for svc-cfg-aeu1",
+      "request":{
+        "resource":{"elements":{"database":"db-1"}},
+        "accessType":"create","user":"svc-cfg-aeu1","requestData":"create db-1 for svc-cfg-aeu1"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-2;' for svc-cfg-aeu2",
+      "request":{
+        "resource":{"elements":{"database":"db-2"}},
+        "accessType":"create","user":"svc-cfg-aeu2","requestData":"create db-2 for svc-cfg-aeu2"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for user1 (in aeg1)",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user1","userGroups":["aeg1"],"requestData":"create db-1.tbl-1;' for user1 (in aeg1)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for user2 (in aeg2)",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user2","userGroups":["aeg2"],"requestData":"create db-1.tbl-2;' for user2 (in aeg2)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for user1 (in svc-cfg-aeg1)",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user1","userGroups":["svc-cfg-aeg1"],"requestData":"create db-1.tbl-1;' for user1 (in svc-cfg-aeg1)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for user2 (in svc-cfg-aeg2)",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user2","userGroups":["svc-cfg-aeg2"],"requestData":"create db-1.tbl-2;' for user2 (in svc-cfg-aeg2)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for user-in-aer1",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user-in-aer1","requestData":"create db-1.tbl-1;' for user1 (in aer1)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for user-in-aer2",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user-in-aer2","requestData":"create db-1.tbl-2;' for user2 (in aer2)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for user-in-svc-cfg-aer1",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user-in-svc-cfg-aer1","requestData":"create db-1.tbl-1;' for user1 (in svc-cfg-aer1)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for user-in-svc-cfg-aer2",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user-in-svc-cfg-aer2","requestData":"create db-1.tbl-2;' for user2 (in svc-cfg-aer2)"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for group-in-aer1",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user3","userGroups":["group-in-aer1"],"requestData":"create db-1.tbl-1;' for group-in-aer1"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for group-in-aer2",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user3", "userGroups":["group-in-aer2"],"requestData":"create db-1.tbl-2;' for group-in-aer2"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-1;' for group-in-svc-cfg-aer1",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-1"}},
+        "accessType":"create","user":"user3","userGroups":["group-in-svc-cfg-aer1"],"requestData":"create db-1.tbl-1;' for group-in-svc-cfg-aer1"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for group-in-svc-cfg-aer2",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user3", "userGroups":["group-in-svc-cfg-aer2"],"requestData":"create db-1.tbl-2;' for group-in-svc-cfg-aer2"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create db-1.tbl-2;' for group-in-svc-cfg-aer2",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-2"}},
+        "accessType":"create","user":"user3", "userGroups":["group-in-svc-cfg-aer2"],"requestData":"create db-1.tbl-2;' for group-in-svc-cfg-aer2"
+      },
+      "result":{"isAudited":false,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"AUDIT 'create db-1.tbl-3;' for user3",
+      "request":{
+        "resource":{"elements":{"database":"db-1", "table":"tbl-3"}},
+        "accessType":"create","user":"user3","userGroups":["users"],"requestData":"create db-1.tbl-3;' for user3"
+      },
+      "result":{"isAudited":true,"isAllowed":true,"policyId":1}
+    }
+  ,
+    {"name":"NO AUDIT 'create testdb;' for aeu1; no audit",
+      "request":{
+        "resource":{"elements":{"database":"testdb"}},
+        "accessType":"create","user":"aeu1","requestData":"create db-1 for aeu1"
+      },
+      "result":{"isAudited":false,"isAllowed":false,"policyId":-1}
+    }
+  ]
+}
+