You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by gc...@apache.org on 2014/03/05 01:14:37 UTC

[1/4] SENTRY-122: Refactor provider/policy API to allow for DB-policy provider (Brock Noland via Gregory Chanan)

Repository: incubator-sentry
Updated Branches:
  refs/heads/db_policy_store 21094febb -> 92212c3d3


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/ResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/ResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/ResourceAuthorizationProvider.java
index 1b5f2c2..4d8551c 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/ResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/ResourceAuthorizationProvider.java
@@ -21,7 +21,7 @@ import static org.apache.sentry.provider.file.PolicyFileConstants.KV_JOINER;
 import static org.apache.sentry.provider.file.PolicyFileConstants.PRIVILEGE_NAME;
 
 import java.util.ArrayList;
-import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -29,32 +29,38 @@ import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.policy.common.PermissionFactory;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.policy.common.PrivilegeFactory;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.provider.common.GroupMappingService;
-import org.apache.shiro.authz.Permission;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 public abstract class ResourceAuthorizationProvider implements AuthorizationProvider {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(ResourceAuthorizationProvider.class);
   private final GroupMappingService groupService;
   private final PolicyEngine policy;
-  private final PermissionFactory permissionFactory;
-  private final List<String> lastFailedPermissions = new ArrayList<String>();
+  private final PrivilegeFactory privilegeFactory;
+  private final ThreadLocal<List<String>> lastFailedPrivileges;
 
   public ResourceAuthorizationProvider(PolicyEngine policy,
       GroupMappingService groupService) {
     this.policy = policy;
     this.groupService = groupService;
-    this.permissionFactory = policy.getPermissionFactory();
+    this.privilegeFactory = policy.getPrivilegeFactory();
+    this.lastFailedPrivileges = new ThreadLocal<List<String>>() {
+      @Override
+      protected List<String> initialValue() {
+        return new ArrayList<String>();
+      }
+    };
   }
 
   /***
@@ -83,40 +89,40 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
 
   private boolean doHasAccess(Subject subject,
       List<? extends Authorizable> authorizables, Set<? extends Action> actions) {
-    List<String> groups =  getGroups(subject);
-    List<String> hierarchy = new ArrayList<String>();
+    Set<String> groups =  getGroups(subject);
+    Set<String> hierarchy = new HashSet<String>();
     for (Authorizable authorizable : authorizables) {
       hierarchy.add(KV_JOINER.join(authorizable.getTypeName(), authorizable.getName()));
     }
-    Iterable<Permission> permissions = getPermissions(authorizables, groups);
-    List<String> requestPermissions = buildPermissions(authorizables, actions);
-    lastFailedPermissions.clear();
+    Iterable<Privilege> privileges = getPermissions(groups);
+    List<String> requestPrivileges = buildPermissions(authorizables, actions);
+    lastFailedPrivileges.get().clear();
 
-    for (String requestPermission : requestPermissions) {
-      for (Permission permission : permissions) {
+    for (String requestPrivilege : requestPrivileges) {
+      for (Privilege permission : privileges) {
         /*
          * Does the permission granted in the policy file imply the requested action?
          */
-        boolean result = permission.implies(permissionFactory.createPermission(requestPermission));
+        boolean result = permission.implies(privilegeFactory.createPrivilege(requestPrivilege));
         if(LOGGER.isDebugEnabled()) {
-          LOGGER.debug("FilePermission {}, RequestPermission {}, result {}",
-              new Object[]{ permission, requestPermission, result});
+          LOGGER.debug("ProviderPrivilege {}, RequestPrivilege {}, result {}",
+              new Object[]{ permission, requestPrivilege, result});
         }
         if (result) {
           return true;
         }
       }
     }
-    lastFailedPermissions.addAll(requestPermissions);
+    lastFailedPrivileges.get().addAll(requestPrivileges);
     return false;
   }
 
