You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sr...@apache.org on 2014/06/11 01:35:50 UTC

git commit: SENTRY-247: Go back to using filter push down once the bugs are fixed (Arun Suresh via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 38c4294ba -> f741870c2


SENTRY-247: Go back to using filter push down once the bugs are fixed (Arun Suresh via Sravya Tirukkovalur)


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

Branch: refs/heads/master
Commit: f741870c2145dd7a060e6408d6e73ca26a175943
Parents: 38c4294
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Tue Jun 10 16:35:08 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Tue Jun 10 16:35:08 2014 -0700

----------------------------------------------------------------------
 .../binding/hive/HiveAuthzBindingHook.java      | 11 ++--
 .../sentry/core/model/db/AccessConstants.java   |  1 +
 .../org/apache/sentry/core/model/db/Table.java  |  1 +
 .../sentry/policy/db/DBModelAuthorizables.java  |  8 ++-
 .../sentry/policy/db/DBWildcardPrivilege.java   |  4 ++
 .../common/ResourceAuthorizationProvider.java   | 34 ++++++++++--
 .../provider/db/SimpleDBProviderBackend.java    |  2 +-
 .../db/service/persistent/SentryStore.java      | 57 ++++++++++++++++++--
 .../thrift/SentryPolicyStoreProcessor.java      | 19 ++++++-
 .../tests/e2e/dbprovider/TestDbEndToEnd.java    | 26 ++-------
 .../tests/e2e/dbprovider/TestDbSandboxOps.java  | 12 +++++
 .../e2e/hive/TestPrivilegeAtTransform.java      |  4 +-
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java | 10 +++-
 13 files changed, 149 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/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 812f310..6c507b8 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
@@ -451,13 +451,18 @@ implements HiveDriverFilterHook {
       List<DBModelAuthorizable> connectHierarchy = new ArrayList<DBModelAuthorizable>();
       connectHierarchy.add(hiveAuthzBinding.getAuthServer());
       // by default allow connect access to default db
-      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(currDB.getName()) &&
+      Table currTbl = Table.ALL;
+      if ((DEFAULT_DATABASE_NAME.equalsIgnoreCase(currDB.getName()) &&
           "false".equalsIgnoreCase(authzConf.
-              get(HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false"))) {
+              get(HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false")))
+              ||stmtOperation.equals(HiveOperation.CREATEFUNCTION)
+              ||stmtOperation.equals(HiveOperation.DROPFUNCTION)) {
         currDB = Database.ALL;
+        currTbl = Table.SOME;
       }
+
       connectHierarchy.add(currDB);
-      connectHierarchy.add(Table.ALL);
+      connectHierarchy.add(currTbl);
 
       inputHierarchy.add(connectHierarchy);
       // check if this is a create temp function and we need to validate URI

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
index 9f5035e..4e89f68 100644
--- a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
+++ b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
@@ -25,6 +25,7 @@ public class AccessConstants {
    * represents all Servers, Databases, or Tables.
    */
   public static final String ALL = "*";
+  public static final String SOME = "+";
 
   public static final String SELECT = "select";
   public static final String INSERT = "insert";

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Table.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Table.java b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Table.java
index 62a0a81..b161d02 100644
--- a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Table.java
+++ b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/Table.java
@@ -22,6 +22,7 @@ public class Table implements TableOrView {
    * Represents all tables
    */
   public static final Table ALL = new Table(AccessConstants.ALL);
+  public static final Table SOME = new Table(AccessConstants.SOME);
 
   private final String name;
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
index f4b32e1..873f789 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/DBModelAuthorizables.java
@@ -29,10 +29,14 @@ public class DBModelAuthorizables {
 
   public static DBModelAuthorizable from(KeyValue keyValue) {
     String prefix = keyValue.getKey().toLowerCase();
-    String name = keyValue.getValue().toLowerCase();
+    String name = keyValue.getValue();
     for(AuthorizableType type : AuthorizableType.values()) {
       if(prefix.equalsIgnoreCase(type.name())) {
-        return from(type, name);
+        if (prefix.equalsIgnoreCase(AuthorizableType.URI.toString())) {
+          return from(type, name);
+        } else {
+          return from(type, name.toLowerCase());
+        }
       }
     }
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/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
index cab1234..d599de9 100644
--- 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
@@ -125,6 +125,10 @@ public class DBWildcardPrivilege implements Privilege {
         && AccessConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {
       /* privilege request is to match with any object of given type */
       return true;
+    } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
+        && AccessConstants.SOME.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());
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
index 3a993b0..51d4248 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
@@ -17,6 +17,7 @@
 package org.apache.sentry.provider.common;
 
 import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_SPLITTER;
 import static org.apache.sentry.provider.common.ProviderConstants.KV_JOINER;
 import static org.apache.sentry.provider.common.ProviderConstants.PRIVILEGE_NAME;
 
@@ -30,20 +31,23 @@ import org.apache.sentry.core.common.ActiveRoleSet;
 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.PolicyEngine;
 import org.apache.sentry.policy.common.Privilege;
 import org.apache.sentry.policy.common.PrivilegeFactory;
-import org.apache.sentry.policy.common.PolicyEngine;
 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.ImmutableSet;
 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 PrivilegeFactory privilegeFactory;
@@ -95,8 +99,8 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     for (Authorizable authorizable : authorizables) {
       hierarchy.add(KV_JOINER.join(authorizable.getTypeName(), authorizable.getName()));
     }
-    Iterable<Privilege> privileges = getPrivileges(groups, roleSet, authorizables.toArray(new Authorizable[0]));
     List<String> requestPrivileges = buildPermissions(authorizables, actions);
+    Iterable<Privilege> privileges = getPrivileges(groups, roleSet, authorizables.toArray(new Authorizable[0]));
     lastFailedPrivileges.get().clear();
 
     for (String requestPrivilege : requestPrivileges) {
@@ -114,12 +118,13 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
         }
       }
     }
+
     lastFailedPrivileges.get().addAll(requestPrivileges);
     return false;
   }
 
   private Iterable<Privilege> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable[] authorizables) {
-    return Iterables.transform(policy.getPrivileges(groups, roleSet, authorizables),
+    return Iterables.transform(appendDefaultDBPriv(policy.getPrivileges(groups, roleSet, authorizables), authorizables),
         new Function<String, Privilege>() {
       @Override
       public Privilege apply(String privilege) {
@@ -128,6 +133,29 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     });
   }
 
+  private ImmutableSet<String> appendDefaultDBPriv(ImmutableSet<String> privileges, Authorizable[] authorizables) {
+    // Only for switch db
+    if ((authorizables != null)&&(authorizables.length == 3)&&(authorizables[2].getName().equals("+"))) {
+      if ((privileges.size() == 1) && hasOnlyServerPrivilege(privileges.asList().get(0))) {
+        // Assuming authorizable[0] will always be the server
+        // This Code is only reachable only when user fires a 'use default'
+        // and the user has a privilege on atleast 1 privilized Object
+        String defaultPriv = "Server=" + authorizables[0].getName() + "->Db=default->Table=*->action=select";
+        HashSet<String> newPrivs = Sets.newHashSet(defaultPriv);
+        return ImmutableSet.copyOf(newPrivs);
+      }
+    }
+    return privileges;
+  }
+
+  private boolean hasOnlyServerPrivilege(String priv) {
+    ArrayList<String> l = Lists.newArrayList(AUTHORIZABLE_SPLITTER.split(priv));
+    if ((l.size() == 1)&&(l.get(0).toLowerCase().startsWith("server"))) {
+      return l.get(0).toLowerCase().split("=")[1].endsWith("+");
+    }
+    return false;
+  }
+
   @Override
   public GroupMappingService getGroupMapping() {
     return groupService;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
index 326b91d..b66037a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
@@ -78,7 +78,7 @@ public class SimpleDBProviderBackend implements ProviderBackend {
       throw new IllegalStateException("Backend has not been properly initialized");
     }
     try {
-      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet, null));
+      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet, authorizableHierarchy));
     } catch (SentryUserException e) {
       String msg = "Unable to obtain privileges from server: " + e.getMessage();
       LOGGER.error(msg, e);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
index 5560729..707641e 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
@@ -41,6 +41,7 @@ import javax.jdo.Transaction;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.model.db.AccessConstants;
+import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.provider.common.ProviderConstants;
 import org.apache.sentry.provider.db.SentryAccessDeniedException;
@@ -597,6 +598,36 @@ public class SentryStore {
     }
   }
 
+  private boolean hasAnyServerPrivileges(Set<String> roleNames, String serverName) {
+    if ((roleNames.size() == 0)||(roleNames == null)) return false;
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryPrivilege.class);
+      query.declareVariables("org.apache.sentry.provider.db.service.model.MSentryRole role");
+      List<String> rolesFiler = new LinkedList<String>();
+      for (String rName : roleNames) {
+        rolesFiler.add("role.roleName == \"" + rName.trim().toLowerCase() + "\"");
+      }
+      StringBuilder filters = new StringBuilder("roles.contains(role) "
+          + "&& (" + Joiner.on(" || ").join(rolesFiler) + ") ");
+      filters.append("&& serverName == \"" + serverName + "\"");
+      query.setFilter(filters.toString());
+      query.setResult("count(this)");
+
+      Long numPrivs = (Long) query.execute();
+      rollbackTransaction = false;
+      commitTransaction(pm);
+      return (numPrivs > 0);
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
+
   List<MSentryPrivilege> getMSentryPrivileges(Set<String> roleNames, TSentryAuthorizable authHierarchy) {
     if ((roleNames.size() == 0)||(roleNames == null)) return new ArrayList<MSentryPrivilege>();
     boolean rollbackTransaction = true;
@@ -625,7 +656,6 @@ public class SentryStore {
           filters.append(" && ((\"" + authHierarchy.getUri() + "\".startsWith(URI)) || (URI == null)) && (dbName == null)");
         }
       }
-      System.out.println("Filter String: " + filters.toString());
       query.setFilter(filters.toString());
       List<MSentryPrivilege> privileges = (List<MSentryPrivilege>) query.execute();
       rollbackTransaction = false;
@@ -799,18 +829,33 @@ public class SentryStore {
   public Set<String> listSentryPrivilegesForProvider(Set<String> groups,
       TSentryActiveRoleSet roleSet, TSentryAuthorizable authHierarchy) throws SentryInvalidInputException {
     Set<String> result = Sets.newHashSet();
-    Set<String> activeRoleNames = toTrimedLower(roleSet.getRoles());
-
-    Set<String> roleNamesForGroups = toTrimedLower(getRoleNamesForGroups(groups));
-    Set<String> rolesToQuery = roleSet.isAll() ? roleNamesForGroups : Sets.intersection(activeRoleNames, roleNamesForGroups);
+    Set<String> rolesToQuery = getRolesToQuery(groups, roleSet);
     List<MSentryPrivilege> mSentryPrivileges = getMSentryPrivileges(rolesToQuery, authHierarchy);
 
     for (MSentryPrivilege priv : mSentryPrivileges) {
       result.add(toAuthorizable(priv));
     }
+
     return result;
   }
 
+
+  public boolean hasAnyServerPrivileges(Set<String> groups, TSentryActiveRoleSet roleSet, String server) {
+    Set<String> rolesToQuery = getRolesToQuery(groups, roleSet);
+    return hasAnyServerPrivileges(rolesToQuery, server);
+  }
+
+
+
+  private Set<String> getRolesToQuery(Set<String> groups,
+      TSentryActiveRoleSet roleSet) {
+    Set<String> activeRoleNames = toTrimedLower(roleSet.getRoles());
+
+    Set<String> roleNamesForGroups = toTrimedLower(getRoleNamesForGroups(groups));
+    Set<String> rolesToQuery = roleSet.isAll() ? roleNamesForGroups : Sets.intersection(activeRoleNames, roleNamesForGroups);
+    return rolesToQuery;
+  }
+
   @VisibleForTesting
   static String toAuthorizable(MSentryPrivilege privilege) {
     List<String> authorizable = new ArrayList<String>(4);
@@ -1010,4 +1055,6 @@ public class SentryStore {
     }
 
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/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 a1cf24a..b324b43 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
@@ -381,8 +381,23 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TListSentryPrivilegesForProviderResponse response = new TListSentryPrivilegesForProviderResponse();
     response.setPrivileges(new HashSet<String>());
     try {
-      response.setPrivileges(sentryStore.listSentryPrivilegesForProvider(
-          request.getGroups(), request.getRoleSet(), request.getAuthorizableHierarchy()));
+      Set<String> privilegesForProvider = sentryStore.listSentryPrivilegesForProvider(
+          request.getGroups(), request.getRoleSet(), request.getAuthorizableHierarchy());
+      response.setPrivileges(privilegesForProvider);
+      if ((privilegesForProvider == null)||(privilegesForProvider.size() == 0)) {
+        if (sentryStore.hasAnyServerPrivileges(
+            request.getGroups(), request.getRoleSet(), request.getAuthorizableHierarchy().getServer())) {
+
+          // REQUIRED for ensuring 'default' Db is accessible by any user
+          // with privileges to atleast 1 object with the specific server as root
+
+          // Need some way to specify that even though user has no privilege
+          // For the specific AuthorizableHierarchy.. he has privilege on
+          // atleast 1 object in the server hierarchy
+          HashSet<String> serverPriv = Sets.newHashSet("server=+");
+          response.setPrivileges(serverPriv);
+        }
+      }
       response.setStatus(Status.OK());
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
index 2198c05..cabd5db 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbEndToEnd.java
@@ -68,9 +68,13 @@ public class TestDbEndToEnd extends AbstractTestWithDbProvider {
     // test a specific role
     statement.execute("SET ROLE user_role");
     statement.execute("SELECT * FROM t1");
+
+    /** Dissabling test : see https://issues.apache.org/jira/browse/HIVE-6629
     // test NONE
     statement.execute("SET ROLE NONE");
     context.assertAuthzException(statement, "SELECT * FROM t1");
+    */
+
     // test ALL
     statement.execute("SET ROLE ALL");
     statement.execute("SELECT * FROM t1");
@@ -199,28 +203,6 @@ public class TestDbEndToEnd extends AbstractTestWithDbProvider {
     // 8
     connection = context.createConnection(USER1_1);
     statement = context.createStatement(connection);
-    Exception ex = null;
-    try {
-      statement.execute("USE " + dbName2);
-    } catch (Exception e) {
-      ex = e;
-    } finally {
-      statement.close();
-      connection.close();
-    }
-    System.out.println("Message : " + ex.getMessage());
-    assertTrue("This should not be allowed !!", ex != null);
-
-    connection = context.createConnection(ADMIN1);
-    statement = context.createStatement(connection);
-    statement.execute("CREATE ROLE all_db2");
-    statement.execute("GRANT ALL ON DATABASE " + dbName2 + " TO ROLE all_db2");
-    statement.execute("GRANT ROLE all_db2 TO GROUP " + USERGROUP1);
-    statement.close();
-    connection.close();
-
-    connection = context.createConnection(USER1_1);
-    statement = context.createStatement(connection);
     statement.execute("USE " + dbName2);
 
     statement.execute("INSERT OVERWRITE TABLE " +

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
index 5eef792..f822785 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSandboxOps.java
@@ -16,9 +16,13 @@
  */
 package org.apache.sentry.tests.e2e.dbprovider;
 
+import static org.junit.Assert.assertTrue;
+
 import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
 import org.apache.sentry.tests.e2e.hive.TestSandboxOps;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
 
 public class TestDbSandboxOps extends TestSandboxOps {
   @BeforeClass
@@ -29,4 +33,12 @@ public class TestDbSandboxOps extends TestSandboxOps {
         .getSentryClient());
   }
 
+  @Ignore
+  @Test
+  public void testPerDbPolicyOnDFS() throws Exception {
+    // TODO : Looks like the test in the base class is specifically meant for
+    // File based providers... Since it is assuming that multiple policy files
+    // in DFS would be handled by the provider..
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
index 732632b..bc20213 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
@@ -25,6 +25,7 @@ import java.sql.Connection;
 import java.sql.Statement;
 
 import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.dbprovider.PolicyProviderForTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,6 +38,7 @@ public class TestPrivilegeAtTransform extends AbstractTestWithStaticConfiguratio
   private File dataFile;
   private PolicyFile policyFile;
 
+  @Override
   @Before
   public void setup() throws Exception {
     context = createContext();
@@ -45,7 +47,7 @@ public class TestPrivilegeAtTransform extends AbstractTestWithStaticConfiguratio
     FileOutputStream to = new FileOutputStream(dataFile);
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = PolicyProviderForTest.setAdminOnServer1(ADMINGROUP);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f741870c/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
index 1e93ec6..029d8d5 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
@@ -49,6 +49,7 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
   Map <String, String >testProperties;
   private static final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
 
+  @Override
   @Before
   public void setup() throws Exception {
     testProperties = new HashMap<String, String>();
@@ -109,7 +110,7 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
     // test CTAS can reference UDFs
     statement.execute("USE DB_1");
     statement.execute("create table table2 as select A, count(A) from TAB_1 GROUP BY A");
-    
+
     // test user can switch db
     statement.execute("USE DB_1");
     //test user can create view
@@ -387,11 +388,18 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
     Connection connection = context.createConnection(ADMIN1);
     Statement statement = context.createStatement(connection);
     statement.execute("use default");
+    statement.execute("create table tab1(a int)");
     context.close();
 
     connection = context.createConnection(USER1_1);
     statement = context.createStatement(connection);
     statement.execute("use default");
+    try {
+      statement.execute("select * from tab1");
+      assertTrue("Should not be allowed !!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
     context.close();
 
     connection = context.createConnection(USER2_1);