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/24 00:32:25 UTC

git commit: SENTRY-311: Metastore plugin needs to be changed to updated privilege model ( Prasad Mujumdar via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 0ebbb3775 -> 5df60185b


SENTRY-311: Metastore plugin needs to be changed to updated privilege model ( Prasad Mujumdar 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/5df60185
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/5df60185
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/5df60185

Branch: refs/heads/master
Commit: 5df60185b402b2246590151252b38e178d46f28e
Parents: 0ebbb37
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Mon Jun 23 15:31:32 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Mon Jun 23 15:31:32 2014 -0700

----------------------------------------------------------------------
 .../metastore/MetastoreAuthzBinding.java        |  84 +++++++---
 .../e2e/metastore/TestMetastoreEndToEnd.java    | 167 ++++++++++++++-----
 2 files changed, 187 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/5df60185/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBinding.java
index ccf6a67..5d7d9a4 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBinding.java
@@ -92,6 +92,7 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
 
     public HierarcyBuilder addDbToOutput(Server server, String dbName) {
       List<DBModelAuthorizable> dbHierarchy = new ArrayList<DBModelAuthorizable>();
+      addServerToOutput(server);
       dbHierarchy.add(server);
       dbHierarchy.add(new Database(dbName));
       authHierarchy.add(dbHierarchy);
@@ -100,6 +101,7 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
 
     public HierarcyBuilder addUriToOutput(Server server, String uriPath) {
       List<DBModelAuthorizable> uriHierarchy = new ArrayList<DBModelAuthorizable>();
+      addServerToOutput(server);
       uriHierarchy.add(server);
       uriHierarchy.add(new AccessURI(uriPath));
       authHierarchy.add(uriHierarchy);
@@ -109,6 +111,7 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
     public HierarcyBuilder addTableToOutput(Server server, String dbName,
         String tableName) {
       List<DBModelAuthorizable> tableHierarchy = new ArrayList<DBModelAuthorizable>();
+      addDbToOutput(server, dbName);
       tableHierarchy.add(server);
       tableHierarchy.add(new Database(dbName));
       tableHierarchy.add(new Table(tableName));
@@ -197,20 +200,24 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
   private void authorizeCreateDatabase(PreCreateDatabaseEvent context)
       throws InvalidOperationException, MetaException {
     authorizeMetastoreAccess(HiveOperation.CREATEDATABASE,
-        new HierarcyBuilder().build(),
+        new HierarcyBuilder().addServerToOutput(getAuthServer()).build(),
         new HierarcyBuilder().addServerToOutput(getAuthServer()).build());
   }
 
   private void authorizeDropDatabase(PreDropDatabaseEvent context)
       throws InvalidOperationException, MetaException {
     authorizeMetastoreAccess(HiveOperation.DROPDATABASE,
-        new HierarcyBuilder().build(),
-        new HierarcyBuilder().addServerToOutput(getAuthServer()).build());
+ new HierarcyBuilder()
+.addDbToOutput(getAuthServer(),
+            context.getDatabase().getName()).build(),
+        new HierarcyBuilder().addDbToOutput(getAuthServer(),
+            context.getDatabase().getName()).build());
   }
 
   private void authorizeCreateTable(PreCreateTableEvent context)
       throws InvalidOperationException, MetaException {
     HierarcyBuilder inputBuilder = new HierarcyBuilder();
+    inputBuilder.addDbToOutput(getAuthServer(), context.getTable().getDbName());
     if (!StringUtils.isEmpty(context.getTable().getSd().getLocation())) {
       String uriPath;
       try {
@@ -230,15 +237,27 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
       throws InvalidOperationException, MetaException {
     authorizeMetastoreAccess(
         HiveOperation.DROPTABLE,
-        new HierarcyBuilder().build(),
-        new HierarcyBuilder().addDbToOutput(getAuthServer(),
-            context.getTable().getDbName()).build());
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getTable().getDbName(), context.getTable().getTableName())
+            .build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getTable().getDbName(), context.getTable().getTableName())
+            .build());
   }
 
   private void authorizeAlterTable(PreAlterTableEvent context)
       throws InvalidOperationException, MetaException {
-
+    /*
+     * There are multiple alter table options and it's tricky to figure which is
+     * attempted here. Currently all alter table needs full level privilege
+     * except the for setting location which also needs a privile on URI. Hence
+     * we set initially set the operation to ALTERTABLE_ADDCOLS. If the client
+     * has specified the location, then change to ALTERTABLE_LOCATION
+     */
+    HiveOperation operation = HiveOperation.ALTERTABLE_ADDCOLS;
     HierarcyBuilder inputBuilder = new HierarcyBuilder();
+    inputBuilder.addTableToOutput(getAuthServer(), context.getOldTable()
+        .getDbName(), context.getOldTable().getTableName());
     // if the operation requires location change, then add URI privilege check
     String oldLocationUri;
     String newLocationUri;
@@ -252,17 +271,21 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
     }
     if (oldLocationUri.compareTo(newLocationUri) != 0) {
       inputBuilder.addUriToOutput(getAuthServer(), newLocationUri);
+      operation = HiveOperation.ALTERTABLE_LOCATION;
     }
     authorizeMetastoreAccess(
-        HiveOperation.ALTERTABLE_ADDCOLS, inputBuilder.build(),
-        new HierarcyBuilder().addDbToOutput(getAuthServer(),
-            context.getOldTable().getDbName()).build());
+        operation,
+        inputBuilder.build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getOldTable().getDbName(),
+            context.getOldTable().getTableName()).build());
   }
 
   private void authorizeAddPartition(PreAddPartitionEvent context)
       throws InvalidOperationException, MetaException, NoSuchObjectException {
     HierarcyBuilder inputBuilder = new HierarcyBuilder();
-
+    inputBuilder.addTableToOutput(getAuthServer(), context.getPartition()
+        .getDbName(), context.getPartition().getTableName());
     // check if we need to validate URI permissions when storage location is
     // non-default, ie something not under the parent table
     String partitionLocation = context.getPartition().getSd().getLocation();
@@ -271,7 +294,6 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
           .getHandler()
           .get_table(context.getPartition().getDbName(),
               context.getPartition().getTableName()).getSd().getLocation();
-
       String uriPath;
       try {
         uriPath = PathUtils.parseDFSURI(warehouseDir, context.getPartition()
@@ -285,25 +307,49 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
     }
     authorizeMetastoreAccess(HiveOperation.ALTERTABLE_ADDPARTS,
         inputBuilder.build(),
-        new HierarcyBuilder().addDbToOutput(getAuthServer(),
-            context.getPartition().getDbName()).build());
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getPartition().getDbName(),
+            context.getPartition().getTableName()).build());
   }
 
   private void authorizeDropPartition(PreDropPartitionEvent context)
       throws InvalidOperationException, MetaException {
     authorizeMetastoreAccess(
         HiveOperation.ALTERTABLE_DROPPARTS,
-        new HierarcyBuilder().build(),
-        new HierarcyBuilder().addDbToOutput(getAuthServer(),
-            context.getPartition().getDbName()).build());
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getPartition().getDbName(),
+            context.getPartition().getTableName()).build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getPartition().getDbName(),
+            context.getPartition().getTableName()).build());
   }
 
   private void authorizeAlterPartition(PreAlterPartitionEvent context)
       throws InvalidOperationException, MetaException {
+    /*
+     * There are multiple alter partition options and it's tricky to figure out
+     * which is attempted here. Currently all alter partition need full level
+     * privilege except the for setting location which also needs a privilege on
+     * URI. Currently we don't try to distinguish the operation type. All alter
+     * partitions are treated as set-location
+     */
+    HierarcyBuilder inputBuilder = new HierarcyBuilder().addTableToOutput(
+        getAuthServer(), context.getDbName(), context.getTableName());
+    String partitionLocation = context.getNewPartition().getSd().getLocation();
+    if (!StringUtils.isEmpty(partitionLocation)) {
+      String uriPath;
+      try {
+        uriPath = PathUtils.parseDFSURI(warehouseDir, partitionLocation);
+      } catch (URISyntaxException e) {
+        throw new MetaException(e.getMessage());
+      }
+      inputBuilder.addUriToOutput(getAuthServer(), uriPath);
+    }
     authorizeMetastoreAccess(
         HiveOperation.ALTERPARTITION_LOCATION,
-        new HierarcyBuilder().build(),
-        new HierarcyBuilder().addServerToOutput(getAuthServer()).build());
+        inputBuilder.build(),
+        new HierarcyBuilder().addTableToOutput(getAuthServer(),
+            context.getDbName(), context.getTableName()).build());
   }
 
   private InvalidOperationException invalidOperationException(Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/5df60185/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetastoreEndToEnd.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetastoreEndToEnd.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetastoreEndToEnd.java
index bae0213..52a2b1e 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetastoreEndToEnd.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetastoreEndToEnd.java
@@ -46,6 +46,12 @@ public class TestMetastoreEndToEnd extends
   private static final String dbName = "db_1";
   private static final String db_all_role = "all_db1";
   private static final String uri_role = "uri_role";
+  private static final String tab1_all_role = "tab1_all_role";
+  private static final String tab1_read_role = "tab1_read_role";
+  private static final String tab2_all_role = "tab2_all_role";
+  private static final String tab2_read_role = "tab2_read_role";
+  private static final String tabName1 = "tab1";
+  private static final String tabName2 = "tab2";
 
   @Before
   public void setup() throws Exception {
@@ -53,9 +59,21 @@ public class TestMetastoreEndToEnd extends
     policyFile
         .addRolesToGroup(USERGROUP1, db_all_role)
         .addRolesToGroup(USERGROUP2, "read_db_role")
+        .addRolesToGroup(USERGROUP2, tab1_all_role)
+        .addRolesToGroup(USERGROUP2, tab2_all_role)
+        .addRolesToGroup(USERGROUP3, tab1_read_role)
+        .addRolesToGroup(USERGROUP3, tab2_read_role)
         .addPermissionsToRole(db_all_role, "server=server1->db=" + dbName)
         .addPermissionsToRole("read_db_role",
             "server=server1->db=" + dbName + "->table=*->action=SELECT")
+        .addPermissionsToRole(tab1_all_role,
+            "server=server1->db=" + dbName + "->table=" + tabName1)
+        .addPermissionsToRole(tab2_all_role,
+            "server=server1->db=" + dbName + "->table=" + tabName2)
+        .addPermissionsToRole(tab1_read_role,
+            "server=server1->db=" + dbName + "->table=" + tabName1 + "->action=SELECT")
+        .addPermissionsToRole(tab2_read_role,
+            "server=server1->db=" + dbName + "->table=" + tabName2 + "->action=SELECT")
         .setUserGroupMapping(StaticUserGroup.getStaticMapping());
     writePolicyFile(policyFile);
 
@@ -105,8 +123,8 @@ public class TestMetastoreEndToEnd extends
       Context.verifyMetastoreAuthException(e);
     }
     try {
-      client.dropDatabase(dbName, true, true, true);
-      fail("drop db should have failed for non-admin user");
+      createMetastoreDB(client, "barDb");
+      fail("create db should have failed for non-admin user");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
     }
@@ -119,9 +137,6 @@ public class TestMetastoreEndToEnd extends
    */
   @Test
   public void testTablePrivileges() throws Exception {
-    String tabName1 = "tab1";
-    String tabName2 = "tab2";
-
     HiveMetaStoreClient client = context.getMetaStoreClient(ADMIN1);
     createMetastoreTable(client, dbName, tabName1,
         Lists.newArrayList(new FieldSchema("col1", "int", "")));
@@ -132,8 +147,11 @@ public class TestMetastoreEndToEnd extends
         Lists.newArrayList(new FieldSchema("col1", "int", "")));
     assertEquals(1, client.getTables(dbName, tabName2).size());
     client.dropTable(dbName, tabName1);
+    createMetastoreTable(client, dbName, tabName1,
+        Lists.newArrayList(new FieldSchema("col1", "int", "")));
     client.close();
 
+    // group2 users can't create the table, but can drop it
     client = context.getMetaStoreClient(USER2_1);
     try {
       createMetastoreTable(client, dbName, "barTab",
@@ -142,9 +160,21 @@ public class TestMetastoreEndToEnd extends
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
     }
+    client.dropTable(dbName, tabName2);
+    client.close();
+
+    // group3 users can't create or drop it
+    client = context.getMetaStoreClient(USER3_1);
+    try {
+      createMetastoreTable(client, dbName, "barTab",
+          Lists.newArrayList(new FieldSchema("col1", "int", "")));
+      fail("Create table should have failed for non-privilege user");
+    } catch (MetaException e) {
+      Context.verifyMetastoreAuthException(e);
+    }
 
     try {
-      client.dropTable(dbName, tabName2);
+      client.dropTable(dbName, tabName1);
       fail("drop table should have failed for non-privilege user");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
@@ -158,7 +188,6 @@ public class TestMetastoreEndToEnd extends
    */
   @Test
   public void testAlterTablePrivileges() throws Exception {
-    String tabName1 = "tab1";
 
     HiveMetaStoreClient client = context.getMetaStoreClient(ADMIN1);
     createMetastoreTable(client, dbName, tabName1,
@@ -174,11 +203,20 @@ public class TestMetastoreEndToEnd extends
     Table metaTable3 = client.getTable(dbName, tabName1);
     assertEquals(metaTable2, metaTable3);
 
-    // verify group2 users can't alter tables in db_1
+    // verify group1 users with DDL privileges can alter tables in db_1
     client = context.getMetaStoreClient(USER2_1);
     metaTable2 = client.getTable(dbName, tabName1);
     metaTable2.getSd().setCols(
         Lists.newArrayList(new FieldSchema("col3", "string", "")));
+    client.alter_table(dbName, tabName1, metaTable2);
+    metaTable3 = client.getTable(dbName, tabName1);
+    assertEquals(metaTable2, metaTable3);
+
+    // verify group3 users can't alter tables in db_1
+    client = context.getMetaStoreClient(USER3_1);
+    metaTable2 = client.getTable(dbName, tabName1);
+    metaTable2.getSd().setCols(
+        Lists.newArrayList(new FieldSchema("col3", "string", "")));
     try {
       client.alter_table(dbName, tabName1, metaTable2);
       fail("alter table should have failed for non-privilege user");
@@ -194,25 +232,32 @@ public class TestMetastoreEndToEnd extends
    */
   @Test
   public void testAddPartitionPrivileges() throws Exception {
-    String tabName = "tab1";
     ArrayList<String> partVals1 = Lists.newArrayList("part1");
     ArrayList<String> partVals2 = Lists.newArrayList("part2");
-    ArrayList<String> partVals3 = Lists.newArrayList("part2");
+    ArrayList<String> partVals3 = Lists.newArrayList("part3");
+    ArrayList<String> partVals4 = Lists.newArrayList("part4");
 
     // user with ALL on DB should be able to add partition
     HiveMetaStoreClient client = context.getMetaStoreClient(USER1_1);
-    Table tbl1 = createMetastoreTableWithPartition(client, dbName,
-        tabName, Lists.newArrayList(new FieldSchema("col1", "int", "")),
+    Table tbl1 = createMetastoreTableWithPartition(client, dbName, tabName1,
+        Lists.newArrayList(new FieldSchema("col1", "int", "")),
         Lists.newArrayList(new FieldSchema("part_col1", "string", "")));
-    assertEquals(1, client.getTables(dbName, tabName).size());
-    addPartition(client, dbName, tabName, partVals1, tbl1);
-    addPartition(client, dbName, tabName, partVals2, tbl1);
+    assertEquals(1, client.getTables(dbName, tabName1).size());
+    addPartition(client, dbName, tabName1, partVals1, tbl1);
+    addPartition(client, dbName, tabName1, partVals2, tbl1);
     client.close();
 
-    // user without ALL on DB should NOT be able to add partition
+    // user with ALL on Table should be able to add partition
     client = context.getMetaStoreClient(USER2_1);
+    tbl1 = client.getTable(dbName, tabName1);
+    addPartition(client, dbName, tabName1, partVals3, tbl1);
+    client.close();
+
+    // user without ALL on DB or Table should NOT be able to add partition
+    client = context.getMetaStoreClient(USER3_1);
+    tbl1 = client.getTable(dbName, tabName1);
     try {
-      addPartition(client, dbName, tabName, partVals3, tbl1);
+      addPartition(client, dbName, tabName1, partVals4, tbl1);
       fail("Add partition should have failed for non-admin user");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
@@ -221,14 +266,20 @@ public class TestMetastoreEndToEnd extends
 
     // user with ALL on DB should be able to drop partition
     client = context.getMetaStoreClient(USER1_1);
-    tbl1 = client.getTable(dbName, tabName);
-    client.dropPartition(dbName, tabName, partVals1, true);
+    tbl1 = client.getTable(dbName, tabName1);
+    client.dropPartition(dbName, tabName1, partVals1, true);
     client.close();
 
-    // user without ALL on DB should NOT be able to drop partition
+    // user with ALL on Table should be able to drop partition
     client = context.getMetaStoreClient(USER2_1);
+    tbl1 = client.getTable(dbName, tabName1);
+    client.dropPartition(dbName, tabName1, partVals2, true);
+    client.close();
+
+    // user without ALL on DB or Table should NOT be able to drop partition
+    client = context.getMetaStoreClient(USER3_1);
     try {
-      addPartition(client, dbName, tabName, partVals2, tbl1);
+      addPartition(client, dbName, tabName1, partVals3, tbl1);
       fail("Drop partition should have failed for non-admin user");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
@@ -241,8 +292,6 @@ public class TestMetastoreEndToEnd extends
    */
   @Test
   public void testUriTablePrivileges() throws Exception {
-    String tabName1 = "tab1";
-    String tabName2 = "tab2";
     String newPath1 = "fooTab1";
     String newPath2 = "fooTab2";
 
@@ -251,27 +300,26 @@ public class TestMetastoreEndToEnd extends
     String tabDir2 = hiveServer.getProperty(HiveServerFactory.WAREHOUSE_DIR)
         + File.separator + newPath2;
     policyFile.addRolesToGroup(USERGROUP1, uri_role)
-        .addRolesToGroup(USERGROUP2, db_all_role)
+        .addRolesToGroup(USERGROUP2, uri_role)
+        .addRolesToGroup(USERGROUP3, db_all_role)
         .addPermissionsToRole(uri_role, "server=server1->URI=" + tabDir1)
         .addPermissionsToRole(uri_role, "server=server1->URI=" + tabDir2);
     writePolicyFile(policyFile);
 
-    // create table
-    HiveMetaStoreClient client = context.getMetaStoreClient(USER2_1);
-    createMetastoreTable(client, dbName, tabName1,
-        Lists.newArrayList(new FieldSchema("col1", "int", "")));
-    client.close();
+    // user with URI privileges should be able to create table with that
+    // specific location
+    HiveMetaStoreClient client = context.getMetaStoreClient(USER1_1);
+    createMetastoreTableWithLocation(client, dbName, tabName1,
+        Lists.newArrayList(new FieldSchema("col1", "int", "")), tabDir1);
 
-    // user with URI privileges should be able to create table with that specific location
-    client = context.getMetaStoreClient(USER1_1);
     createMetastoreTableWithLocation(client, dbName, tabName2,
         Lists.newArrayList(new FieldSchema("col1", "int", "")), tabDir2);
     client.close();
 
     // user without URI privileges should be NOT able to create table with that specific location
-    client = context.getMetaStoreClient(USER2_1);
+    client = context.getMetaStoreClient(USER3_1);
     try {
-      createMetastoreTableWithLocation(client, dbName, tabName2,
+      createMetastoreTableWithLocation(client, dbName, "fooTab",
           Lists.newArrayList(new FieldSchema("col1", "int", "")), tabDir2);
       fail("Create table with location should fail without URI privilege");
     } catch (MetaException e) {
@@ -282,17 +330,25 @@ public class TestMetastoreEndToEnd extends
     // user with URI privileges should be able to alter table to set that specific location
     client = context.getMetaStoreClient(USER1_1);
     Table metaTable1 = client.getTable(dbName, tabName1);
-    metaTable1.getSd().setLocation(tabDir1);
+    metaTable1.getSd().setLocation(tabDir2);
     client.alter_table(dbName, tabName1, metaTable1);
     client.close();
 
+    // user with URI privileges and table all should be able to alter table to
+    // set that specific location
+    client = context.getMetaStoreClient(USER2_1);
+    metaTable1 = client.getTable(dbName, tabName2);
+    metaTable1.getSd().setLocation(tabDir1);
+    client.alter_table(dbName, tabName2, metaTable1);
+    client.close();
+
     // user without URI privileges should be NOT able to alter table to set that
     // specific location
-    client = context.getMetaStoreClient(USER2_1);
+    client = context.getMetaStoreClient(USER3_1);
     Table metaTable2 = client.getTable(dbName, tabName2);
-    metaTable1.getSd().setLocation(tabDir1);
+    metaTable2.getSd().setLocation(tabDir2);
     try {
-      client.alter_table(dbName, tabName1, metaTable2);
+      client.alter_table(dbName, tabName2, metaTable2);
       fail("Alter table with location should fail without URI privilege");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
@@ -323,7 +379,6 @@ public class TestMetastoreEndToEnd extends
         .addPermissionsToRole(uri_role, "server=server1->URI=" + tabDir2);
     writePolicyFile(policyFile);
 
-
     // user with URI privileges should be able to alter partition to set that specific location
     HiveMetaStoreClient client = context.getMetaStoreClient(USER1_1);
     Table tbl1 = createMetastoreTableWithPartition(client, dbName,
@@ -357,24 +412,46 @@ public class TestMetastoreEndToEnd extends
    */
   @Test
   public void testAlterSetLocationPrivileges() throws Exception {
-    String tabName1 = "tab1";
+    String newPath1 = "fooTab1";
     ArrayList<String> partVals1 = Lists.newArrayList("part1");
+    ArrayList<String> partVals2 = Lists.newArrayList("part2");
+    String tabDir1 = hiveServer.getProperty(HiveServerFactory.WAREHOUSE_DIR)
+        + File.separator + newPath1;
+
+    policyFile.addRolesToGroup(USERGROUP1, uri_role)
+        .addRolesToGroup(USERGROUP2, uri_role)
+        .addPermissionsToRole(uri_role, "server=server1->URI=" + tabDir1);
+    writePolicyFile(policyFile);
 
-    // user with Server privileges should be able to alter partition
     HiveMetaStoreClient client = context.getMetaStoreClient(ADMIN1);
     Table tbl1 = createMetastoreTableWithPartition(client, dbName,
         tabName1, Lists.newArrayList(new FieldSchema("col1", "int", "")),
         Lists.newArrayList(new FieldSchema("part_col1", "string", "")));
     addPartition(client, dbName, tabName1, partVals1, tbl1);
+    tbl1 = client.getTable(dbName, tabName1);
+    addPartition(client, dbName, tabName1, partVals2, tbl1);
+    client.close();
+
+    // user with DB and URI privileges should be able to alter partition set location
+    client = context.getMetaStoreClient(USER1_1);
     Partition newPartition = client.getPartition(dbName, tabName1, partVals1);
+    newPartition.getSd().setLocation(tabDir1);
     client.alter_partition(dbName, tabName1, newPartition);
     client.close();
 
-    // user without SERVER privileges should be able to alter partition to set
-    // that specific location
-    client = context.getMetaStoreClient(USER1_1);
-    tbl1 = client.getTable(dbName, tabName1);
-    newPartition = client.getPartition(dbName, tabName1, partVals1);
+    // user with Table and URI privileges should be able to alter partition set location
+    client = context.getMetaStoreClient(USER2_1);
+    newPartition = client.getPartition(dbName, tabName1, partVals2);
+    newPartition.getSd().setLocation(tabDir1);
+    client.alter_partition(dbName, tabName1, newPartition);
+    client.close();
+
+    policyFile.addRolesToGroup(USERGROUP3, db_all_role);
+    writePolicyFile(policyFile);
+    // user without URI privileges should not be able to alter partition set location
+    client = context.getMetaStoreClient(USER3_1);
+    newPartition = client.getPartition(dbName, tabName1, partVals2);
+    newPartition.getSd().setLocation(tabDir1);
     try {
       client.alter_partition(dbName, tabName1, newPartition);
       fail("alter partition with location should have failed");