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/19 04:11:15 UTC

git commit: SENTRY-303: Allow users to grant/revoke SELECT/INSERT to ALL tables in a Database ( Arun Suresh via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master eefdec94a -> f42bc7734


SENTRY-303: Allow users to grant/revoke SELECT/INSERT to ALL tables in a Database ( 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/f42bc773
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/f42bc773
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/f42bc773

Branch: refs/heads/master
Commit: f42bc7734c709ad5d96160271f8e62660574e133
Parents: eefdec9
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Wed Jun 18 19:10:12 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Wed Jun 18 19:10:12 2014 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/SentryGrantRevokeTask.java     |  29 ++++-
 .../sentry/policy/db/DBWildcardPrivilege.java   |   7 +-
 .../db/service/persistent/SentryStore.java      |   7 +-
 .../thrift/SentryPolicyServiceClient.java       |  10 +-
 .../thrift/TestSentryServerWithoutKerberos.java |   9 +-
 .../thrift/TestSentryServiceIntegration.java    |   5 +-
 .../e2e/dbprovider/TestDatabaseProvider.java    | 110 +++++++++++++++++++
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java |  29 +++++
 8 files changed, 187 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index f29078a..0b65008 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -278,7 +278,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   private int processGrantDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       String server, GrantDesc desc) throws SentryUserException {
-    return processGrantRevokeDDL(console, sentryClient, subject, 
+    return processGrantRevokeDDL(console, sentryClient, subject,
         server, true, desc.getPrincipals(), desc.getPrivileges(), desc.getPrivilegeSubjectDesc());
   }
 
@@ -286,7 +286,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   private int processRevokeDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       String server, RevokeDesc desc) throws SentryUserException {
-    return processGrantRevokeDDL(console, sentryClient, subject, 
+    return processGrantRevokeDDL(console, sentryClient, subject,
         server, false, desc.getPrincipals(), desc.getPrivileges(),
         desc.getPrivilegeSubjectDesc());
   }
@@ -539,7 +539,8 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             } else if (uriPath != null) {
               sentryClient.grantURIPrivilege(subject, princ.getName(), server, uriPath);
             } else if (tableName == null) {
-              sentryClient.grantDatabasePrivilege(subject, princ.getName(), server, dbName);
+              sentryClient.grantDatabasePrivilege(subject, princ.getName(), server, dbName,
+                  toDbSentryAction(privDesc.getPrivilege().getPriv()));
             } else {
               sentryClient.grantTablePrivilege(subject, princ.getName(), server, dbName,
                   tableName, toSentryAction(privDesc.getPrivilege().getPriv()));
@@ -550,7 +551,8 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
             } else if (uriPath != null) {
               sentryClient.revokeURIPrivilege(subject, princ.getName(), server, uriPath);
             } else if (tableName == null) {
-              sentryClient.revokeDatabasePrivilege(subject, princ.getName(), server, dbName);
+              sentryClient.revokeDatabasePrivilege(subject, princ.getName(), server, dbName,
+                  toDbSentryAction(privDesc.getPrivilege().getPriv()));
             } else {
               sentryClient.revokeTablePrivilege(subject, princ.getName(), server, dbName,
                   tableName, toSentryAction(privDesc.getPrivilege().getPriv()));
@@ -567,6 +569,25 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     }
   }
 
