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:40 UTC

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

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);
+      }
+    }
+  }
+}