You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by an...@apache.org on 2016/05/23 21:51:57 UTC

sentry git commit: SENTRY-1230: Add filesystem tests to test Sentry with user data storage on S3. The tests can also be run on other types of fileSystems (ms azure, hdfs). Just ensure pass correct parameters. (Anne Yu, reviewed by Haohao) [Forced Update!]

Repository: sentry
Updated Branches:
  refs/heads/master ff1515093 -> b4a64f976 (forced update)


SENTRY-1230: Add filesystem tests to test Sentry with user data storage on S3. The tests can also be run on other types of fileSystems (ms azure, hdfs). Just ensure pass correct parameters. (Anne Yu, reviewed by Haohao)

For example,
   -Dsentry.e2etest.DFSType=S3DFS
   -Dsentry.e2etest.storage.uri=s3a://sentry-s3
   -Dfs.s3a.access.key=a
   -Dfs.s3a.secret.key=s
   export HIVE_CONF_DIR=/etc/hive/conf/hite-site.xml


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

Branch: refs/heads/master
Commit: b4a64f976dd3b98225bbbe8fcf4bf8d7ecddce90
Parents: f6f730c
Author: Anne Yu <an...@cloudera.com>
Authored: Fri May 20 16:00:50 2016 -0700
Committer: Anne Yu <an...@cloudera.com>
Committed: Mon May 23 15:24:18 2016 -0700

----------------------------------------------------------------------
 pom.xml                                         |   1 +
 sentry-tests/sentry-tests-hive/pom.xml          |  13 +-
 .../sentry/tests/e2e/hdfs/TestDbHdfsBase.java   |  12 +-
 .../AbstractTestWithStaticConfiguration.java    |   5 +-
 .../sentry/tests/e2e/hive/fs/AbstractDFS.java   |  10 +-
 .../sentry/tests/e2e/hive/fs/ClusterDFS.java    |   7 +-
 .../apache/sentry/tests/e2e/hive/fs/DFS.java    |   2 +
 .../sentry/tests/e2e/hive/fs/DFSFactory.java    |   9 +-
 .../apache/sentry/tests/e2e/hive/fs/S3DFS.java  |  49 +++++
 .../sentry/tests/e2e/hive/fs/TestFSBase.java    | 216 +++++++++++++++++++
 .../tests/e2e/hive/fs/TestFSContants.java       |  33 +++
 .../e2e/hive/fs/TestHiveWarehouseOnExtFs.java   |  95 ++++++++
 .../tests/e2e/hive/fs/TestTableOnExtFS.java     | 107 +++++++++
 .../e2e/hive/hiveserver/AbstractHiveServer.java |   5 +
 .../e2e/hive/hiveserver/EmbeddedHiveServer.java |   5 +
 .../tests/e2e/hive/hiveserver/HiveServer.java   |   1 +
 .../e2e/hive/hiveserver/InternalHiveServer.java |   5 +
 .../hive/hiveserver/UnmanagedHiveServer.java    |   6 +
 18 files changed, 562 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a82c66d..44a1e5a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -95,6 +95,7 @@ limitations under the License.
     <zookeeper.version>3.4.5</zookeeper.version>
     <kafka.version>0.9.0.0</kafka.version>
     <commons-io.version>1.3.2</commons-io.version>
+    <hadoop-aws.version>2.7.0</hadoop-aws.version>
   </properties>
 
   <dependencyManagement>

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/pom.xml b/sentry-tests/sentry-tests-hive/pom.xml
index e36b5c0..02bfa49 100644
--- a/sentry-tests/sentry-tests-hive/pom.xml
+++ b/sentry-tests/sentry-tests-hive/pom.xml
@@ -283,6 +283,13 @@ limitations under the License.
         <version>${joda-time.version}</version>
         <scope>test</scope>
       </dependency>
