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/23 23:40:10 UTC

git commit: SENTRY-309: Metastore binding should use fully qualified URI for validating alter table operations ( Prasad Mujumdar via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master b41d9ee37 -> 07b0c9b86


SENTRY-309: Metastore binding should use fully qualified URI for validating alter table operations ( 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/07b0c9b8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/07b0c9b8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/07b0c9b8

Branch: refs/heads/master
Commit: 07b0c9b86c8923dcec18d8797e395188c1607584
Parents: b41d9ee
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Mon Jun 23 14:39:39 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Mon Jun 23 14:39:39 2014 -0700

----------------------------------------------------------------------
 .../SentryHiveAuthorizationTaskFactoryImpl.java |  2 +-
 .../metastore/MetastoreAuthzBinding.java        | 36 ++++++++++++++------
 .../e2e/dbprovider/TestDatabaseProvider.java    |  3 +-
 .../AbstractTestWithStaticConfiguration.java    | 12 ++++++-
 .../sentry/tests/e2e/hive/fs/DFSFactory.java    |  7 ++--
 .../sentry/tests/e2e/hive/fs/MiniDFS.java       |  6 +++-
 .../e2e/hive/hiveserver/HiveServerFactory.java  |  1 +
 .../hiveserver/InternalMetastoreServer.java     |  5 +++
 ...actMetastoreTestWithStaticConfiguration.java | 18 ++++++++++
 .../e2e/metastore/TestMetaStoreWithPigHCat.java |  2 +-
 .../e2e/metastore/TestMetastoreEndToEnd.java    |  6 ++--
 11 files changed, 78 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
index 9da020c..991d734 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -294,7 +294,7 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
         } else if (astChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
           throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
         }else if (astChild.getToken().getType() == HiveParser.TOK_URI) {
-          privilegeObject = privilegeObject.replaceAll("'", "");
+          privilegeObject = privilegeObject.replaceAll("'", "").replaceAll("\"", "");
           subject.setUri(true);
         } else if (astChild.getToken().getType() == HiveParser.TOK_SERVER) {
           subject.setServer(true);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/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 c87e258..ccf6a67 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
@@ -45,7 +45,6 @@ import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
@@ -127,6 +126,7 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
   private final HiveConf hiveConf;
   private final ImmutableSet<String> serviceUsers;
   private HiveAuthzBinding hiveAuthzBinding;
+  private String warehouseDir;
 
   public MetastoreAuthzBinding(Configuration config) throws Exception {
     super(config);
@@ -150,6 +150,8 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
     serviceUsers = ImmutableSet.copyOf(toTrimedLower(Sets.newHashSet(authzConf
         .getStrings(AuthzConfVars.AUTHZ_METASTORE_SERVICE_USERS.getVar(),
             new String[] { "" }))));
+    warehouseDir = hiveConf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
+
   }
 
   /**
@@ -210,7 +212,6 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
       throws InvalidOperationException, MetaException {
     HierarcyBuilder inputBuilder = new HierarcyBuilder();
     if (!StringUtils.isEmpty(context.getTable().getSd().getLocation())) {
-      String warehouseDir = hiveConf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
       String uriPath;
       try {
         uriPath = PathUtils.parseDFSURI(warehouseDir, context.getTable().getSd()
@@ -239,10 +240,18 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
 
     HierarcyBuilder inputBuilder = new HierarcyBuilder();
     // if the operation requires location change, then add URI privilege check
-    if (context.getOldTable().getSd().getLocation()
-        .compareTo(context.getNewTable().getSd().getLocation()) != 0) {
-      inputBuilder.addUriToOutput(getAuthServer(), context.getNewTable()
-          .getSd().getLocation());
+    String oldLocationUri;
+    String newLocationUri;
+    try {
+      oldLocationUri = PathUtils.parseDFSURI(warehouseDir, context
+          .getOldTable().getSd().getLocation());
+      newLocationUri = PathUtils.parseDFSURI(warehouseDir, context
+          .getNewTable().getSd().getLocation());
+    } catch (URISyntaxException e) {
+      throw new MetaException(e.getMessage());
+    }
+    if (oldLocationUri.compareTo(newLocationUri) != 0) {
+      inputBuilder.addUriToOutput(getAuthServer(), newLocationUri);
     }
     authorizeMetastoreAccess(
         HiveOperation.ALTERTABLE_ADDCOLS, inputBuilder.build(),
@@ -262,9 +271,16 @@ 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()
+            .getSd().getLocation());
+      } catch (URISyntaxException e) {
+        throw new MetaException(e.getMessage());
+      }
       if (!partitionLocation.startsWith(tableLocation + File.separator)) {
-        inputBuilder.addUriToOutput(getAuthServer(), context.getPartition()
-          .getSd().getLocation());
+        inputBuilder.addUriToOutput(getAuthServer(), uriPath);
       }
     }
     authorizeMetastoreAccess(HiveOperation.ALTERTABLE_ADDPARTS,
@@ -287,9 +303,7 @@ public class MetastoreAuthzBinding extends MetaStorePreEventListener {
     authorizeMetastoreAccess(
         HiveOperation.ALTERPARTITION_LOCATION,
         new HierarcyBuilder().build(),
-        new HierarcyBuilder()
-            .addDbToOutput(getAuthServer(),
-            context.getNewPartition().getDbName()).build());
+        new HierarcyBuilder().addServerToOutput(getAuthServer()).build());
   }
 
   private InvalidOperationException invalidOperationException(Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/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 e32aae8..200ea55 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
@@ -382,7 +382,8 @@ public class TestDatabaseProvider extends AbstractTestWithStaticConfiguration {
     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_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);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
index 422c623..dbc01f9 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
@@ -45,8 +45,10 @@ import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.policy.db.DBModelAuthorizables;
 import org.apache.sentry.provider.db.SimpleDBProviderBackend;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.service.thrift.SentryService;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
 import org.apache.sentry.service.thrift.SentryServiceFactory;
 import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
@@ -190,7 +192,7 @@ public abstract class AbstractTestWithStaticConfiguration {
     policyFileLocation = new File(confDir, HiveServerFactory.AUTHZ_PROVIDER_FILENAME);
 
     String dfsType = System.getProperty(DFSFactory.FS_TYPE);
-    dfs = DFSFactory.create(dfsType, baseDir);
+    dfs = DFSFactory.create(dfsType, baseDir, testServerType);
     fileSystem = dfs.getFileSystem();
 
     String policyURI;
@@ -358,6 +360,14 @@ public abstract class AbstractTestWithStaticConfiguration {
     }
   }
 
+  public static SentryPolicyServiceClient getSentryClient() throws Exception {
+    if (sentryServer == null) {
+      throw new IllegalAccessException("Sentry service not initialized");
+    }
+    SentryServiceClientFactory factory = new SentryServiceClientFactory();
+    return factory.create(sentryServer.getConf());
+  }
+
   @Before
   public void setup() throws Exception{
     dfs.createBaseDir();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
index c897b49..3f03ef0 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFSFactory.java
@@ -18,12 +18,15 @@ package org.apache.sentry.tests.e2e.hive.fs;
 
 import java.io.File;
 
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory.HiveServer2Type;
+
 import com.google.common.annotations.VisibleForTesting;
 
 public class DFSFactory {
   public static final String FS_TYPE = "sentry.e2etest.DFSType";
 
-  public static DFS create(String dfsType, File baseDir) throws Exception{
+  public static DFS create(String dfsType, File baseDir,
+      String serverType) throws Exception {
     DFSType type;
     if(dfsType!=null) {
       type = DFSType.valueOf(dfsType.trim());
@@ -32,7 +35,7 @@ public class DFSFactory {
     }
     switch (type) {
       case MiniDFS:
-        return new MiniDFS(baseDir);
+        return new MiniDFS(baseDir, serverType);
       case ClusterDFS:
         return new ClusterDFS();
       default:

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
index a96a1ce..f632461 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/MiniDFS.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.GroupMappingServiceProvider;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory.HiveServer2Type;
 
 import com.google.common.collect.Lists;
 
@@ -52,7 +53,10 @@ public class MiniDFS extends AbstractDFS {
 
   private static MiniDFSCluster dfsCluster;
 
-  MiniDFS(File baseDir) throws Exception {
+  MiniDFS(File baseDir, String serverType) throws Exception {
+    if (HiveServer2Type.InternalMetastore.name().equalsIgnoreCase(serverType)) {
+      Configuration.addDefaultResource("core-site-for-sentry-test.xml");
+    }
     Configuration conf = new Configuration();
     File dfsDir = assertCreateDir(new File(baseDir, "dfs"));
     conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dfsDir.getPath());

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
index 51acbf0..44331f6 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
@@ -145,6 +145,7 @@ public class HiveServerFactory {
           properties.put(METASTORE_HOOK,
               "org.apache.sentry.binding.metastore.MetastoreAuthzBinding");
         }
+        properties.put(ConfVars.METASTORESERVERMINTHREADS.varname, "5");
       }
     }
     if (!properties.containsKey(METASTORE_BYPASS)) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalMetastoreServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalMetastoreServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalMetastoreServer.java
index b1f404f..50e0e4f 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalMetastoreServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalMetastoreServer.java
@@ -44,6 +44,11 @@ public class InternalMetastoreServer extends AbstractHiveServer {
   }
 
   @Override
+  public String getURL() {
+    return "jdbc:hive2://";
+  }
+
+  @Override
   public void start() throws Exception {
     startMetastore();
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/AbstractMetastoreTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/AbstractMetastoreTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/AbstractMetastoreTestWithStaticConfiguration.java
index 7084400..549a0fa 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/AbstractMetastoreTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/AbstractMetastoreTestWithStaticConfiguration.java
@@ -38,8 +38,10 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.pig.PigServer;
+import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory.HiveServer2Type;
+import org.junit.After;
 import org.junit.BeforeClass;
 
 public abstract class AbstractMetastoreTestWithStaticConfiguration extends
@@ -52,6 +54,22 @@ public abstract class AbstractMetastoreTestWithStaticConfiguration extends
     AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
   }
 
+  @Override
+  @After
+  public void clearDB() throws Exception {
+
+  }
+
+  @Override
+  protected void writePolicyFile(PolicyFile policyFile) throws Exception {
+    policyFile.write(context.getPolicyFile());
+  }
+
+  public static PolicyFile setAdminOnServer1(String adminGroup)
+      throws Exception {
+    return SentryPolicyProviderForDb.setAdminOnServer1(adminGroup,
+        getSentryClient());
+  }
   /**
    * create a metastore table using the given attributes
    * @param client

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetaStoreWithPigHCat.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetaStoreWithPigHCat.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetaStoreWithPigHCat.java
index 8a2bf59..00d0492 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetaStoreWithPigHCat.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/metastore/TestMetaStoreWithPigHCat.java
@@ -51,7 +51,7 @@ public class TestMetaStoreWithPigHCat extends
     Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
     to.close();
 
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = setAdminOnServer1(ADMINGROUP);
     policyFile
         .addRolesToGroup(USERGROUP1, db_all_role)
         .addRolesToGroup(USERGROUP2, "read_db_role")

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/07b0c9b8/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 c672ef0..bae0213 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
@@ -18,7 +18,6 @@
 
 package org.apache.sentry.tests.e2e.metastore;
 
-import org.apache.sentry.provider.file.PolicyFile;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -30,6 +29,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.tests.e2e.hive.Context;
 import org.apache.sentry.tests.e2e.hive.StaticUserGroup;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
@@ -49,7 +49,7 @@ public class TestMetastoreEndToEnd extends
 
   @Before
   public void setup() throws Exception {
-    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    policyFile = setAdminOnServer1(ADMINGROUP); // PolicyFile.setAdminOnServer1(ADMINGROUP);
     policyFile
         .addRolesToGroup(USERGROUP1, db_all_role)
         .addRolesToGroup(USERGROUP2, "read_db_role")
@@ -341,6 +341,7 @@ public class TestMetastoreEndToEnd extends
       tbl1 = client.getTable(dbName, tabName1);
       addPartitionWithLocation(client, dbName, tabName1, partVals3,
           tbl1, tabDir2);
+      fail("Add partition with location should have failed");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
     }
@@ -376,6 +377,7 @@ public class TestMetastoreEndToEnd extends
     newPartition = client.getPartition(dbName, tabName1, partVals1);
     try {
       client.alter_partition(dbName, tabName1, newPartition);
+      fail("alter partition with location should have failed");
     } catch (MetaException e) {
       Context.verifyMetastoreAuthException(e);
     }