-  private Iterable<Permission> getPermissions(List<? extends Authorizable> authorizables, List<String> groups) {
-    return Iterables.transform(policy.getPermissions(authorizables, groups).values(),
-        new Function<String, Permission>() {
+  private Iterable<Privilege> getPermissions(Set<String> groups) {
+    return Iterables.transform(policy.getPrivileges(groups),
+        new Function<String, Privilege>() {
       @Override
-      public Permission apply(String permission) {
-        return permissionFactory.createPermission(permission);
+      public Privilege apply(String privilege) {
+        return privilegeFactory.createPrivilege(privilege);
       }
     });
   }
@@ -126,7 +132,7 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     return groupService;
   }
 
-  private List<String> getGroups(Subject subject) {
+  private Set<String> getGroups(Subject subject) {
     return groupService.getGroups(subject.getName());
   }
 
@@ -136,18 +142,18 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
   }
 
   @Override
-  public Set<String> listPermissionsForSubject(Subject subject) throws SentryConfigurationException {
-    return policy.listPermissions(getGroups(subject));
+  public Set<String> listPrivilegesForSubject(Subject subject) throws SentryConfigurationException {
+    return policy.getPrivileges(getGroups(subject));
   }
 
   @Override
-  public Set<String> listPermissionsForGroup(String groupName) throws SentryConfigurationException {
-    return policy.listPermissions(groupName);
+  public Set<String> listPrivilegesForGroup(String groupName) throws SentryConfigurationException {
+    return policy.getPrivileges(Sets.newHashSet(groupName));
   }
 
   @Override
-  public List<String> getLastFailedPermissions() {
-    return lastFailedPermissions;
+  public List<String> getLastFailedPrivileges() {
+    return lastFailedPrivileges.get();
   }
 
   private List<String> buildPermissions(List<? extends Authorizable> authorizables,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
index 9eabb53..f02da69 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
@@ -22,10 +22,8 @@ import static org.apache.sentry.provider.file.PolicyFileConstants.ROLES;
 import static org.apache.sentry.provider.file.PolicyFileConstants.ROLE_SPLITTER;
 import static org.apache.sentry.provider.file.PolicyFileConstants.USERS;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -36,27 +34,25 @@ import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.policy.common.RoleValidator;
+import org.apache.sentry.policy.common.PrivilegeUtils;
+import org.apache.sentry.policy.common.PrivilegeValidator;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.sentry.provider.common.ProviderBackend;
-import org.apache.sentry.provider.common.Roles;
-import org.apache.shiro.config.ConfigurationException;
+import org.apache.sentry.provider.common.ProviderBackendContext;
 import org.apache.shiro.config.Ini;
-import org.apache.shiro.util.PermissionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
 import com.google.common.collect.HashMultimap;
-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;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.SetMultimap;
 import com.google.common.collect.Sets;
 
 public class SimpleFileProviderBackend implements ProviderBackend {
@@ -64,46 +60,89 @@ public class SimpleFileProviderBackend implements ProviderBackend {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(SimpleFileProviderBackend.class);
 
-
-
   private final FileSystem fileSystem;
   private final Path resourcePath;
-  private final List<Path> perDbResources = Lists.newArrayList();
-  private Roles rolesStorage;
   private final Configuration conf;
-  private boolean processed;
-  private final List<String> configErrors = new ArrayList<String>();
-  private final List<String> configWarnings = new ArrayList<String>();
+  private final List<String> configErrors;
+  private final List<String> configWarnings;
+  private final SetMultimap<String, String> groupToPrivilegeMap;
+
+  private ImmutableList<PrivilegeValidator> validators;
+  private boolean allowPerDatabaseSection;
+  private volatile boolean initialized;
 
   public SimpleFileProviderBackend(String resourcePath) throws IOException {
-    this(new Configuration(), resourcePath);
+    this(new Configuration(), new Path(resourcePath));
   }
 
   public SimpleFileProviderBackend(Configuration conf, String resourcePath) throws IOException {
     this(conf, new Path(resourcePath));
   }
 
-  @VisibleForTesting
   public SimpleFileProviderBackend(Configuration conf, Path resourcePath) throws IOException {
     this.resourcePath = resourcePath;
     this.fileSystem = resourcePath.getFileSystem(conf);
-    this.rolesStorage = new Roles();
+    this.groupToPrivilegeMap = HashMultimap.create();
     this.conf = conf;
-    this.processed = false;
+    this.configErrors = Lists.newArrayList();
+    this.configWarnings = Lists.newArrayList();
+    this.validators = ImmutableList.of();
+    this.allowPerDatabaseSection = true;
+    this.initialized = false;
   }
 
   /**
    * {@inheritDoc}
    */
-  public void process(List<? extends RoleValidator> validators) {
+  @Override
+  public void initialize(ProviderBackendContext context) {
+    if (initialized) {
+      throw new IllegalStateException("Backend has already been initialized, cannot be initialized twice");
+    }
+    this.validators = context.getValidators();
+    this.allowPerDatabaseSection = context.isAllowPerDatabase();
+    parse();
+    this.initialized = true;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public ImmutableSet<String> getPrivileges(Set<String> groups) {
+    if (!initialized) {
+      throw new IllegalStateException("Backend has not been properly initialized");
+    }
+    ImmutableSet.Builder<String> resultBuilder = ImmutableSet.builder();
+    for (String group : groups) {
+      resultBuilder.addAll(groupToPrivilegeMap.get(group));
+    }
+    return resultBuilder.build();
+  }
+
+  @Override
+  public void validatePolicy(boolean strictValidation) throws SentryConfigurationException {
+    if (!initialized) {
+      throw new IllegalStateException("Backend has not been properly initialized");
+    }
+    List<String> localConfigErrors = Lists.newArrayList(configErrors);
+    List<String> localConfigWarnings = Lists.newArrayList(configWarnings);
+    if ((strictValidation && !localConfigWarnings.isEmpty()) || !localConfigErrors.isEmpty()) {
+      localConfigErrors.add("Failed to process global policy file " + resourcePath);
+      SentryConfigurationException e = new SentryConfigurationException("");
+      e.setConfigErrors(localConfigErrors);
+      e.setConfigWarnings(localConfigWarnings);
+      throw e;
+    }
+  }
+
+  private void parse() {
     configErrors.clear();
-    perDbResources.clear();
+    configWarnings.clear();
+    SetMultimap<String, String> groupToPrivilegeMapTemp = HashMultimap.create();
     Ini ini;
-
     LOGGER.info("Parsing " + resourcePath);
-    Roles roles = new Roles();
     try {
-      perDbResources.clear();
       try {
         ini = PolicyFiles.loadFromPath(fileSystem, resourcePath);
       } catch (IOException e) {
@@ -126,12 +165,13 @@ public class SimpleFileProviderBackend implements ProviderBackend {
           }
         }
       }
-      ImmutableSetMultimap<String, String> globalRoles;
-      Map<String, ImmutableSetMultimap<String, String>> perDatabaseRoles = Maps.newHashMap();
-      globalRoles = parseIni(null, ini, validators, resourcePath);
+      groupToPrivilegeMapTemp.putAll(parseIni(null, ini, validators, resourcePath));
       Ini.Section filesSection = ini.getSection(DATABASES);
       if(filesSection == null) {
         LOGGER.info("Section " + DATABASES + " needs no further processing");
+      } else if (!allowPerDatabaseSection) {
+        String msg = "Per-db policy file is not expected in this configuration.";
+        throw new SentryConfigurationException(msg);
       } else {
         for(Map.Entry<String, String> entry : filesSection.entrySet()) {
           String database = Strings.nullToEmpty(entry.getKey()).trim().toLowerCase();
@@ -144,16 +184,14 @@ public class SimpleFileProviderBackend implements ProviderBackend {
             Ini perDbIni = PolicyFiles.loadFromPath(perDbPolicy.getFileSystem(conf), perDbPolicy);
             if(perDbIni.containsKey(USERS)) {
               configErrors.add("Per-db policy file cannot contain " + USERS + " section in " +  perDbPolicy);
-              throw new ConfigurationException("Per-db policy files cannot contain " + USERS + " section");
+              throw new SentryConfigurationException("Per-db policy files cannot contain " + USERS + " section");
             }
             if(perDbIni.containsKey(DATABASES)) {
               configErrors.add("Per-db policy files cannot contain " + DATABASES
                   + " section in " + perDbPolicy);
-              throw new ConfigurationException("Per-db policy files cannot contain " + DATABASES + " section");
+              throw new SentryConfigurationException("Per-db policy files cannot contain " + DATABASES + " section");
             }
-            ImmutableSetMultimap<String, String> currentDbRoles = parseIni(database, perDbIni, validators, perDbPolicy);
-            perDatabaseRoles.put(database, currentDbRoles);
-            perDbResources.add(perDbPolicy);
+            groupToPrivilegeMapTemp.putAll(parseIni(database, perDbIni, validators, perDbPolicy));
           } catch (Exception e) {
             configErrors.add("Failed to read per-DB policy file " + perDbPolicy +
                " Error: " + e.getMessage());
@@ -161,14 +199,14 @@ public class SimpleFileProviderBackend implements ProviderBackend {
           }
         }
       }
-      roles = new Roles(globalRoles, ImmutableMap.copyOf(perDatabaseRoles));
+      groupToPrivilegeMap.clear();
+      groupToPrivilegeMap.putAll(groupToPrivilegeMapTemp);
     } catch (Exception e) {
       configErrors.add("Error processing file " + resourcePath + e.getMessage());
       LOGGER.error("Error processing file, ignoring " + resourcePath, e);
     }
-    rolesStorage = roles;
-    this.processed = true;
   }
+
   /**
    * Relative for our purposes is no scheme, no authority
    * and a non-absolute path portion.
@@ -178,22 +216,8 @@ public class SimpleFileProviderBackend implements ProviderBackend {
     return uri.getAuthority() == null && uri.getScheme() == null && !path.isUriPathAbsolute();
   }
 
-  protected long getModificationTime() throws IOException {
-    // if resource path has been deleted, throw all exceptions
-    long result = fileSystem.getFileStatus(resourcePath).getModificationTime();
-    for(Path perDbPolicy : perDbResources) {
-      try {
-        result = Math.max(result, fileSystem.getFileStatus(perDbPolicy).getModificationTime());
-      } catch (FileNotFoundException e) {
-        // if a per-db file has been deleted, wait until the main
-        // policy file has been updated before refreshing
-      }
-    }
-    return result;
-  }
-
-  private ImmutableSetMultimap<String, String> parseIni(String database, Ini ini, List<? extends RoleValidator> validators,
-      Path policyPath) {
+  private ImmutableSetMultimap<String, String> parseIni(String database, Ini ini,
+      List<? extends PrivilegeValidator> validators, Path policyPath) {
     Ini.Section privilegesSection = ini.getSection(ROLES);
     boolean invalidConfiguration = false;
     if (privilegesSection == null) {
@@ -210,13 +234,13 @@ public class SimpleFileProviderBackend implements ProviderBackend {
       invalidConfiguration = true;
     }
     if (!invalidConfiguration) {
-      return parsePermissions(database, privilegesSection, groupsSection, validators, policyPath);
+      return parsePrivileges(database, privilegesSection, groupsSection, validators, policyPath);
     }
     return ImmutableSetMultimap.of();
   }
 
-  private ImmutableSetMultimap<String, String> parsePermissions(@Nullable String database,
-      Ini.Section rolesSection, Ini.Section groupsSection, List<? extends RoleValidator> validators,
+  private ImmutableSetMultimap<String, String> parsePrivileges(@Nullable String database,
+      Ini.Section rolesSection, Ini.Section groupsSection, List<? extends PrivilegeValidator> validators,
       Path policyPath) {
     ImmutableSetMultimap.Builder<String, String> resultBuilder = ImmutableSetMultimap.builder();
     Multimap<String, String> roleNameToPrivilegeMap = HashMultimap
@@ -242,15 +266,14 @@ public class SimpleFileProviderBackend implements ProviderBackend {
         LOGGER.warn(warnMsg);
         configWarnings.add(warnMsg);
       }
-      Set<String> roles = PermissionUtils
-          .toPermissionStrings(roleValue);
-      if (!invalidConfiguration && roles != null) {
-        for(String role : roles) {
-          for(RoleValidator validator : validators) {
-            validator.validate(database, role.trim());
+      Set<String> privileges = PrivilegeUtils.toPrivilegeStrings(roleValue);
+      if (!invalidConfiguration && privileges != null) {
+        for(String privilege : privileges) {
+          for(PrivilegeValidator validator : validators) {
+            validator.validate(new PrivilegeValidatorContext(database, privilege.trim()));
           }
         }
-        roleNameToPrivilegeMap.putAll(roleName, roles);
+        roleNameToPrivilegeMap.putAll(roleName, privileges);
       }
     }
     Splitter roleSplitter = ROLE_SPLITTER.omitEmptyStrings().trimResults();
@@ -273,26 +296,4 @@ public class SimpleFileProviderBackend implements ProviderBackend {
     }
     return resultBuilder.build();
   }
-
-  /*
-   * {@inheritDoc}
-   */
-  public Roles getRoles() {
-    if (!processed) throw new UnsupportedOperationException("Process has not been called");
-
-    return rolesStorage;
-  }
-
-  @Override
-  public void validatePolicy(List<? extends RoleValidator> validators, boolean strictValidation)
-      throws SentryConfigurationException {
-    if ((strictValidation && !configWarnings.isEmpty()) || !configErrors.isEmpty()) {
-      configErrors.add("Failed to process global policy file " + resourcePath);
-      SentryConfigurationException e = new SentryConfigurationException("");
-      e.setConfigErrors(configErrors);
-      e.setConfigWarnings(configWarnings);
-      throw e;
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestGetGroupMapping.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestGetGroupMapping.java b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestGetGroupMapping.java
index f223bee..39625f7 100644
--- a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestGetGroupMapping.java
+++ b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestGetGroupMapping.java
@@ -16,17 +16,18 @@
  */
 package org.apache.sentry.provider.file;
 
-import java.util.Arrays;
-import java.util.List;
-import org.apache.sentry.core.common.Authorizable;
+import static org.junit.Assert.assertSame;
+
+import java.util.Set;
+
 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.provider.common.GroupMappingService;
-import com.google.common.collect.ImmutableSetMultimap;
-import com.google.common.collect.ImmutableSet;
 import org.junit.Test;
-import static org.junit.Assert.assertSame;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 
 public class TestGetGroupMapping {
 
@@ -39,23 +40,14 @@ public class TestGetGroupMapping {
 
   @Test
   public void testResourceAuthorizationProvider() {
-    final List<String> list = Arrays.asList("a", "b", "c");
+    final Set<String> set = Sets.newHashSet("a", "b", "c");
     GroupMappingService mappingService = new GroupMappingService() {
-      public List<String> getGroups(String user) { return list; }
+      public Set<String> getGroups(String user) { return set; }
     };
     PolicyEngine policyEngine = new PolicyEngine() {
-      public PermissionFactory getPermissionFactory() { return null; }
-
-      public ImmutableSetMultimap<String, String> getPermissions(List<? extends Authorizable> authorizables, List<String> groups) { return null; }
-      public ImmutableSet<String> listPermissions(String groupName)
-          throws SentryConfigurationException {
-        return null;
-      }
+      public PrivilegeFactory getPrivilegeFactory() { return null; }
 
-      public ImmutableSet<String> listPermissions(List<String> groupName)
-          throws SentryConfigurationException {
-        return null;
-      }
+      public ImmutableSet<String> getPrivileges(Set<String> groups) { return null; }
 
       public void validatePolicy(boolean strictValidation)
           throws SentryConfigurationException {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestKeyValue.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestKeyValue.java b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestKeyValue.java
index 1fd64f1..1d8c9ae 100644
--- a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestKeyValue.java
+++ b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestKeyValue.java
@@ -21,7 +21,6 @@ import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertFalse;
 import static org.apache.sentry.provider.file.PolicyFileConstants.KV_JOINER;
 
-import org.apache.sentry.provider.file.KeyValue;
 import org.junit.Test;
 
 public class TestKeyValue {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestLocalGroupMapping.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestLocalGroupMapping.java b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestLocalGroupMapping.java
index f1d8192..c436009 100644
--- a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestLocalGroupMapping.java
+++ b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestLocalGroupMapping.java
@@ -19,25 +19,25 @@ package org.apache.sentry.provider.file;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.Path;
-import org.apache.sentry.provider.file.LocalGroupMappingService;
-import org.apache.sentry.provider.file.PolicyFiles;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
 public class TestLocalGroupMapping {
 
-  private String resourcePath = "test-authz-provider-local-group-mapping.ini";
+  private static final String resourcePath = "test-authz-provider-local-group-mapping.ini";
+  private static final Set<String> fooGroups = Sets.newHashSet("admin", "analyst");
+  private static final Set<String> barGroups = Sets.newHashSet("jranalyst");
+
   private LocalGroupMappingService localGroupMapping;
-  private String[] fooGroups = new String[] {"admin", "analyst" };
-  private String[] barGroups = new String[] {"jranalyst"};
 
   private File baseDir;
 
@@ -57,13 +57,13 @@ public class TestLocalGroupMapping {
 
   @Test
   public void testGroupMapping() {
-    List <String> fooGroupsFromResource = localGroupMapping.getGroups("foo");
-    Assert.assertArrayEquals(fooGroupsFromResource.toArray(), fooGroups);
+    Set<String> fooGroupsFromResource = localGroupMapping.getGroups("foo");
+    Assert.assertEquals(fooGroupsFromResource, fooGroups);
 
-    List <String> barGroupsFromResource = localGroupMapping.getGroups("bar");
-    Assert.assertArrayEquals(barGroupsFromResource.toArray(), barGroups);
+    Set<String> barGroupsFromResource = localGroupMapping.getGroups("bar");
+    Assert.assertEquals(barGroupsFromResource, barGroups);
 
-    List <String> unknownGroupsFromResource = localGroupMapping.getGroups("unknown");
+    Set<String> unknownGroupsFromResource = localGroupMapping.getGroups("unknown");
     Assert.assertTrue("List not empty " + unknownGroupsFromResource, unknownGroupsFromResource.isEmpty());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
index 6ae3776..6444407 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
@@ -26,8 +26,8 @@ import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.sentry.tests.e2e.hive.fs.DFSFactory;
 import org.apache.sentry.tests.e2e.hive.fs.DFS;
+import org.apache.sentry.tests.e2e.hive.fs.DFSFactory;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServer;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
 import org.junit.AfterClass;
@@ -173,8 +173,12 @@ public abstract class AbstractTestWithStaticConfiguration {
       }
       baseDir = null;
     }
-    if(dfs!=null) {
-      dfs.tearDown();
+    if(dfs != null) {
+      try {
+        dfs.tearDown();
+      } catch (Exception e) {
+        LOGGER.info("Exception shutting down dfs", e);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
index 2f83678..4f7dd2d 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.URI;
 import java.sql.Connection;
-import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Set;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestConfigTool.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestConfigTool.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestConfigTool.java
index 6968cc0..bb7bec2 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestConfigTool.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestConfigTool.java
@@ -17,36 +17,25 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Set;
 
-import junit.framework.Assert;
-
 import org.apache.sentry.binding.hive.authz.SentryConfigTool;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.provider.file.PolicyFile;
-
-import com.google.common.io.Resources;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 public class TestConfigTool extends AbstractTestWithStaticConfiguration {
   private static final String DB2_POLICY_FILE = "db2-policy-file.ini";
@@ -188,18 +177,18 @@ public class TestConfigTool extends AbstractTestWithStaticConfiguration {
     configTool.validatePolicy();
 
     Set<String> permList = configTool.getSentryProvider()
-        .listPermissionsForSubject(new Subject(USER1_1));
+        .listPrivilegesForSubject(new Subject(USER1_1));
     assertTrue(permList
         .contains("server=server1->db=db1->table=tab1->action=select"));
     assertTrue(permList
         .contains("server=server1->db=db1->table=tab2->action=insert"));
 
-    permList = configTool.getSentryProvider().listPermissionsForSubject(
+    permList = configTool.getSentryProvider().listPrivilegesForSubject(
         new Subject(USER2_1));
     assertTrue(permList
         .contains("server=server1->db=db1->table=tab3->action=select"));
 
-    permList = configTool.getSentryProvider().listPermissionsForSubject(
+    permList = configTool.getSentryProvider().listPrivilegesForSubject(
         new Subject(ADMIN1));
     assertTrue(permList.contains("server=server1"));
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPerDBConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPerDBConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPerDBConfiguration.java
index 80912a3..f782613 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPerDBConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPerDBConfiguration.java
@@ -26,8 +26,8 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 
-import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.policy.db.SimpleDBPolicyEngine;
+import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -71,6 +71,8 @@ public class TestPerDBConfiguration extends AbstractTestWithStaticConfiguration
 
   @After
   public void teardown() throws Exception {
+    // one test turns this on so let's disable it in the teardown method
+    System.setProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE, "false");
     if (context != null) {
       context.close();
     }
@@ -336,6 +338,13 @@ public class TestPerDBConfiguration extends AbstractTestWithStaticConfiguration
     context.assertAuthzException(statement, "SELECT COUNT(*) FROM db1.tbl1");
     context.assertAuthzException(statement, "USE db1");
 
+    // once we disable this property all queries should fail
+    System.setProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE, "false");
+    context.assertAuthzException(statement, "USE db2");
+
+    // re-enable for clean
+    System.setProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE, "true");
+
     statement.close();
     connection.close();
 
@@ -346,7 +355,6 @@ public class TestPerDBConfiguration extends AbstractTestWithStaticConfiguration
     statement.execute("DROP DATABASE db2 CASCADE");
     statement.close();
     connection.close();
-    System.setProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE, "false");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
index c267ea6..56ed06a 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtTableScope.java
@@ -17,8 +17,8 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
-import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSentryOnFailureHookLoading.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSentryOnFailureHookLoading.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSentryOnFailureHookLoading.java
index 8222590..cae270b 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSentryOnFailureHookLoading.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSentryOnFailureHookLoading.java
@@ -17,13 +17,9 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
-import com.google.common.io.Resources;
-import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
-import org.apache.sentry.provider.file.PolicyFile;
-import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.sql.Connection;
@@ -31,10 +27,17 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.HashMap;
 import java.util.Map;
+
 import junit.framework.Assert;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.io.Resources;
 
 public class TestSentryOnFailureHookLoading extends AbstractTestWithHiveServer {
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
index 1068dbe..145584d 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
@@ -16,12 +16,13 @@
  */
 package org.apache.sentry.tests.e2e.hive.fs;
 
+import java.io.IOException;
+
 import junit.framework.Assert;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import java.io.IOException;
-
 public abstract class AbstractDFS implements DFS{
   protected static FileSystem fileSystem;
   protected static Path dfsBaseDir;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
index 1e2c01e..d5db811 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
@@ -16,17 +16,16 @@
  */
 package org.apache.sentry.tests.e2e.hive.fs;
 
+import java.security.PrivilegedExceptionAction;
+import java.util.Random;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.security.PrivilegedExceptionAction;
-import java.util.Random;
-
 public class ClusterDFS extends AbstractDFS{
   private static final Logger LOGGER = LoggerFactory
       .getLogger(ClusterDFS.class);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
index b9764bc..9e9bb27 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
@@ -19,8 +19,6 @@ package org.apache.sentry.tests.e2e.hive.fs;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import java.io.IOException;
-
 public interface DFS {
   public FileSystem getFileSystem();
   public void tearDown() throws Exception;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
index c3e5bf3..c897b49 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
@@ -16,10 +16,10 @@
  */
 package org.apache.sentry.tests.e2e.hive.fs;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import java.io.File;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class DFSFactory {
   public static final String FS_TYPE = "sentry.e2etest.DFSType";
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
index dba2a54..de684a9 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
@@ -16,14 +16,14 @@
  */
 package org.apache.sentry.tests.e2e.hive.fs;
 
+import java.io.File;
+
 import junit.framework.Assert;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 
-import java.io.File;
-import java.io.IOException;
-
 public class MiniDFS extends AbstractDFS {
   private static MiniDFSCluster dfsCluster;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
index ce3b97c..52ba09e 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
@@ -17,12 +17,12 @@
 
 package org.apache.sentry.tests.e2e.hive.hiveserver;
 
-import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.fest.reflect.core.Reflection;
-
 import java.sql.Connection;
 import java.sql.DriverManager;
 
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.fest.reflect.core.Reflection;
+
 public class EmbeddedHiveServer implements HiveServer {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
index 0751e91..8af3f45 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
@@ -24,7 +24,6 @@ import java.net.ServerSocket;
 import java.net.URL;
 import java.util.Map;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -36,6 +35,7 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.io.Resources;
 
 public class HiveServerFactory {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
index 3a257bf..02d8024 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
@@ -18,6 +18,7 @@
 package org.apache.sentry.tests.e2e.hive.hiveserver;
 
 import java.io.IOException;
+
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hive.service.server.HiveServer2;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
index 4425efa..42a274f 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
@@ -16,16 +16,16 @@
  */
 package org.apache.sentry.tests.e2e.hive.hiveserver;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.util.Properties;
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.util.Properties;
+import com.google.common.base.Preconditions;
 
 public class UnmanagedHiveServer implements HiveServer {
   private static final Logger LOGGER = LoggerFactory.getLogger(UnmanagedHiveServer.class);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
index 05c5263..5103a79 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
@@ -19,7 +19,6 @@ package org.apache.sentry.tests.e2e.solr;
 import java.io.File;
 import java.io.IOException;
 import java.net.MalformedURLException;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.Random;
 import java.util.SortedMap;
@@ -47,12 +46,10 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.servlet.SolrDispatchFilter;
-
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
index f68fd28..bb566bb 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
@@ -9,7 +9,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrTestCaseJ4;
-import org.junit.Assert;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/ModifiableUserAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/ModifiableUserAuthenticationFilter.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/ModifiableUserAuthenticationFilter.java
index b7081ba..8df4ba3 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/ModifiableUserAuthenticationFilter.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/ModifiableUserAuthenticationFilter.java
@@ -28,7 +28,6 @@ import javax.servlet.http.HttpServletRequest;
 
 import org.apache.solr.servlet.SolrHadoopAuthenticationFilter;
 import org.apache.solr.servlet.SolrRequestParsers;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestCollAdminCoreOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestCollAdminCoreOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestCollAdminCoreOperations.java
index 865fd10..1313a75 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestCollAdminCoreOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestCollAdminCoreOperations.java
@@ -16,12 +16,6 @@
  */
 package org.apache.sentry.tests.e2e.solr;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
-
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -29,10 +23,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Properties;
 import java.util.Random;
 
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 
 @ThreadLeakScope(Scope.NONE) // hdfs client currently leaks thread(s)
 public class TestCollAdminCoreOperations extends AbstractSolrSentryTestBase {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestQueryOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestQueryOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestQueryOperations.java
index ace0d0f..271e6ef 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestQueryOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestQueryOperations.java
@@ -16,12 +16,6 @@
  */
 package org.apache.sentry.tests.e2e.solr;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
-
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -30,6 +24,11 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.solr.common.SolrInputDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 
 @ThreadLeakScope(Scope.NONE) // hdfs client currently leaks thread(s)
 public class TestQueryOperations extends AbstractSolrSentryTestBase {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestUpdateOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestUpdateOperations.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestUpdateOperations.java
index aaca7b4..0f2167a 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestUpdateOperations.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/TestUpdateOperations.java
@@ -16,12 +16,6 @@
  */
 package org.apache.sentry.tests.e2e.solr;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
-
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -30,6 +24,11 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.solr.common.SolrInputDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 
 @ThreadLeakScope(Scope.NONE) // hdfs client currently leaks thread(s)
 public class TestUpdateOperations extends AbstractSolrSentryTestBase {


[3/4] SENTRY-122: Refactor provider/policy API to allow for DB-policy provider (Brock Noland via Gregory Chanan)

Posted by gc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInRole.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInRole.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInRole.java
deleted file mode 100644
index 48b36a6..0000000
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInRole.java
+++ /dev/null
@@ -1,70 +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.db;
-
-import javax.annotation.Nullable;
-
-import org.apache.sentry.core.model.db.AccessURI;
-import org.apache.sentry.core.model.db.Database;
-import org.apache.sentry.core.model.db.DBModelAuthorizable;
-import org.apache.shiro.config.ConfigurationException;
-
-public class DatabaseRequiredInRole extends AbstractDBRoleValidator {
-
-  @Override
-  public void validate(@Nullable String database, String role) throws ConfigurationException {
-    /*
-     *  Rule only applies to rules in per database policy file
-     */
-    if(database != null) {
-      Iterable<DBModelAuthorizable> authorizables = parseRole(role);
-      /*
-       * Each permission in a non-global file must have a database
-       * object except for URIs.
-       *
-       * We allow URIs to be specified in the per DB policy file for
-       * ease of mangeability. URIs will contain to remain server scope
-       * objects.
-       */
-      boolean foundDatabaseInAuthorizables = false;
-      boolean foundURIInAuthorizables = false;
-      boolean allowURIInAuthorizables = false;
-
-      if ("true".equalsIgnoreCase(
-          System.getProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE))) {
-        allowURIInAuthorizables = true;
-      }
-
-      for(DBModelAuthorizable authorizable : authorizables) {
-        if(authorizable instanceof Database) {
-          foundDatabaseInAuthorizables = true;
-        }
-        if (authorizable instanceof AccessURI) {
-          if (foundDatabaseInAuthorizables) {
-            String msg = "URI object is specified at DB scope in " + role;
-            throw new ConfigurationException(msg);
-          }
-          foundURIInAuthorizables = true;
-        }
-      }
-      if(!foundDatabaseInAuthorizables && !(foundURIInAuthorizables && allowURIInAuthorizables)) {
-        String msg = "Missing database object in " + role;
-        throw new ConfigurationException(msg);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServerNameMustMatch.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServerNameMustMatch.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServerNameMustMatch.java
index 8ddf1dd..1848a32 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServerNameMustMatch.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServerNameMustMatch.java
@@ -16,25 +16,25 @@
  */
 package org.apache.sentry.policy.db;
 
-import javax.annotation.Nullable;
-
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.shiro.config.ConfigurationException;
 
-public class ServerNameMustMatch extends AbstractDBRoleValidator {
+public class ServerNameMustMatch extends AbstractDBPrivilegeValidator {
 
   private final String serverName;
   public ServerNameMustMatch(String serverName) {
     this.serverName = serverName;
   }
   @Override
-  public void validate(@Nullable String database, String role) throws ConfigurationException {
-    Iterable<DBModelAuthorizable> authorizables = parseRole(role);
+  public void validate(PrivilegeValidatorContext context) throws ConfigurationException {
+    String privilege = context.getPrivilege();
+    Iterable<DBModelAuthorizable> authorizables = parsePrivilege(privilege);
     for(DBModelAuthorizable authorizable : authorizables) {
       if(authorizable instanceof Server && !serverName.equalsIgnoreCase(authorizable.getName())) {
         String msg = "Server name " + authorizable.getName() + " in "
-      + role + " is invalid. Expected " + serverName;
+            + privilege + " is invalid. Expected " + serverName;
         throw new ConfigurationException(msg);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServersAllIsInvalid.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServersAllIsInvalid.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServersAllIsInvalid.java
index 9445b0b..b729ec3 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServersAllIsInvalid.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/ServersAllIsInvalid.java
@@ -16,21 +16,21 @@
  */
 package org.apache.sentry.policy.db;
 
-import javax.annotation.Nullable;
-
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.shiro.config.ConfigurationException;
 
-public class ServersAllIsInvalid extends AbstractDBRoleValidator {
+public class ServersAllIsInvalid extends AbstractDBPrivilegeValidator {
 
   @Override
-  public void validate(@Nullable String database, String role) throws ConfigurationException {
-    Iterable<DBModelAuthorizable> authorizables = parseRole(role);
+  public void validate(PrivilegeValidatorContext context) throws ConfigurationException {
+    String privilege = context.getPrivilege();
+    Iterable<DBModelAuthorizable> authorizables = parsePrivilege(privilege);
     for(DBModelAuthorizable authorizable : authorizables) {
       if(authorizable instanceof Server &&
           authorizable.getName().equals(Server.ALL.getName())) {
-        String msg = "Invalid value for " + authorizable.getAuthzType() + " in " + role;
+        String msg = "Invalid value for " + authorizable.getAuthzType() + " in " + privilege;
         throw new ConfigurationException(msg);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
index 1d01b47..7ea5a06 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
@@ -16,33 +16,19 @@
  */
 package org.apache.sentry.policy.db;
 
-import javax.annotation.Nullable;
+import java.util.Set;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.core.model.db.AccessURI;
-import org.apache.sentry.core.model.db.Database;
-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;
 
 public class SimpleDBPolicyEngine implements PolicyEngine {
 
@@ -51,119 +37,47 @@ public class SimpleDBPolicyEngine implements PolicyEngine {
 
   public final static String ACCESS_ALLOW_URI_PER_DB_POLICYFILE = "sentry.allow.uri.db.policyfile";
 
-  private ProviderBackend providerBackend;
-  private String serverName;
-  private List<? extends RoleValidator> validators;
+  private final ProviderBackend providerBackend;
 
   public SimpleDBPolicyEngine(String serverName, ProviderBackend providerBackend) {
-    validators = Lists.newArrayList(new ServersAllIsInvalid(), new DatabaseMustMatch(),
-          new DatabaseRequiredInRole(), new ServerNameMustMatch(serverName));
     this.providerBackend = providerBackend;
-    this.providerBackend.process(validators);
-    this.serverName = serverName;
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setAllowPerDatabase(true);
+    context.setValidators(createPrivilegeValidators(serverName));
+    this.providerBackend.initialize(context);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public PermissionFactory getPermissionFactory() {
-    return new DBWildcardPermission.DBWildcardPermissionFactory();
+  public PrivilegeFactory getPrivilegeFactory() {
+    return new DBWildcardPrivilege.DBWildcardPrivilegeFactory();
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public ImmutableSetMultimap<String, String> getPermissions(
-      List<? extends Authorizable> authorizables, List<String> groups)
+  public ImmutableSet<String> getPrivileges(Set<String> groups)
       throws SentryConfigurationException {
-    String database = null;
-    Boolean isURI = false;
-    for(Authorizable authorizable : authorizables) {
-      if(authorizable instanceof Database) {
-        database = authorizable.getName();
-      }
-      if (authorizable instanceof AccessURI) {
-        isURI = true;
-      }
-    }
-
     if(LOGGER.isDebugEnabled()) {
-      LOGGER.debug("Getting permissions for {} via {}", groups, database);
+      LOGGER.debug("Getting permissions for {}", groups);
     }
-    ImmutableSetMultimap.Builder<String, String> resultBuilder = ImmutableSetMultimap.builder();
-    for(String group : groups) {
-      resultBuilder.putAll(group, getDBRoles(database, group, isURI, providerBackend.getRoles()));
-    }
-    ImmutableSetMultimap<String, String> result = resultBuilder.build();
+    ImmutableSet<String> result = providerBackend.getPrivileges(groups);
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("result = " + result);
     }
     return result;
   }
 
-  private ImmutableSet<String> getDBRoles(@Nullable String database,
-      String group, Boolean isURI, Roles roles) {
-    ImmutableSetMultimap<String, String> globalRoles = roles.getGlobalRoles();
-    ImmutableMap<String, ImmutableSetMultimap<String, String>> perDatabaseRoles = roles.getPerDatabaseRoles();
-    ImmutableSet.Builder<String> resultBuilder = ImmutableSet.builder();
-    String allowURIPerDbFile =
-        System.getProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE);
-    Boolean consultPerDbRolesForURI = isURI && ("true".equalsIgnoreCase(allowURIPerDbFile));
-
-    // handle Database.ALL
-    if (Database.ALL.getName().equals(database)) {
-      for(Entry<String, ImmutableSetMultimap<String, String>> dbListEntry : perDatabaseRoles.entrySet()) {
-        if (dbListEntry.getValue().containsKey(group)) {
-          resultBuilder.addAll(dbListEntry.getValue().get(group));
-        }
-      }
-    } else if(database != null) {
-      ImmutableSetMultimap<String, String> dbPolicies =  perDatabaseRoles.get(database);
-      if(dbPolicies != null && dbPolicies.containsKey(group)) {
-        resultBuilder.addAll(dbPolicies.get(group));
-      }
-    }
-
-    if (consultPerDbRolesForURI) {
-      for(String db : perDatabaseRoles.keySet()) {
-        ImmutableSetMultimap<String, String> dbPolicies =  perDatabaseRoles.get(db);
-        if(dbPolicies != null && dbPolicies.containsKey(group)) {
-          resultBuilder.addAll(dbPolicies.get(group));
-        }
-      }
-    }
-
-    if(globalRoles.containsKey(group)) {
-      resultBuilder.addAll(globalRoles.get(group));
-    }
-    ImmutableSet<String> result = resultBuilder.build();
-    if(LOGGER.isDebugEnabled()) {
-      LOGGER.debug("Database {}, Group {}, Result {}",
-          new Object[]{ database, group, result});
-    }
-    return result;
-  }
-
   @Override
   public void validatePolicy(boolean strictValidation) throws SentryConfigurationException {
-    this.providerBackend.validatePolicy(validators, strictValidation);
+    this.providerBackend.validatePolicy(strictValidation);
   }
 
-  @Override
-  public ImmutableSet<String> listPermissions(String groupName) throws SentryConfigurationException {
-    return getDBRoles(Database.ALL.getName(), groupName, true, providerBackend.getRoles());
+  public static ImmutableList<PrivilegeValidator> createPrivilegeValidators(String serverName) {
+    return ImmutableList.<PrivilegeValidator>of(new ServersAllIsInvalid(), new DatabaseMustMatch(),
+        new DatabaseRequiredInPrivilege(), new ServerNameMustMatch(serverName));
   }
-
-  @Override
-  public ImmutableSet<String> listPermissions(List<String> groupNames)
-      throws SentryConfigurationException {
-    ImmutableSet.Builder<String> resultBuilder = ImmutableSet.builder();
-    for (String groupName : groupNames) {
-      resultBuilder.addAll(listPermissions(groupName));
-    }
-    return resultBuilder.build();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
index 89ca737..b4ed2e5 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
@@ -18,15 +18,12 @@ package org.apache.sentry.policy.db;
 
 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.db.Database;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -34,7 +31,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;
 
@@ -49,7 +45,6 @@ public abstract class AbstractTestSimplePolicyEngine {
   private static final String PERM_SERVER1_ADMIN = "server=server1";
   private PolicyEngine policy;
   private static File baseDir;
-  private List<Authorizable> authorizables = Lists.newArrayList();
 
   @BeforeClass
   public static void setupClazz() throws IOException {
@@ -93,7 +88,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT
         ));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("manager")).values())
+        new TreeSet<String>(policy.getPrivileges(set("manager")))
         .toString());
   }
 
@@ -103,7 +98,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         PERM_SERVER1_CUSTOMERS_SELECT, PERM_SERVER1_ANALYST_ALL,
         PERM_SERVER1_JUNIOR_ANALYST_READ));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("analyst")).values())
+        new TreeSet<String>(policy.getPrivileges(set("analyst")))
         .toString());
   }
 
@@ -113,7 +108,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         .newHashSet(PERM_SERVER1_JUNIOR_ANALYST_ALL,
             PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("jranalyst")).values())
+        new TreeSet<String>(policy.getPrivileges(set("jranalyst")))
         .toString());
   }
 
@@ -121,43 +116,40 @@ public abstract class AbstractTestSimplePolicyEngine {
   public void testAdmin() throws Exception {
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(PERM_SERVER1_ADMIN));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("admin")).values())
+        new TreeSet<String>(policy.getPrivileges(set("admin")))
         .toString());
   }
 
 
   @Test
   public void testOtherGroup() throws Exception {
-    authorizables.add(new Database("other_group_db"));
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(
         PERM_SERVER1_OTHER_GROUP_DB_CUSTOMERS_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("other_group")).values())
+        new TreeSet<String>(policy.getPrivileges(set("other_group")))
         .toString());
   }
 
   @Test
   public void testDbAll() throws Exception {
-    authorizables.add(new Database(Database.ALL.getName()));
     Set<String> expected = Sets.newTreeSet(Sets
         .newHashSet(PERM_SERVER1_JUNIOR_ANALYST_ALL,
             PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("jranalyst")).values())
+        new TreeSet<String>(policy.getPrivileges(set("jranalyst")))
         .toString());
   }
 
   @Test
   public void testDbAllforOtherGroup() throws Exception {
-    authorizables.add(new Database(Database.ALL.getName()));
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(
         PERM_SERVER1_OTHER_GROUP_DB_CUSTOMERS_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPermissions(authorizables, list("other_group")).values())
+        new TreeSet<String>(policy.getPrivileges(set("other_group")))
         .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/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
index d8d68b7..661deff 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
@@ -17,10 +17,11 @@
 package org.apache.sentry.policy.db;
 
 import java.io.IOException;
+
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class DBPolicyFileBackend extends SimpleDBPolicyEngine {
-  public DBPolicyFileBackend(String resource, String server) throws IOException{
+  public DBPolicyFileBackend(String server, String resource) throws IOException{
     super(server, new SimpleFileProviderBackend(resource));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBModelAuthorizables.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBModelAuthorizables.java
index 23b03d4..70f5e79 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBModelAuthorizables.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBModelAuthorizables.java
@@ -20,13 +20,11 @@ package org.apache.sentry.policy.db;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNull;
 
-
 import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
 import org.apache.sentry.core.model.db.View;
-import org.apache.sentry.policy.db.DBModelAuthorizables;
 import org.junit.Test;
 
 public class TestDBModelAuthorizables {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPermission.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPermission.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPermission.java
deleted file mode 100644
index 2024cd8..0000000
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPermission.java
+++ /dev/null
@@ -1,286 +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.db;
-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.db.AccessConstants;
-import org.apache.sentry.provider.file.KeyValue;
-import org.apache.shiro.authz.Permission;
-import org.junit.Test;
-
-public class TestDBWildcardPermission {
-
-  private static final String ALL = AccessConstants.ALL;
-
-  private static final Permission ROLE_SERVER_SERVER1_DB_ALL =
-      create(new KeyValue("server", "server1"), new KeyValue("db", ALL));
-  private static final Permission ROLE_SERVER_SERVER1_DB_DB1 =
-      create(new KeyValue("server", "server1"), new KeyValue("db", "db1"));
-  private static final Permission ROLE_SERVER_SERVER2_DB_ALL =
-      create(new KeyValue("server", "server2"), new KeyValue("db", ALL));
-  private static final Permission ROLE_SERVER_SERVER2_DB_DB1 =
-      create(new KeyValue("server", "server2"), new KeyValue("db", "db1"));
-  private static final Permission ROLE_SERVER_ALL_DB_ALL =
-      create(new KeyValue("server", ALL), new KeyValue("db", ALL));
-  private static final Permission ROLE_SERVER_ALL_DB_DB1 =
-      create(new KeyValue("server", ALL), new KeyValue("db", "db1"));
-
-  private static final Permission ROLE_SERVER_SERVER1_URI_URI1 =
-      create(new KeyValue("server", "server1"), new KeyValue("uri",
-          "hdfs://namenode:8020/path/to/uri1"));
-  private static final Permission ROLE_SERVER_SERVER1_URI_URI2 =
-      create(new KeyValue("server", "server1"), new KeyValue("uri",
-          "hdfs://namenode:8020/path/to/uri2/"));
-  private static final Permission ROLE_SERVER_SERVER1_URI_ALL =
-      create(new KeyValue("server", "server1"), new KeyValue("uri", ALL));
-
-
-  private static final Permission ROLE_SERVER_SERVER1 =
-      create(new KeyValue("server", "server1"));
-
-
-  private static final Permission REQUEST_SERVER1_DB1 =
-      create(new KeyValue("server", "server1"), new KeyValue("db", "db1"));
-  private static final Permission REQUEST_SERVER2_DB1 =
-      create(new KeyValue("server", "server2"), new KeyValue("db", "db1"));
-  private static final Permission REQUEST_SERVER1_DB2 =
-      create(new KeyValue("server", "server1"), new KeyValue("db", "db2"));
-  private static final Permission REQUEST_SERVER2_DB2 =
-      create(new KeyValue("server", "server2"), new KeyValue("db", "db2"));
-
-  private static final Permission REQUEST_SERVER1_URI1 =
-      create(new KeyValue("server", "server1"), new KeyValue("uri",
-          "hdfs://namenode:8020/path/to/uri1/some/file"));
-  private static final Permission REQUEST_SERVER1_URI2 =
-      create(new KeyValue("server", "server1"), new KeyValue("uri",
-          "hdfs://namenode:8020/path/to/uri2/some/other/file"));
-
-  private static final Permission REQUEST_SERVER1_OTHER =
-      create(new KeyValue("server", "server2"), new KeyValue("other", "thing"));
-
-  private static final Permission REQUEST_SERVER1 =
-      create(new KeyValue("server", "server2"));
-
-  @Test
-  public void testOther() throws Exception {
-    assertFalse(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_OTHER));
-    assertFalse(REQUEST_SERVER1_OTHER.implies(ROLE_SERVER_ALL_DB_ALL));
-  }
-  @Test
-  public void testRoleShorterThanRequest() throws Exception {
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_DB1));
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_DB2));
-    assertFalse(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER2_DB1));
-    assertFalse(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER2_DB2));
-
-    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1));
-    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1));
-  }
-  @Test
-  public void testRolesAndRequests() throws Exception {
-    // ROLE_SERVER_SERVER1_DB_ALL
-    assertTrue(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER1_DB1));
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER2_DB1));
-    assertTrue(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER1_DB2));
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER2_DB2));
-
-    // test inverse
-    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER1_DB_ALL));
-    assertFalse(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER1_DB_ALL));
-    assertTrue(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER1_DB_ALL));
-    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER1_DB_ALL));
-
-    // ROLE_SERVER_SERVER1_DB_DB1
-    assertTrue(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER1_DB1));
-    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER2_DB1));
-    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER1_DB2));
-    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER2_DB2));
-
-    // test inverse
-    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER1_DB_DB1));
-    assertFalse(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER1_DB_DB1));
-    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER1_DB_DB1));
-    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER1_DB_DB1));
-
-    // ROLE_SERVER_SERVER2_DB_ALL
-    assertFalse(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER1_DB1));
-    assertTrue(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER2_DB1));
-    assertFalse(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER1_DB2));
-    assertTrue(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER2_DB2));
-
-    // test inverse
-    assertFalse(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER2_DB_ALL));
-    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER2_DB_ALL));
-    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER2_DB_ALL));
-    assertTrue(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER2_DB_ALL));
-
-    // ROLE_SERVER_SERVER2_DB_DB1
-    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER1_DB1));
-    assertTrue(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER2_DB1));
-    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER1_DB2));
-    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER2_DB2));
-
-    assertFalse(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER2_DB_DB1));
-    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER2_DB_DB1));
-    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER2_DB_DB1));
-    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER2_DB_DB1));
-
-    // ROLE_SERVER_ALL_DB_ALL
-    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_DB1));
-    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER2_DB1));
-    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_DB2));
-    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER2_DB2));
-
-    // test inverse
-    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_ALL_DB_ALL));
-    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_ALL_DB_ALL));
-    assertTrue(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_ALL_DB_ALL));
-    assertTrue(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_ALL_DB_ALL));
-
-    // ROLE_SERVER_ALL_DB_DB1
-    assertTrue(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_DB1));
-    assertTrue(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER2_DB1));
-    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_DB2));
-    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER2_DB2));
-
-    // test inverse
-    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_ALL_DB_DB1));
-    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_ALL_DB_DB1));
-    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_ALL_DB_DB1));
-    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_ALL_DB_DB1));
-
-    // uri
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI1));
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
-    assertTrue(ROLE_SERVER_SERVER1_URI_ALL.implies(REQUEST_SERVER1_URI1));
-    assertTrue(ROLE_SERVER_SERVER1_URI_ALL.implies(REQUEST_SERVER1_URI2));
-    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
-    assertTrue(ROLE_SERVER_SERVER1_URI_URI1.implies(REQUEST_SERVER1_URI1));
-    assertFalse(ROLE_SERVER_SERVER1_URI_URI1.implies(REQUEST_SERVER1_URI2));
-    assertTrue(ROLE_SERVER_SERVER1_URI_URI2.implies(REQUEST_SERVER1_URI2));
-    assertFalse(ROLE_SERVER_SERVER1_URI_URI2.implies(REQUEST_SERVER1_URI1));
-    assertFalse(REQUEST_SERVER2_DB2.implies(REQUEST_SERVER1_URI1));
-    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_URI1));
-    // test inverse
-    assertTrue(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_ALL));
-    assertTrue(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_ALL));
-    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1));
-    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_URI1));
-    assertFalse(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_URI1));
-    assertFalse(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_URI2));
-    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_URI2));
-  };
-  @Test
-  public void testUnexpected() throws Exception {
-    Permission p = new Permission() {
-      @Override
-      public boolean implies(Permission p) {
-        return false;
-      }
-    };
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(null));
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(p));
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.equals(null));
-    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.equals(p));
-
-    assertEquals(ROLE_SERVER_SERVER1_DB_ALL.hashCode(),
-        create(ROLE_SERVER_SERVER1_DB_ALL.toString()).hashCode());
-  }
-  @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("", "db1")));
-  }
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyValue() throws Exception {
-    System.out.println(create(KV_JOINER.join("db", "")));
-  }
-  @Test(expected=IllegalArgumentException.class)
-  public void testEmptyPart() throws Exception {
-    System.out.println(create(AUTHORIZABLE_JOINER.
-        join(KV_JOINER.join("server", "server1"), "")));
-  }
-  @Test(expected=IllegalArgumentException.class)
-  public void testOnlySeperators() throws Exception {
-    System.out.println(create(AUTHORIZABLE_JOINER.
-        join(KV_SEPARATOR, KV_SEPARATOR, KV_SEPARATOR)));
-  }
-  @Test
-  public void testImpliesURIPositive() throws Exception {
-    assertTrue(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "hdfs://namenode:8020/path/to/some/dir"));
-    assertTrue(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "hdfs://namenode:8020/path"));
-    assertTrue(DBWildcardPermission.impliesURI("file:///path",
-        "file:///path/to/some/dir"));
-    assertTrue(DBWildcardPermission.impliesURI("file:///path",
-        "file:///path"));
-  }
-  @Test
-  public void testImpliesURINegative() throws Exception {
-    // relative path
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "hdfs://namenode:8020/path/to/../../other"));
-    assertFalse(DBWildcardPermission.impliesURI("file:///path",
-        "file:///path/to/../../other"));
-    // bad policy
-    assertFalse(DBWildcardPermission.impliesURI("blah",
-        "hdfs://namenode:8020/path/to/some/dir"));
-    // bad request
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "blah"));
-    // scheme
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "file:///path/to/some/dir"));
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "file://namenode:8020/path/to/some/dir"));
-    // hostname
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode1:8020/path",
-        "hdfs://namenode2:8020/path/to/some/dir"));
-    // port
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "hdfs://namenode:8021/path/to/some/dir"));
-    // mangled path
-    assertFalse(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path",
-        "hdfs://namenode:8020/pathFooBar"));
-    // ends in /
-    assertTrue(DBWildcardPermission.impliesURI("hdfs://namenode:8020/path/",
-        "hdfs://namenode:8020/path/FooBar"));
-  }
-  static DBWildcardPermission create(KeyValue... keyValues) {
-    return create(AUTHORIZABLE_JOINER.join(keyValues));
-
-  }
-  static DBWildcardPermission create(String s) {
-    return new DBWildcardPermission(s);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPrivilege.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPrivilege.java
new file mode 100644
index 0000000..f4862e0
--- /dev/null
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDBWildcardPrivilege.java
@@ -0,0 +1,286 @@
+/*
+ * 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.db;
+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.db.AccessConstants;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.provider.file.KeyValue;
+import org.junit.Test;
+
+public class TestDBWildcardPrivilege {
+
+  private static final String ALL = AccessConstants.ALL;
+
+  private static final Privilege ROLE_SERVER_SERVER1_DB_ALL =
+      create(new KeyValue("server", "server1"), new KeyValue("db", ALL));
+  private static final Privilege ROLE_SERVER_SERVER1_DB_DB1 =
+      create(new KeyValue("server", "server1"), new KeyValue("db", "db1"));
+  private static final Privilege ROLE_SERVER_SERVER2_DB_ALL =
+      create(new KeyValue("server", "server2"), new KeyValue("db", ALL));
+  private static final Privilege ROLE_SERVER_SERVER2_DB_DB1 =
+      create(new KeyValue("server", "server2"), new KeyValue("db", "db1"));
+  private static final Privilege ROLE_SERVER_ALL_DB_ALL =
+      create(new KeyValue("server", ALL), new KeyValue("db", ALL));
+  private static final Privilege ROLE_SERVER_ALL_DB_DB1 =
+      create(new KeyValue("server", ALL), new KeyValue("db", "db1"));
+
+  private static final Privilege ROLE_SERVER_SERVER1_URI_URI1 =
+      create(new KeyValue("server", "server1"), new KeyValue("uri",
+          "hdfs://namenode:8020/path/to/uri1"));
+  private static final Privilege ROLE_SERVER_SERVER1_URI_URI2 =
+      create(new KeyValue("server", "server1"), new KeyValue("uri",
+          "hdfs://namenode:8020/path/to/uri2/"));
+  private static final Privilege ROLE_SERVER_SERVER1_URI_ALL =
+      create(new KeyValue("server", "server1"), new KeyValue("uri", ALL));
+
+
+  private static final Privilege ROLE_SERVER_SERVER1 =
+      create(new KeyValue("server", "server1"));
+
+
+  private static final Privilege REQUEST_SERVER1_DB1 =
+      create(new KeyValue("server", "server1"), new KeyValue("db", "db1"));
+  private static final Privilege REQUEST_SERVER2_DB1 =
+      create(new KeyValue("server", "server2"), new KeyValue("db", "db1"));
+  private static final Privilege REQUEST_SERVER1_DB2 =
+      create(new KeyValue("server", "server1"), new KeyValue("db", "db2"));
+  private static final Privilege REQUEST_SERVER2_DB2 =
+      create(new KeyValue("server", "server2"), new KeyValue("db", "db2"));
+
+  private static final Privilege REQUEST_SERVER1_URI1 =
+      create(new KeyValue("server", "server1"), new KeyValue("uri",
+          "hdfs://namenode:8020/path/to/uri1/some/file"));
+  private static final Privilege REQUEST_SERVER1_URI2 =
+      create(new KeyValue("server", "server1"), new KeyValue("uri",
+          "hdfs://namenode:8020/path/to/uri2/some/other/file"));
+
+  private static final Privilege REQUEST_SERVER1_OTHER =
+      create(new KeyValue("server", "server2"), new KeyValue("other", "thing"));
+
+  private static final Privilege REQUEST_SERVER1 =
+      create(new KeyValue("server", "server2"));
+
+  @Test
+  public void testOther() throws Exception {
+    assertFalse(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_OTHER));
+    assertFalse(REQUEST_SERVER1_OTHER.implies(ROLE_SERVER_ALL_DB_ALL));
+  }
+  @Test
+  public void testRoleShorterThanRequest() throws Exception {
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_DB1));
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_DB2));
+    assertFalse(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER2_DB1));
+    assertFalse(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER2_DB2));
+
+    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1));
+    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1));
+  }
+  @Test
+  public void testRolesAndRequests() throws Exception {
+    // ROLE_SERVER_SERVER1_DB_ALL
+    assertTrue(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER1_DB1));
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER2_DB1));
+    assertTrue(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER1_DB2));
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(REQUEST_SERVER2_DB2));
+
+    // test inverse
+    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER1_DB_ALL));
+    assertFalse(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER1_DB_ALL));
+    assertTrue(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER1_DB_ALL));
+    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER1_DB_ALL));
+
+    // ROLE_SERVER_SERVER1_DB_DB1
+    assertTrue(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER1_DB1));
+    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER2_DB1));
+    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER1_DB2));
+    assertFalse(ROLE_SERVER_SERVER1_DB_DB1.implies(REQUEST_SERVER2_DB2));
+
+    // test inverse
+    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER1_DB_DB1));
+    assertFalse(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER1_DB_DB1));
+    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER1_DB_DB1));
+    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER1_DB_DB1));
+
+    // ROLE_SERVER_SERVER2_DB_ALL
+    assertFalse(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER1_DB1));
+    assertTrue(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER2_DB1));
+    assertFalse(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER1_DB2));
+    assertTrue(ROLE_SERVER_SERVER2_DB_ALL.implies(REQUEST_SERVER2_DB2));
+
+    // test inverse
+    assertFalse(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER2_DB_ALL));
+    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER2_DB_ALL));
+    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER2_DB_ALL));
+    assertTrue(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER2_DB_ALL));
+
+    // ROLE_SERVER_SERVER2_DB_DB1
+    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER1_DB1));
+    assertTrue(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER2_DB1));
+    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER1_DB2));
+    assertFalse(ROLE_SERVER_SERVER2_DB_DB1.implies(REQUEST_SERVER2_DB2));
+
+    assertFalse(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_SERVER2_DB_DB1));
+    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_SERVER2_DB_DB1));
+    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_SERVER2_DB_DB1));
+    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_SERVER2_DB_DB1));
+
+    // ROLE_SERVER_ALL_DB_ALL
+    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_DB1));
+    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER2_DB1));
+    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER1_DB2));
+    assertTrue(ROLE_SERVER_ALL_DB_ALL.implies(REQUEST_SERVER2_DB2));
+
+    // test inverse
+    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_ALL_DB_ALL));
+    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_ALL_DB_ALL));
+    assertTrue(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_ALL_DB_ALL));
+    assertTrue(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_ALL_DB_ALL));
+
+    // ROLE_SERVER_ALL_DB_DB1
+    assertTrue(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_DB1));
+    assertTrue(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER2_DB1));
+    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_DB2));
+    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER2_DB2));
+
+    // test inverse
+    assertTrue(REQUEST_SERVER1_DB1.implies(ROLE_SERVER_ALL_DB_DB1));
+    assertTrue(REQUEST_SERVER2_DB1.implies(ROLE_SERVER_ALL_DB_DB1));
+    assertFalse(REQUEST_SERVER1_DB2.implies(ROLE_SERVER_ALL_DB_DB1));
+    assertFalse(REQUEST_SERVER2_DB2.implies(ROLE_SERVER_ALL_DB_DB1));
+
+    // uri
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI1));
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
+    assertTrue(ROLE_SERVER_SERVER1_URI_ALL.implies(REQUEST_SERVER1_URI1));
+    assertTrue(ROLE_SERVER_SERVER1_URI_ALL.implies(REQUEST_SERVER1_URI2));
+    assertTrue(ROLE_SERVER_SERVER1.implies(REQUEST_SERVER1_URI2));
+    assertTrue(ROLE_SERVER_SERVER1_URI_URI1.implies(REQUEST_SERVER1_URI1));
+    assertFalse(ROLE_SERVER_SERVER1_URI_URI1.implies(REQUEST_SERVER1_URI2));
+    assertTrue(ROLE_SERVER_SERVER1_URI_URI2.implies(REQUEST_SERVER1_URI2));
+    assertFalse(ROLE_SERVER_SERVER1_URI_URI2.implies(REQUEST_SERVER1_URI1));
+    assertFalse(REQUEST_SERVER2_DB2.implies(REQUEST_SERVER1_URI1));
+    assertFalse(ROLE_SERVER_ALL_DB_DB1.implies(REQUEST_SERVER1_URI1));
+    // test inverse
+    assertTrue(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_ALL));
+    assertTrue(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_ALL));
+    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1));
+    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_URI1));
+    assertFalse(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_URI1));
+    assertFalse(REQUEST_SERVER1_URI2.implies(ROLE_SERVER_SERVER1_URI_URI2));
+    assertFalse(REQUEST_SERVER1_URI1.implies(ROLE_SERVER_SERVER1_URI_URI2));
+  };
+  @Test
+  public void testUnexpected() throws Exception {
+    Privilege p = new Privilege() {
+      @Override
+      public boolean implies(Privilege p) {
+        return false;
+      }
+    };
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(null));
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.implies(p));
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.equals(null));
+    assertFalse(ROLE_SERVER_SERVER1_DB_ALL.equals(p));
+
+    assertEquals(ROLE_SERVER_SERVER1_DB_ALL.hashCode(),
+        create(ROLE_SERVER_SERVER1_DB_ALL.toString()).hashCode());
+  }
+  @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("", "db1")));
+  }
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyValue() throws Exception {
+    System.out.println(create(KV_JOINER.join("db", "")));
+  }
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyPart() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_JOINER.join("server", "server1"), "")));
+  }
+  @Test(expected=IllegalArgumentException.class)
+  public void testOnlySeperators() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_SEPARATOR, KV_SEPARATOR, KV_SEPARATOR)));
+  }
+  @Test
+  public void testImpliesURIPositive() throws Exception {
+    assertTrue(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "hdfs://namenode:8020/path/to/some/dir"));
+    assertTrue(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "hdfs://namenode:8020/path"));
+    assertTrue(DBWildcardPrivilege.impliesURI("file:///path",
+        "file:///path/to/some/dir"));
+    assertTrue(DBWildcardPrivilege.impliesURI("file:///path",
+        "file:///path"));
+  }
+  @Test
+  public void testImpliesURINegative() throws Exception {
+    // relative path
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "hdfs://namenode:8020/path/to/../../other"));
+    assertFalse(DBWildcardPrivilege.impliesURI("file:///path",
+        "file:///path/to/../../other"));
+    // bad policy
+    assertFalse(DBWildcardPrivilege.impliesURI("blah",
+        "hdfs://namenode:8020/path/to/some/dir"));
+    // bad request
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "blah"));
+    // scheme
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "file:///path/to/some/dir"));
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "file://namenode:8020/path/to/some/dir"));
+    // hostname
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode1:8020/path",
+        "hdfs://namenode2:8020/path/to/some/dir"));
+    // port
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "hdfs://namenode:8021/path/to/some/dir"));
+    // mangled path
+    assertFalse(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path",
+        "hdfs://namenode:8020/pathFooBar"));
+    // ends in /
+    assertTrue(DBWildcardPrivilege.impliesURI("hdfs://namenode:8020/path/",
+        "hdfs://namenode:8020/path/FooBar"));
+  }
+  static DBWildcardPrivilege create(KeyValue... keyValues) {
+    return create(AUTHORIZABLE_JOINER.join(keyValues));
+
+  }
+  static DBWildcardPrivilege create(String s) {
+    return new DBWildcardPrivilege(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDatabaseRequiredInRole.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDatabaseRequiredInRole.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDatabaseRequiredInRole.java
index 948b7ac..f9b00b4 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDatabaseRequiredInRole.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestDatabaseRequiredInRole.java
@@ -20,6 +20,7 @@ package org.apache.sentry.policy.db;
 
 import junit.framework.Assert;
 
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.shiro.config.ConfigurationException;
 import org.junit.Test;
 
@@ -27,19 +28,19 @@ public class TestDatabaseRequiredInRole {
 
   @Test
   public void testURIInPerDbPolicyFile() throws Exception {
-    DatabaseRequiredInRole dbRequiredInRole = new DatabaseRequiredInRole();
+    DatabaseRequiredInPrivilege dbRequiredInRole = new DatabaseRequiredInPrivilege();
     System.setProperty("sentry.allow.uri.db.policyfile", "true");
-    dbRequiredInRole.validate("db1",
-      "server=server1->URI=file:///user/db/warehouse/tab1");
+    dbRequiredInRole.validate(new PrivilegeValidatorContext("db1",
+      "server=server1->URI=file:///user/db/warehouse/tab1"));
     System.setProperty("sentry.allow.uri.db.policyfile", "false");
   }
 
   @Test
   public void testURIWithDBInPerDbPolicyFile() throws Exception {
-    DatabaseRequiredInRole dbRequiredInRole = new DatabaseRequiredInRole();
+    DatabaseRequiredInPrivilege dbRequiredInRole = new DatabaseRequiredInPrivilege();
     try {
-      dbRequiredInRole.validate("db1",
-        "server=server1->db=db1->URI=file:///user/db/warehouse/tab1");
+      dbRequiredInRole.validate(new PrivilegeValidatorContext("db1",
+        "server=server1->db=db1->URI=file:///user/db/warehouse/tab1"));
       Assert.fail("Expected ConfigurationException");
     } catch (ConfigurationException e) {
       ;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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..01f428b 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,10 @@ 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.policy.common.PolicyEngine;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.After;
@@ -36,7 +32,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 +71,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"));
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
   @Test
@@ -93,33 +85,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"));
     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"));
     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"));
     Assert.assertEquals(permissions.toString(), "[server=server1]");
   }
+
   @Test
   public void testDatabaseRequiredInRole() throws Exception {
     append("[databases]", globalPolicyFile);
@@ -128,40 +112,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"));
     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"));
     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"));
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -171,12 +145,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"));
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -211,30 +181,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"));
     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"));
     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/92212c3d/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..e34b3ee 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
@@ -90,7 +90,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));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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..57f7575 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,8 @@ 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.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,7 +73,7 @@ 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(),
@@ -89,7 +89,7 @@ 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);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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..f39eacd 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,14 @@ 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.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 +69,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 +102,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);
     Assert.assertEquals("No DB permissions found", 1, dbPerms.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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/92212c3d/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/92212c3d/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/92212c3d/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/92212c3d/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/92212c3d/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);
-    }
-  }
-}


[2/4] SENTRY-122: Refactor provider/policy API to allow for DB-policy provider (Brock Noland via Gregory Chanan)

Posted by gc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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/92212c3d/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..3519d05 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,19 @@
  */
 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.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 +38,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) {
     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);
     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/92212c3d/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..495ec0d 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,12 @@ 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.policy.common.PolicyEngine;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -34,7 +31,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 +46,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 +89,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")))
         .toString());
   }
 