+  private static String toDbSentryAction(PrivilegeType privilegeType) {
+    if (PrivilegeType.ALL.equals(privilegeType)) {
+      return AccessConstants.ALL;
+    } else {
+      if (PrivilegeType.SELECT.equals(privilegeType)) {
+        return AccessConstants.SELECT;
+      } else if (PrivilegeType.INSERT.equals(privilegeType)) {
+        return AccessConstants.INSERT;
+      } else {
+        // Should we throw an Exception here ?
+        // On second thought... I don't think we should..
+        // Earlier, we were sending everything as ALL..
+        // So with the patch, it should default to old
+        // behavior for something other than INSERT or SELECT
+        return AccessConstants.ALL;
+      }
+    }
+  }
+
   private static SentryHivePrivilegeObjectDesc toSentryHivePrivilegeObjectDesc(PrivilegeObjectDesc privSubjectObjDesc)
     throws HiveException{
     if (!(privSubjectObjDesc instanceof SentryHivePrivilegeObjectDesc)) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/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 d599de9..896283c 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
@@ -93,6 +93,10 @@ public class DBWildcardPrivilege implements Privilege {
         return true;
       } else {
         KeyValue part = parts.get(index);
+        // Support for action inheritance from parent to child (eg. Db -> Table)
+        if (part.getKey().equalsIgnoreCase("action") && !(otherPart.getKey().equalsIgnoreCase("action"))) {
+          continue;
+        }
         // are the keys even equal
         if(!part.getKey().equalsIgnoreCase(otherPart.getKey())) {
           return false;
@@ -119,7 +123,8 @@ public class DBWildcardPrivilege implements Privilege {
   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)) {
+    if(policyPart.getValue().equals(AccessConstants.ALL) ||
+        policyPart.getValue().equalsIgnoreCase("ALL") || policyPart.equals(requestPart)) {
       return true;
     } else if (!PolicyFileConstants.PRIVILEGE_NAME.equalsIgnoreCase(policyPart.getKey())
         && AccessConstants.ALL.equalsIgnoreCase(requestPart.getValue())) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/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 56fc3df..fb8cfc2 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
@@ -276,7 +276,7 @@ public class SentryStore {
         throw new SentryNoSuchObjectException("Role: " + roleName);
       } else {
 
-        if (privilege.getTableName() != null) {
+        if ((privilege.getTableName() != null)||(privilege.getDbName() != null)) {
           // If Grant is for ALL and Either INSERT/SELECT already exists..
           // need to remove it and GRANT ALL..
           if (privilege.getAction().equalsIgnoreCase("*")) {
@@ -516,8 +516,9 @@ public class SentryStore {
 
     if (AccessConstants.SELECT.equalsIgnoreCase(action) ||
         AccessConstants.INSERT.equalsIgnoreCase(action)) {
-      if (Strings.nullToEmpty(tableName).trim().isEmpty()) {
-        throw new SentryInvalidInputException("Table name can't be null for SELECT/INSERT privilege");
+      if (Strings.nullToEmpty(tableName).trim().isEmpty()
+          &&Strings.nullToEmpty(dbName).trim().isEmpty()) {
+        throw new SentryInvalidInputException("Either Table name or Db name must be NON-NULL for SELECT/INSERT privilege");
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
index 9e2c200..27f617f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
@@ -231,7 +231,7 @@ public class SentryPolicyServiceClient {
     }
   }
 
-  public Set<TSentryPrivilege> listAllPrivilegesByRoleName(String requestorUserName, String roleName) 
+  public Set<TSentryPrivilege> listAllPrivilegesByRoleName(String requestorUserName, String roleName)
 		  throws SentryUserException {
     return listPrivilegesByRoleName(requestorUserName, roleName, null);
   }
@@ -298,10 +298,10 @@ public class SentryPolicyServiceClient {
   }
 
   public void grantDatabasePrivilege(String requestorUserName,
-      String roleName, String server, String db)
+      String roleName, String server, String db, String action)
   throws SentryUserException {
     grantPrivilege(requestorUserName, roleName,
-        PrivilegeScope.DATABASE, server, null, db, null, AccessConstants.ALL);
+        PrivilegeScope.DATABASE, server, null, db, null, action);
   }
 
   public void grantTablePrivilege(String requestorUserName,
@@ -352,10 +352,10 @@ public class SentryPolicyServiceClient {
   }
 
   public void revokeDatabasePrivilege(String requestorUserName,
-      String roleName, String server, String db)
+      String roleName, String server, String db, String action)
   throws SentryUserException {
     revokePrivilege(requestorUserName, roleName,
-        PrivilegeScope.DATABASE, server, null, db, null, AccessConstants.ALL);
+        PrivilegeScope.DATABASE, server, null, db, null, action);
   }
 
   public void revokeTablePrivilege(String requestorUserName,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
index 9f89302..79579c6 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import junit.framework.Assert;
 
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
@@ -53,7 +54,7 @@ public class TestSentryServerWithoutKerberos extends SentryServiceIntegrationBas
     client.createRole(requestorUserName, roleName);
     client.dropRole(requestorUserName, roleName);
   }
-  
+
   @Test
   public void testQueryPushDown() throws Exception {
     String requestorUserName = ADMIN_USER;
@@ -115,7 +116,7 @@ public class TestSentryServerWithoutKerberos extends SentryServiceIntegrationBas
     Assert.assertEquals("Privilege not correctly assigned to roles !!", new HashSet<String>(), listPrivilegesForProvider);
   }
 
-  
+
 
   /**
    * Create role, add privileges and grant it to a group drop the role and
@@ -135,7 +136,7 @@ public class TestSentryServerWithoutKerberos extends SentryServiceIntegrationBas
     client.dropRoleIfExists(requestorUserName, roleName);
     client.createRole(requestorUserName, roleName);
     client.grantRoleToGroup(requestorUserName, ADMIN_GROUP, roleName);
-    client.grantDatabasePrivilege(requestorUserName, roleName, "server1", "db2");
+    client.grantDatabasePrivilege(requestorUserName, roleName, "server1", "db2", AccessConstants.ALL);
     client.grantTablePrivilege(requestorUserName, roleName, "server1", "db3", "tab3", "ALL");
     assertEquals(2, client.listPrivilegesForProvider(requestorUserGroupNames,
             ActiveRoleSet.ALL).size());
@@ -152,7 +153,7 @@ public class TestSentryServerWithoutKerberos extends SentryServiceIntegrationBas
             ActiveRoleSet.ALL).size());
 
     // grant different privileges and verify
-    client.grantDatabasePrivilege(requestorUserName, roleName, "server1", "db2");
+    client.grantDatabasePrivilege(requestorUserName, roleName, "server1", "db2", AccessConstants.ALL);
     assertEquals(1, client.listPrivilegesForProvider(requestorUserGroupNames,
             ActiveRoleSet.ALL).size());
     client.dropRole(requestorUserName, roleName);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
index f0bf127..a4ae291 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Set;
 
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.service.thrift.SentryServiceIntegrationBase;
 import org.junit.Test;
@@ -182,7 +183,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
     Set<TSentryRole> roles = client.listRoles(requestorUserName);
     assertEquals("Incorrect number of roles", 1, roles.size());
 
-    client.grantDatabasePrivilege(requestorUserName, roleName, server, db);
+    client.grantDatabasePrivilege(requestorUserName, roleName, server, db, AccessConstants.ALL);
     Set<TSentryPrivilege> privileges = client.listAllPrivilegesByRoleName(requestorUserName, roleName);
     assertTrue(privileges.size() == 1);
     for (TSentryPrivilege privilege:privileges) {
@@ -190,7 +191,7 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
         privilege.getPrivilegeName().equalsIgnoreCase(SentryStore.constructPrivilegeName(privilege)));
     }
 
-    client.revokeDatabasePrivilege(requestorUserName, roleName, server, db);
+    client.revokeDatabasePrivilege(requestorUserName, roleName, server, db, AccessConstants.ALL);
     client.dropRole(requestorUserName, roleName);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/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 27ef9ce..52c9e9e 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
@@ -121,6 +121,116 @@ public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
   }
 
 
+  // SENTRY-303 tests
+  @Test
+  public void testGrantSELECTonDb() throws Exception {
+    File dataFile = doSetupForGrantDbTests();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+
+    // Grant only SELECT on Database
+    statement.execute("GRANT SELECT ON DATABASE db1 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);
+    // SELECT is allowed
+    statement.execute("SELECT * FROM db1.t1");
+    statement.execute("SELECT * FROM db1.t2");
+    try {
+      // INSERT is not allowed
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+      assertTrue("only SELECT allowed on t1!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    try {
+      // INSERT is not allowed
+      statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t2");
+      assertTrue("only SELECT allowed on t2!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+  }
+
+  @Test
+  public void testGrantINSERTonDb() throws Exception {
+    File dataFile = doSetupForGrantDbTests();
+
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+
+    // Grant only INSERT on Database
+    statement.execute("GRANT INSERT ON DATABASE db1 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);
+    // INSERT is allowed
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t1");
+    statement.execute("LOAD DATA LOCAL INPATH '" + dataFile.getPath() + "' INTO TABLE db1.t2");
+    try {
+      // SELECT is not allowed
+      statement.execute("SELECT * FROM db1.t1");
+      assertTrue("only SELECT allowed on t1!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    try {
+      // SELECT is not allowed
+      statement.execute("SELECT * FROM db1.t2");
+      assertTrue("only INSERT allowed on t2!!", false);
+    } catch (Exception e) {
+      // Ignore
+    }
+    statement.close();
+    connection.close();
+  }
+
+  private File doSetupForGrantDbTests() 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);
+    try {
+      statement.execute("DROP ROLE user_role");
+    } catch (Exception e) {
+      // Ignore
+    }
+    try {
+      statement.execute("CREATE ROLE user_role");
+    } catch (Exception e) {
+      // Ignore
+    }
+    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("DROP TABLE IF EXISTS t2");
+    statement.execute("CREATE TABLE t2 (c2 string)");
+    statement.close();
+    connection.close();
+
+    return dataFile;
+  }
+
   @Test
   public void testRevokeDbALLAfterGrantTable() throws Exception {
     doSetup();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/f42bc773/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 ff73c0a..c9658ab 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
@@ -55,6 +55,35 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
     policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
   }
 
+  // SENTRY-285 test
+  @Test
+  public void testAllOnDb() throws Exception {
+
+    policyFile
+        .addRolesToGroup(USERGROUP1, "all_db1")
+        .addPermissionsToRole("all_db1", "server=server1->db=DB_1->action=all")
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping());
+    writePolicyFile(policyFile);
+
+    // setup db objects needed by the test
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("create database db_1");
+    statement.execute("create table db_1.tab1(a int)");
+
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("use db_1");
+    statement.execute("select * from tab1");
+
+    policyFile
+        .addPermissionsToRole("all_db1", "server=server1->db=DB_1");
+    writePolicyFile(policyFile);
+    statement.execute("use db_1");
+    statement.execute("select * from tab1");
+  }
+
+
   /* Admin creates database DB_1
    * Admin grants ALL to USER_GROUP of which user1 is a member.
    */