You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sh...@apache.org on 2014/03/13 22:21:26 UTC

[11/13] SENTRY-143: Merge db_policy_store branch into master (Brock Noland via Shreepadma Venugopalan)

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
index f348e0e..e88ae32 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
@@ -18,14 +18,11 @@ package org.apache.sentry.policy.db;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Arrays;
 
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.model.db.Database;
-import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.After;
@@ -36,7 +33,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
 public class TestPolicyParsingNegative {
@@ -75,12 +72,8 @@ public class TestPolicyParsingNegative {
     append("other_group = malicious_role", otherPolicyFile);
     append("[roles]", otherPolicyFile);
     append("malicious_role = server=server1->db=customers->table=purchases->action=select", otherPolicyFile);
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1"),
-            new Database("other_group_db")
-    }), Lists.newArrayList("other_group")).get("other_group");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
   @Test
@@ -93,33 +86,25 @@ public class TestPolicyParsingNegative {
     policyFile.addGroupsToUser("admin1", "admin");
     policyFile.write(globalPolicyFile);
     policyFile.write(otherPolicyFile);
-    policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1")
-    }), Lists.newArrayList("admin")).get("admin");
+    policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
     // test to ensure [users] fails parsing of per-db file
     policyFile.addDatabase("other", otherPolicyFile.getPath());
     policyFile.write(globalPolicyFile);
     policyFile.write(otherPolicyFile);
-    policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1")
-    }), Lists.newArrayList("admin")).get("admin");
+    policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
     // test to ensure [databases] fails parsing of per-db file
     // by removing the user mapping from the per-db policy file
     policyFile.removeGroupsFromUser("admin1", "admin")
       .write(otherPolicyFile);
-    policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1")
-    }), Lists.newArrayList("admin")).get("admin");
+    policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
   }
+
   @Test
   public void testDatabaseRequiredInRole() throws Exception {
     append("[databases]", globalPolicyFile);
@@ -128,40 +113,30 @@ public class TestPolicyParsingNegative {
     append("other_group = malicious_role", otherPolicyFile);
     append("[roles]", otherPolicyFile);
     append("malicious_role = server=server1", otherPolicyFile);
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1"),
-            new Database("other_group_db")
-    }), Lists.newArrayList("other_group")).get("other_group");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
+
   @Test
   public void testServerAll() throws Exception {
     append("[groups]", globalPolicyFile);
     append("group = malicious_role", globalPolicyFile);
     append("[roles]", globalPolicyFile);
     append("malicious_role = server=*", globalPolicyFile);
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            Server.ALL,
-            new Database("some_db")
-    }), Lists.newArrayList("group")).get("group");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
+
   @Test
   public void testServerIncorrect() throws Exception {
     append("[groups]", globalPolicyFile);
     append("group = malicious_role", globalPolicyFile);
     append("[roles]", globalPolicyFile);
     append("malicious_role = server=server2", globalPolicyFile);
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            Server.ALL,
-            new Database("some_db")
-    }), Lists.newArrayList("group")).get("group");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -171,12 +146,8 @@ public class TestPolicyParsingNegative {
     append("group = malicious_role", globalPolicyFile);
     append("[roles]", globalPolicyFile);
     append("malicious_role = *", globalPolicyFile);
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            Server.ALL,
-            new Database("some_db")
-    }), Lists.newArrayList("group")).get("group");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -211,30 +182,13 @@ public class TestPolicyParsingNegative {
     append("[roles]", db2PolicyFile);
     append("db2_rule = server=server1->db=db2->table=purchases->action=select", db2PolicyFile);
 
-    PolicyEngine policy = new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
+    PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
 
     // verify that the db1 rule is empty
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1"),
-            new Database("db1")
-    }), Lists.newArrayList("db1_group")).get("db1_group");
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("db1_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
 
-    permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1"),
-            new Database("db2")
-    }), Lists.newArrayList("db2_group")).get("db2_group");
+    permissions = policy.getPrivileges(Sets.newHashSet("db2_group"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), 1, permissions.size());
-
-    permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Server("server1"),
-            new Database("db2")
-    }), Lists.newArrayList("db2_group")).get("db2_group");
-    Assert.assertEquals(permissions.toString(), 1, permissions.size());
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
index 2f4c20e..469be14 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
@@ -27,6 +27,7 @@ import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.AccessConstants;
@@ -90,7 +91,8 @@ public class TestResourceAuthorizationProviderGeneralCases {
     baseDir = Files.createTempDir();
     PolicyFiles.copyToDir(baseDir, "test-authz-provider.ini", "test-authz-provider-other-group.ini");
     authzProvider = new HadoopGroupResourceAuthorizationProvider(
-        new DBPolicyFileBackend(new File(baseDir, "test-authz-provider.ini").getPath(), "server1"),
+        new DBPolicyFileBackend("server1",
+        new File(baseDir, "test-authz-provider.ini").getPath()),
         new MockGroupMappingServiceProvider(USER_TO_GROUP_MAP));
 
   }
@@ -110,7 +112,7 @@ public class TestResourceAuthorizationProviderGeneralCases {
       helper.add("authorizables", authzHierarchy).add("Privileges", privileges);
     LOGGER.info("Running with " + helper.toString());
     Assert.assertEquals(helper.toString(), expected,
-        authzProvider.hasAccess(subject, authzHierarchy, privileges));
+        authzProvider.hasAccess(subject, authzHierarchy, privileges, ActiveRoleSet.ALL));
     LOGGER.info("Passed " + helper.toString());
   }
 