+    <!-- http://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-aws</artifactId>
+      <version>${hadoop-aws.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <build>
     <plugins>
@@ -479,7 +486,11 @@ limitations under the License.
           <include>**/TestDbPrivilegesAtColumnScope.java</include>
           <include>**/TestColumnEndToEnd.java</include>
           <include>**/TestDbComplexView.java</include>
-          <include>**/TestConcurrentClients</include>
+          <include>**/TestConcurrentClients.java</include>
+          <include>**/TestDbHdfsMaxGroups.java</include>
+          <include>**/TestDbHdfsExtMaxGroups.java</include>
+          <include>**/TestHiveWarehouseOnExtFs.java</include>
+          <include>**/TestTableOnExtFS.java</include>
         </includes>
         <argLine>-Dsentry.e2etest.hiveServer2Type=UnmanagedHiveServer2 -Dsentry.e2etest.DFSType=ClusterDFS -Dsentry.e2etest.external.sentry=true</argLine>
        </configuration>

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestDbHdfsBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestDbHdfsBase.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestDbHdfsBase.java
index 001f5a4..44ce575 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestDbHdfsBase.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestDbHdfsBase.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import static org.apache.sentry.tests.e2e.hive.fs.DFSFactory.DFSType;
 import static org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory.HiveServer2Type;
 
+import org.apache.sentry.tests.e2e.hive.fs.TestFSContants;
 import org.junit.After;
 import org.junit.BeforeClass;
 
@@ -85,15 +86,16 @@ public abstract class TestDbHdfsBase extends AbstractTestWithStaticConfiguration
   protected static UserGroupInformation adminUgi;
   protected static UserGroupInformation hiveUgi;
   protected static int NUM_RETRIES_FOR_ACLS = 12;
-  protected static int WAIT_SECS_FOR_ACLS = 1000; //seconds
+  protected static int WAIT_SECS_FOR_ACLS = Integer.parseInt(
+      System.getProperty(TestFSContants.SENTRY_E2E_TEST_HDFS_ACLS_SYNCUP_SECS, "1000")); // seconds
   protected static String testExtPathDir =
-      System.getProperty("test.hdfs.e2e.ext.path");
+      System.getProperty(TestFSContants.SENTRY_E2E_TEST_HDFS_EXT_PATH);
   protected static final String KEYTAB_LOCATION =
-      System.getProperty("sentry.e2e.hive.keytabs.location", "/cdep/keytabs");
+      System.getProperty(TestFSContants.SENTRY_E2E_TEST_HIVE_KEYTAB_LOC, "/cdep/keytabs");
   protected static String DFS_TYPE =
-      System.getProperty("sentry.e2etest.DFSType", DFSType.MiniDFS.name());
+      System.getProperty(TestFSContants.SENTRY_E2E_TEST_DFS_TYPE, DFSType.MiniDFS.name());
 
-  protected final static String dfsAdmin = System.getProperty("dfs.cluster.administrators", "hdfs");
+  protected final static String dfsAdmin = System.getProperty(TestFSContants.SENTRY_E2E_TEST_DFS_ADMIN, "hdfs");
 
   @BeforeClass
   public static void setupTestStaticConfiguration() throws Exception {

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/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 0e4b3ca..56654db 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
@@ -35,6 +35,7 @@ import java.util.Map;
 import java.util.HashSet;
 
 import com.google.common.collect.Sets;
+import org.apache.sentry.tests.e2e.hive.fs.TestFSContants;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.AfterClass;
@@ -284,7 +285,8 @@ public abstract class AbstractTestWithStaticConfiguration {
     dataDir = assertCreateDir(new File(baseDir, "data"));
     policyFileLocation = new File(confDir, HiveServerFactory.AUTHZ_PROVIDER_FILENAME);
 
-    dfsType = System.getProperty(DFSFactory.FS_TYPE, DFSFactory.DFSType.MiniDFS.toString());
+    dfsType = System.getProperty(TestFSContants.SENTRY_E2E_TEST_DFS_TYPE,
+        DFSFactory.DFSType.MiniDFS.toString());
     dfs = DFSFactory.create(dfsType, baseDir, testServerType, enableHDFSAcls);
     fileSystem = dfs.getFileSystem();
 
@@ -318,7 +320,6 @@ public abstract class AbstractTestWithStaticConfiguration {
           "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
     }
 
-    HiveConf hiveConf = new HiveConf();
     hiveServer = create(properties, baseDir, confDir, logDir, policyURI, fileSystem);
     hiveServer.start();
     createContext();

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
index 32aabb4..d000c12 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/AbstractDFS.java
@@ -25,14 +25,15 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 
-public abstract class AbstractDFS implements DFS{
+public abstract class AbstractDFS implements DFS {
   private static final Logger LOGGER = LoggerFactory
       .getLogger(AbstractDFS.class);
+  protected static final String TEST_USER = System.getProperty("sentry.e2etest.hive.test.user", "hive");
+  protected static final String KEYTAB_LOCATION = System.getProperty("sentry.e2e.hive.keytabs.location");
   protected static FileSystem fileSystem;
   protected static Path dfsBaseDir;
   public Path sentryDir;
 
-
   @Override
   public FileSystem getFileSystem(){
     return fileSystem;
@@ -84,4 +85,9 @@ public abstract class AbstractDFS implements DFS{
     return dir;
   }
 
+  @Override
+  public String getTestUser() {return TEST_USER;}
+
+  @Override
+  public String getKeytabLocation() {return KEYTAB_LOCATION;}
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
index 06415cb..df9c521 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/ClusterDFS.java
@@ -29,16 +29,13 @@ import java.util.Random;
 public class ClusterDFS extends AbstractDFS{
   private static final Logger LOGGER = LoggerFactory
       .getLogger(ClusterDFS.class);
-  public static final String TEST_USER = "sentry.e2etest.hive.test.user";
-  private static final String testUser = System.getProperty(TEST_USER, "hive");
-  private static final String KEYTAB_LOCATION = System.getProperty("sentry.e2e.hive.keytabs.location");
   private UserGroupInformation ugi;
 
   ClusterDFS() throws Exception{
-    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(testUser, KEYTAB_LOCATION + "/" + testUser + ".keytab");
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(TEST_USER, KEYTAB_LOCATION + "/" + TEST_USER + ".keytab");
     fileSystem = getFS(ugi);
     LOGGER.info("File system uri for policy files: " + fileSystem.getUri());
-    LOGGER.info("Creating basedir as user : " + testUser);
+    LOGGER.info("Creating basedir as user : " + TEST_USER);
     String policyDir = System.getProperty("sentry.e2etest.hive.policy.location", "/user/hive/sentry");
     sentryDir = super.assertCreateDfsDir(new Path(fileSystem.getUri() + policyDir));
     dfsBaseDir = super.assertCreateDfsDir(new Path(fileSystem.getUri() + "/tmp/" + (new Random()).nextInt()));

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
index 67ba338..78337a6 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/DFS.java
@@ -29,4 +29,6 @@ public interface DFS {
   Path getBaseDir();
   void createBaseDir() throws Exception;
   void writePolicyFile(File srcFile) throws IOException;
+  String getTestUser();
+  String getKeytabLocation();
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/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 7f650ce..cbd1cfc 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
@@ -21,8 +21,6 @@ import java.io.File;
 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,
       String serverType, boolean enableHDFSAcls) throws Exception {
     DFSType type;
@@ -36,6 +34,8 @@ public class DFSFactory {
         return new MiniDFS(baseDir, serverType, enableHDFSAcls);
       case ClusterDFS:
         return new ClusterDFS();
+      case S3DFS:
+        return new S3DFS();
       default:
         throw new UnsupportedOperationException(type.name());
     }
@@ -49,6 +49,7 @@ public class DFSFactory {
   @VisibleForTesting
   public static enum DFSType {
     MiniDFS,
-    ClusterDFS;
-  };
+    ClusterDFS,
+    S3DFS
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/S3DFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/S3DFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/S3DFS.java
new file mode 100644
index 0000000..c4ba71e
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/S3DFS.java
@@ -0,0 +1,49 @@
+package org.apache.sentry.tests.e2e.hive.fs;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.PrivilegedExceptionAction;
+
+public class S3DFS extends AbstractDFS {
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(S3DFS.class);
+
+  private UserGroupInformation ugi;
+
+  S3DFS() throws Exception{
+    ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(TEST_USER, KEYTAB_LOCATION + "/" + TEST_USER + ".keytab");
+    fileSystem = getFS(ugi);
+    LOGGER.info("fileSystem URI = " + fileSystem.getUri());
+    LOGGER.info("Kinited as testUser = " + TEST_USER);
+  }
+
+  private FileSystem getFS(UserGroupInformation ugi) throws Exception {
+    return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      public FileSystem run() throws Exception {
+        Configuration conf = new Configuration();
+        return FileSystem.get(conf);
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSBase.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSBase.java
new file mode 100644
index 0000000..fd4c7c8
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSBase.java
@@ -0,0 +1,216 @@
+package org.apache.sentry.tests.e2e.hive.fs;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import com.google.common.base.Strings;
+import java.io.IOException;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Statement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.apache.sentry.tests.e2e.hive.fs.DFSFactory.DFSType;
+
+/**
+ * This is an abstract base test class for any file system tests:
+ * 1. tests will be run on a real cluster, which is not managed by this framework;
+ * 2. the real cluster can be configured to have HDFS, S3 or MS Azure as an
+ * external storage file system, tests can be run on any of these file systems;
+ * 3. defaultFS might or might not be the same as storage file system: for example,
+ * defaultFS=hdfs, while S3 is configured as an external storage system;
+ * 3. The condition to trigger tests: a. hdfs cluster; b. S3 or MS Azure cluster with
+ * explicitly specified sentry.e2etest.storage.uri;
+ * 4. To run test, could run the below mvn command:
+ * mvn test -P cluster-hadoop-provider-db -f pom.xml \
+ * -Dsentry.e2etest.DFSType=S3Cluster \ (or if not specify DFSType will be derived from storage.uri)
+ * -Dsentry.e2e.hive.keytabs.location=/root/keytabs \
+ * -Dsentry.e2etest.dfs.admin=hdfs \
+ * -Dsentry.e2etest.storage.uri=s3a://bucketname (use lowercase here)
+ */
+public class TestFSBase extends AbstractTestWithStaticConfiguration {
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(TestFSBase.class);
+
+  protected static String authenticationType;
+  protected static final String dfsAdmin =
+      System.getProperty(TestFSContants.SENTRY_E2E_TEST_DFS_ADMIN, "hdfs");
+  protected static final String KEYTAB_LOCATION = System.getProperty(
+      TestFSContants.SENTRY_E2E_TEST_HIVE_KEYTAB_LOC);
+  protected static final DFSType DFS_TYPE =
+      DFSType.valueOf(System.getProperty(TestFSContants.SENTRY_E2E_TEST_DFS_TYPE, "ClusterDFS"));
+  protected static DFSType storageDFSType = DFS_TYPE;
+  protected static URI defaultStorageUri;
+  protected static FileSystem storageFileSystem;
+  protected static String StrWarehouseDirFromConfFile;
+
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+    AbstractTestWithStaticConfiguration.setupAdmin();
+    authenticationType = System.getProperty(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION.varname);
+    LOGGER.info("authenticationType = " + authenticationType);
+    Assume.assumeThat(hiveServer2Type, equalTo(HiveServerFactory.HiveServer2Type.UnmanagedHiveServer2));
+
+    if (DFSType.ClusterDFS.equals(DFS_TYPE) ||
+        DFSType.S3DFS.equals(DFS_TYPE)) {
+      LOGGER.info("DFSType = " + DFS_TYPE);
+    } else {
+      LOGGER.warn("Incorrect DFSType " + DFS_TYPE);
+      Assume.assumeTrue(false);
+    }
+
+    String storageUriStr = System.getProperty(TestFSContants.SENTRY_E2E_TEST_STORAGE_URI);
+    if (!Strings.isNullOrEmpty(storageUriStr)) {
+      storageUriStr = storageUriStr.toLowerCase();
+      if (storageUriStr.startsWith("hdfs") || storageUriStr.startsWith("file")) {
+        storageDFSType = DFSType.ClusterDFS;
+      } else if (storageUriStr.startsWith("s3a")) {
+        storageDFSType = DFSType.S3DFS;
+      }
+     }
+
+    storageFileSystem = fileSystem;
+    if (storageDFSType.equals(DFSType.ClusterDFS)) {
+      // hdfs cluster
+      defaultStorageUri = FileSystem.getDefaultUri(fileSystem.getConf());
+    } else {
+      // non-hdfs file sytem must specify defaultStorageUri
+      if (!Strings.isNullOrEmpty(storageUriStr)) {
+        defaultStorageUri = URI.create(storageUriStr);
+      } else {
+        LOGGER.warn("Skipping test: Unknown sentry.e2etest.storage.uri, " +
+            "for example, s3a://bucketname");
+        Assume.assumeTrue(false);
+      }
+      LOGGER.info("defaultStorageUri = " + defaultStorageUri.toString());
+
+      if (storageDFSType.equals(DFSType.S3DFS)) {
+        // currently defaultFS = s3a doesn't work for NN
+        // needs to explicitly specify s3a's defaultUri
+        String accessKey = System.getProperty(TestFSContants.S3A_ACCESS_KEY,
+            hiveServer.getProperty(TestFSContants.S3A_ACCESS_KEY));
+        String secretKey = System.getProperty(TestFSContants.S3A_SECRET_KEY,
+            hiveServer.getProperty(TestFSContants.S3A_SECRET_KEY));
+        LOGGER.info("accessKey = " + accessKey);
+        LOGGER.info("secretKey = " + secretKey);
+        Assume.assumeTrue(Strings.isNullOrEmpty(accessKey) == false);
+        Assume.assumeTrue(Strings.isNullOrEmpty(secretKey) == false);
+
+        Configuration conf = new Configuration();
+        conf.set(TestFSContants.S3A_ACCESS_KEY, accessKey);
+        conf.set(TestFSContants.S3A_SECRET_KEY, secretKey);
+        storageFileSystem = new S3AFileSystem();
+        Assume.assumeNotNull(storageFileSystem);
+        LOGGER.info("Configuring S3DFS defaultStorageUri = " + defaultStorageUri.toString());
+        storageFileSystem.initialize(defaultStorageUri, conf);
+      }
+      /*
+      else if (DFS_TYPE.equals(DFSType.MSAZUREDFS)) {
+      }
+      */
+    }
+    // Get warehouse dir from hite-site.xml conf file
+    StrWarehouseDirFromConfFile = hiveServer.getOrgWarehouseDir();
+  }
+
+  @Override
+  @Before
+  public void setup() throws Exception {
+    LOGGER.info("TestFSBase setup");
+    //no-op
+  }
+
+  /**
+   * Return a full path starting with scheme and authority
+   * hdfs:/nameserver/relativePath; s3a://bucketname/relativePath
+   * @param relativePath
+   * @return full path
+   */
+  protected static Path getFullPathWithSchemeAndAuthority(Path relativePath) {
+    return relativePath.makeQualified(defaultStorageUri, relativePath);
+  }
+
+  protected void createPath(Path relativePath) throws Exception {
+    Path fullPath = getFullPathWithSchemeAndAuthority(relativePath);
+    FileSystem adminFS = storageFileSystem;
+    LOGGER.info("Creating path " + fullPath);
+    if (storageDFSType.equals(DFSType.ClusterDFS)) {
+      UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+          dfsAdmin, KEYTAB_LOCATION + "/" + dfsAdmin + ".keytab");
+      adminFS = getFS(ugi);
+    }
+    if (adminFS.exists(fullPath)) {
+      adminFS.delete(fullPath, true);
+    }
+    adminFS.mkdirs(fullPath);
+  }
+
+  /**
+   * Drop and create role, in case the previous
+   * tests leave same roles uncleaned up
+   * @param statement
+   * @param roleName
+   * @throws Exception
+   */
+  protected void dropRecreateRole(Statement statement, String roleName) throws Exception {
+    try {
+      exec(statement, "DROP ROLE " + roleName);
+    } catch (Exception ex) {
+      //noop
+      LOGGER.info("Role " + roleName  + " does not exist. But it's ok.");
+    } finally {
+      exec(statement, "CREATE ROLE " + roleName);
+    }
+  }
+
+  protected static void kinitFromKeytabFile (String user, String keyTabFile) throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(TestFSContants.SENTRY_E2E_TEST_SECURITY_AUTH, authenticationType);
+    UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.loginUserFromKeytab(user, keyTabFile);
+  }
+
+  protected static String getKeyTabFileFullPath(String user) {
+    return KEYTAB_LOCATION + "/" + user + ".keytab";
+  }
+
+  protected FileSystem getFS(UserGroupInformation ugi) throws Exception {
+    return ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
+      public FileSystem run() throws Exception {
+        Configuration conf = new Configuration();
+        conf.set(TestFSContants.SENTRY_E2E_TEST_SECURITY_AUTH, authenticationType);
+        return FileSystem.get(conf);
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSContants.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSContants.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSContants.java
new file mode 100644
index 0000000..ce1124b
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestFSContants.java
@@ -0,0 +1,33 @@
+package org.apache.sentry.tests.e2e.hive.fs;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public final class TestFSContants {
+  private TestFSContants() {
+    // no-op
+  }
+  public static final String SENTRY_E2E_TEST_HDFS_EXT_PATH = "test.hdfs.e2e.ext.path";
+  public static final String SENTRY_E2E_TEST_HIVE_KEYTAB_LOC = "sentry.e2e.hive.keytabs.location";
+  public static final String SENTRY_E2E_TEST_HDFS_ACLS_SYNCUP_SECS = "1000";
+  public static final String SENTRY_E2E_TEST_DFS_ADMIN = "sentry.e2etest.dfs.admin";
+  public static final String SENTRY_E2E_TEST_DFS_TYPE = "sentry.e2etest.DFSType";
+  // storage default URI could be like s3a://bucketname, hdfs://nameservice, file://
+  public static final String SENTRY_E2E_TEST_STORAGE_URI = "sentry.e2etest.storage.uri";
+  public static final String S3A_ACCESS_KEY = "fs.s3a.access.key";
+  public static final String S3A_SECRET_KEY = "fs.s3a.secret.key";
+  public static final String SENTRY_E2E_TEST_SECURITY_AUTH = "hadoop.security.authentication";
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestHiveWarehouseOnExtFs.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestHiveWarehouseOnExtFs.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestHiveWarehouseOnExtFs.java
new file mode 100644
index 0000000..ed965c1
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestHiveWarehouseOnExtFs.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.tests.e2e.hive.fs;
+
+import com.google.common.base.Strings;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.fs.Path;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Configure Hive warehouse dir to an external storage FS system:
+ * for example,
+ * <property>
+ * <name>hive.metastore.warehouse.dir</name>
+ * <value>s3a://sentry-s3/user/hive/warehouse</value>
+ * </property>
+ * Test basic db and tbl permissions
+ * Ensure export HIVE_CONF_DIR=/etc/hive/conf/hite-site.xml
+ */
+public class TestHiveWarehouseOnExtFs extends TestFSBase {
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(TestHiveWarehouseOnExtFs.class);
+  private static final String TEST_DB = "test_warehouse_db";
+  private static final String TEST_TBL = "test_warehouse_ext_tbl";
+  private static final String TEST_ROLE = "test_warehouse_role";
+
+  @Before
+  public void createDbAndRole() throws Exception {
+
+    LOGGER.info("StrWarehouseDirFromConfFile = " + StrWarehouseDirFromConfFile);
+    // Ensure Hive warehouse is configured to s3a or other file system than hdfs
+    // warehouse dir is extracted from ${HIVE_CONF_DIR}/hive-site.xml
+    try (Connection connection = context.createConnection(ADMIN1)) {
+      try (Statement statement = connection.createStatement()) {
+        exec(statement, "DROP DATABASE IF EXISTS " + TEST_DB + " CASCADE");
+        exec(statement, "CREATE DATABASE " + TEST_DB);
+        dropRecreateRole(statement, TEST_ROLE);
+        exec(statement, "GRANT ALL ON DATABASE " + TEST_DB + " TO ROLE " + TEST_ROLE);
+        exec(statement, "GRANT ROLE " + TEST_ROLE + " TO GROUP " + USERGROUP1);
+      }
+    }
+  }
+
+  @Test
+  public void TestCreateTable() throws Exception {
+    try (Connection connection = context.createConnection(USER1_1)) {
+      try (Statement statement = connection.createStatement()) {
+        exec(statement, "USE " + TEST_DB);
+        exec(statement, "CREATE TABLE " + TEST_TBL
+            + " (value STRING, number INT) PARTITIONED BY (par INT)");
+        exec(statement, "INSERT INTO TABLE " + TEST_TBL + " PARTITION (par=1) VALUES ('test1', 1)");
+        try (ResultSet rs = execQuery(statement, "SELECT number FROM " + TEST_TBL + " LIMIT 1")) {
+          assertTrue("No number returned", rs.next());
+          int number = rs.getInt("number");
+          assertEquals("expected number 1, actual is " + number, number, 1);
+        }
+        String tblDir = StrWarehouseDirFromConfFile + "/" + TEST_DB + ".db/" + TEST_TBL;
+        String msg = "tblDir = " + tblDir + " does not exist";
+        if (new Path(StrWarehouseDirFromConfFile).isAbsoluteAndSchemeAuthorityNull()) {
+          // Warehouse locates on default fileSystem, for example ClusterDFS
+          assertTrue(msg, fileSystem.exists(new Path(tblDir)));
+        } else {
+          // Warehosue locates on other storage file system, for example s3a
+          assertTrue(msg, storageFileSystem.exists(new Path(tblDir)));
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestTableOnExtFS.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestTableOnExtFS.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestTableOnExtFS.java
new file mode 100644
index 0000000..58746c5
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/fs/TestTableOnExtFS.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.tests.e2e.hive.fs;
+
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test create an external db or table with its loc on external FS:
+ * For example, create external tbl location 's3a://sentry-s3/db/tbl'
+ * To run this test, ensure pass the below parameters:
+ * -Dsentry.e2etest.DFSType=S3DFS
+ * -Dsentry.e2etest.storage.uri=s3a://sentry-s3
+ * -Dfs.s3a.access.key=a
+ * -Dfs.s3a.secret.key=s
+ * export HIVE_CONF_DIR=/etc/hive/conf/hite-site.xml
+ */
+public class TestTableOnExtFS extends TestFSBase {
+
+  private static final String TEST_DB = "test_dfs_db";
+  private static final String TEST_TBL = "test_dfs_ext_tbl";
+  private static final String TEST_ROLE = "test_dfs_role";
+  private static final String StrTestUriWithoutSchema = StrWarehouseDirFromConfFile + "/exttbl";
+  // TEST_PATH could be:
+  // hdfs://nameservice1/tblpath,
+  // s3a://bucketname/tblpath
+  private static final Path TEST_PATH =
+      getFullPathWithSchemeAndAuthority(new Path(StrTestUriWithoutSchema));
+
+  @Before
+  public void setTestPath() throws Exception {
+    createPath(TEST_PATH);
+    try (Connection connection = context.createConnection(ADMIN1)) {
+      try (Statement statement = connection.createStatement()) {
+        exec(statement, "DROP DATABASE IF EXISTS " + TEST_DB + " CASCADE");
+        exec(statement, "CREATE DATABASE " + TEST_DB);
+        dropRecreateRole(statement, TEST_ROLE);
+        exec(statement, "GRANT ALL ON DATABASE " + TEST_DB + " TO ROLE " + TEST_ROLE);
+        exec(statement, "GRANT ROLE " + TEST_ROLE + " TO GROUP " + USERGROUP1);
+      }
+    }
+  }
+
+  private void testTableWithUriHelper(String strUri) throws Exception {
+    try (Connection connection = context.createConnection((ADMIN1))) {
+      try (Statement statement = connection.createStatement()) {
+        exec(statement, "GRANT ALL ON URI '" + strUri + "' TO ROLE " + TEST_ROLE);
+      }
+    }
+    try (Connection connection = context.createConnection(USER1_1)) {
+      try (Statement statement = connection.createStatement()) {
+        exec(statement, "USE " + TEST_DB);
+        exec(statement, "CREATE EXTERNAL TABLE " + TEST_TBL
+            + " (value STRING, number INT) PARTITIONED BY (par INT) LOCATION '"
+            + strUri + "'");
+        exec(statement, "INSERT INTO TABLE " + TEST_TBL + " PARTITION (par=1) VALUES ('test1', 1)");
+        try (ResultSet rs = execQuery(statement, "SELECT number FROM " + TEST_TBL + " LIMIT 1")) {
+          assertTrue("No number returned", rs.next());
+          int number = rs.getInt("number");
+          assertEquals("expected number 1, actual is " + number, number, 1);
+        }
+        exec(statement, "ALTER TABLE " + TEST_TBL + " DROP PARTITION (par=1) PURGE");
+      }
+    }
+  }
+
+  /**
+   * Test full URI case: with Scheme and Authority
+   * @throws Exception
+   */
+  @Test
+  public void TestCreateExtTableWithFullUri() throws Exception {
+    testTableWithUriHelper(TEST_PATH.toString());
+  }
+
+  /**
+   * Test a URI without Scheme and Authority
+   * @throws Exception
+   */
+  @Test
+  public void TestCreateExtTableWithoutScheme() throws Exception {
+    testTableWithUriHelper(StrTestUriWithoutSchema);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
index dda7dba..9a008d7 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/AbstractHiveServer.java
@@ -92,4 +92,9 @@ public abstract class AbstractHiveServer implements HiveServer {
       }
     } while (true);
   }
+
+  @Override
+  public String getOrgWarehouseDir() {
+    return configuration.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, null);
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
index 52ba09e..028153a 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/EmbeddedHiveServer.java
@@ -57,4 +57,9 @@ public class EmbeddedHiveServer implements HiveServer {
   public String getProperty(String key) {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public String getOrgWarehouseDir() {
+    return (String)null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServer.java
index 175e84c..8b084ab 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServer.java
@@ -31,4 +31,5 @@ public interface HiveServer {
 
   Connection createConnection(String user, String password) throws Exception;
 
+  String getOrgWarehouseDir();
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
index 45f0ef2..ceda5d0 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/InternalHiveServer.java
@@ -44,4 +44,9 @@ public class InternalHiveServer extends AbstractHiveServer {
       hiveServer2.stop();
     }
   }
+
+  @Override
+  public String getOrgWarehouseDir() {
+    return conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, null);
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/b4a64f97/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
index beae8e8..d5c5db4 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/UnmanagedHiveServer.java
@@ -110,4 +110,10 @@ public class UnmanagedHiveServer implements HiveServer {
     UserGroupInformation.loginUserFromKeytab(user, KEYTAB_LOCATION + "/" + user + ".keytab");
     LOGGER.info("Kinited user: "+ user+" keytab: "+KEYTAB_LOCATION+"/"+user+".keytab");
   }
+
+  // return original warehouse dir, may not include scheme and authority
+  // it is the exact string defined in the hive.metastore.warehouse.dir
+  public String getOrgWarehouseDir() {
+    return getSystemAndConfigProperties(HiveConf.ConfVars.METASTOREWAREHOUSE.varname.toString(), null);
+  }
 }