@@ -105,7 +100,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")))
         .toString());
   }
 
@@ -115,7 +110,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")))
         .toString());
   }
 
@@ -123,11 +118,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")))
         .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/92212c3d/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/92212c3d/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/92212c3d/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..cd271a5 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
@@ -29,9 +29,8 @@ import org.apache.commons.io.FileUtils;
 import org.apache.sentry.core.common.Action;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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..aa849ef 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,8 @@ 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.Authorizable;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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/92212c3d/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/92212c3d/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..e95aca3 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,12 @@ 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.policy.common.PolicyEngine;
-import org.apache.sentry.provider.file.PolicyFile;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,7 +32,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 +68,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")));
   }
 
   @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"));
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -102,10 +94,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"));
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/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/92212c3d/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/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
index 4887678..8dc2f52 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
@@ -19,11 +19,17 @@ package org.apache.sentry.provider.common;
 import java.util.List;
 import java.util.Set;
 
+import javax.annotation.concurrent.ThreadSafe;
+
 import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.core.common.Subject;
 
+/**
+ * Implementations of AuthorizationProvider must be threadsafe.
+ */
+@ThreadSafe
 public interface AuthorizationProvider {
 
   /***
@@ -59,7 +65,7 @@ public interface AuthorizationProvider {
    * @return
    * @throws SentryConfigurationException
    */
-  public Set<String> listPermissionsForSubject(Subject subject) throws SentryConfigurationException;
+  public Set<String> listPrivilegesForSubject(Subject subject) throws SentryConfigurationException;
 
   /**
    * Returns the list privileges for the given group
@@ -67,11 +73,11 @@ public interface AuthorizationProvider {
    * @return
    * @throws SentryConfigurationException
    */
-  public Set<String> listPermissionsForGroup(String groupName) throws SentryConfigurationException;
+  public Set<String> listPrivilegesForGroup(String groupName) throws SentryConfigurationException;
 
   /***
    * Returns the list of missing privileges of the last access request
    * @return
    */
-  public List<String> getLastFailedPermissions();
+  public List<String> getLastFailedPrivileges();
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/GroupMappingService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/GroupMappingService.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/GroupMappingService.java
index 226cc88..22371d1 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/GroupMappingService.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/GroupMappingService.java
@@ -16,15 +16,20 @@
  */
 package org.apache.sentry.provider.common;
 
-import java.util.List;
+import java.util.Set;
+
+import javax.annotation.concurrent.ThreadSafe;
 
 /**
  * Interface so the Groups class is easier to unit test with.
+ * Implementations of this class are expected to be thread safe
+ * after construction.
  */
+@ThreadSafe
 public interface GroupMappingService {
 
   /**
    * @return non-null list of groups for user
    */
-  public List<String> getGroups(String user);
+  public Set<String> getGroups(String user);
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
index 8f18926..309f270 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
@@ -46,19 +46,19 @@ public class NoAuthorizationProvider implements AuthorizationProvider {
   }
 
   @Override
-  public Set<String> listPermissionsForSubject(Subject subject)
+  public Set<String> listPrivilegesForSubject(Subject subject)
       throws SentryConfigurationException {
     return new HashSet<String>();
   }
 
   @Override
-  public Set<String> listPermissionsForGroup(String groupName)
+  public Set<String> listPrivilegesForGroup(String groupName)
       throws SentryConfigurationException {
     return new HashSet<String>();
   }
 
   @Override
-  public List<String> getLastFailedPermissions() {
+  public List<String> getLastFailedPrivileges() {
     return new ArrayList<String>();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoGroupMappingService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoGroupMappingService.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoGroupMappingService.java
index e1bc6d2..e44cbc4 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoGroupMappingService.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoGroupMappingService.java
@@ -16,8 +16,8 @@
  */
 package org.apache.sentry.provider.common;
 
-import java.util.LinkedList;
-import java.util.List;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * GroupMappingService that always returns an empty list of groups
@@ -27,7 +27,7 @@ public class NoGroupMappingService implements GroupMappingService {
   /**
    * @return empty list of groups for every user
    */
-  public List<String> getGroups(String user) {
-    return new LinkedList<String>();
+  public Set<String> getGroups(String user) {
+    return new HashSet<String>();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
index 327a3a5..3582d36 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
@@ -16,31 +16,43 @@
  */
 package org.apache.sentry.provider.common;
 
-import javax.annotation.Nullable;
+import java.util.Set;
 
-import java.util.List;
+import javax.annotation.concurrent.ThreadSafe;
 
 import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.policy.common.RoleValidator;
 
 import com.google.common.collect.ImmutableSet;
 
 /**
- * Interface for getting roles from a specific provider backend.
+ * Interface for getting roles from a specific provider backend. Implementations
+ * are expected to be thread safe after initialize() has
+ * been called.
  */
+@ThreadSafe
 public interface ProviderBackend {
+
   /**
-   * Process roles from the backend.  Checks the validity of each role
-   * by running it through each validator passed via validators.
+   * Set the privilege validators to be used on the backend. This is required
+   * because the Backend must be created before the policy engine and only the
+   * policy engine knows the validators. Ideally we could change but since
+   * both the policy engine and backend are exposed via configuration properties
+   * that would be backwards incompatible.
+   * @param validators
    */
-  public void process(List<? extends RoleValidator> validators);
+  public void initialize(ProviderBackendContext context);
 
   /**
-   * Get the roles from the backend.  Requires that process(...) is invoked at
-   * least once prior.
+   * Get the privileges from the backend.
    */
-  public Roles getRoles();
+  public ImmutableSet<String> getPrivileges(Set<String> groups);
 
-  public void validatePolicy(List<? extends RoleValidator> validators, boolean strictValidation)
-      throws SentryConfigurationException;
+  /**
+   * If strictValidation is true then an error is thrown for warnings
+   * as well as errors.
+   *
+   * @param strictValidation
+   * @throws SentryConfigurationException
+   */
+  public void validatePolicy(boolean strictValidation) throws SentryConfigurationException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackendContext.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackendContext.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackendContext.java
new file mode 100644
index 0000000..f45d23d
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackendContext.java
@@ -0,0 +1,50 @@
+/*
+ * 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.provider.common;
+
+import org.apache.sentry.policy.common.PrivilegeValidator;
+
+import com.google.common.collect.ImmutableList;
+
+public class ProviderBackendContext {
+
+  private boolean allowPerDatabase;
+  private ImmutableList<PrivilegeValidator> validators;
+
+  public ProviderBackendContext() {
+    validators = ImmutableList.of();
+  }
+
+  public boolean isAllowPerDatabase() {
+    return allowPerDatabase;
+  }
+
+  public void setAllowPerDatabase(boolean allowPerDatabase) {
+    this.allowPerDatabase = allowPerDatabase;
+  }
+
+  public ImmutableList<PrivilegeValidator> getValidators() {
+    return validators;
+  }
+
+  public void setValidators(ImmutableList<PrivilegeValidator> validators) {
+    if (validators == null) {
+      validators = ImmutableList.of();
+    }
+    this.validators = validators;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/Roles.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/Roles.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/Roles.java
deleted file mode 100644
index a8f36a3..0000000
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/Roles.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.provider.common;
-
-import javax.annotation.Nullable;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSetMultimap;
-
-/**
- * Class providing storage of roles
- */
-public class Roles {
-  private final ImmutableSetMultimap<String, String> globalRoles;
-  private final ImmutableMap<String, ImmutableSetMultimap<String, String>> perDatabaseRoles;
-
-  public Roles() {
-    this(ImmutableSetMultimap.<String, String>of(),
-        ImmutableMap.<String, ImmutableSetMultimap<String, String>>of());
-  }
-
-  public Roles(ImmutableSetMultimap<String, String> globalRoles,
-      ImmutableMap<String, ImmutableSetMultimap<String, String>> perDatabaseRoles) {
-    this.globalRoles = globalRoles;
-    this.perDatabaseRoles = perDatabaseRoles;
-  }
-
-  public ImmutableSetMultimap<String, String> getGlobalRoles() {
-    return globalRoles;
-  }
-
-  public ImmutableMap<String, ImmutableSetMultimap<String, String>> getPerDatabaseRoles() {
-    return perDatabaseRoles;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/MockGroupMappingServiceProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/MockGroupMappingServiceProvider.java b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/MockGroupMappingServiceProvider.java
index 806b42e..1e885f4 100644
--- a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/MockGroupMappingServiceProvider.java
+++ b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/MockGroupMappingServiceProvider.java
@@ -17,14 +17,13 @@
 package org.apache.sentry.provider.common;
 
 import java.util.Collection;
-import java.util.List;
+import java.util.Set;
 
-import org.apache.sentry.provider.common.GroupMappingService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
 
 public class MockGroupMappingServiceProvider implements GroupMappingService {
   private static final Logger LOGGER = LoggerFactory
@@ -36,10 +35,10 @@ public class MockGroupMappingServiceProvider implements GroupMappingService {
   }
 
   @Override
-  public List<String> getGroups(String user) {
+  public Set<String> getGroups(String user) {
     Collection<String> groups = userToGroupMap.get(user);
     LOGGER.info("Mapping " + user + " to " + groups);
-    return Lists.newArrayList(groups);
+    return Sets.newHashSet(groups);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestNoAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestNoAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestNoAuthorizationProvider.java
index 3f48f49..dbcf05b 100644
--- a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestNoAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestNoAuthorizationProvider.java
@@ -16,11 +16,11 @@
  */
 package org.apache.sentry.provider.common;
 
-import org.junit.Test;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
+import org.junit.Test;
+
 /**
  * Tests around the NoAuthorizationProvider
  */

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
index 549a9db..4c3e6ea 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
@@ -19,7 +19,6 @@
 package org.apache.sentry.provider.db.service.model;
 
 import java.util.HashSet;
-
 import java.util.Set;
 
 import javax.jdo.annotations.PersistenceCapable;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryConfigurationException.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryConfigurationException.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryConfigurationException.java
index 2aeea42..0e5ad32 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryConfigurationException.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryConfigurationException.java
@@ -21,6 +21,9 @@ import org.apache.sentry.SentryUserException;
 
 public class SentryConfigurationException extends SentryUserException {
   private static final long serialVersionUID = 1298632655835L;
+  public SentryConfigurationException(String msg) {
+    super(msg);
+  }
   public SentryConfigurationException(String msg, Throwable t) {
     super(msg, t);
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
index 55dbcdd..a451f58 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
@@ -71,7 +71,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
 
   @VisibleForTesting
   static List<NotificationHandler> createHandlers(Configuration conf)
-      throws Exception {
+      throws SentryConfigurationException {
     List<NotificationHandler> handlers = Lists.newArrayList();
     Iterable<String> notificationHandlers = Splitter.onPattern("[\\s,]").trimResults()
         .omitEmptyStrings().split(conf.get(PolicyStoreServerConfig.NOTIFICATION_HANDLERS, ""));
@@ -80,7 +80,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       try {
         clazz = Class.forName(notificationHandler);
         if (!NotificationHandler.class.isAssignableFrom(clazz)) {
-          throw new IllegalArgumentException("Class " + notificationHandler + " is not a " +
+          throw new SentryConfigurationException("Class " + notificationHandler + " is not a " +
               NotificationHandler.class.getName());
         }
       } catch (ClassNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/KerberosConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/KerberosConfiguration.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/KerberosConfiguration.java
index 8323959..3022f67 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/KerberosConfiguration.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/KerberosConfiguration.java
@@ -18,8 +18,9 @@
 package org.apache.sentry.service.thrift;
 
 import java.io.File;
-import java.util.Map;
 import java.util.HashMap;
+import java.util.Map;
+
 import javax.security.auth.login.AppConfigurationEntry;
 
 public class KerberosConfiguration extends javax.security.auth.login.Configuration {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyStoreProcessor.java
index ab4fff6..46f8fb8 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryPolicyStoreProcessor.java
@@ -32,18 +32,18 @@ public class TestSentryPolicyStoreProcessor {
   public void setup() {
     conf = new Configuration(false);
   }
-  @Test(expected=IllegalArgumentException.class)
+  @Test(expected=SentryConfigurationException.class)
   public void testConfigNotNotificationHandler() throws Exception {
     conf.set(PolicyStoreServerConfig.NOTIFICATION_HANDLERS, Object.class.getName());
     SentryPolicyStoreProcessor.createHandlers(conf);
   }
-  @Test(expected=IllegalStateException.class)
+  @Test(expected=SentryConfigurationException.class)
   public void testConfigCannotCreateNotificationHandler() throws Exception {
     conf.set(PolicyStoreServerConfig.NOTIFICATION_HANDLERS,
         ExceptionInConstructorNotificationHandler.class.getName());
     SentryPolicyStoreProcessor.createHandlers(conf);
   }
-  @Test(expected=IllegalArgumentException.class)
+  @Test(expected=SentryConfigurationException.class)
   public void testConfigNotAClassNotificationHandler() throws Exception {
     conf.set(PolicyStoreServerConfig.NOTIFICATION_HANDLERS, "junk");
     SentryPolicyStoreProcessor.createHandlers(conf);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
index 692dbfa..8e1be52 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
@@ -18,16 +18,17 @@
 
 package org.apache.sentry.provider.db.service.thrift;
 import java.util.HashSet;
-import com.google.common.base.Preconditions;
-
 import java.util.Set;
 
 import org.apache.sentry.service.thrift.SentryServiceIntegrationBase;
 import org.apache.sentry.service.thrift.ServiceConstants.ThriftConstants;
+import org.apache.sentry.service.thrift.Status;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
   private static final Logger LOGGER = LoggerFactory.getLogger(TestSentryServiceIntegration.class);
 
@@ -38,7 +39,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
     dropReq.setRoleName("admin_r");
     dropReq.setUserName("user_1");
     TDropSentryRoleResponse dropResp = client.dropRole(dropReq);
-    assertOK(dropResp.getStatus());
+    assertStatus(Status.NO_SUCH_OBJECT, dropResp.getStatus());
     LOGGER.info("Successfully dropped role: admin_r");
 
     TCreateSentryRoleRequest createReq = new TCreateSentryRoleRequest();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
index c1bb887..db76aa8 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
@@ -155,9 +155,13 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
 
   }
   protected static void assertOK(TSentryResponseStatus resp) {
-    if (resp.getValue() !=  Status.OK.getCode()) {
-      String message = "Response: " + Status.fromCode(resp.getValue()) + ", Code: "
-          + resp.getValue() + ", Message: " + resp.getMessage();
+    assertStatus(Status.OK, resp);
+  }
+
+  protected static void assertStatus(Status status, TSentryResponseStatus resp) {
+    if (resp.getValue() !=  status.getCode()) {
+      String message = "Expected: " + status + ", Response: " + Status.fromCode(resp.getValue())
+          + ", Code: " + resp.getValue() + ", Message: " + resp.getMessage();
       String stackTrace = Strings.nullToEmpty(resp.getStack()).trim();
       if (!stackTrace.isEmpty()) {
         message += ", StackTrace: " + stackTrace;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupMappingService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupMappingService.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupMappingService.java
index 4db465d..f2bb39c 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupMappingService.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupMappingService.java
@@ -18,7 +18,8 @@ package org.apache.sentry.provider.file;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.hadoop.security.Groups;
 import org.apache.sentry.provider.common.GroupMappingService;
@@ -36,12 +37,12 @@ public class HadoopGroupMappingService implements GroupMappingService {
   }
 
   @Override
-  public List<String> getGroups(String user) {
+  public Set<String> getGroups(String user) {
     try {
-      return groups.getGroups(user);
+      return new HashSet<String>(groups.getGroups(user));
     } catch (IOException e) {
       LOGGER.warn("Unable to obtain groups for " + user, e);
     }
-    return Collections.emptyList();
+    return Collections.emptySet();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
index ff3adf1..b2e4196 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
@@ -21,11 +21,8 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Groups;
-import org.apache.sentry.policy.common.RoleValidator;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.GroupMappingService;
-import org.apache.sentry.provider.file.HadoopGroupMappingService;
-import org.apache.sentry.provider.file.ResourceAuthorizationProvider;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupMappingService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupMappingService.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupMappingService.java
index c399117..a4d9cba 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupMappingService.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupMappingService.java
@@ -20,9 +20,9 @@ package org.apache.sentry.provider.file;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 /**
  * Mapping users to groups
@@ -62,8 +62,8 @@ public class LocalGroupMappingService implements GroupMappingService {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(LocalGroupMappingService.class);
 
-  private final Map <String, List<String>> groupMap =
-      new HashMap <String, List<String>> ();
+  private final Map <String, Set<String>> groupMap =
+      new HashMap <String, Set<String>> ();
 
   public LocalGroupMappingService(Path resourcePath) throws IOException {
     this(new Configuration(), resourcePath);
@@ -75,11 +75,11 @@ public class LocalGroupMappingService implements GroupMappingService {
   }
 
   @Override
-  public List<String> getGroups(String user) {
+  public Set<String> getGroups(String user) {
     if (groupMap.containsKey(user)) {
       return groupMap.get(user);
     } else {
-      return Collections.emptyList();
+      return Collections.emptySet();
     }
   }
 
@@ -102,7 +102,7 @@ public class LocalGroupMappingService implements GroupMappingService {
             " in the " + resourcePath);
         continue;
       }
-      List<String> groupList = Lists.newArrayList(
+      Set<String> groupList = Sets.newHashSet(
           PolicyFileConstants.ROLE_SPLITTER.trimResults().split(groupNames));
       LOGGER.debug("Got user mapping: " + userName + ", Groups: " + groupNames);
       groupMap.put(userName, groupList);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
index 374e989..e8293f6 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
@@ -17,12 +17,10 @@
 
 package org.apache.sentry.provider.file;
 
-import org.apache.sentry.policy.common.PolicyEngine;
-import org.apache.sentry.provider.file.LocalGroupMappingService;
-import org.apache.sentry.provider.file.ResourceAuthorizationProvider;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.sentry.policy.common.PolicyEngine;
 
 
 public class LocalGroupResourceAuthorizationProvider extends

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
index bed3202..0189f85 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFile.java
@@ -17,27 +17,29 @@
 
 package org.apache.sentry.provider.file;
 
+import static org.apache.sentry.provider.file.PolicyFileConstants.DATABASES;
+import static org.apache.sentry.provider.file.PolicyFileConstants.GROUPS;
+import static org.apache.sentry.provider.file.PolicyFileConstants.ROLES;
+import static org.apache.sentry.provider.file.PolicyFileConstants.USERS;
+
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
-import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.io.Files;
 
-import static org.apache.sentry.provider.file.PolicyFileConstants.*;
-
 /**
  * PolicyFile creator. Written specifically to be used with tests. Specifically
  * due to the fact that methods that would typically return true or false to

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFiles.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFiles.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFiles.java
index 295ce78..a908ec3 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFiles.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/PolicyFiles.java
@@ -63,7 +63,7 @@ public class PolicyFiles {
       throws IOException {
     InputStream input = new FileInputStream(inputFile.getPath());
     FSDataOutputStream out = fs.create(new Path(dest, inputFile.getName()));
-    long bytes = ByteStreams.copy(input, out);
+    ByteStreams.copy(input, out);
     input.close();
     out.hflush();
     out.close();


[4/4] git commit: SENTRY-122: Refactor provider/policy API to allow for DB-policy provider (Brock Noland via Gregory Chanan)

Posted by gc...@apache.org.
SENTRY-122: Refactor provider/policy API to allow for DB-policy provider (Brock Noland via Gregory Chanan)


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

Branch: refs/heads/db_policy_store
Commit: 92212c3d397cf2687ada33dfed3da0adb1cee87f
Parents: 21094fe
Author: Gregory Chanan <gr...@gmail.com>
Authored: Tue Mar 4 16:13:45 2014 -0800
Committer: Gregory Chanan <gr...@gmail.com>
Committed: Tue Mar 4 16:13:45 2014 -0800

----------------------------------------------------------------------
 bin/sentry                                      |   1 -
 pom.xml                                         |   4 +-
 .../binding/hive/HiveAuthzBindingHook.java      |  26 +-
 .../hive/HiveAuthzBindingPreExecHook.java       |   2 +-
 .../hive/SentryOnFailureHookContext.java        |  13 +-
 .../hive/SentryOnFailureHookContextImpl.java    |  13 +-
 .../binding/hive/authz/HiveAuthzBinding.java    |  35 +--
 .../hive/authz/HiveAuthzPrivilegesMap.java      |   1 -
 .../binding/hive/authz/SentryConfigTool.java    |  58 ++--
 .../binding/hive/MockUserToGroupMapping.java    |   8 +-
 .../sentry/binding/hive/TestHiveAuthzConf.java  |   1 +
 .../org/apache/sentry/binding/hive/TestURI.java |   1 -
 .../authz/SentrySolrAuthorizationException.java |   2 +
 .../binding/solr/authz/SolrAuthzBinding.java    |   8 +-
 .../sentry/binding/solr/conf/SolrAuthzConf.java |   4 -
 .../binding/solr/TestSolrAuthzBinding.java      |  39 +--
 .../common/SentryConfigurationException.java    |   1 +
 .../sentry/core/common/utils/PathUtils.java     |   4 +-
 .../sentry/core/common/utils/TestPathUtils.java |   1 +
 .../sentry/core/search/TestCollection.java      |   3 +-
 .../sentry/policy/common/PermissionFactory.java |  26 --
 .../sentry/policy/common/PolicyEngine.java      |  38 ++-
 .../apache/sentry/policy/common/Privilege.java  |  21 ++
 .../sentry/policy/common/PrivilegeFactory.java  |  24 ++
 .../sentry/policy/common/PrivilegeUtils.java    |  27 ++
 .../policy/common/PrivilegeValidator.java       |  24 ++
 .../common/PrivilegeValidatorContext.java       |  38 +++
 .../sentry/policy/common/RoleValidator.java     |  26 --
 .../policy/db/AbstractDBPrivilegeValidator.java |  50 ++++
 .../policy/db/AbstractDBRoleValidator.java      |  50 ----
 .../sentry/policy/db/DBWildcardPermission.java  | 181 ------------
 .../sentry/policy/db/DBWildcardPrivilege.java   | 179 ++++++++++++
 .../sentry/policy/db/DatabaseMustMatch.java     |  11 +-
 .../policy/db/DatabaseRequiredInPrivilege.java  |  71 +++++
 .../policy/db/DatabaseRequiredInRole.java       |  70 -----
 .../sentry/policy/db/ServerNameMustMatch.java   |  12 +-
 .../sentry/policy/db/ServersAllIsInvalid.java   |  12 +-
 .../sentry/policy/db/SimpleDBPolicyEngine.java  | 124 ++------
 .../db/AbstractTestSimplePolicyEngine.java      |  26 +-
 .../sentry/policy/db/DBPolicyFileBackend.java   |   3 +-
 .../policy/db/TestDBModelAuthorizables.java     |   2 -
 .../policy/db/TestDBWildcardPermission.java     | 286 -------------------
 .../policy/db/TestDBWildcardPrivilege.java      | 286 +++++++++++++++++++
 .../policy/db/TestDatabaseRequiredInRole.java   |  13 +-
 .../policy/db/TestPolicyParsingNegative.java    |  93 ++----
 ...sourceAuthorizationProviderGeneralCases.java |   3 +-
 ...sourceAuthorizationProviderSpecialCases.java |   6 +-
 .../policy/db/TestSimpleDBPolicyEngineDFS.java  |  23 +-
 .../db/TestSimpleDBPolicyEngineLocalFS.java     |   3 +-
 .../AbstractSearchPrivilegeValidator.java       |  51 ++++
 .../search/AbstractSearchRoleValidator.java     |  50 ----
 .../search/CollectionRequiredInPrivilege.java   |  43 +++
 .../policy/search/CollectionRequiredInRole.java |  44 ---
 .../policy/search/SearchWildcardPermission.java | 152 ----------
 .../policy/search/SearchWildcardPrivilege.java  | 146 ++++++++++
 .../policy/search/SimpleSearchPolicyEngine.java |  87 ++----
 .../search/AbstractTestSearchPolicyEngine.java  |  17 +-
 .../policy/search/SearchPolicyFileBackend.java  |   1 +
 .../search/TestCollectionRequiredInRole.java    |  19 +-
 ...SearchAuthorizationProviderGeneralCases.java |   3 +-
 ...SearchAuthorizationProviderSpecialCases.java |   2 +-
 .../search/TestSearchModelAuthorizables.java    |   2 -
 .../search/TestSearchPolicyEngineDFS.java       |  11 +-
 .../policy/search/TestSearchPolicyNegative.java |  33 +--
 .../search/TestSearchWildcardPermission.java    | 206 -------------
 .../search/TestSearchWildcardPrivilege.java     | 205 +++++++++++++
 .../provider/common/AuthorizationProvider.java  |  12 +-
 .../provider/common/GroupMappingService.java    |   9 +-
 .../common/NoAuthorizationProvider.java         |   6 +-
 .../provider/common/NoGroupMappingService.java  |   8 +-
 .../sentry/provider/common/ProviderBackend.java |  36 ++-
 .../provider/common/ProviderBackendContext.java |  50 ++++
 .../apache/sentry/provider/common/Roles.java    |  50 ----
 .../common/MockGroupMappingServiceProvider.java |   9 +-
 .../common/TestNoAuthorizationProvider.java     |   4 +-
 .../db/service/model/MSentryPrivilege.java      |   1 -
 .../thrift/SentryConfigurationException.java    |   3 +
 .../thrift/SentryPolicyStoreProcessor.java      |   4 +-
 .../service/thrift/KerberosConfiguration.java   |   3 +-
 .../thrift/TestSentryPolicyStoreProcessor.java  |   6 +-
 .../thrift/TestSentryServiceIntegration.java    |   7 +-
 .../thrift/SentryServiceIntegrationBase.java    |  10 +-
 .../file/HadoopGroupMappingService.java         |   9 +-
 ...adoopGroupResourceAuthorizationProvider.java |   3 -
 .../provider/file/LocalGroupMappingService.java |  14 +-
 ...LocalGroupResourceAuthorizationProvider.java |   4 +-
 .../apache/sentry/provider/file/PolicyFile.java |  10 +-
 .../sentry/provider/file/PolicyFiles.java       |   2 +-
 .../file/ResourceAuthorizationProvider.java     |  66 +++--
 .../file/SimpleFileProviderBackend.java         | 171 +++++------
 .../provider/file/TestGetGroupMapping.java      |  32 +--
 .../sentry/provider/file/TestKeyValue.java      |   1 -
 .../provider/file/TestLocalGroupMapping.java    |  22 +-
 .../AbstractTestWithStaticConfiguration.java    |  10 +-
 .../apache/sentry/tests/e2e/hive/Context.java   |   1 -
 .../sentry/tests/e2e/hive/TestConfigTool.java   |  27 +-
 .../tests/e2e/hive/TestPerDBConfiguration.java  |  12 +-
 .../e2e/hive/TestPrivilegesAtTableScope.java    |   4 +-
 .../hive/TestSentryOnFailureHookLoading.java    |  21 +-
 .../sentry/tests/e2e/hive/fs/AbstractDFS.java   |   5 +-
 .../sentry/tests/e2e/hive/fs/ClusterDFS.java    |   7 +-
 .../apache/sentry/tests/e2e/hive/fs/DFS.java    |   2 -
 .../sentry/tests/e2e/hive/fs/DFSFactory.java    |   4 +-
 .../sentry/tests/e2e/hive/fs/MiniDFS.java       |   6 +-
 .../e2e/hive/hiveserver/EmbeddedHiveServer.java |   6 +-
 .../e2e/hive/hiveserver/HiveServerFactory.java  |   2 +-
 .../e2e/hive/hiveserver/InternalHiveServer.java |   1 +
 .../hive/hiveserver/UnmanagedHiveServer.java    |  12 +-
 .../e2e/solr/AbstractSolrSentryTestBase.java    |   3 -
 .../sentry/tests/e2e/solr/HdfsTestUtil.java     |   1 -
 .../ModifiableUserAuthenticationFilter.java     |   1 -
 .../e2e/solr/TestCollAdminCoreOperations.java   |  12 +-
 .../tests/e2e/solr/TestQueryOperations.java     |  11 +-
 .../tests/e2e/solr/TestUpdateOperations.java    |  11 +-
 114 files changed, 1795 insertions(+), 1941 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/bin/sentry
----------------------------------------------------------------------
diff --git a/bin/sentry b/bin/sentry
index 9f2ce77..6c40f68 100755
--- a/bin/sentry
+++ b/bin/sentry
@@ -62,4 +62,3 @@ while [ $# -gt 0 ]; do    # Until you run out of parameters . . .
         ;;
   esac
 done
-

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 06649bd..ee10c07 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,8 +50,8 @@ limitations under the License.
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <maven.compile.source>1.6</maven.compile.source>
-    <maven.compile.target>1.6</maven.compile.target>
+    <maven.compile.source>1.7</maven.compile.source>
+    <maven.compile.target>1.7</maven.compile.target>
     <!-- versions are in alphabetical order -->
     <ant.contrib.version>1.0b3</ant.contrib.version>
     <maven.antrun.plugin.version>1.7</maven.antrun.plugin.version>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
index 506f185..eb54807 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
@@ -18,11 +18,9 @@ package org.apache.sentry.binding.hive;
 
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
-import java.io.File;
 import java.io.Serializable;
 import java.net.MalformedURLException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.CodeSource;
 import java.util.ArrayList;
@@ -44,8 +42,6 @@ import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
@@ -60,14 +56,13 @@ import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationSco
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivilegesMap;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
-import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.common.utils.PathUtils;
 import org.apache.sentry.core.model.db.AccessURI;
-import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Table;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -278,7 +273,7 @@ implements HiveDriverFilterHook {
     } catch (AuthorizationException e) {
       executeOnFailureHooks(context, stmtOperation, e);
       String permsRequired = "";
-      for (String perm : hiveAuthzBinding.getLastQueryPermissionErrors()) {
+      for (String perm : hiveAuthzBinding.getLastQueryPrivilegeErrors()) {
         permsRequired += perm + ";";
       }
       context.getConf().set(HiveAuthzConf.HIVE_SENTRY_AUTH_ERRORS, permsRequired);
@@ -707,22 +702,15 @@ implements HiveDriverFilterHook {
       throws Exception {
 
     List<T> hooks = new ArrayList<T>();
-    String csHooks = authzConf.get(hookConfVar.getVar(), "");
-    if (csHooks == null) {
+    String csHooks = authzConf.get(hookConfVar.getVar(), "").trim();
+    if (csHooks.isEmpty()) {
       return hooks;
     }
-
-    csHooks = csHooks.trim();
-    if (csHooks.equals("")) {
-      return hooks;
-    }
-
-    String[] hookClasses = csHooks.split(",");
-
-    for (String hookClass : hookClasses) {
+    for (String hookClass : Splitter.on(",").omitEmptyStrings().trimResults().split(csHooks)) {
       try {
+        @SuppressWarnings("unchecked")
         T hook =
-            (T) Class.forName(hookClass.trim(), true, JavaUtils.getClassLoader()).newInstance();
+            (T) Class.forName(hookClass, true, JavaUtils.getClassLoader()).newInstance();
         hooks.add(hook);
       } catch (ClassNotFoundException e) {
         LOG.error(hookConfVar.getVar() + " Class not found:" + e.getMessage());

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
index f120c77..bed7917 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
-import org.apache.sentry.binding.hive.authz.HiveAuthzPrivilegesMap;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveExtendedOperation;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivilegesMap;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContext.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContext.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContext.java
index 2beacd0..a380651 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContext.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContext.java
@@ -18,19 +18,16 @@
 
 package org.apache.sentry.binding.hive;
 
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Table;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Set;
 
 /**
  * Context information provided by Access to implementations

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContextImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContextImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContextImpl.java
index d8ffe23..f97d7f3 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContextImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryOnFailureHookContextImpl.java
@@ -18,19 +18,16 @@
 
 package org.apache.sentry.binding.hive;
 
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.sentry.core.model.db.AccessURI;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Table;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Set;
 
 public class SentryOnFailureHookContextImpl implements SentryOnFailureHookContext {
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index f6a1ecc..7e8995d 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
-import org.apache.sentry.core.common.Action;
+import org.apache.sentry.binding.hive.conf.InvalidConfigurationException;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
@@ -44,7 +44,6 @@ import org.apache.sentry.provider.common.NoAuthorizationProvider;
 import org.apache.sentry.provider.common.ProviderBackend;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.sentry.binding.hive.conf.InvalidConfigurationException;
 
 import com.google.common.base.Strings;
 
@@ -56,12 +55,10 @@ public class HiveAuthzBinding {
   private static final AtomicInteger queryID = new AtomicInteger();
   public static final String HIVE_BINDING_TAG = "hive.authz.bindings.tag";
 
-  private final HiveAuthzConf authzConf;
   private final Server authServer;
   private final AuthorizationProvider authProvider;
 
   public HiveAuthzBinding (HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception {
-    this.authzConf = authzConf;
     this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
     this.authProvider = getAuthProvider(hiveConf, authzConf, authServer.getName());
   }
@@ -130,27 +127,25 @@ public class HiveAuthzBinding {
     String authProviderName = authzConf.get(AuthzConfVars.AUTHZ_PROVIDER.getVar());
     String resourceName =
         authzConf.get(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar());
-    String providerBackendName =
-      authzConf.get(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar());
-    String policyEngineName =
-      authzConf.get(AuthzConfVars.AUTHZ_POLICY_ENGINE.getVar());
+    String providerBackendName = authzConf.get(AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar());
+    String policyEngineName = authzConf.get(AuthzConfVars.AUTHZ_POLICY_ENGINE.getVar());
 
     LOG.debug("Using authorization provider " + authProviderName +
-      " with resource " + resourceName + ", policy engine "
-      + policyEngineName + ", provider backend " + providerBackendName);
-    // load the provider backend class
-    Constructor<?> providerBackendConstructor =
-      Class.forName(providerBackendName).getDeclaredConstructor(String.class);
-    providerBackendConstructor.setAccessible(true);
-    ProviderBackend providerBackend =
-      (ProviderBackend) providerBackendConstructor.newInstance(new Object[] {resourceName});
+        " with resource " + resourceName + ", policy engine "
+        + policyEngineName + ", provider backend " + providerBackendName);
+      // load the provider backend class
+      Constructor<?> providerBackendConstructor =
+        Class.forName(providerBackendName).getDeclaredConstructor(String.class);
+      providerBackendConstructor.setAccessible(true);
+    ProviderBackend providerBackend = (ProviderBackend) providerBackendConstructor.
+        newInstance(new Object[] {resourceName});
 
     // load the policy engine class
     Constructor<?> policyConstructor =
       Class.forName(policyEngineName).getDeclaredConstructor(String.class, ProviderBackend.class);
     policyConstructor.setAccessible(true);
-    PolicyEngine policyEngine =
-      (PolicyEngine) policyConstructor.newInstance(new Object[] {serverName, providerBackend});
+    PolicyEngine policyEngine = (PolicyEngine) policyConstructor.
+        newInstance(new Object[] {serverName, providerBackend});
 
 
     // load the authz provider class
@@ -234,7 +229,7 @@ public class HiveAuthzBinding {
     return hierarchy.get(hierarchy.size() -1).getAuthzType();
   }
 
-  public List<String> getLastQueryPermissionErrors() {
-    return authProvider.getLastFailedPermissions();
+  public List<String> getLastQueryPrivilegeErrors() {
+    return authProvider.getLastFailedPrivileges();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
index b20ec34..7d241d0 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveExtendedOperation;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
-import org.apache.sentry.core.common.Action;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
index d7a518d..e59b747 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryConfigTool.java
@@ -25,7 +25,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Set;
 
-import org.apache.sentry.binding.hive.HiveAuthzBindingSessionHook;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -43,12 +42,13 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.HiveAuthzBindingSessionHook;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
-import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.core.common.SentryConfigurationException;
-import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Server;
+import org.apache.sentry.provider.common.AuthorizationProvider;
 
 public class SentryConfigTool {
   private String sentrySiteFile = null;
@@ -57,7 +57,7 @@ public class SentryConfigTool {
   private String jdbcURL = null;
   private String user = null;
   private String passWord = null;
-  private boolean listPerms = false;
+  private boolean listPrivs = false;
   private boolean validate = false;
   private HiveConf hiveConf = null;
   private HiveAuthzConf authzConf = null;
@@ -147,12 +147,12 @@ public class SentryConfigTool {
     this.passWord = passWord;
   }
 
-  public boolean isListPerms() {
-    return listPerms;
+  public boolean isListPrivs() {
+    return listPrivs;
   }
 
-  public void setListPerms(boolean listPerms) {
-    this.listPerms = listPerms;
+  public void setListPrivs(boolean listPrivs) {
+    this.listPrivs = listPrivs;
   }
 
   /**
@@ -229,10 +229,10 @@ public class SentryConfigTool {
   }
 
   // list permissions for given user
-  public void listPerms() throws Exception {
+  public void listPrivs() throws Exception {
     getSentryProvider().validateResource(true);
     System.out.println("Available privileges for user " + getUser() + ":");
-    Set<String> permList = getSentryProvider().listPermissionsForSubject(
+    Set<String> permList = getSentryProvider().listPrivilegesForSubject(
         new Subject(getUser()));
     for (String perms : permList) {
       System.out.println("\t" + perms);
@@ -359,17 +359,18 @@ public class SentryConfigTool {
 
   /**
    *  parse arguments
-   *
-   *   -d,--debug               enable debug output
-   *   -e,--query <arg>         Query privilege verification, requires -u
-   *    -h,--help                Print usage
-   *   -i,--policyIni <arg>     Policy file path
-   *   -j,--jdbcURL <arg>       JDBC URL
-   *   -l,--listPerms           list permissions for given user, requires -u
-   *   -p,--password <arg>      Password
-   *   -s,--sentry-site <arg>   sentry-site file path
-   *   -u,--user <arg>          user name
-   *   -v,--validate            Validate policy file
+   * <pre>
+   *   -d,--debug                  Enable debug output
+   *   -e,--query <arg>            Query privilege verification, requires -u
+   *    -h,--help                  Print usage
+   *   -i,--policyIni <arg>        Policy file path
+   *   -j,--jdbcURL <arg>          JDBC URL
+   *   -l,--listPrivs,--listPerms  List privilges for given user, requires -u
+   *   -p,--password <arg>         Password
+   *   -s,--sentry-site <arg>      sentry-site file path
+   *   -u,--user <arg>             user name
+   *   -v,--validate               Validate policy file
+   * </pre>
    * @param args
    */
   private void parseArgs(String[] args) {
@@ -391,6 +392,9 @@ public class SentryConfigTool {
     Option listPermsOpt = new Option("l", "listPerms", false,
         "list permissions for given user, requires -u");
     listPermsOpt.setRequired(false);
+    Option listPrivsOpt = new Option("listPrivs", false,
+        "list privileges for given user, requires -u");
+    listPrivsOpt.setRequired(false);
 
     // required args
     OptionGroup sentryOptGroup = new OptionGroup();
@@ -398,6 +402,7 @@ public class SentryConfigTool {
     sentryOptGroup.addOption(validateOpt);
     sentryOptGroup.addOption(queryOpt);
     sentryOptGroup.addOption(listPermsOpt);
+    sentryOptGroup.addOption(listPrivsOpt);
     sentryOptGroup.setRequired(true);
     sentryOptions.addOptionGroup(sentryOptGroup);
 
@@ -445,8 +450,8 @@ public class SentryConfigTool {
           setUser(opt.getValue());
         } else if (opt.getOpt().equals("p")) {
           setPassWord(opt.getValue());
-        } else if (opt.getOpt().equals("l")) {
-          setListPerms(true);
+        } else if (opt.getOpt().equals("l") || opt.getOpt().equals("listPrivs")) {
+          setListPrivs(true);
         } else if (opt.getOpt().equals("v")) {
           setValidate(true);
         } else if (opt.getOpt().equals("h")) {
@@ -456,7 +461,7 @@ public class SentryConfigTool {
         }
       }
 
-      if (isListPerms() && (getUser() == null)) {
+      if (isListPrivs() && (getUser() == null)) {
         throw new ParseException("Can't use -l without -u ");
       }
       if ((getQuery() != null) && (getUser() == null)) {
@@ -489,8 +494,8 @@ public class SentryConfigTool {
       }
 
       // list permissions for give user
-      if (sentryTool.isListPerms()) {
-        sentryTool.listPerms();
+      if (sentryTool.isListPrivs()) {
+        sentryTool.listPrivs();
       }
 
       // verify given query
@@ -503,6 +508,7 @@ public class SentryConfigTool {
       }
     } catch (Exception e) {
       System.out.println("Sentry tool reported Errors: " + e.getMessage());
+      e.printStackTrace(System.out);
       System.exit(1);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/MockUserToGroupMapping.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/MockUserToGroupMapping.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/MockUserToGroupMapping.java
index 83432ca..c095603 100644
--- a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/MockUserToGroupMapping.java
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/MockUserToGroupMapping.java
@@ -17,18 +17,18 @@
 
 package org.apache.sentry.binding.hive;
 
-import java.util.List;
+import java.util.Set;
 
 import org.apache.sentry.provider.common.GroupMappingService;
 
-import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 public class MockUserToGroupMapping implements GroupMappingService {
 
   // User to group 1-to-1 map
   @Override
-  public List<String> getGroups(String user) {
-    return Lists.newArrayList(user);
+  public Set<String> getGroups(String user) {
+    return Sets.newHashSet(user);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzConf.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzConf.java
index ea2c7ea..1942e03 100644
--- a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzConf.java
@@ -18,6 +18,7 @@ package org.apache.sentry.binding.hive;
 
 import java.util.Arrays;
 import java.util.List;
+
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestURI.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestURI.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestURI.java
index 1853559..e99d37f 100644
--- a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestURI.java
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestURI.java
@@ -20,7 +20,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SentrySolrAuthorizationException.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SentrySolrAuthorizationException.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SentrySolrAuthorizationException.java
index 134eaeb..938dbfd 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SentrySolrAuthorizationException.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SentrySolrAuthorizationException.java
@@ -17,6 +17,8 @@
 package org.apache.sentry.binding.solr.authz;
 
 public class SentrySolrAuthorizationException extends Exception {
+  private static final long serialVersionUID = -263787088321897523L;
+
   public SentrySolrAuthorizationException(String message) {
     super(message);
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
index c6ce53e..6e31f18 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
@@ -20,22 +20,20 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.util.Arrays;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
+import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;
-import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
-import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.provider.common.GroupMappingService;
 import org.apache.sentry.provider.common.ProviderBackend;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,7 +129,7 @@ public class SolrAuthzBinding {
    * @param user
    * @return list of groups the user belongs to
    */
-  public List<String> getGroups(String user) {
+  public Set<String> getGroups(String user) {
     return groupMapping.getGroups(user);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
index c9ee8ba..70983c4 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
@@ -17,12 +17,8 @@
 package org.apache.sentry.binding.solr.conf;
 
 import java.net.URL;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
-import org.mortbay.log.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
index b061eec..e2e3403 100644
--- a/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
+++ b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
@@ -16,31 +16,32 @@
  */
 package org.apache.sentry.binding.solr;
 
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+
 import java.io.File;
 import java.io.FileNotFoundException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.List;
 import java.lang.reflect.InvocationTargetException;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Set;
 
 import junit.framework.Assert;
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertTrue;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.sentry.binding.solr.authz.SentrySolrAuthorizationException;
+import org.apache.sentry.binding.solr.authz.SolrAuthzBinding;
+import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
+import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
 import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;
 import org.apache.sentry.provider.file.PolicyFiles;
-import org.apache.sentry.binding.solr.authz.SolrAuthzBinding;
-import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
-import org.apache.sentry.binding.solr.conf.SolrAuthzConf.AuthzConfVars;
-import org.apache.sentry.binding.solr.authz.SentrySolrAuthorizationException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 import com.google.common.io.Resources;
 
@@ -59,11 +60,11 @@ public class TestSolrAuthzBinding {
   private Subject sergeant1 = new Subject("sergeant1");
   private Subject general1 = new Subject("general1");
 
-  private EnumSet querySet = EnumSet.of(SearchModelAction.QUERY);
-  private EnumSet updateSet = EnumSet.of(SearchModelAction.UPDATE);
-  private EnumSet allSet = EnumSet.of(SearchModelAction.ALL);
-  private EnumSet allOfSet = EnumSet.allOf(SearchModelAction.class);
-  private EnumSet emptySet = EnumSet.noneOf(SearchModelAction.class);
+  private EnumSet<SearchModelAction> querySet = EnumSet.of(SearchModelAction.QUERY);
+  private EnumSet<SearchModelAction> updateSet = EnumSet.of(SearchModelAction.UPDATE);
+  private EnumSet<SearchModelAction> allSet = EnumSet.of(SearchModelAction.ALL);
+  private EnumSet<SearchModelAction> allOfSet = EnumSet.allOf(SearchModelAction.class);
+  private EnumSet<SearchModelAction> emptySet = EnumSet.noneOf(SearchModelAction.class);
 
   @Before
   public void setUp() throws Exception {
@@ -170,7 +171,7 @@ public class TestSolrAuthzBinding {
       new SolrAuthzConf(Resources.getResource("sentry-site.xml"));
     setUsableAuthzConf(solrAuthzConf);
     SolrAuthzBinding binding = new SolrAuthzBinding(solrAuthzConf);
-    List<String> emptyList = Arrays.asList();
+    Set<String> emptyList = Collections.emptySet();
 
     // check non-existant users
     assertEquals(binding.getGroups(null), emptyList);
@@ -183,9 +184,9 @@ public class TestSolrAuthzBinding {
     assertEquals(binding.getGroups("othergeneralgroup"), emptyList);
 
     // check valid group names
-    assertEquals(binding.getGroups("corporal1"), Arrays.asList("corporal"));
-    assertEquals(binding.getGroups("sergeant1"), Arrays.asList("sergeant"));
-    assertEquals(binding.getGroups("general1"), Arrays.asList("general", "othergeneralgroup"));
+    assertEquals(binding.getGroups("corporal1"), Sets.newHashSet("corporal"));
+    assertEquals(binding.getGroups("sergeant1"), Sets.newHashSet("sergeant"));
+    assertEquals(binding.getGroups("general1"), Sets.newHashSet("general", "othergeneralgroup"));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/SentryConfigurationException.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/SentryConfigurationException.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/SentryConfigurationException.java
index 516b2da..10d66f5 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/SentryConfigurationException.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/SentryConfigurationException.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.shiro.config.ConfigurationException;
 
 public class SentryConfigurationException extends ConfigurationException {
+  private static final long serialVersionUID = -116202866086371884L;
   private List<String> configErrors = new ArrayList<String>();
   private List<String> configWarnings = new ArrayList<String>();
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
index 1659450..962179f 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
@@ -43,8 +43,8 @@ public class PathUtils {
     // request path does not contain relative parts /a/../b &&
     // request path starts with privilege path &&
     // authorities (nullable) are equal
-    String requestPath = ensureEndsWithSeparator(requestURI.getPath());
-    String privilegePath = ensureEndsWithSeparator(privilegeURI.getPath());
+    String requestPath = ensureEndsWithSeparator(requestURI.getPath()).replace("//", "/");
+    String privilegePath = ensureEndsWithSeparator(privilegeURI.getPath()).replace("//", "/");
     if (requestURI.getPath().equals(requestURI.normalize().getPath()) &&
         requestPath.startsWith(privilegePath) &&
         Strings.nullToEmpty(privilegeURI.getAuthority()).equals(Strings.nullToEmpty(requestURI.getAuthority()))) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java b/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
index 28818ba..d30305b 100644
--- a/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
+++ b/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
@@ -21,6 +21,7 @@ import static junit.framework.Assert.assertFalse;
 import static junit.framework.Assert.assertTrue;
 
 import java.net.URI;
+
 import org.junit.Test;
 
 public class TestPathUtils {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-core/sentry-core-model-search/src/test/java/org/apache/sentry/core/search/TestCollection.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-search/src/test/java/org/apache/sentry/core/search/TestCollection.java b/sentry-core/sentry-core-model-search/src/test/java/org/apache/sentry/core/search/TestCollection.java
index bc00b62..1bf7069 100644
--- a/sentry-core/sentry-core-model-search/src/test/java/org/apache/sentry/core/search/TestCollection.java
+++ b/sentry-core/sentry-core-model-search/src/test/java/org/apache/sentry/core/search/TestCollection.java
@@ -16,9 +16,10 @@ package org.apache.sentry.core.search;
  * limitations under the License.
  */
 
+import junit.framework.Assert;
+
 import org.apache.sentry.core.model.search.Collection;
 import org.junit.Test;
-import junit.framework.Assert;
 
 public class TestCollection {
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PermissionFactory.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PermissionFactory.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PermissionFactory.java
deleted file mode 100644
index 45fd7bd..0000000
--- a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PermissionFactory.java
+++ /dev/null
@@ -1,26 +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.common;
-
-import org.apache.shiro.authz.Permission;
-
-/**
- * Factory for creating Shiro permissions
- */
-public interface PermissionFactory {
-  Permission createPermission(String permission);
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
index c08d082..79c48d4 100644
--- a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
@@ -17,41 +17,37 @@
 
 package org.apache.sentry.policy.common;
 
-import java.util.List;
+import java.util.Set;
+
+import javax.annotation.concurrent.ThreadSafe;
 
-import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSetMultimap;
-
+/**
+ * Implementations of this interface are expected to be thread safe
+ * after construction.
+ */
+@ThreadSafe
 public interface PolicyEngine {
 
   /**
-   * The permission factory to use in order to compare permissions in {@link getPermission}.
-   * This is typically a factory that returns a permission used to evaluate wildcards.
-   * @return the permission factory
+   * The privilege factory to use in order to compare privileges in {@link getPermission}.
+   * This is typically a factory that returns a privilege used to evaluate wildcards.
+   * @return the privilege factory
    */
-  public PermissionFactory getPermissionFactory();
+  public PrivilegeFactory getPrivilegeFactory();
 
   /**
-   * Get permissions associated with a group. Returns Strings which can be resolved
+   * Get privileges associated with a group. Returns Strings which can be resolved
    * by the caller. Strings are returned to separate the PolicyFile class from the
-   * type of permissions used in a policy file. Additionally it is possible further
-   * processing of the permissions is needed before resolving to a permission object.
-   * @param authorizeable object
+   * type of privileges used in a policy file. Additionally it is possible further
+   * processing of the privileges is needed before resolving to a privilege object.
    * @param group name
-   * @return non-null immutable set of permissions
+   * @return non-null immutable set of privileges
    */
-  public ImmutableSetMultimap<String, String> getPermissions(
-      List<? extends Authorizable> authorizables, List<String> groups)
+  public ImmutableSet<String> getPrivileges(Set<String> groups)
       throws SentryConfigurationException;
 
-  public ImmutableSet<String> listPermissions(String groupName)
-    throws SentryConfigurationException;
-
-  public ImmutableSet<String> listPermissions(List<String> groupName)
-    throws SentryConfigurationException;
-
   public void validatePolicy(boolean strictValidation) throws SentryConfigurationException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/Privilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/Privilege.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/Privilege.java
new file mode 100644
index 0000000..c7e1734
--- /dev/null
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/Privilege.java
@@ -0,0 +1,21 @@
+/*
+ * 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.common;
+
+public interface Privilege {
+  public boolean implies(Privilege p);
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeFactory.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeFactory.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeFactory.java
new file mode 100644
index 0000000..2f8296b
--- /dev/null
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeFactory.java
@@ -0,0 +1,24 @@
+/*
+ * 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.common;
+
+/**
+ * Factory for creating Privilege
+ */
+public interface PrivilegeFactory {
+  Privilege createPrivilege(String permission);
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeUtils.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeUtils.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeUtils.java
new file mode 100644
index 0000000..7387ad0
--- /dev/null
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeUtils.java
@@ -0,0 +1,27 @@
+/*
+ * 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.common;
+
+import java.util.Set;
+
+import org.apache.shiro.util.PermissionUtils;
+
+public class PrivilegeUtils {
+  public static Set<String> toPrivilegeStrings(String s) {
+    return PermissionUtils.toPermissionStrings(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidator.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidator.java
new file mode 100644
index 0000000..5548f04
--- /dev/null
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidator.java
@@ -0,0 +1,24 @@
+/*
+ * 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.common;
+
+import org.apache.shiro.config.ConfigurationException;
+
+public interface PrivilegeValidator {
+
+  public void validate(PrivilegeValidatorContext context) throws ConfigurationException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidatorContext.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidatorContext.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidatorContext.java
new file mode 100644
index 0000000..2b7fd1a
--- /dev/null
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PrivilegeValidatorContext.java
@@ -0,0 +1,38 @@
+/*
+ * 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.common;
+
+import javax.annotation.Nullable;
+
+public class PrivilegeValidatorContext {
+  private final String database;
+  private final String privilege;
+  public PrivilegeValidatorContext(String privilege) {
+    this(null, privilege);
+  }
+  public PrivilegeValidatorContext(@Nullable String database, String privilege) {
+    super();
+    this.database = database;
+    this.privilege = privilege;
+  }
+  public @Nullable String getDatabase() {
+    return database;
+  }
+  public String getPrivilege() {
+    return privilege;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/RoleValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/RoleValidator.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/RoleValidator.java
deleted file mode 100644
index 8390364..0000000
--- a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/RoleValidator.java
+++ /dev/null
@@ -1,26 +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.common;
-
-import javax.annotation.Nullable;
-
-import org.apache.shiro.config.ConfigurationException;
-
-public interface RoleValidator {
-
-  public void validate(@Nullable String database, String role) throws ConfigurationException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBPrivilegeValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBPrivilegeValidator.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBPrivilegeValidator.java
new file mode 100644
index 0000000..1b774ee
--- /dev/null
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBPrivilegeValidator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.db;
+
+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.db.DBModelAuthorizable;
+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 AbstractDBPrivilegeValidator implements PrivilegeValidator {
+
+  @VisibleForTesting
+  public static Iterable<DBModelAuthorizable> parsePrivilege(String string) {
+    List<DBModelAuthorizable> 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)) {
+        DBModelAuthorizable authorizable = DBModelAuthorizables.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/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBRoleValidator.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBRoleValidator.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBRoleValidator.java
deleted file mode 100644
index 722a4be..0000000
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/AbstractDBRoleValidator.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.db;
-
-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.db.DBModelAuthorizable;
-import org.apache.shiro.config.ConfigurationException;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-
-public abstract class AbstractDBRoleValidator implements RoleValidator {
-
-  @VisibleForTesting
-  public static Iterable<DBModelAuthorizable> parseRole(String string) {
-    List<DBModelAuthorizable> 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)) {
-        DBModelAuthorizable authorizable = DBModelAuthorizables.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/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPermission.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPermission.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPermission.java
deleted file mode 100644
index 01981d1..0000000
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPermission.java
+++ /dev/null
@@ -1,181 +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.db;
-
-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.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-
-import org.apache.commons.lang.text.StrSubstitutor;
-import org.apache.sentry.core.common.utils.PathUtils;
-import org.apache.sentry.core.model.db.AccessConstants;
-import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
-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.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-// XXX this class is made ugly by the fact that Action is not a Authorizable.
-public class DBWildcardPermission implements Permission, Serializable {
-  private static final Logger LOGGER = LoggerFactory
-      .getLogger(DBWildcardPermission.class);
-  private static final long serialVersionUID = -6785051263922740818L;
-
-  private final ImmutableList<KeyValue> parts;
-
-  public DBWildcardPermission(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 DBWildcardPermissions
-    if (!(p instanceof DBWildcardPermission)) {
-      return false;
-    }
-
-    DBWildcardPermission wp = (DBWildcardPermission) 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(AccessConstants.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(AccessConstants.ALL) || policyPart.equals(requestPart)) {
-      return true;
-    } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
-        && AccessConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {
-      /* permission request is to match with any object of given type */
-      return true;
-    } else if(policyPart.getKey().equalsIgnoreCase(AuthorizableType.URI.name())) {
-      return impliesURI(policyPart.getValue(), requestPart.getValue());
-    }
-    return false;
-  }
-
-  @VisibleForTesting
-  protected static boolean impliesURI(String privilege, String request) {
-    try {
-    URI privilegeURI = new URI(new StrSubstitutor(System.getProperties()).replace(privilege));
-    URI requestURI = new URI(request);
-    if(privilegeURI.getScheme() == null || privilegeURI.getPath() == null) {
-      LOGGER.warn("Privilege URI " + request + " is not valid. Either no scheme or no path.");
-      return false;
-    }
-    if(requestURI.getScheme() == null || requestURI.getPath() == null) {
-      LOGGER.warn("Request URI " + request + " is not valid. Either no scheme or no path.");
-      return false;
-    }
-      return PathUtils.impliesURI(privilegeURI, requestURI);
-    } catch (URISyntaxException e) {
-      LOGGER.warn("Request URI " + request + " is not a URI", e);
-      return false;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return AUTHORIZABLE_JOINER.join(parts);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof DBWildcardPermission) {
-      DBWildcardPermission wp = (DBWildcardPermission) o;
-      return parts.equals(wp.parts);
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return parts.hashCode();
-  }
-
-  public static class DBWildcardPermissionFactory implements PermissionFactory {
-    @Override
-    public Permission createPermission(String permission) {
-      return new DBWildcardPermission(permission);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPrivilege.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPrivilege.java
new file mode 100644
index 0000000..cab1234
--- /dev/null
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBWildcardPrivilege.java
@@ -0,0 +1,179 @@
+/*
+ * 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.db;
+
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_SPLITTER;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+
+import org.apache.commons.lang.text.StrSubstitutor;
+import org.apache.sentry.core.common.utils.PathUtils;
+import org.apache.sentry.core.model.db.AccessConstants;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.provider.file.KeyValue;
+import org.apache.sentry.provider.file.PolicyFileConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+// XXX this class is made ugly by the fact that Action is not a Authorizable.
+public class DBWildcardPrivilege implements Privilege {
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(DBWildcardPrivilege.class);
+
+  private final ImmutableList<KeyValue> parts;
+
+  public DBWildcardPrivilege(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 DBWildcardPermissions
+    if (!(p instanceof DBWildcardPrivilege)) {
+      return false;
+    }
+
+    DBWildcardPrivilege wp = (DBWildcardPrivilege) 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(AccessConstants.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(AccessConstants.ALL) || policyPart.equals(requestPart)) {
+      return true;
+    } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
+        && AccessConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {
+      /* privilege request is to match with any object of given type */
+      return true;
+    } else if(policyPart.getKey().equalsIgnoreCase(AuthorizableType.URI.name())) {
+      return impliesURI(policyPart.getValue(), requestPart.getValue());
+    }
+    return false;
+  }
+
+  @VisibleForTesting
+  protected static boolean impliesURI(String privilege, String request) {
+    try {
+    URI privilegeURI = new URI(new StrSubstitutor(System.getProperties()).replace(privilege));
+    URI requestURI = new URI(request);
+    if(privilegeURI.getScheme() == null || privilegeURI.getPath() == null) {
+      LOGGER.warn("Privilege URI " + request + " is not valid. Either no scheme or no path.");
+      return false;
+    }
+    if(requestURI.getScheme() == null || requestURI.getPath() == null) {
+      LOGGER.warn("Request URI " + request + " is not valid. Either no scheme or no path.");
+      return false;
+    }
+      return PathUtils.impliesURI(privilegeURI, requestURI);
+    } catch (URISyntaxException e) {
+      LOGGER.warn("Request URI " + request + " is not a URI", e);
+      return false;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return AUTHORIZABLE_JOINER.join(parts);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof DBWildcardPrivilege) {
+      DBWildcardPrivilege wp = (DBWildcardPrivilege) o;
+      return parts.equals(wp.parts);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return parts.hashCode();
+  }
+
+  public static class DBWildcardPrivilegeFactory implements PrivilegeFactory {
+    @Override
+    public Privilege createPrivilege(String privilege) {
+      return new DBWildcardPrivilege(privilege);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseMustMatch.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseMustMatch.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseMustMatch.java
index a7c2091..d280c41 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseMustMatch.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseMustMatch.java
@@ -18,21 +18,24 @@ package org.apache.sentry.policy.db;
 
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.Database;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
 import org.apache.shiro.config.ConfigurationException;
 
-public class DatabaseMustMatch extends AbstractDBRoleValidator {
+public class DatabaseMustMatch extends AbstractDBPrivilegeValidator {
 
   @Override
-  public void validate(String database, String role) throws ConfigurationException {
+  public void validate(PrivilegeValidatorContext context) throws ConfigurationException {
+    String database = context.getDatabase();
+    String privilege = context.getPrivilege();
     /*
      *  Rule only applies to rules in per database policy file
      */
     if(database != null) {
-      Iterable<DBModelAuthorizable> authorizables = parseRole(role);
+      Iterable<DBModelAuthorizable> authorizables = parsePrivilege(privilege);
       for(DBModelAuthorizable authorizable : authorizables) {
         if(authorizable instanceof Database &&
             !database.equalsIgnoreCase(authorizable.getName())) {
-          String msg = "Role " + role + " references db " +
+          String msg = "Privilege " + privilege + " references db " +
               authorizable.getName() + ", but is only allowed to reference "
               + database;
           throw new ConfigurationException(msg);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/92212c3d/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInPrivilege.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInPrivilege.java
new file mode 100644
index 0000000..e89aa16
--- /dev/null
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DatabaseRequiredInPrivilege.java
@@ -0,0 +1,71 @@
+/*
+ * 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.db;
+
+import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.Database;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
+import org.apache.shiro.config.ConfigurationException;
+
+public class DatabaseRequiredInPrivilege extends AbstractDBPrivilegeValidator {
+
+  @Override
+  public void validate(PrivilegeValidatorContext context) throws ConfigurationException {
+    String database = context.getDatabase();
+    String privilege = context.getPrivilege();
+    /*
+     *  Rule only applies to rules in per database policy file
+     */
+    if(database != null) {
+      Iterable<DBModelAuthorizable> authorizables = parsePrivilege(privilege);
+      /*
+       * Each permission in a non-global file must have a database
+       * object except for URIs.
+       *
+       * We allow URIs to be specified in the per DB policy file for
+       * ease of mangeability. URIs will contain to remain server scope
+       * objects.
+       */
+      boolean foundDatabaseInAuthorizables = false;
+      boolean foundURIInAuthorizables = false;
+      boolean allowURIInAuthorizables = false;
+
+      if ("true".equalsIgnoreCase(
+          System.getProperty(SimpleDBPolicyEngine.ACCESS_ALLOW_URI_PER_DB_POLICYFILE))) {
+        allowURIInAuthorizables = true;
+      }
+
+      for(DBModelAuthorizable authorizable : authorizables) {
+        if(authorizable instanceof Database) {
+          foundDatabaseInAuthorizables = true;
+        }
+        if (authorizable instanceof AccessURI) {
+          if (foundDatabaseInAuthorizables) {
+            String msg = "URI object is specified at DB scope in " + privilege;
+            throw new ConfigurationException(msg);
+          }
+          foundURIInAuthorizables = true;
+        }
+      }
+      if(!foundDatabaseInAuthorizables && !(foundURIInAuthorizables && allowURIInAuthorizables)) {
+        String msg = "Missing database object in " + privilege;
+        throw new ConfigurationException(msg);
+      }
+    }
+  }
+}