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/18 22:22:31 UTC

git commit: SENTRY-281: Revoking a parent privilege should revoke all child privileges ( Arun Suresh via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master c248a3dc1 -> 5134a092a


SENTRY-281: Revoking a parent privilege should revoke all child privileges ( 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/5134a092
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/5134a092
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/5134a092

Branch: refs/heads/master
Commit: 5134a092a0d45d608cdbef7a1763bc9092d7c4ca
Parents: c248a3d
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Wed Jun 18 12:05:39 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Wed Jun 18 12:05:39 2014 -0700

----------------------------------------------------------------------
 .../db/service/persistent/SentryStore.java      | 202 +++--
 .../e2e/dbprovider/TestDatabaseProvider.java    | 731 ++++++++++++++++++-
 .../tests/e2e/dbprovider/TestDbEndToEnd.java    |   9 +-
 3 files changed, 890 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/5134a092/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 91669d6..56fc3df 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
@@ -275,6 +275,38 @@ public class SentryStore {
       if (mRole == null) {
         throw new SentryNoSuchObjectException("Role: " + roleName);
       } else {
+
+        if (privilege.getTableName() != null) {
+          // If Grant is for ALL and Either INSERT/SELECT already exists..
+          // need to remove it and GRANT ALL..
+          if (privilege.getAction().equalsIgnoreCase("*")) {
+            TSentryPrivilege tNotAll = new TSentryPrivilege(privilege);
+            tNotAll.setAction(AccessConstants.SELECT);
+            MSentryPrivilege mSelect = getMSentryPrivilege(constructPrivilegeName(tNotAll), pm);
+            tNotAll.setAction(AccessConstants.INSERT);
+            MSentryPrivilege mInsert = getMSentryPrivilege(constructPrivilegeName(tNotAll), pm);
+            if ((mSelect != null)&&(mRole.getPrivileges().contains(mSelect))) {
+              mSelect.removeRole(mRole);
+              pm.makePersistent(mSelect);
+            }
+            if ((mInsert != null)&&(mRole.getPrivileges().contains(mInsert))) {
+              mInsert.removeRole(mRole);
+              pm.makePersistent(mInsert);
+            }
+          } else {
+            // If Grant is for Either INSERT/SELECT and ALL already exists..
+            // do nothing..
+            TSentryPrivilege tAll = new TSentryPrivilege(privilege);
+            tAll.setAction(AccessConstants.ALL);
+            MSentryPrivilege mAll = getMSentryPrivilege(constructPrivilegeName(tAll), pm);
+            if ((mAll != null)&&(mRole.getPrivileges().contains(mAll))) {
+              CommitContext commit = commitUpdateTransaction(pm);
+              rollbackTransaction = false;
+              return commit;
+            }
+          }
+        }
+
         MSentryPrivilege mPrivilege = getMSentryPrivilege(constructPrivilegeName(privilege), pm);
         if (mPrivilege == null) {
           mPrivilege = convertToMSentryPrivilege(privilege);
@@ -313,20 +345,21 @@ public class SentryStore {
         query = pm.newQuery(MSentryPrivilege.class);
         MSentryPrivilege mPrivilege = getMSentryPrivilege(constructPrivilegeName(tPrivilege), pm);
         if (mPrivilege == null) {
-          revokePartialPrivilege(pm, mRole, tPrivilege);
-          CommitContext commit = commitUpdateTransaction(pm);
-          rollbackTransaction = false;
-          return commit;
+          mPrivilege = convertToMSentryPrivilege(tPrivilege);
         } else {
-          // remove privilege and role objects from each other's set. needed by
-          // datanucleus to model m:n relationships correctly through a join table.
-          mRole.removePrivilege(mPrivilege);
-          mPrivilege.removeRole(mRole);
-          CommitContext commit = commitUpdateTransaction(pm);
-          rollbackTransaction = false;
-          return commit;
+          mPrivilege = (MSentryPrivilege)pm.detachCopy(mPrivilege);
+        }
 
+        Set<MSentryPrivilege> privilegeGraph = Sets.newHashSet(mPrivilege);
+        // Get the privilege graph
+        populateChildren(Sets.newHashSet(roleName), mPrivilege, privilegeGraph);
+        for (MSentryPrivilege childPriv : privilegeGraph) {
+          revokePartial(pm, tPrivilege, mRole, childPriv);
         }
+        pm.makePersistent(mRole);
+        CommitContext commit = commitUpdateTransaction(pm);
+        rollbackTransaction = false;
+        return commit;
       }
     } finally {
       if (rollbackTransaction) {
@@ -335,38 +368,124 @@ public class SentryStore {
     }
   }
 
+
   /**
-   * Our privilege model at present only allows ALL on server and databases.
-   * However, roles can be granted ALL, SELECT, and INSERT on tables. When
+   * Roles can be granted ALL, SELECT, and INSERT on tables. When
    * a role has ALL and SELECT or INSERT are revoked, we need to remove the ALL
    * privilege and add SELECT (INSERT was revoked) or INSERT (SELECT was revoked).
    */
-  private void revokePartialPrivilege(PersistenceManager pm, MSentryRole role,
-      TSentryPrivilege tPrivilege)
-          throws SentryNoSuchObjectException, SentryInvalidInputException {
-    // only perform partial revoke if INSERT/SELECT were the action
-    // and the privilege being revoked is on a table
-    String action = tPrivilege.getAction();
-    if (AccessConstants.ALL.equalsIgnoreCase(action) ||
-        StringUtils.isEmpty(tPrivilege.getDbName()) || StringUtils.isEmpty(tPrivilege.getTableName())) {
-      throw new SentryNoSuchObjectException("Unknown privilege: " + tPrivilege);
+  private void revokePartial(PersistenceManager pm,
+      TSentryPrivilege requestedPrivToRevoke, MSentryRole mRole,
+      MSentryPrivilege currentPrivilege) throws SentryInvalidInputException {
+    MSentryPrivilege persistedPriv = getMSentryPrivilege(constructPrivilegeName(convertToTSentryPrivilege(currentPrivilege)), pm);
+    if (persistedPriv == null) {
+      persistedPriv = convertToMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege));
     }
-    TSentryPrivilege tPrivilegeAll = new TSentryPrivilege(tPrivilege);
-    tPrivilegeAll.setAction(AccessConstants.ALL);
-    String allPrivilegeName = constructPrivilegeName(tPrivilegeAll);
-    MSentryPrivilege allPrivilege = getMSentryPrivilege(allPrivilegeName, pm);
-    if (allPrivilege == null) {
-      throw new SentryNoSuchObjectException("Unknown privilege: " + tPrivilege);
+
+    if (requestedPrivToRevoke.getAction().equalsIgnoreCase("ALL") || requestedPrivToRevoke.getAction().equalsIgnoreCase("*")) {
+      persistedPriv.removeRole(mRole);
+      pm.makePersistent(persistedPriv);
+    } else if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.SELECT)
+        && (!currentPrivilege.getAction().equalsIgnoreCase(AccessConstants.INSERT))) {
+      revokeRolePartial(pm, mRole, currentPrivilege, persistedPriv, AccessConstants.INSERT);
+    } else if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.INSERT)
+        && (!currentPrivilege.getAction().equalsIgnoreCase(AccessConstants.SELECT))) {
+      revokeRolePartial(pm, mRole, currentPrivilege, persistedPriv, AccessConstants.SELECT);
     }
-    role.removePrivilege(allPrivilege);
-    if (AccessConstants.SELECT.equalsIgnoreCase(action)) {
-      tPrivilege.setAction(AccessConstants.INSERT);
-    } else if (AccessConstants.INSERT.equalsIgnoreCase(action)) {
-      tPrivilege.setAction(AccessConstants.SELECT);
-    } else {
-      throw new IllegalStateException("Unexpected action: " + action);
+  }
+
+  private void revokeRolePartial(PersistenceManager pm, MSentryRole mRole,
+      MSentryPrivilege currentPrivilege, MSentryPrivilege persistedPriv, String addAction)
+      throws SentryInvalidInputException {
+    // If table / URI, remove ALL
+    persistedPriv.removeRole(mRole);
+    pm.makePersistent(persistedPriv);
+
+    currentPrivilege.setAction(AccessConstants.ALL);
+    persistedPriv = getMSentryPrivilege(constructPrivilegeName(convertToTSentryPrivilege(currentPrivilege)), pm);
+    if ((persistedPriv != null)&&(mRole.getPrivileges().contains(persistedPriv))) {
+      persistedPriv.removeRole(mRole);
+      pm.makePersistent(persistedPriv);
+
+      currentPrivilege.setAction(addAction);
+      persistedPriv = getMSentryPrivilege(constructPrivilegeName(convertToTSentryPrivilege(currentPrivilege)), pm);
+      if (persistedPriv == null) {
+        persistedPriv = convertToMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege));
+        mRole.appendPrivilege(persistedPriv);
+      }
+      persistedPriv.appendRole(mRole);
+      pm.makePersistent(persistedPriv);
+    }
+
+  }
+
+
+  /**
+   * Explore Privilege graph and collect child privileges
+   */
+  private void populateChildren(Set<String> roleNames, MSentryPrivilege priv,
+      Set<MSentryPrivilege> children) throws SentryInvalidInputException {
+    if ((priv.getServerName() != null) || (priv.getDbName() != null)) {
+      // Get all DBLevel Privs
+      Set<MSentryPrivilege> childPrivs = getChildPrivileges(roleNames, priv);
+      for (MSentryPrivilege childPriv : childPrivs) {
+        // Only recurse for db level privs..
+        if ((childPriv.getDbName() != null) && (childPriv.getTableName() == null)) {
+          populateChildren(roleNames, childPriv, children);
+        }
+        children.add(childPriv);
+      }
+    }
+  }
+
+  private Set<MSentryPrivilege> getChildPrivileges(Set<String> roleNames,
+      MSentryPrivilege parent) throws SentryInvalidInputException {
+    // Table and URI do not have children
+    if ((parent.getTableName() != null)||(parent.getURI() != null)) return new HashSet<MSentryPrivilege>();
+    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 == \"" + parent.getServerName() + "\"");
+      if (parent.getDbName() != null) {
+        filters.append(" && dbName == \"" + parent.getDbName() + "\"");
+        filters.append(" && tableName != null");
+      } else {
+        filters.append(" && (dbName != null || URI != null)");
+      }
+      query.setFilter(filters.toString());
+      query
+          .setResult("privilegeScope, serverName, dbName, tableName, URI, action, grantorPrincipal");
+      Set<MSentryPrivilege> privileges = new HashSet<MSentryPrivilege>();
+      for (Object[] privObj : (List<Object[]>) query.execute()) {
+        MSentryPrivilege priv = new MSentryPrivilege();
+        priv.setPrivilegeScope((String) privObj[0]);
+        priv.setServerName((String) privObj[1]);
+        priv.setDbName((String) privObj[2]);
+        priv.setTableName((String) privObj[3]);
+        priv.setURI((String) privObj[4]);
+        priv.setAction((String) privObj[5]);
+        priv.setGrantorPrincipal((String) privObj[6]);
+        priv.setPrivilegeName(constructPrivilegeName(convertToTSentryPrivilege(priv)));
+        privileges.add(priv);
+      }
+      rollbackTransaction = false;
+      commitTransaction(pm);
+      return privileges;
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
     }