@@ -125,7 +127,7 @@ public class TestResourceAuthorizationProviderGeneralCases {
     .add("Table", table).add("Privileges", privileges).add("authzHierarchy", authzHierarchy);
     LOGGER.info("Running with " + helper.toString());
     Assert.assertEquals(helper.toString(), expected,
-        authzProvider.hasAccess(subject, authzHierarchy, privileges));
+        authzProvider.hasAccess(subject, authzHierarchy, privileges, ActiveRoleSet.ALL));
     LOGGER.info("Passed " + helper.toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderSpecialCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderSpecialCases.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderSpecialCases.java
index 688b845..3ae901e 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderSpecialCases.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderSpecialCases.java
@@ -25,8 +25,9 @@ import java.util.Set;
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.DBModelAction;
@@ -73,11 +74,11 @@ public class TestResourceAuthorizationProviderSpecialCases {
       .addPermissionsToRole("role1", true, "server=" + server1.getName() + "->uri=" + uri.getName(),
           "server=" + server1.getName() + "->uri=" + uri.getName());
     policyFile.write(iniFile);
-    DBPolicyFileBackend policy = new DBPolicyFileBackend(initResource, server1.getName());
+    DBPolicyFileBackend policy = new DBPolicyFileBackend(server1.getName(), initResource);
     authzProvider = new LocalGroupResourceAuthorizationProvider(initResource, policy);
     List<? extends Authorizable> authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertTrue(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
   }
   @Test
   public void testNonAbolutePath() throws Exception {
@@ -89,30 +90,30 @@ public class TestResourceAuthorizationProviderSpecialCases {
       .addRolesToGroup("group1", "role1")
       .addPermissionsToRole("role1", "server=" + server1.getName() + "->uri=" + uri.getName());
     policyFile.write(iniFile);
-    DBPolicyFileBackend policy = new DBPolicyFileBackend(initResource, server1.getName());
+    DBPolicyFileBackend policy = new DBPolicyFileBackend(server1.getName(), initResource);
     authzProvider = new LocalGroupResourceAuthorizationProvider(initResource, policy);
     // positive test
     List<? extends Authorizable> authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertTrue(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
     // negative tests
     // TODO we should support the case of /path/to/./ but let's to that later
     uri = new AccessURI("file:///path/to/./");
     authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertFalse(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
     uri = new AccessURI("file:///path/to/../");
     authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertFalse(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
     uri = new AccessURI("file:///path/to/../../");
     authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertFalse(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
     uri = new AccessURI("file:///path/to/dir/../../");
     authorizableHierarchy = ImmutableList.of(server1, uri);
     Assert.assertFalse(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
   }
   @Test(expected=IllegalArgumentException.class)
   public void testInvalidPath() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
index c093dde..08f84a3 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
@@ -18,7 +18,7 @@ package org.apache.sentry.policy.db;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.List;
+import java.util.Set;
 
 import junit.framework.Assert;
 
@@ -26,17 +26,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.model.db.Database;
-import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.provider.file.PolicyFiles;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableSetMultimap;
-import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
 public class TestSimpleDBPolicyEngineDFS extends AbstractTestSimplePolicyEngine {
@@ -72,7 +70,8 @@ public class TestSimpleDBPolicyEngineDFS extends AbstractTestSimplePolicyEngine
     fileSystem.delete(etc, true);
     fileSystem.mkdirs(etc);
     PolicyFiles.copyToDir(fileSystem, etc, "test-authz-provider.ini", "test-authz-provider-other-group.ini");
-    setPolicy(new DBPolicyFileBackend(new Path(etc, "test-authz-provider.ini").toString(), "server1"));
+    setPolicy(new DBPolicyFileBackend("server1",
+        new Path(etc, "test-authz-provider.ini").toString()));
   }
   @Override
   protected void beforeTeardown() throws IOException {
@@ -104,15 +103,12 @@ public class TestSimpleDBPolicyEngineDFS extends AbstractTestSimplePolicyEngine
     PolicyFiles.copyFilesToDir(fileSystem, etc, globalPolicyFile);
     PolicyFiles.copyFilesToDir(fileSystem, etc, dbPolicyFile);
     DBPolicyFileBackend multiFSEngine =
-        new DBPolicyFileBackend(globalPolicyFile.getPath(), "server1");
+        new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
 
-    List<Authorizable> dbAuthorizables = Lists.newArrayList();
-    dbAuthorizables.add(new Server("server1"));
-    dbAuthorizables.add(new Database("db11"));
-    List<String> dbGroups = Lists.newArrayList();
+    Set<String> dbGroups = Sets.newHashSet();
     dbGroups.add("group1");
-    ImmutableSetMultimap <String, String> dbPerms =
-        multiFSEngine.getPermissions(dbAuthorizables, dbGroups);
+    ImmutableSet<String> dbPerms =
+        multiFSEngine.getPrivileges(dbGroups, ActiveRoleSet.ALL);
     Assert.assertEquals("No DB permissions found", 1, dbPerms.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineLocalFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineLocalFS.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineLocalFS.java
index 86ec2fa..cb4e1a2 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineLocalFS.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineLocalFS.java
@@ -32,7 +32,8 @@ public class TestSimpleDBPolicyEngineLocalFS extends AbstractTestSimplePolicyEng
     Assert.assertNotNull(baseDir);
     Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs());
     PolicyFiles.copyToDir(baseDir, "test-authz-provider.ini", "test-authz-provider-other-group.ini");
-    setPolicy(new DBPolicyFileBackend(new File(baseDir, "test-authz-provider.ini").getPath(), "server1"));
+    setPolicy(new DBPolicyFileBackend("server1",
+        new File(baseDir, "test-authz-provider.ini").getPath()));
   }
   @Override
   protected void beforeTeardown() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchPrivilegeValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchPrivilegeValidator.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchPrivilegeValidator.java
new file mode 100644
index 0000000..a4e611c
--- /dev/null
+++ b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchPrivilegeValidator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.sentry.policy.search;
+
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_SPLITTER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.PRIVILEGE_PREFIX;
+
+import java.util.List;
+
+import org.apache.sentry.core.model.search.SearchModelAuthorizable;
+import org.apache.sentry.policy.common.PrivilegeValidator;
+import org.apache.shiro.config.ConfigurationException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+
+public abstract class AbstractSearchPrivilegeValidator implements PrivilegeValidator {
+
+  @VisibleForTesting
+  public static Iterable<SearchModelAuthorizable> parsePrivilege(String string) {
+    List<SearchModelAuthorizable> result = Lists.newArrayList();
+    System.err.println("privilege = " + string);
+    for(String section : AUTHORIZABLE_SPLITTER.split(string)) {
+      // XXX this ugly hack is because action is not an authorizable
+      if(!section.toLowerCase().startsWith(PRIVILEGE_PREFIX)) {
+        SearchModelAuthorizable authorizable = SearchModelAuthorizables.from(section);
+        if(authorizable == null) {
+          String msg = "No authorizable found for " + section;
+          throw new ConfigurationException(msg);
+        }
+        result.add(authorizable);
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchRoleValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchRoleValidator.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchRoleValidator.java
deleted file mode 100644
index 8e7c19f..0000000
--- a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/AbstractSearchRoleValidator.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.sentry.policy.search;
-
-import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_SPLITTER;
-import static org.apache.sentry.provider.file.PolicyFileConstants.PRIVILEGE_PREFIX;
-
-import java.util.List;
-
-import org.apache.sentry.policy.common.RoleValidator;
-import org.apache.sentry.core.model.search.SearchModelAuthorizable;
-import org.apache.shiro.config.ConfigurationException;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-
-public abstract class AbstractSearchRoleValidator implements RoleValidator {
-
-  @VisibleForTesting
-  public static Iterable<SearchModelAuthorizable> parseRole(String string) {
-    List<SearchModelAuthorizable> result = Lists.newArrayList();
-    for(String section : AUTHORIZABLE_SPLITTER.split(string)) {
-      // XXX this ugly hack is because action is not an authorizeable
-      if(!section.toLowerCase().startsWith(PRIVILEGE_PREFIX)) {
-        SearchModelAuthorizable authorizable = SearchModelAuthorizables.from(section);
-        if(authorizable == null) {
-          String msg = "No authorizable found for " + section;
-          throw new ConfigurationException(msg);
-        }
-        result.add(authorizable);
-      }
-    }
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInPrivilege.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInPrivilege.java
new file mode 100644
index 0000000..81ff67f
--- /dev/null
+++ b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInPrivilege.java
@@ -0,0 +1,43 @@
+/*
+ * 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.sentry.policy.search;
+
+import org.apache.sentry.core.common.SentryConfigurationException;
+import org.apache.sentry.core.model.search.Collection;
+import org.apache.sentry.core.model.search.SearchModelAuthorizable;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
+
+public class CollectionRequiredInPrivilege extends AbstractSearchPrivilegeValidator {
+
+  @Override
+  public void validate(PrivilegeValidatorContext context) throws SentryConfigurationException {
+    String privilege = context.getPrivilege();
+    Iterable<SearchModelAuthorizable> authorizables = parsePrivilege(privilege);
+    boolean foundCollectionInAuthorizables = false;
+
+    for(SearchModelAuthorizable authorizable : authorizables) {
+      if(authorizable instanceof Collection) {
+        foundCollectionInAuthorizables = true;
+        break;
+      }
+    }
+    if(!foundCollectionInAuthorizables) {
+      String msg = "Missing collection object in " + privilege;
+      throw new SentryConfigurationException(msg);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInRole.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInRole.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInRole.java
deleted file mode 100644
index 7f152d9..0000000
--- a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/CollectionRequiredInRole.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.sentry.policy.search;
-
-import javax.annotation.Nullable;
-
-import org.apache.sentry.core.model.search.Collection;
-import org.apache.sentry.core.model.search.SearchModelAuthorizable;
-import org.apache.shiro.config.ConfigurationException;
-
-public class CollectionRequiredInRole extends AbstractSearchRoleValidator {
-
-  @Override
-  public void validate(@Nullable String database, String role) throws ConfigurationException {
-    Iterable<SearchModelAuthorizable> authorizables = parseRole(role);
-    boolean foundCollectionInAuthorizables = false;
-
-    for(SearchModelAuthorizable authorizable : authorizables) {
-      if(authorizable instanceof Collection) {
-        foundCollectionInAuthorizables = true;
-        break;
-      }
-    }
-
-    if(!foundCollectionInAuthorizables) {
-      String msg = "Missing collection object in " + role;
-      throw new ConfigurationException(msg);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPermission.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPermission.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPermission.java
deleted file mode 100644
index 2d2e0bb..0000000
--- a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPermission.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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.
- */
-
-// copied from apache shiro
-
-package org.apache.sentry.policy.search;
-
-import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
-import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_SPLITTER;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.apache.sentry.core.model.search.SearchConstants;
-import org.apache.sentry.policy.common.PermissionFactory;
-import org.apache.sentry.provider.file.KeyValue;
-import org.apache.sentry.provider.file.PolicyFileConstants;
-import org.apache.shiro.authz.Permission;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-public class SearchWildcardPermission implements Permission, Serializable {
-  private static final Logger LOGGER = LoggerFactory
-      .getLogger(SearchWildcardPermission.class);
-  private static final long serialVersionUID = -6785051263922740819L;
-
-  private final ImmutableList<KeyValue> parts;
-
-  public SearchWildcardPermission(String wildcardString) {
-    wildcardString = Strings.nullToEmpty(wildcardString).trim();
-    if (wildcardString.isEmpty()) {
-      throw new IllegalArgumentException("Wildcard string cannot be null or empty.");
-    }
-    List<KeyValue>parts = Lists.newArrayList();
-    for (String authorizable : AUTHORIZABLE_SPLITTER.trimResults().split(wildcardString)) {
-      if (authorizable.isEmpty()) {
-        throw new IllegalArgumentException("Privilege '" + wildcardString + "' has an empty section");
-      }
-      parts.add(new KeyValue(authorizable));
-    }
-    if (parts.isEmpty()) {
-      throw new AssertionError("Should never occur: " + wildcardString);
-    }
-    this.parts = ImmutableList.copyOf(parts);
-  }
-
-
-  @Override
-  public boolean implies(Permission p) {
-    // By default only supports comparisons with other SearchWildcardPermissions
-    if (!(p instanceof SearchWildcardPermission)) {
-      return false;
-    }
-
-    SearchWildcardPermission wp = (SearchWildcardPermission) p;
-
-    List<KeyValue> otherParts = wp.parts;
-    if(equals(wp)) {
-      return true;
-    }
-    int index = 0;
-    for (KeyValue otherPart : otherParts) {
-      // If this permission has less parts than the other permission, everything
-      // after the number of parts contained
-      // in this permission is automatically implied, so return true
-      if (parts.size() - 1 < index) {
-        return true;
-      } else {
-        KeyValue part = parts.get(index);
-        // are the keys even equal
-        if(!part.getKey().equalsIgnoreCase(otherPart.getKey())) {
-          return false;
-        }
-        if (!impliesKeyValue(part, otherPart)) {
-          return false;
-        }
-        index++;
-      }
-    }
-    // If this permission has more parts than
-    // the other parts, only imply it if
-    // all of the other parts are wildcards
-    for (; index < parts.size(); index++) {
-      KeyValue part = parts.get(index);
-      if (!part.getValue().equals(SearchConstants.ALL)) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  private boolean impliesKeyValue(KeyValue policyPart, KeyValue requestPart) {
-    Preconditions.checkState(policyPart.getKey().equalsIgnoreCase(requestPart.getKey()),
-        "Please report, this method should not be called with two different keys");
-    if(policyPart.getValue().equals(SearchConstants.ALL) || policyPart.equals(requestPart)) {
-      return true;
-    } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
-        && SearchConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {
-      /* permission request is to match with any object of given type */
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return AUTHORIZABLE_JOINER.join(parts);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof SearchWildcardPermission) {
-      SearchWildcardPermission wp = (SearchWildcardPermission) o;
-      return parts.equals(wp.parts);
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return parts.hashCode();
-  }
-
-  public static class SearchWildcardPermissionFactory implements PermissionFactory {
-    @Override
-    public Permission createPermission(String permission) {
-      return new SearchWildcardPermission(permission);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPrivilege.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPrivilege.java
new file mode 100644
index 0000000..9a33fcf
--- /dev/null
+++ b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SearchWildcardPrivilege.java
@@ -0,0 +1,146 @@
+/*
+ * 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.
+ */
+
+// copied from apache shiro
+
+package org.apache.sentry.policy.search;
+
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_SPLITTER;
+
+import java.util.List;
+
+import org.apache.sentry.core.model.search.SearchConstants;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.provider.file.KeyValue;
+import org.apache.sentry.provider.file.PolicyFileConstants;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class SearchWildcardPrivilege implements Privilege {
+
+  private final ImmutableList<KeyValue> parts;
+
+  public SearchWildcardPrivilege(String wildcardString) {
+    wildcardString = Strings.nullToEmpty(wildcardString).trim();
+    if (wildcardString.isEmpty()) {
+      throw new IllegalArgumentException("Wildcard string cannot be null or empty.");
+    }
+    List<KeyValue>parts = Lists.newArrayList();
+    for (String authorizable : AUTHORIZABLE_SPLITTER.trimResults().split(wildcardString)) {
+      if (authorizable.isEmpty()) {
+        throw new IllegalArgumentException("Privilege '" + wildcardString + "' has an empty section");
+      }
+      parts.add(new KeyValue(authorizable));
+    }
+    if (parts.isEmpty()) {
+      throw new AssertionError("Should never occur: " + wildcardString);
+    }
+    this.parts = ImmutableList.copyOf(parts);
+  }
+
+
+  @Override
+  public boolean implies(Privilege p) {
+    // By default only supports comparisons with other SearchWildcardPermissions
+    if (!(p instanceof SearchWildcardPrivilege)) {
+      return false;
+    }
+
+    SearchWildcardPrivilege wp = (SearchWildcardPrivilege) p;
+
+    List<KeyValue> otherParts = wp.parts;
+    if(equals(wp)) {
+      return true;
+    }
+    int index = 0;
+    for (KeyValue otherPart : otherParts) {
+      // If this privilege has less parts than the other privilege, everything
+      // after the number of parts contained
+      // in this privilege is automatically implied, so return true
+      if (parts.size() - 1 < index) {
+        return true;
+      } else {
+        KeyValue part = parts.get(index);
+        // are the keys even equal
+        if(!part.getKey().equalsIgnoreCase(otherPart.getKey())) {
+          return false;
+        }
+        if (!impliesKeyValue(part, otherPart)) {
+          return false;
+        }
+        index++;
+      }
+    }
+    // If this privilege has more parts than
+    // the other parts, only imply it if
+    // all of the other parts are wildcards
+    for (; index < parts.size(); index++) {
+      KeyValue part = parts.get(index);
+      if (!part.getValue().equals(SearchConstants.ALL)) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  private boolean impliesKeyValue(KeyValue policyPart, KeyValue requestPart) {
+    Preconditions.checkState(policyPart.getKey().equalsIgnoreCase(requestPart.getKey()),
+        "Please report, this method should not be called with two different keys");
+    if(policyPart.getValue().equals(SearchConstants.ALL) || policyPart.equals(requestPart)) {
+      return true;
+    } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
+        && SearchConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {
+      /* privilege request is to match with any object of given type */
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return AUTHORIZABLE_JOINER.join(parts);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof SearchWildcardPrivilege) {
+      SearchWildcardPrivilege wp = (SearchWildcardPrivilege) o;
+      return parts.equals(wp.parts);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return parts.hashCode();
+  }
+
+  public static class SearchWildcardPrivilegeFactory implements PrivilegeFactory {
+    @Override
+    public Privilege createPrivilege(String privilege) {
+      return new SearchWildcardPrivilege(privilege);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
index 51ab35d..728e356 100644
--- a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
+++ b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
@@ -16,29 +16,20 @@
  */
 package org.apache.sentry.policy.search;
 
-import javax.annotation.Nullable;
+import java.util.Set;
 
-import java.io.IOException;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.shiro.config.ConfigurationException;
-import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.policy.common.PermissionFactory;
+import org.apache.sentry.policy.common.PrivilegeFactory;
 import org.apache.sentry.policy.common.PolicyEngine;
-import org.apache.sentry.policy.common.RoleValidator;
+import org.apache.sentry.policy.common.PrivilegeValidator;
 import org.apache.sentry.provider.common.ProviderBackend;
-import org.apache.sentry.provider.common.Roles;
-import org.apache.sentry.provider.file.SimpleFileProviderBackend;
+import org.apache.sentry.provider.common.ProviderBackendContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSetMultimap;
-import com.google.common.collect.Lists;
 
 /**
  * A PolicyEngine for a search service.
@@ -48,87 +39,46 @@ public class SimpleSearchPolicyEngine implements PolicyEngine {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(SimpleSearchPolicyEngine.class);
 
-  private ProviderBackend providerBackend;
+  private final ProviderBackend providerBackend;
 
   public SimpleSearchPolicyEngine(ProviderBackend providerBackend) {
-    List<? extends RoleValidator> validators =
-      Lists.newArrayList(new CollectionRequiredInRole());
     this.providerBackend = providerBackend;
-    this.providerBackend.process(validators);
-
-    if (!this.providerBackend.getRoles().getPerDatabaseRoles().isEmpty()) {
-      throw new ConfigurationException(
-        "SimpleSearchPolicyEngine does not support per-database roles, " +
-        "but per-database roles were specified.  Ignoring.");
-    }
-  }
-
-  /*
-   * Note: finalize is final because constructor throws exception, see:
-   * OBJ11-J.
-   */
-  public final void finalize() {
-    // do nothing
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setAllowPerDatabase(false);
+    context.setValidators(createPrivilegeValidators());
+    this.providerBackend.initialize(context);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public PermissionFactory getPermissionFactory() {
-    return new SearchWildcardPermission.SearchWildcardPermissionFactory();
+  public PrivilegeFactory getPrivilegeFactory() {
+    return new SearchWildcardPrivilege.SearchWildcardPrivilegeFactory();
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public ImmutableSetMultimap<String, String> getPermissions(List<? extends Authorizable> authorizables, List<String> groups) {
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("Getting permissions for {}", groups);
     }
-    ImmutableSetMultimap.Builder<String, String> resultBuilder = ImmutableSetMultimap.builder();
-    for(String group : groups) {
-      resultBuilder.putAll(group, getSearchRoles(group,providerBackend.getRoles()));
-    }
-    ImmutableSetMultimap<String, String> result = resultBuilder.build();
+    ImmutableSet<String> result = providerBackend.getPrivileges(groups, roleSet);
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("result = " + result);
     }
     return result;
   }
 
-  private ImmutableSet<String> getSearchRoles(String group, Roles roles) {
-    ImmutableSetMultimap<String, String> globalRoles = roles.getGlobalRoles();
-    ImmutableSet.Builder<String> resultBuilder = ImmutableSet.builder();
-
-    if(globalRoles.containsKey(group)) {
-      resultBuilder.addAll(globalRoles.get(group));
-    }
-    ImmutableSet<String> result = resultBuilder.build();
-    if(LOGGER.isDebugEnabled()) {
-      LOGGER.debug("Group {}, Result {}",
-          new Object[]{ group, result});
-    }
-    return result;
-  }
-
-  @Override
-  public ImmutableSet<String> listPermissions(String groupName)
-      throws SentryConfigurationException {
-    // TODO: not supported yet
-    throw new SentryConfigurationException("Not implemented yet");
-  }
-
   @Override
-  public ImmutableSet<String> listPermissions(List<String> groupName)
+  public void validatePolicy(boolean strictValidation)
       throws SentryConfigurationException {
     throw new SentryConfigurationException("Not implemented yet");
   }
 
-  @Override
-  public void validatePolicy(boolean strictValidation)
-      throws SentryConfigurationException {
-    throw new SentryConfigurationException("Not implemented yet");
+  public static ImmutableList<PrivilegeValidator> createPrivilegeValidators() {
+    return ImmutableList.<PrivilegeValidator>of(new CollectionRequiredInPrivilege());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/AbstractTestSearchPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/AbstractTestSearchPolicyEngine.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/AbstractTestSearchPolicyEngine.java
index 24e9521..d1c415b 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/AbstractTestSearchPolicyEngine.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/AbstractTestSearchPolicyEngine.java
@@ -18,15 +18,13 @@ package org.apache.sentry.policy.search;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.model.search.Collection;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -34,7 +32,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
@@ -50,7 +47,6 @@ public abstract class AbstractTestSearchPolicyEngine {
 
   private PolicyEngine policy;
   private static File baseDir;
-  private List<Authorizable> authorizables = Lists.newArrayList();
 
   @BeforeClass
   public static void setupClazz() throws IOException {
@@ -94,7 +90,7 @@ public abstract class AbstractTestSearchPolicyEngine {
         ANALYST_TMPCOLLECTION_QUERY, JRANALYST_JRANALYST1_ALL,
         JRANALYST_PURCHASES_PARTIAL_QUERY));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("manager")).values())
+        new TreeSet<String>(policy.getPrivileges(set("manager"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -105,7 +101,7 @@ public abstract class AbstractTestSearchPolicyEngine {
         ANALYST_JRANALYST1_ACTION_ALL, ANALYST_TMPCOLLECTION_UPDATE,
         ANALYST_TMPCOLLECTION_QUERY));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("analyst")).values())
+        new TreeSet<String>(policy.getPrivileges(set("analyst"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -115,7 +111,7 @@ public abstract class AbstractTestSearchPolicyEngine {
         .newHashSet(JRANALYST_JRANALYST1_ALL,
             JRANALYST_PURCHASES_PARTIAL_QUERY));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("jranalyst")).values())
+        new TreeSet<String>(policy.getPrivileges(set("jranalyst"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -123,11 +119,11 @@ public abstract class AbstractTestSearchPolicyEngine {
   public void testAdmin() throws Exception {
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(ADMIN_COLLECTION_ALL));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("admin")).values())
+        new TreeSet<String>(policy.getPrivileges(set("admin"), ActiveRoleSet.ALL))
         .toString());
   }
 
-  private static List<String> list(String... values) {
-    return Lists.newArrayList(values);
+  private static Set<String> set(String... values) {
+    return Sets.newHashSet(values);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
index 874f2db..fd8af78 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
@@ -17,6 +17,7 @@
 package org.apache.sentry.policy.search;
 
 import java.io.IOException;
+
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class SearchPolicyFileBackend extends SimpleSearchPolicyEngine {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestCollectionRequiredInRole.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestCollectionRequiredInRole.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestCollectionRequiredInRole.java
index a56aabd..b626f1a 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestCollectionRequiredInRole.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestCollectionRequiredInRole.java
@@ -20,6 +20,7 @@ package org.apache.sentry.policy.search;
 
 import junit.framework.Assert;
 
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.shiro.config.ConfigurationException;
 import org.junit.Test;
 
@@ -27,11 +28,11 @@ public class TestCollectionRequiredInRole {
 
   @Test
   public void testEmptyRole() throws Exception {
-    CollectionRequiredInRole collRequiredInRole = new CollectionRequiredInRole();
+    CollectionRequiredInPrivilege collRequiredInRole = new CollectionRequiredInPrivilege();
 
     // check no db
     try {
-      collRequiredInRole.validate(null,"index=index1");
+      collRequiredInRole.validate(new PrivilegeValidatorContext("index=index1"));
       Assert.fail("Expected ConfigurationException");
     } catch (ConfigurationException e) {
       ;
@@ -39,7 +40,7 @@ public class TestCollectionRequiredInRole {
 
     // check with db
     try {
-      collRequiredInRole.validate("db1","index=index2");
+      collRequiredInRole.validate(new PrivilegeValidatorContext("db1","index=index2"));
       Assert.fail("Expected ConfigurationException");
     } catch (ConfigurationException e) {
       ;
@@ -48,15 +49,15 @@ public class TestCollectionRequiredInRole {
 
   @Test
   public void testCollectionWithoutAction() throws Exception {
-    CollectionRequiredInRole collRequiredInRole = new CollectionRequiredInRole();
-    collRequiredInRole.validate(null,"collection=nodb");
-    collRequiredInRole.validate("db2","collection=db");
+    CollectionRequiredInPrivilege collRequiredInRole = new CollectionRequiredInPrivilege();
+    collRequiredInRole.validate(new PrivilegeValidatorContext("collection=nodb"));
+    collRequiredInRole.validate(new PrivilegeValidatorContext("db2","collection=db"));
   }
 
   @Test
   public void testCollectionWithAction() throws Exception {
-    CollectionRequiredInRole collRequiredInRole = new CollectionRequiredInRole();
-    collRequiredInRole.validate(null,"collection=nodb->action=query");
-    collRequiredInRole.validate("db2","collection=db->action=update");
+    CollectionRequiredInPrivilege collRequiredInRole = new CollectionRequiredInPrivilege();
+    collRequiredInRole.validate(new PrivilegeValidatorContext(null,"collection=nodb->action=query"));
+    collRequiredInRole.validate(new PrivilegeValidatorContext("db2","collection=db->action=update"));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
index 4bbaf3a..6f36243 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
@@ -27,11 +27,11 @@ import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.core.model.search.SearchConstants;
-import org.apache.sentry.core.model.search.SearchModelAction;
 import org.apache.sentry.core.model.search.Collection;
+import org.apache.sentry.core.model.search.SearchModelAction;
 import org.apache.sentry.provider.common.MockGroupMappingServiceProvider;
 import org.apache.sentry.provider.file.HadoopGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.PolicyFiles;
@@ -117,7 +117,7 @@ public class TestSearchAuthorizationProviderGeneralCases {
       .add("Privileges", privileges).add("authzHierarchy", authzHierarchy);
     LOGGER.info("Running with " + helper.toString());
     Assert.assertEquals(helper.toString(), expected,
-        authzProvider.hasAccess(subject, authzHierarchy, privileges));
+        authzProvider.hasAccess(subject, authzHierarchy, privileges, ActiveRoleSet.ALL));
     LOGGER.info("Passed " + helper.toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderSpecialCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderSpecialCases.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderSpecialCases.java
index 2a7872d..801a702 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderSpecialCases.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderSpecialCases.java
@@ -25,8 +25,9 @@ import java.util.Set;
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;
@@ -75,7 +76,7 @@ public class TestSearchAuthorizationProviderSpecialCases {
     authzProvider = new LocalGroupResourceAuthorizationProvider(initResource, policy);
     List<? extends Authorizable> authorizableHierarchy = ImmutableList.of(collection1);
     Assert.assertTrue(authorizableHierarchy.toString(),
-        authzProvider.hasAccess(user1, authorizableHierarchy, actions));
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchModelAuthorizables.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchModelAuthorizables.java
index bd06b7e..c68cd75 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchModelAuthorizables.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchModelAuthorizables.java
@@ -20,9 +20,7 @@ package org.apache.sentry.policy.search;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNull;
 
-
 import org.apache.sentry.core.model.search.Collection;
-import org.apache.sentry.policy.search.SearchModelAuthorizables;
 import org.junit.Test;
 
 public class TestSearchModelAuthorizables {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyEngineDFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyEngineDFS.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyEngineDFS.java
index 1683eec..735935e 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyEngineDFS.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyEngineDFS.java
@@ -18,7 +18,6 @@ package org.apache.sentry.policy.search;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.List;
 
 import junit.framework.Assert;
 
@@ -26,16 +25,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.provider.file.PolicyFiles;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableSetMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
 
 public class TestSearchPolicyEngineDFS extends AbstractTestSearchPolicyEngine {
 
@@ -71,7 +63,8 @@ public class TestSearchPolicyEngineDFS extends AbstractTestSearchPolicyEngine {
     fileSystem.delete(etc, true);
     fileSystem.mkdirs(etc);
     PolicyFiles.copyToDir(fileSystem, etc, "test-authz-provider.ini");
-    setPolicy(new SearchPolicyFileBackend(new Path(etc, "test-authz-provider.ini").toString()));
+    setPolicy(new SearchPolicyFileBackend(new Path(etc,
+        "test-authz-provider.ini").toString()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyNegative.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyNegative.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyNegative.java
index 0770aa8..2abe8f2 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyNegative.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchPolicyNegative.java
@@ -18,16 +18,13 @@ package org.apache.sentry.policy.search;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.Collections;
 
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.shiro.config.ConfigurationException;
-import org.apache.sentry.core.common.Authorizable;
-import org.apache.sentry.core.model.search.Collection;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.policy.common.PolicyEngine;
-import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,7 +33,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
 public class TestSearchPolicyNegative {
@@ -72,26 +69,22 @@ public class TestSearchPolicyNegative {
     append("[databases]", globalPolicyFile);
     append("other_group_db = " + otherPolicyFile.getPath(), globalPolicyFile);
     append("[groups]", otherPolicyFile);
-    append("other_group = malicious_role", otherPolicyFile);
+    append("other_group = some_role", otherPolicyFile);
     append("[roles]", otherPolicyFile);
-    append("malicious_role = collection=*", otherPolicyFile);
-    try {
-      PolicyEngine policy = new SearchPolicyFileBackend(globalPolicyFile.getPath());
-      Assert.fail("Excepted ConfigurationException");
-    } catch (ConfigurationException ce) {}
+    append("some_role = collection=c1", otherPolicyFile);
+    SearchPolicyFileBackend policy = new SearchPolicyFileBackend(globalPolicyFile.getPath());
+    Assert.assertEquals(Collections.emptySet(),
+        policy.getPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL));
   }
 
   @Test
   public void testCollectionRequiredInRole() throws Exception {
     append("[groups]", globalPolicyFile);
-    append("group = malicious_role", globalPolicyFile);
+    append("group = some_role", globalPolicyFile);
     append("[roles]", globalPolicyFile);
-    append("malicious_role = action=query", globalPolicyFile);
+    append("some_role = action=query", globalPolicyFile);
     PolicyEngine policy = new SearchPolicyFileBackend(globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            new Collection("collection1"),
-    }), Lists.newArrayList("group")).get("group");
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -102,10 +95,7 @@ public class TestSearchPolicyNegative {
     append("[roles]", globalPolicyFile);
     append("malicious_role = collection=*", globalPolicyFile);
     PolicyEngine policy = new SearchPolicyFileBackend(globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPermissions(
-        Arrays.asList(new Authorizable[] {
-            Collection.ALL
-    }), Lists.newArrayList("incorrectGroup")).get("incorrectGroup");
+    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("incorrectGroup"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPermission.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPermission.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPermission.java
deleted file mode 100644
index b20595d..0000000
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPermission.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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.sentry.policy.search;
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertTrue;
-import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
-import static org.apache.sentry.provider.file.PolicyFileConstants.KV_JOINER;
-import static org.apache.sentry.provider.file.PolicyFileConstants.KV_SEPARATOR;
-
-import org.apache.sentry.core.model.search.SearchConstants;
-import org.apache.sentry.provider.file.KeyValue;
-import org.apache.shiro.authz.Permission;
-import org.junit.Test;
-
-public class TestSearchWildcardPermission {
-
-  private static final String ALL = SearchConstants.ALL;
-
-  @Test
-  public void testSimpleNoAction() throws Exception {
-    Permission collection1 = create(new KeyValue("collection", "coll1"));
-    Permission collection2 = create(new KeyValue("collection", "coll2"));
-    Permission collection1Case = create(new KeyValue("colleCtIon", "coLl1"));
-
-    assertTrue(collection1.implies(collection1));
-    assertTrue(collection2.implies(collection2));
-    assertTrue(collection1.implies(collection1Case));
-    assertTrue(collection1Case.implies(collection1));
-
-    assertFalse(collection1.implies(collection2));
-    assertFalse(collection1Case.implies(collection2));
-    assertFalse(collection2.implies(collection1));
-    assertFalse(collection2.implies(collection1Case));
-  }
-
-  @Test
-  public void testSimpleAction() throws Exception {
-    Permission query =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
-    Permission update =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
-    Permission queryCase =
-      create(new KeyValue("colleCtIon", "coLl1"), new KeyValue("AcTiOn", "QuERy"));
-
-    assertTrue(query.implies(query));
-    assertTrue(update.implies(update));
-    assertTrue(query.implies(queryCase));
-    assertTrue(queryCase.implies(query));
-
-    assertFalse(query.implies(update));
-    assertFalse(queryCase.implies(update));
-    assertFalse(update.implies(query));
-    assertFalse(update.implies(queryCase));
-  }
-
-  @Test
-  public void testRoleShorterThanRequest() throws Exception {
-    Permission collection1 = create(new KeyValue("collection", "coll1"));
-    Permission query =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
-    Permission update =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
-    Permission all =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", ALL));
-
-    assertTrue(collection1.implies(query));
-    assertTrue(collection1.implies(update));
-    assertTrue(collection1.implies(all));
-
-    assertFalse(query.implies(collection1));
-    assertFalse(update.implies(collection1));
-    assertTrue(all.implies(collection1));
-  }
-
-  @Test
-  public void testCollectionAll() throws Exception {
-    Permission collectionAll = create(new KeyValue("collection", ALL));
-    Permission collection1 = create(new KeyValue("collection", "coll1"));
-    assertTrue(collectionAll.implies(collection1));
-    assertTrue(collection1.implies(collectionAll));
-
-    Permission allUpdate =
-      create(new KeyValue("collection", ALL), new KeyValue("action", "update"));
-    Permission allQuery =
-      create(new KeyValue("collection", ALL), new KeyValue("action", "query"));
-    Permission coll1Update =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
-    Permission coll1Query =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
-    assertTrue(allUpdate.implies(coll1Update));
-    assertTrue(allQuery.implies(coll1Query));
-    assertTrue(coll1Update.implies(allUpdate));
-    assertTrue(coll1Query.implies(allQuery));
-    assertFalse(allUpdate.implies(coll1Query));
-    assertFalse(coll1Update.implies(coll1Query));
-    assertFalse(allQuery.implies(coll1Update));
-    assertFalse(coll1Query.implies(allUpdate));
-    assertFalse(allUpdate.implies(allQuery));
-    assertFalse(allQuery.implies(allUpdate));
-    assertFalse(coll1Update.implies(coll1Query));
-    assertFalse(coll1Query.implies(coll1Update));
-
-    // test different length paths
-    assertTrue(collectionAll.implies(allUpdate));
-    assertTrue(collectionAll.implies(allQuery));
-    assertTrue(collectionAll.implies(coll1Update));
-    assertTrue(collectionAll.implies(coll1Query));
-    assertFalse(allUpdate.implies(collectionAll));
-    assertFalse(allQuery.implies(collectionAll));
-    assertFalse(coll1Update.implies(collectionAll));
-    assertFalse(coll1Query.implies(collectionAll));
-  }
-
-  @Test
-  public void testActionAll() throws Exception {
-    Permission coll1All =
-       create(new KeyValue("collection", "coll1"), new KeyValue("action", ALL));
-    Permission coll1Update =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
-    Permission coll1Query =
-      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
-    assertTrue(coll1All.implies(coll1All));
-    assertTrue(coll1All.implies(coll1Update));
-    assertTrue(coll1All.implies(coll1Query));
-    assertFalse(coll1Update.implies(coll1All));
-    assertFalse(coll1Query.implies(coll1All));
-
-    // test different lengths
-    Permission coll1 =
-       create(new KeyValue("collection", "coll1"));
-    assertTrue(coll1All.implies(coll1));
-    assertTrue(coll1.implies(coll1All));
-  }
-
-  @Test
-  public void testUnexpected() throws Exception {
-    Permission p = new Permission() {
-      @Override
-      public boolean implies(Permission p) {
-        return false;
-      }
-    };
-    Permission collection1 = create(new KeyValue("collection", "coll1"));
-    assertFalse(collection1.implies(null));
-    assertFalse(collection1.implies(p));
-    assertFalse(collection1.equals(null));
-    assertFalse(collection1.equals(p));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testNullString() throws Exception {
-    System.out.println(create((String)null));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyString() throws Exception {
-    System.out.println(create(""));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyKey() throws Exception {
-    System.out.println(create(KV_JOINER.join("collection", "")));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyValue() throws Exception {
-    System.out.println(create(KV_JOINER.join("", "coll1")));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyPart() throws Exception {
-    System.out.println(create(AUTHORIZABLE_JOINER.
-        join(KV_JOINER.join("collection1", "coll1"), "")));
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testOnlySeperators() throws Exception {
-    System.out.println(create(AUTHORIZABLE_JOINER.
-        join(KV_SEPARATOR, KV_SEPARATOR, KV_SEPARATOR)));
-  }
-
-  static SearchWildcardPermission create(KeyValue... keyValues) {
-    return create(AUTHORIZABLE_JOINER.join(keyValues));
-
-  }
-  static SearchWildcardPermission create(String s) {
-    return new SearchWildcardPermission(s);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPrivilege.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPrivilege.java
new file mode 100644
index 0000000..cb5531f
--- /dev/null
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchWildcardPrivilege.java
@@ -0,0 +1,205 @@
+/*
+ * 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.sentry.policy.search;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertTrue;
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.KV_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.KV_SEPARATOR;
+
+import org.apache.sentry.core.model.search.SearchConstants;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.provider.file.KeyValue;
+import org.junit.Test;
+
+public class TestSearchWildcardPrivilege {
+
+  private static final String ALL = SearchConstants.ALL;
+
+  @Test
+  public void testSimpleNoAction() throws Exception {
+    Privilege collection1 = create(new KeyValue("collection", "coll1"));
+    Privilege collection2 = create(new KeyValue("collection", "coll2"));
+    Privilege collection1Case = create(new KeyValue("colleCtIon", "coLl1"));
+
+    assertTrue(collection1.implies(collection1));
+    assertTrue(collection2.implies(collection2));
+    assertTrue(collection1.implies(collection1Case));
+    assertTrue(collection1Case.implies(collection1));
+
+    assertFalse(collection1.implies(collection2));
+    assertFalse(collection1Case.implies(collection2));
+    assertFalse(collection2.implies(collection1));
+    assertFalse(collection2.implies(collection1Case));
+  }
+
+  @Test
+  public void testSimpleAction() throws Exception {
+    Privilege query =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
+    Privilege update =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
+    Privilege queryCase =
+      create(new KeyValue("colleCtIon", "coLl1"), new KeyValue("AcTiOn", "QuERy"));
+
+    assertTrue(query.implies(query));
+    assertTrue(update.implies(update));
+    assertTrue(query.implies(queryCase));
+    assertTrue(queryCase.implies(query));
+
+    assertFalse(query.implies(update));
+    assertFalse(queryCase.implies(update));
+    assertFalse(update.implies(query));
+    assertFalse(update.implies(queryCase));
+  }
+
+  @Test
+  public void testRoleShorterThanRequest() throws Exception {
+    Privilege collection1 = create(new KeyValue("collection", "coll1"));
+    Privilege query =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
+    Privilege update =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
+    Privilege all =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", ALL));
+
+    assertTrue(collection1.implies(query));
+    assertTrue(collection1.implies(update));
+    assertTrue(collection1.implies(all));
+
+    assertFalse(query.implies(collection1));
+    assertFalse(update.implies(collection1));
+    assertTrue(all.implies(collection1));
+  }
+
+  @Test
+  public void testCollectionAll() throws Exception {
+    Privilege collectionAll = create(new KeyValue("collection", ALL));
+    Privilege collection1 = create(new KeyValue("collection", "coll1"));
+    assertTrue(collectionAll.implies(collection1));
+    assertTrue(collection1.implies(collectionAll));
+
+    Privilege allUpdate =
+      create(new KeyValue("collection", ALL), new KeyValue("action", "update"));
+    Privilege allQuery =
+      create(new KeyValue("collection", ALL), new KeyValue("action", "query"));
+    Privilege coll1Update =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
+    Privilege coll1Query =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
+    assertTrue(allUpdate.implies(coll1Update));
+    assertTrue(allQuery.implies(coll1Query));
+    assertTrue(coll1Update.implies(allUpdate));
+    assertTrue(coll1Query.implies(allQuery));
+    assertFalse(allUpdate.implies(coll1Query));
+    assertFalse(coll1Update.implies(coll1Query));
+    assertFalse(allQuery.implies(coll1Update));
+    assertFalse(coll1Query.implies(allUpdate));
+    assertFalse(allUpdate.implies(allQuery));
+    assertFalse(allQuery.implies(allUpdate));
+    assertFalse(coll1Update.implies(coll1Query));
+    assertFalse(coll1Query.implies(coll1Update));
+
+    // test different length paths
+    assertTrue(collectionAll.implies(allUpdate));
+    assertTrue(collectionAll.implies(allQuery));
+    assertTrue(collectionAll.implies(coll1Update));
+    assertTrue(collectionAll.implies(coll1Query));
+    assertFalse(allUpdate.implies(collectionAll));
+    assertFalse(allQuery.implies(collectionAll));
+    assertFalse(coll1Update.implies(collectionAll));
+    assertFalse(coll1Query.implies(collectionAll));
+  }
+
+  @Test
+  public void testActionAll() throws Exception {
+    Privilege coll1All =
+       create(new KeyValue("collection", "coll1"), new KeyValue("action", ALL));
+    Privilege coll1Update =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "update"));
+    Privilege coll1Query =
+      create(new KeyValue("collection", "coll1"), new KeyValue("action", "query"));
+    assertTrue(coll1All.implies(coll1All));
+    assertTrue(coll1All.implies(coll1Update));
+    assertTrue(coll1All.implies(coll1Query));
+    assertFalse(coll1Update.implies(coll1All));
+    assertFalse(coll1Query.implies(coll1All));
+
+    // test different lengths
+    Privilege coll1 =
+       create(new KeyValue("collection", "coll1"));
+    assertTrue(coll1All.implies(coll1));
+    assertTrue(coll1.implies(coll1All));
+  }
+
+  @Test
+  public void testUnexpected() throws Exception {
+    Privilege p = new Privilege() {
+      @Override
+      public boolean implies(Privilege p) {
+        return false;
+      }
+    };
+    Privilege collection1 = create(new KeyValue("collection", "coll1"));
+    assertFalse(collection1.implies(null));
+    assertFalse(collection1.implies(p));
+    assertFalse(collection1.equals(null));
+    assertFalse(collection1.equals(p));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testNullString() throws Exception {
+    System.out.println(create((String)null));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyString() throws Exception {
+    System.out.println(create(""));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyKey() throws Exception {
+    System.out.println(create(KV_JOINER.join("collection", "")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyValue() throws Exception {
+    System.out.println(create(KV_JOINER.join("", "coll1")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyPart() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_JOINER.join("collection1", "coll1"), "")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testOnlySeperators() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_SEPARATOR, KV_SEPARATOR, KV_SEPARATOR)));
+  }
+
+  static SearchWildcardPrivilege create(KeyValue... keyValues) {
+    return create(AUTHORIZABLE_JOINER.join(keyValues));
+
+  }
+  static SearchWildcardPrivilege create(String s) {
+    return new SearchWildcardPrivilege(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/644e8be3/sentry-provider/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-provider/pom.xml b/sentry-provider/pom.xml
index b1594cf..9bec058 100644
--- a/sentry-provider/pom.xml
+++ b/sentry-provider/pom.xml
@@ -32,6 +32,7 @@ limitations under the License.
   <modules>
     <module>sentry-provider-common</module>
     <module>sentry-provider-file</module>
+    <module>sentry-provider-db</module>
   </modules>
 
 </project>