-    role.appendPrivilege(convertToMSentryPrivilege(tPrivilege));
   }
 
   private MSentryPrivilege getMSentryPrivilege(String privilegeName, PersistenceManager pm) {
@@ -384,9 +503,9 @@ public class SentryStore {
   @VisibleForTesting
   public static String constructPrivilegeName(TSentryPrivilege privilege) throws SentryInvalidInputException {
     StringBuilder privilegeName = new StringBuilder();
-    String serverName = privilege.getServerName();
-    String dbName = privilege.getDbName();
-    String tableName = privilege.getTableName();
+    String serverName = safeTrimLower(privilege.getServerName());
+    String dbName = safeTrimLower(privilege.getDbName());
+    String tableName = safeTrimLower(privilege.getTableName());
     String uri = privilege.getURI();
     String action = privilege.getAction();
     PrivilegeScope scope;
@@ -921,7 +1040,6 @@ public class SentryStore {
     TSentryRole role = new TSentryRole();
     role.setRoleName(mSentryRole.getRoleName());
     role.setGrantorPrincipal(mSentryRole.getGrantorPrincipal());
-
     Set<TSentryGroup> sentryGroups = new HashSet<TSentryGroup>();
     for(MSentryGroup mSentryGroup:mSentryRole.getGroups()) {
       TSentryGroup group = convertToTSentryGroup(mSentryGroup);
@@ -971,13 +1089,13 @@ public class SentryStore {
     mSentryPrivilege.setPrivilegeName(constructPrivilegeName(privilege));
     return mSentryPrivilege;
   }
-  private String safeTrim(String s) {
+  private static String safeTrim(String s) {
     if (s == null) {
       return null;
     }
     return s.trim();
   }
-  private String safeTrimLower(String s) {
+  private static String safeTrimLower(String s) {
     if (s == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/5134a092/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDatabaseProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDatabaseProvider.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDatabaseProvider.java
index 6187692..27ef9ce 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDatabaseProvider.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDatabaseProvider.java
@@ -17,16 +17,13 @@
 
 package org.apache.sentry.tests.e2e.dbprovider;
 
-import org.apache.sentry.provider.db.SentryAccessDeniedException;
-import org.apache.sentry.provider.db.SentryAlreadyExistsException;
-import org.apache.sentry.provider.db.SentryNoSuchObjectException;
-import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
 import static org.hamcrest.Matchers.equalToIgnoringCase;
 import static org.hamcrest.Matchers.is;
-import org.junit.After;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -37,11 +34,17 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.junit.Before;
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
+import org.apache.sentry.provider.db.SentryAlreadyExistsException;
+import org.apache.sentry.provider.db.SentryNoSuchObjectException;
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.io.Resources;
+
 public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
 
   @BeforeClass
@@ -91,6 +94,7 @@ public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
     statement.execute("CREATE ROLE admin_role");
     statement.execute("GRANT ALL ON DATABASE default TO ROLE admin_role");
     statement.execute("GRANT ROLE admin_role TO GROUP " + ADMINGROUP);
+    statement.execute("DROP TABLE t1");
     statement.execute("CREATE TABLE t1 (c1 string)");
     statement.execute("CREATE ROLE user_role");
     statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
@@ -117,6 +121,719 @@ public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
   }
 
 
+  @Test
+  public void testRevokeDbALLAfterGrantTable() throws Exception {
+    doSetup();
+
+    // Revoke ALL on Db
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("REVOKE ALL ON DATABASE db1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("SELECT * FROM t1");
+    try {
+      statement.execute("SELECT * FROM db1.t2");
+      assertTrue("SELECT should not be allowed after revoke on parent!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    ResultSet resultSet = statement.executeQuery("SHOW GRANT ROLE user_role");
+    assertResultSize(resultSet, 1);
+    statement.close();
+    connection.close();
+  }
+
+  @Test
+  public void testRevokeServerAfterGrantTable() throws Exception {
+    doSetup();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    ResultSet resultSet = statement.executeQuery("SHOW GRANT ROLE user_role");
+    assertResultSize(resultSet, 2);
+    statement.close();
+    connection.close();;
+
+    // Revoke on Server
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("REVOKE ALL ON SERVER server1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    try {
+      statement.execute("SELECT * FROM t1");
+      assertTrue("SELECT should not be allowed after revoke on parent!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    try {
+      statement.execute("SELECT * FROM db1.t2");
+      assertTrue("SELECT should not be allowed after revoke on parent!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    resultSet = statement.executeQuery("SHOW GRANT ROLE user_role");
+    assertResultSize(resultSet, 0);
+    statement.close();
+    connection.close();
+  }
+
+
+  /**
+   * - Create db db1
+   * - Create role user_role
+   * - Create tables (t1, db1.t2)
+   * - Grant all on table t2 to user_role
+   * @throws Exception
+   */
+  private void doSetup() throws Exception {
+    super.setupAdmin();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+    statement.execute("DROP TABLE IF EXISTS t2");
+    statement.execute("CREATE TABLE t2 (c1 string)");
+    statement.execute("GRANT ALL ON TABLE t2 TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("SELECT * FROM t1");
+    statement.execute("SELECT * FROM db1.t2");
+
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * SENTRY-299
+   *
+   * 1. Create 2 Roles (user_role & user_role2)
+   * 2. Create a Table t1
+   * 3. grant ALL on t1 to user_role
+   * 4. grant INSERT on t1 to user_role2
+   * 5. Revoke INSERT on t1 from user_role
+   *     - This would imply user_role can still SELECT
+   *     - But user_role should NOT be allowed to LOAD
+   * 6. Ensure Presense of another role will still enforce the revoke
+   * @throws Exception
+   */
+
+  @Test
+  public void testRevokeFailAnotherRoleExist() throws Exception {
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+    statement.execute("CREATE ROLE user_role2");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role2");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role2");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role2");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.execute("GRANT ROLE user_role2 TO GROUP " + USERGROUP2);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("SELECT * FROM db1.t1");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    ResultSet resultSet = statement.executeQuery("SHOW GRANT ROLE user_role");
+    assertResultSize(resultSet, 2);
+    statement.close();
+    connection.close();
+
+    // Revoke ALL on Db
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE INSERT ON TABLE t1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // This Should pass
+    statement.execute("SELECT * FROM db1.t1");
+
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT Should Not be allowed since we Revoked INSERT privileges on the table !!", false);
+    } catch (Exception e) {
+
+    } finally {
+      statement.close();
+      connection.close();
+    }
+
+    // user_role2 can still insert into table
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+    statement.close();
+    connection.close();
+
+    // Grant changed from ALL to SELECT
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    resultSet = statement.executeQuery("SHOW GRANT ROLE user_role");
+    assertResultSize(resultSet, 2);
+    statement.close();
+    connection.close();
+
+  }
+
+
+  /**
+   * SENTRY-302
+   *
+   * 1. Create Role user_role
+   * 2. Create a Table t1
+   * 3. grant ALL on t1 to user_role
+   * 4. grant INSERT on t1 to user_role
+   * 5. Revoke INSERT on t1 from user_role
+   *     - This would imply user_role can still SELECT
+   *     - But user_role should NOT be allowed to LOAD
+   * 6. Ensure INSERT is revoked on table
+   * @throws Exception
+   */
+
+  @Test
+  public void testRevokeFailMultipleGrantsExist() throws Exception {
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("SELECT * FROM db1.t1");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+    // Revoke INSERT on Db
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE INSERT ON TABLE t1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // This Should pass
+    statement.execute("SELECT * FROM db1.t1");
+
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT Should Not be allowed since we Revoked INSERT privileges on the table !!", false);
+    } catch (Exception e) {
+
+    } finally {
+      statement.close();
+      connection.close();
+    }
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+  }
+
+
+  /**
+   * Revoke all on server after:
+   *  - grant all on db
+   *  - grant all on table
+   *  - grant select on table
+   *  - grant insert on table
+   *  - grant all on URI
+   *  - grant select on URI
+   *  - grant insert on URI
+   */
+  @Test
+  public void testRevokeAllOnServer() throws Exception{
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+
+    statement.execute("GRANT ALL ON DATABASE db1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT SELECT ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT INSERT ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure everything works
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 3);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE ALL ON SERVER server1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure nothing works
+    try {
+      statement.execute("SELECT * FROM db1.t1");
+      assertTrue("SELECT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 0);
+    statement.close();
+    connection.close();
+  }
+
+
+  /**
+   * Revoke all on database after:
+   *  - grant all on db
+   *  - grant all on table
+   *  - grant select on table
+   *  - grant insert on table
+   */
+  @Test
+  public void testRevokeAllOnDb() throws Exception{
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+
+    statement.execute("GRANT ALL ON DATABASE db1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 3);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure everything works
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE ALL ON DATABASE db1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure nothing works
+    try {
+      statement.execute("SELECT * FROM db1.t1");
+      assertTrue("SELECT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 1);
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * Revoke all on table after:
+   *  - grant all on table
+   *  - grant select on table
+   *  - grant insert on table
+   */
+  @Test
+  public void testRevokeAllOnTable() throws Exception{
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure everything works
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE ALL ON TABLE t1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure nothing works
+    try {
+      statement.execute("SELECT * FROM db1.t1");
+      assertTrue("SELECT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 1);
+    statement.close();
+    connection.close();
+  }
+
+  /**
+   * Revoke select on table after:
+   *  - grant all on table
+   *  - grant select on table
+   *  - grant insert on table
+   */
+  @Test
+  public void testRevokeSELECTOnTable() throws Exception{
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure everything works
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE SELECT ON TABLE t1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure select not allowed
+    try {
+      statement.execute("SELECT * FROM db1.t1");
+      assertTrue("SELECT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+
+    // Ensure insert allowed
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+    statement.close();
+    connection.close();
+
+    // This removes the ALL and SELECT privileges
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+  }
+
+
+  /**
+   * Revoke insert on table after:
+   *  - grant all on table
+   *  - grant select on table
+   *  - grant insert on table
+   */
+  @Test
+  public void testRevokeINSERTOnTable() throws Exception{
+    super.setupAdmin();
+
+    //copy data file to test dir
+    File dataDir = context.getDataDir();
+    File dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE user_role");
+
+    statement.execute("DROP DATABASE IF EXISTS db1 CASCADE");
+    statement.execute("CREATE DATABASE db1");
+    statement.execute("USE db1");
+
+    statement.execute("DROP TABLE IF EXISTS t1");
+    statement.execute("CREATE TABLE t1 (c1 string)");
+
+    statement.execute("GRANT ALL ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT INSERT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ALL ON URI 'file://" + dataFile.getPath() + "' TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure everything works
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("REVOKE INSERT ON TABLE t1 from ROLE user_role");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    // Ensure insert not allowed
+    try {
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("INSERT should not be allowed !!", false);
+    } catch (SQLException se) {
+      // Ignore
+    }
+
+    // Ensure select allowed
+    statement.execute("SELECT * FROM db1.t1");
+    statement.close();
+    connection.close();
+
+    // This removes the INSERT and ALL privileges
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    assertResultSize(statement.executeQuery("SHOW GRANT ROLE user_role"), 2);
+    statement.close();
+    connection.close();
+  }
+
+
+
+
   /**
    * Grant/Revoke privilege - Positive cases
    * @throws Exception
@@ -433,7 +1150,7 @@ public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
     statement.execute("GRANT ALL ON TABLE tab1 to ROLE role2");
     statement.execute("GRANT ALL,INSERT ON TABLE tab1 to ROLE role2");
     resultSet = statement.executeQuery("SHOW GRANT ROLE role2");
-    assertResultSize(resultSet, 3);
+    assertResultSize(resultSet, 2);
     statement.execute("DROP role role2");
 
     //Revoke privilege when privilege doesnt exist

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/5134a092/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 46d9332..9e97b21 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
@@ -17,8 +17,6 @@
 
 package org.apache.sentry.tests.e2e.dbprovider;
 
-import org.apache.sentry.provider.db.SentryAccessDeniedException;
-import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -26,9 +24,12 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.sql.Connection;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
 
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
 import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -141,7 +142,9 @@ public class TestDbEndToEnd extends AbstractTestWithStaticConfiguration {
 
   }
 
-  /**
+
+
+/**
    * Steps:
    * 1. admin create a new experimental database
    * 2. admin create a new production database, create table, load data