You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2017/04/08 02:29:56 UTC

[2/2] hive git commit: HIVE-16392 : Remove hive.warehouse.subdir.inherit.perms and all permissions inheritance logic (Sahil Takiar via Ashutosh Chauhan)

HIVE-16392 : Remove hive.warehouse.subdir.inherit.perms and all permissions inheritance logic (Sahil Takiar via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: ccc96b47ee764172021edd34342708ef6c47d812
Parents: f478de3
Author: Sahil Takiar <ta...@gmail.com>
Authored: Fri Apr 7 19:28:04 2017 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Apr 7 19:28:04 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    |  59 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   4 -
 .../hadoop/hive/common/TestFileUtils.java       |   1 -
 .../hive/ql/security/TestExtendedAcls.java      | 166 ----
 ...edMetastoreAuthorizationProviderWithACL.java |   1 -
 .../metastore/TestEmbeddedHiveMetaStore.java    |   2 -
 .../hive/metastore/TestHiveMetaStore.java       |  24 +-
 .../hive/ql/security/FolderPermissionBase.java  | 792 -------------------
 .../hive/ql/security/TestFolderPermissions.java |  52 --
 ...StorageBasedMetastoreAuthorizationDrops.java | 205 -----
 ...StorageBasedMetastoreAuthorizationReads.java | 127 ---
 .../hadoop/hive/metastore/HiveAlterHandler.java |   4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  10 +-
 .../apache/hadoop/hive/metastore/Warehouse.java |  12 +-
 .../metastore/TestHiveMetaStoreTimeout.java     |   1 -
 .../java/org/apache/hadoop/hive/ql/Context.java |   4 +-
 .../apache/hadoop/hive/ql/exec/CopyTask.java    |   3 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   5 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |   3 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  48 +-
 .../ql/metadata/SessionHiveMetaStoreClient.java |   2 +-
 21 files changed, 28 insertions(+), 1497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index e586015..f401b6f 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -511,43 +511,13 @@ public final class FileUtils {
    * Creates the directory and all necessary parent directories.
    * @param fs FileSystem to use
    * @param f path to create.
-   * @param inheritPerms whether directory inherits the permission of the last-existing parent path
    * @param conf Hive configuration
    * @return true if directory created successfully.  False otherwise, including if it exists.
    * @throws IOException exception in creating the directory
    */
-  public static boolean mkdir(FileSystem fs, Path f, boolean inheritPerms, Configuration conf) throws IOException {
+  public static boolean mkdir(FileSystem fs, Path f, Configuration conf) throws IOException {
     LOG.info("Creating directory if it doesn't exist: " + f);
-    if (!inheritPerms) {
-      //just create the directory
-      return fs.mkdirs(f);
-    } else {
-      //Check if the directory already exists. We want to change the permission
-      //to that of the parent directory only for newly created directories.
-      try {
-        return fs.getFileStatus(f).isDir();
-      } catch (FileNotFoundException ignore) {
-      }
-      //inherit perms: need to find last existing parent path, and apply its permission on entire subtree.
-      Path lastExistingParent = f;
-      Path firstNonExistentParent = null;
-      while (!fs.exists(lastExistingParent)) {
-        firstNonExistentParent = lastExistingParent;
-        lastExistingParent = lastExistingParent.getParent();
-      }
-      boolean success = fs.mkdirs(f);
-      if (!success) {
-        return false;
-      } else {
-        //set on the entire subtree
-        if (inheritPerms) {
-          HdfsUtils.setFullFileStatus(conf,
-                  new HdfsUtils.HadoopFileStatus(conf, fs, lastExistingParent), fs,
-                  firstNonExistentParent, true);
-        }
-        return true;
-      }
-    }
+    return fs.mkdirs(f);
   }
 
   public static Path makeAbsolute(FileSystem fileSystem, Path path) throws IOException {
@@ -600,11 +570,6 @@ public final class FileUtils {
     if (!triedDistcp) {
       copied = FileUtil.copy(srcFS, src, dstFS, dst, deleteSource, overwrite, conf);
     }
-
-    boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-    if (copied && inheritPerms) {
-      HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, dstFS, dst.getParent()), dstFS, dst, true);
-    }
     return copied;
   }
 
@@ -643,9 +608,8 @@ public final class FileUtils {
     return result;
   }
 
-  public static boolean renameWithPerms(FileSystem fs, Path sourcePath,
-                               Path destPath, boolean inheritPerms,
-                               Configuration conf) throws IOException {
+  public static boolean rename(FileSystem fs, Path sourcePath,
+                               Path destPath, Configuration conf) throws IOException {
     LOG.info("Renaming " + sourcePath + " to " + destPath);
 
     // If destPath directory exists, rename call will move the sourcePath
@@ -654,20 +618,7 @@ public final class FileUtils {
       throw new IOException("Cannot rename the source path. The destination "
           + "path already exists.");
     }
-
-    if (!inheritPerms) {
-      //just rename the directory
-      return fs.rename(sourcePath, destPath);
-    } else {
-      //rename the directory
-      if (fs.rename(sourcePath, destPath)) {
-        HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, destPath.getParent()), fs, destPath,
-                true);
-        return true;
-      }
-
-      return false;
-    }
+    return fs.rename(sourcePath, destPath);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 4f7f1e7..dedf6a2 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2253,10 +2253,6 @@ public class HiveConf extends Configuration {
     HIVE_INSERT_INTO_MULTILEVEL_DIRS("hive.insert.into.multilevel.dirs", false,
         "Where to insert into multilevel directories like\n" +
         "\"insert directory '/HIVEFT25686/chinna/' from table\""),
-    HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS("hive.warehouse.subdir.inherit.perms", true,
-        "Set this to false if the table directories should be created\n" +
-        "with the permissions derived from dfs umask instead of\n" +
-        "inheriting the permission of the warehouse or database directory."),
     HIVE_INSERT_INTO_EXTERNAL_TABLES("hive.insert.into.external.tables", true,
         "whether insert into external tables is allowed"),
     HIVE_TEMPORARY_TABLE_STORAGE(

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java b/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
index 03fcaeb..adc9b0c 100644
--- a/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
+++ b/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
@@ -213,7 +213,6 @@ public class TestFileUtils {
     Path copySrc = new Path("copySrc");
     Path copyDst = new Path("copyDst");
     HiveConf conf = new HiveConf(TestFileUtils.class);
-    conf.set(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS.varname, "false");
 
     FileSystem mockFs = mock(FileSystem.class);
     when(mockFs.getUri()).thenReturn(URI.create("hdfs:///"));

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestExtendedAcls.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestExtendedAcls.java b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestExtendedAcls.java
deleted file mode 100644
index b798379..0000000
--- a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestExtendedAcls.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * 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.hadoop.hive.ql.security;
-
-import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
-import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
-import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
-import static org.apache.hadoop.fs.permission.AclEntryType.USER;
-
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclEntryScope;
-import org.apache.hadoop.fs.permission.AclEntryType;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-public class TestExtendedAcls extends FolderPermissionBase {
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    conf = new HiveConf(TestExtendedAcls.class);
-    //setup the mini DFS with acl's enabled.
-    conf.set("dfs.namenode.acls.enabled", "true");
-    conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
-    baseSetup();
-  }
-
-  private final ImmutableList<AclEntry> aclSpec1 = ImmutableList.of(
-      aclEntry(ACCESS, USER, FsAction.ALL),
-      aclEntry(ACCESS, GROUP, FsAction.ALL),
-      aclEntry(ACCESS, OTHER, FsAction.ALL),
-      aclEntry(ACCESS, USER, "bar", FsAction.READ_WRITE),
-      aclEntry(ACCESS, USER, "foo", FsAction.READ_EXECUTE),
-      aclEntry(ACCESS, GROUP, "bar", FsAction.READ_WRITE),
-      aclEntry(ACCESS, GROUP, "foo", FsAction.READ_EXECUTE));
-
-  private final ImmutableList<AclEntry> aclSpec2 = ImmutableList.of(
-      aclEntry(ACCESS, USER, FsAction.ALL),
-      aclEntry(ACCESS, GROUP, FsAction.ALL),
-      aclEntry(ACCESS, OTHER, FsAction.READ_EXECUTE),
-      aclEntry(ACCESS, USER, "bar2", FsAction.READ_WRITE),
-      aclEntry(ACCESS, USER, "foo2", FsAction.READ_EXECUTE),
-      aclEntry(ACCESS, GROUP, "bar2", FsAction.READ),
-      aclEntry(ACCESS, GROUP, "foo2", FsAction.READ_EXECUTE));
-
-  @Override
-  public void setPermission(String locn, int permIndex) throws Exception {
-    switch (permIndex) {
-      case 0:
-        setAcl(locn, aclSpec1);
-        break;
-      case 1:
-        setAcl(locn, aclSpec2);
-        break;
-      default:
-        throw new RuntimeException("Only 2 permissions by this test");
-    }
-  }
-
-  @Override
-  public void verifyPermission(String locn, int permIndex) throws Exception {
-    switch (permIndex) {
-      case 0:
-        FsPermission perm = fs.getFileStatus(new Path(locn)).getPermission();
-        Assert.assertEquals("Location: " + locn, "rwxrwxrwx", String.valueOf(perm));
-
-        List<AclEntry> actual = getAcl(locn);
-        verifyAcls(aclSpec1, actual);
-        break;
-      case 1:
-        perm = fs.getFileStatus(new Path(locn)).getPermission();
-        Assert.assertEquals("Location: " + locn, "rwxrwxr-x", String.valueOf(perm));
-
-        List<AclEntry> acls = getAcl(locn);
-        verifyAcls(aclSpec2, acls);
-        break;
-      default:
-        throw new RuntimeException("Only 2 permissions by this test: " + permIndex);
-    }
-  }
-
-  /**
-   * Create a new AclEntry with scope, type and permission (no name).
-   *
-   * @param scope
-   *          AclEntryScope scope of the ACL entry
-   * @param type
-   *          AclEntryType ACL entry type
-   * @param permission
-   *          FsAction set of permissions in the ACL entry
-   * @return AclEntry new AclEntry
-   */
-  private AclEntry aclEntry(AclEntryScope scope, AclEntryType type,
-      FsAction permission) {
-    return new AclEntry.Builder().setScope(scope).setType(type)
-        .setPermission(permission).build();
-  }
-
-  /**
-   * Create a new AclEntry with scope, type, name and permission.
-   *
-   * @param scope
-   *          AclEntryScope scope of the ACL entry
-   * @param type
-   *          AclEntryType ACL entry type
-   * @param name
-   *          String optional ACL entry name
-   * @param permission
-   *          FsAction set of permissions in the ACL entry
-   * @return AclEntry new AclEntry
-   */
-  private AclEntry aclEntry(AclEntryScope scope, AclEntryType type,
-      String name, FsAction permission) {
-    return new AclEntry.Builder().setScope(scope).setType(type).setName(name)
-        .setPermission(permission).build();
-  }
-
-  private void verifyAcls(List<AclEntry> expectedList, List<AclEntry> actualList) {
-    for (AclEntry expected : expectedList) {
-      if (expected.getName() != null) {
-        //the non-named acl's are coming as regular permission, and not as aclEntries.
-        boolean found = false;
-        for (AclEntry actual : actualList) {
-          if (actual.equals(expected)) {
-            found = true;
-          }
-        }
-        if (!found) {
-          Assert.fail("Following Acl does not have a match: " + expected);
-        }
-      }
-    }
-  }
-
-  private void setAcl(String locn, List<AclEntry> aclSpec) throws Exception {
-    fs.setAcl(new Path(locn), aclSpec);
-  }
-
-  private List<AclEntry> getAcl(String locn) throws Exception {
-    return fs.getAclStatus(new Path(locn)).getEntries();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
index 028c117..62c109c 100644
--- a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
+++ b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
@@ -84,7 +84,6 @@ public class TestStorageBasedMetastoreAuthorizationProviderWithACL
     warehouseDir = new Path(new Path(fs.getUri()), "/warehouse");
     fs.mkdirs(warehouseDir);
     conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, warehouseDir.toString());
-    conf.setBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS, true);
 
     // Set up scratch directory
     Path scratchDir = new Path(new Path(fs.getUri()), "/scratchdir");

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
index c6a906a..bc00d11 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
@@ -26,8 +26,6 @@ public class TestEmbeddedHiveMetaStore extends TestHiveMetaStore {
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    hiveConf.setBoolean(
-        HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS.varname, true);
     warehouse = new Warehouse(hiveConf);
     try {
       client = new HiveMetaStoreClient(hiveConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 1b0b537..b95c25c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -171,14 +171,6 @@ public abstract class TestHiveMetaStore extends TestCase {
       db = client.getDatabase(dbName);
       Path dbPath = new Path(db.getLocationUri());
       FileSystem fs = FileSystem.get(dbPath.toUri(), hiveConf);
-      boolean inheritPerms = hiveConf.getBoolVar(
-          HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-      FsPermission dbPermission = fs.getFileStatus(dbPath).getPermission();
-      if (inheritPerms) {
-         //Set different perms for the database dir for further tests
-         dbPermission = new FsPermission((short)488);
-         fs.setPermission(dbPath, dbPermission);
-      }
 
       client.dropType(typeName);
       Type typ1 = new Type();
@@ -239,9 +231,6 @@ public abstract class TestHiveMetaStore extends TestCase {
         tbl = client.getTable(dbName, tblName);
       }
 
-      assertEquals(dbPermission, fs.getFileStatus(new Path(tbl.getSd().getLocation()))
-          .getPermission());
-
       Partition part = makePartitionObject(dbName, tblName, vals, tbl, "/part1");
       Partition part2 = makePartitionObject(dbName, tblName, vals2, tbl, "/part2");
       Partition part3 = makePartitionObject(dbName, tblName, vals3, tbl, "/part3");
@@ -259,20 +248,12 @@ public abstract class TestHiveMetaStore extends TestCase {
       assertTrue("getPartition() should have thrown NoSuchObjectException", exceptionThrown);
       Partition retp = client.add_partition(part);
       assertNotNull("Unable to create partition " + part, retp);
-      assertEquals(dbPermission, fs.getFileStatus(new Path(retp.getSd().getLocation()))
-          .getPermission());
       Partition retp2 = client.add_partition(part2);
       assertNotNull("Unable to create partition " + part2, retp2);
-      assertEquals(dbPermission, fs.getFileStatus(new Path(retp2.getSd().getLocation()))
-          .getPermission());
       Partition retp3 = client.add_partition(part3);
       assertNotNull("Unable to create partition " + part3, retp3);
-      assertEquals(dbPermission, fs.getFileStatus(new Path(retp3.getSd().getLocation()))
-          .getPermission());
       Partition retp4 = client.add_partition(part4);
       assertNotNull("Unable to create partition " + part4, retp4);
-      assertEquals(dbPermission, fs.getFileStatus(new Path(retp4.getSd().getLocation()))
-          .getPermission());
 
       Partition part_get = client.getPartition(dbName, tblName, part.getValues());
       if(isThriftClient) {
@@ -394,8 +375,6 @@ public abstract class TestHiveMetaStore extends TestCase {
       // tested
       retp = client.add_partition(part);
       assertNotNull("Unable to create partition " + part, retp);
-      assertEquals(dbPermission, fs.getFileStatus(new Path(retp.getSd().getLocation()))
-          .getPermission());
 
       // test add_partitions
 
@@ -431,9 +410,8 @@ public abstract class TestHiveMetaStore extends TestCase {
 
       // create dir for /mpart5
       Path mp5Path = new Path(mpart5.getSd().getLocation());
-      warehouse.mkdirs(mp5Path, true);
+      warehouse.mkdirs(mp5Path);
       assertTrue(fs.exists(mp5Path));
-      assertEquals(dbPermission, fs.getFileStatus(mp5Path).getPermission());
 
       // add_partitions(5,4) : err = duplicate keyvals on mpart4
       savedException = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
deleted file mode 100644
index 2ae9cc0..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/FolderPermissionBase.java
+++ /dev/null
@@ -1,792 +0,0 @@
-/**
- * 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.hadoop.hive.ql.security;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This test the flag 'hive.warehouse.subdir.inherit.perms'.
- */
-public abstract class FolderPermissionBase {
-  protected static HiveConf conf;
-  protected static Driver driver;
-  protected static String dataFileDir;
-  protected static Path dataFilePath;
-  protected static FileSystem fs;
-
-  protected static Path warehouseDir;
-  protected static Path baseDfsDir;
-
-  protected static final PathFilter hiddenFileFilter = new PathFilter(){
-    public boolean accept(Path p){
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
-
-  public abstract void setPermission(String locn, int permIndex) throws Exception;
-
-  public abstract void verifyPermission(String locn, int permIndex) throws Exception;
-
-
-  public void setPermission(String locn) throws Exception {
-    setPermission(locn, 0);
-  }
-
-  public void verifyPermission(String locn) throws Exception {
-    verifyPermission(locn, 0);
-  }
-
-
-  public static void baseSetup() throws Exception {
-    MiniDFSShim dfs = ShimLoader.getHadoopShims().getMiniDfs(conf, 4, true, null);
-    fs = dfs.getFileSystem();
-    baseDfsDir =  new Path(new Path(fs.getUri()), "/base");
-    fs.mkdirs(baseDfsDir);
-    warehouseDir = new Path(baseDfsDir, "warehouse");
-    fs.mkdirs(warehouseDir);
-    conf.setVar(ConfVars.METASTOREWAREHOUSE, warehouseDir.toString());
-
-    // Assuming the tests are run either in C or D drive in Windows OS!
-    dataFileDir = conf.get("test.data.files").replace('\\', '/')
-        .replace("c:", "").replace("C:", "").replace("D:", "").replace("d:", "");
-    dataFilePath = new Path(dataFileDir, "kv1.txt");
-
-    // Set up scratch directory
-    Path scratchDir = new Path(baseDfsDir, "scratchdir");
-    conf.setVar(HiveConf.ConfVars.SCRATCHDIR, scratchDir.toString());
-
-    //set hive conf vars
-    conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
-    conf.setBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS, true);
-    conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
-    int port = MetaStoreUtils.findFreePort();
-    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
-
-    SessionState.start(new CliSessionState(conf));
-    driver = new Driver(conf);
-    setupDataTable();
-  }
-
-
-  private static void setupDataTable() throws Exception {
-    CommandProcessorResponse ret = driver.run("DROP TABLE IF EXISTS mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("CREATE TABLE mysrc (key STRING, value STRING) PARTITIONED BY (part1 string, part2 string) STORED AS TEXTFILE");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("LOAD DATA LOCAL INPATH '" + dataFilePath + "' INTO TABLE mysrc PARTITION (part1='1',part2='1')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("LOAD DATA LOCAL INPATH '" + dataFilePath + "' INTO TABLE mysrc PARTITION (part1='2',part2='2')");
-    Assert.assertEquals(0,ret.getResponseCode());
-  }
-
-  @Before
-  public void setupBeforeTest() throws Exception {
-    driver.run("USE default");
-  }
-
-  @Test
-  public void testCreateDb() throws Exception {
-    //see if db inherits permission from warehouse directory.
-    String testDb = "mydb";
-    String tableName = "createtable";
-
-    setPermission(warehouseDir.toString());
-    verifyPermission(warehouseDir.toString());
-
-    CommandProcessorResponse ret = driver.run("CREATE DATABASE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db");
-    verifyPermission(warehouseDir + "/" + testDb + ".db");
-
-    ret = driver.run("USE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("CREATE TABLE " + tableName + " (key string, value string)");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + testDb + ".db/" + tableName);
-
-    ret = driver.run("insert into table " + tableName + " select key,value from default.mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db/" + tableName);
-    verifyPermission(warehouseDir + "/" + testDb + ".db/" + tableName);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + testDb + ".db/" + tableName).size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + testDb + ".db/" + tableName)) {
-      verifyPermission(child);
-    }
-
-    ret = driver.run("USE default");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    //cleanup after the test.
-    fs.delete(warehouseDir, true);
-    fs.mkdirs(warehouseDir);
-    Assert.assertEquals(listStatus(warehouseDir.toString()).size(), 0);
-    setupDataTable();
-  }
-
-  @Test
-  public void testCreateTable() throws Exception {
-    String testDb = "mydb2";
-    String tableName = "createtable";
-    CommandProcessorResponse ret = driver.run("CREATE DATABASE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db");
-    setPermission(warehouseDir + "/" + testDb + ".db");
-    verifyPermission(warehouseDir + "/" + testDb + ".db");
-
-    ret = driver.run("USE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("CREATE TABLE " + tableName + " (key string, value string)");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + testDb + ".db/" + tableName);
-
-    ret = driver.run("insert into table " + tableName + " select key,value from default.mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db/" + tableName);
-    verifyPermission(warehouseDir + "/" + testDb + ".db/" + tableName);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + testDb + ".db/" + tableName).size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + testDb + ".db/" + tableName)) {
-      verifyPermission(child);
-    }
-
-    ret = driver.run("USE default");
-    Assert.assertEquals(0,ret.getResponseCode());
-  }
-
-
-  @Test
-  public void testInsertNonPartTable() throws Exception {
-    //case 1 is non-partitioned table.
-    String tableName = "nonpart";
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string)");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    String tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(warehouseDir + "/" + tableName);
-
-    //case1A: insert into non-partitioned table.
-    setPermission(warehouseDir + "/" + tableName);
-    ret = driver.run("insert into table " + tableName + " select key,value from mysrc");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName);
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child);
-    }
-
-    //case1B: insert overwrite non-partitioned-table
-    setPermission(warehouseDir + "/" + tableName, 1);
-    ret = driver.run("insert overwrite table " + tableName + " select key,value from mysrc");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName, 1);
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  @Test
-  public void testInsertStaticSinglePartition() throws Exception {
-    String tableName = "singlestaticpart";
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 string)");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    setPermission(warehouseDir + "/" + tableName);
-
-    //insert into test
-    ret = driver.run("insert into table " + tableName + " partition(part1='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1");
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + tableName + "/part1=1").size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=1")) {
-      verifyPermission(child);
-    }
-
-    //insert overwrite test
-    setPermission(warehouseDir + "/" + tableName, 1);
-    setPermission(warehouseDir + "/" + tableName + "/part1=1", 1);
-    ret = driver.run("insert overwrite table " + tableName + " partition(part1='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName, 1);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1", 1);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + tableName + "/part1=1").size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=1")) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  @Test
-  public void testInsertStaticDualPartition() throws Exception {
-    String tableName = "dualstaticpart";
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 string, part2 string)");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    setPermission(warehouseDir + "/" + tableName);
-
-    //insert into test
-    ret = driver.run("insert into table " + tableName + " partition(part1='1', part2='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1");
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1/part2=1");
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + tableName + "/part1=1/part2=1").size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=1/part2=1")) {
-      verifyPermission(child);
-    }
-
-    //insert overwrite test
-    setPermission(warehouseDir + "/" + tableName, 1);
-    setPermission(warehouseDir + "/" + tableName + "/part1=1", 1);
-    setPermission(warehouseDir + "/" + tableName + "/part1=1/part2=1", 1);
-
-    ret = driver.run("insert overwrite table " + tableName + " partition(part1='1', part2='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName, 1);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1", 1);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=1/part2=1", 1);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + tableName + "/part1=1/part2=1").size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=1/part2=1")) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  @Test
-  public void testInsertDualDynamicPartitions() throws Exception {
-    String tableName = "dualdynamicpart";
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 string, part2 string)");
-    Assert.assertEquals(0, ret.getResponseCode());
-    assertExistence(warehouseDir + "/" + tableName);
-
-    //Insert into test, with permission set 0.
-    setPermission(warehouseDir + "/" + tableName, 0);
-    ret = driver.run("insert into table " + tableName + " partition (part1,part2) select key,value,part1,part2 from mysrc");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyDualPartitionTable(warehouseDir + "/" + tableName, 0);
-
-    //Insert overwrite test, with permission set 1.  We need reset existing partitions to 1 since the permissions
-    //should be inherited from existing partition
-    setDualPartitionTable(warehouseDir + "/" + tableName, 1);
-    ret = driver.run("insert overwrite table " + tableName + " partition (part1,part2) select key,value,part1,part2 from mysrc");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyDualPartitionTable(warehouseDir + "/" + tableName, 1);
-  }
-
-  @Test
-  public void testInsertSingleDynamicPartition() throws Exception {
-    String tableName = "singledynamicpart";
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 string)");
-    Assert.assertEquals(0,ret.getResponseCode());
-    String tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(tableLoc);
-
-    //Insert into test, with permission set 0.
-    setPermission(tableLoc, 0);
-    ret = driver.run("insert into table " + tableName + " partition (part1) select key,value,part1 from mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-    verifySinglePartition(tableLoc, 0);
-
-    //Insert overwrite test, with permission set 1. We need reset existing partitions to 1 since the permissions
-    //should be inherited from existing partition
-    setSinglePartition(tableLoc, 1);
-    ret = driver.run("insert overwrite table " + tableName + " partition (part1) select key,value,part1 from mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-    verifySinglePartition(tableLoc, 1);
-
-    //delete and re-insert using insert overwrite.  There's different code paths insert vs insert overwrite for new tables.
-    ret = driver.run("DROP TABLE " + tableName);
-    Assert.assertEquals(0, ret.getResponseCode());
-    ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 string)");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    setPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("insert overwrite table " + tableName + " partition (part1) select key,value,part1 from mysrc");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifySinglePartition(tableLoc, 0);
-  }
-
-  @Test
-  public void testPartition() throws Exception {
-    String tableName = "alterpart";
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 int, part2 int, part3 int)");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    setPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("insert into table " + tableName + " partition(part1='1',part2='1',part3='1') select key,value from mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    setPermission(warehouseDir + "/" + tableName, 1);
-
-    //alter partition
-    ret = driver.run("alter table " + tableName + " partition (part1='1',part2='1',part3='1') rename to partition (part1='2',part2='2',part3='2')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=2", 1);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=2/part2=2", 1);
-    verifyPermission(warehouseDir + "/" + tableName + "/part1=2/part2=2/part3=2", 1);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + tableName + "/part1=2/part2=2/part3=2").size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + tableName + "/part1=2/part2=2/part3=2")) {
-      verifyPermission(child, 1);
-    }
-
-    String tableName2 = "alterpart2";
-    ret = driver.run("CREATE TABLE " + tableName2 + " (key string, value string) partitioned by (part1 int, part2 int, part3 int)");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName2);
-    setPermission(warehouseDir + "/" + tableName2);
-    ret = driver.run("alter table " + tableName2 + " exchange partition (part1='2',part2='2',part3='2') with table " + tableName);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    //alter exchange can not change base table's permission
-    //alter exchange can only control final partition folder's permission
-    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2", 0);
-    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2/part2=2", 0);
-    verifyPermission(warehouseDir + "/" + tableName2 + "/part1=2/part2=2/part3=2", 1);
-  }
-
-  @Test
-  public void testExternalTable() throws Exception {
-    String tableName = "externaltable";
-
-    String myLocation = warehouseDir + "/myfolder";
-    FileSystem fs = FileSystem.get(new URI(myLocation), conf);
-    fs.mkdirs(new Path(myLocation));
-    setPermission(myLocation);
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) LOCATION '" + myLocation + "'");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("insert into table " + tableName + " select key,value from mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(myLocation).size() > 0);
-    for (String child : listStatus(myLocation)) {
-      verifyPermission(child);
-    }
-  }
-
-  @Test
-  public void testLoadLocal() throws Exception {
-    //case 1 is non-partitioned table.
-    String tableName = "loadlocal";
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string)");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    String tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(warehouseDir + "/" + tableName);
-
-    //case1A: load data local into non-partitioned table.
-    setPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("load data local inpath '" + dataFilePath + "' into table " + tableName);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child);
-    }
-
-    //case1B: load data local into overwrite non-partitioned-table
-    setPermission(warehouseDir + "/" + tableName, 1);
-    for (String child : listStatus(tableLoc)) {
-      setPermission(child, 1);
-    }
-    ret = driver.run("load data local inpath '" + dataFilePath + "' overwrite into table " + tableName);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child, 1);
-    }
-
-    //case 2 is partitioned table.
-    tableName = "loadlocalpartition";
-
-    ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 int, part2 int)");
-    Assert.assertEquals(0,ret.getResponseCode());
-    tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(tableLoc);
-
-    //case 2A: load data local into partitioned table.
-    setPermission(tableLoc);
-    ret = driver.run("LOAD DATA LOCAL INPATH '" + dataFilePath + "' INTO TABLE " + tableName + " PARTITION (part1='1',part2='1')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    String partLoc = warehouseDir + "/" + tableName + "/part1=1/part2=1";
-    Assert.assertTrue(listStatus(partLoc).size() > 0);
-    for (String child : listStatus(partLoc)) {
-      verifyPermission(child);
-    }
-
-    //case 2B: insert data overwrite into partitioned table. set testing table/partition folder hierarchy 1
-    //local load overwrite just overwrite the existing partition content but not the permission
-    setPermission(tableLoc, 1);
-    setPermission(partLoc, 1);
-    for (String child : listStatus(partLoc)) {
-      setPermission(child, 1);
-    }
-    ret = driver.run("LOAD DATA LOCAL INPATH '" + dataFilePath + "' OVERWRITE INTO TABLE " + tableName + " PARTITION (part1='1',part2='1')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(partLoc)) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  @Test
-  public void testLoad() throws Exception {
-    String tableName = "load";
-    String location = "/hdfsPath";
-    fs.copyFromLocalFile(dataFilePath, new Path(location));
-
-    //case 1: load data
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key string, value string)");
-    Assert.assertEquals(0,ret.getResponseCode());
-    String tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(warehouseDir + "/" + tableName);
-
-    //case1A: load data into non-partitioned table.
-    setPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("load data inpath '" + location + "' into table " + tableName);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child);
-    }
-
-    //case1B: load data into overwrite non-partitioned-table
-    setPermission(warehouseDir + "/" + tableName, 1);
-    for (String child : listStatus(tableLoc)) {
-      setPermission(child, 1);
-    }
-
-    fs.copyFromLocalFile(dataFilePath, new Path(location));
-    ret = driver.run("load data inpath '" + location + "' overwrite into table " + tableName);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(tableLoc)) {
-      verifyPermission(child, 1);
-    }
-
-    //case 2 is partitioned table.
-    tableName = "loadpartition";
-
-    ret = driver.run("CREATE TABLE " + tableName + " (key string, value string) partitioned by (part1 int, part2 int)");
-    Assert.assertEquals(0,ret.getResponseCode());
-    tableLoc = warehouseDir + "/" + tableName;
-    assertExistence(tableLoc);
-
-    //case 2A: load data into partitioned table.
-    setPermission(tableLoc);
-    fs.copyFromLocalFile(dataFilePath, new Path(location));
-    ret = driver.run("LOAD DATA INPATH '" + location + "' INTO TABLE " + tableName + " PARTITION (part1='1',part2='1')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    String partLoc = warehouseDir + "/" + tableName + "/part1=1/part2=1";
-    Assert.assertTrue(listStatus(partLoc).size() > 0);
-    for (String child : listStatus(partLoc)) {
-      verifyPermission(child);
-    }
-
-    //case 2B: insert data overwrite into partitioned table. set testing table/partition folder hierarchy 1
-    //load overwrite just overwrite the existing partition content but not the permission
-    setPermission(tableLoc, 1);
-    setPermission(partLoc, 1);
-    Assert.assertTrue(listStatus(partLoc).size() > 0);
-    for (String child : listStatus(partLoc)) {
-      setPermission(child, 1);
-    }
-
-    fs.copyFromLocalFile(dataFilePath, new Path(location));
-    ret = driver.run("LOAD DATA INPATH '" + location + "' OVERWRITE INTO TABLE " + tableName + " PARTITION (part1='1',part2='1')");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    Assert.assertTrue(listStatus(tableLoc).size() > 0);
-    for (String child : listStatus(partLoc)) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  @Test
-  public void testCtas() throws Exception {
-    String testDb = "ctasdb";
-    String tableName = "createtable";
-    CommandProcessorResponse ret = driver.run("CREATE DATABASE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db");
-    setPermission(warehouseDir + "/" + testDb + ".db");
-    verifyPermission(warehouseDir + "/" + testDb + ".db");
-
-    ret = driver.run("USE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("create table " + tableName + " as select key,value from default.mysrc");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + testDb + ".db/" + tableName);
-    verifyPermission(warehouseDir + "/" + testDb + ".db/" + tableName);
-
-    Assert.assertTrue(listStatus(warehouseDir + "/" + testDb + ".db/" + tableName).size() > 0);
-    for (String child : listStatus(warehouseDir + "/" + testDb + ".db/" + tableName)) {
-      verifyPermission(child);
-    }
-
-    ret = driver.run("USE default");
-    Assert.assertEquals(0,ret.getResponseCode());
-  }
-
-  @Test
-  public void testExim() throws Exception {
-
-    //export the table to external file.
-    String myLocation = warehouseDir + "/exim";
-    FileSystem fs = FileSystem.get(new URI(myLocation), conf);
-    fs.mkdirs(new Path(myLocation));
-    setPermission(myLocation);
-    myLocation = myLocation + "/temp";
-
-    CommandProcessorResponse ret = driver.run("export table mysrc to '" + myLocation + "'");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    //check if exported data has inherited the permissions.
-    assertExistence(myLocation);
-    verifyPermission(myLocation);
-
-    assertExistence(myLocation + "/part1=1/part2=1");
-    verifyPermission(myLocation + "/part1=1/part2=1");
-    Assert.assertTrue(listStatus(myLocation + "/part1=1/part2=1").size() > 0);
-    for (String child : listStatus(myLocation + "/part1=1/part2=1")) {
-      verifyPermission(child);
-    }
-
-    assertExistence(myLocation + "/part1=2/part2=2");
-    verifyPermission(myLocation + "/part1=2/part2=2");
-    Assert.assertTrue(listStatus(myLocation + "/part1=2/part2=2").size() > 0);
-    for (String child : listStatus(myLocation + "/part1=2/part2=2")) {
-      verifyPermission(child);
-    }
-
-    //import the table back into another database
-    String testDb = "eximdb";
-    ret = driver.run("CREATE DATABASE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    //use another permission for this import location, to verify that it is really set (permIndex=2)
-    assertExistence(warehouseDir + "/" + testDb + ".db");
-    setPermission(warehouseDir + "/" + testDb + ".db", 1);
-
-    ret = driver.run("USE " + testDb);
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    ret = driver.run("import from '" + myLocation + "'");
-    Assert.assertEquals(0,ret.getResponseCode());
-
-    //check permissions of imported, from the exported table
-    assertExistence(warehouseDir + "/" + testDb + ".db/mysrc");
-    verifyPermission(warehouseDir + "/" + testDb + ".db/mysrc", 1);
-
-    myLocation = warehouseDir + "/" + testDb + ".db/mysrc";
-    assertExistence(myLocation);
-    verifyPermission(myLocation, 1);
-
-    assertExistence(myLocation + "/part1=1/part2=1");
-    verifyPermission(myLocation + "/part1=1/part2=1", 1);
-    Assert.assertTrue(listStatus(myLocation + "/part1=1/part2=1").size() > 0);
-    for (String child : listStatus(myLocation + "/part1=1/part2=1")) {
-      verifyPermission(child, 1);
-    }
-
-    assertExistence(myLocation + "/part1=2/part2=2");
-    verifyPermission(myLocation + "/part1=2/part2=2", 1);
-    Assert.assertTrue(listStatus(myLocation + "/part1=2/part2=2").size() > 0);
-    for (String child : listStatus(myLocation + "/part1=2/part2=2")) {
-      verifyPermission(child, 1);
-    }
-  }
-
-  /**
-   * Tests the permission to the table doesn't change after the truncation
-   * @throws Exception
-   */
-  @Test
-  public void testTruncateTable() throws Exception {
-    String tableName = "truncatetable";
-    String partition = warehouseDir + "/" + tableName + "/part1=1";
-
-    CommandProcessorResponse ret = driver.run("CREATE TABLE " + tableName + " (key STRING, value STRING) PARTITIONED BY (part1 INT)");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    setPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("insert into table " + tableName + " partition(part1='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-
-    verifyPermission(warehouseDir + "/" + tableName);
-    verifyPermission(partition);
-
-    ret = driver.run("TRUNCATE TABLE " + tableName);
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(warehouseDir + "/" + tableName);
-    verifyPermission(warehouseDir + "/" + tableName);
-
-    ret = driver.run("insert into table " + tableName + " partition(part1='1') select key,value from mysrc where part1='1' and part2='1'");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    verifyPermission(warehouseDir + "/" + tableName);
-
-    assertExistence(partition);
-    verifyPermission(partition);    
-
-    // Also test the partition folder if the partition is truncated
-    ret = driver.run("TRUNCATE TABLE " + tableName + " partition(part1='1')");
-    Assert.assertEquals(0, ret.getResponseCode());
-
-    assertExistence(partition);
-    verifyPermission(partition);
-  }
-
-  private void setSinglePartition(String tableLoc, int index) throws Exception {
-    setPermission(tableLoc + "/part1=1", index);
-    setPermission(tableLoc + "/part1=2", index);
-  }
-
-  private void verifySinglePartition(String tableLoc, int index) throws Exception {
-    verifyPermission(tableLoc + "/part1=1", index);
-    verifyPermission(tableLoc + "/part1=2", index);
-
-    Assert.assertTrue(listStatus(tableLoc + "/part1=1").size() > 0);
-    for (String child : listStatus(tableLoc + "/part1=1")) {
-      verifyPermission(child, index);
-    }
-
-    Assert.assertTrue(listStatus(tableLoc + "/part1=2").size() > 0);
-    for (String child : listStatus(tableLoc + "/part1=2")) {
-      verifyPermission(child, index);
-    }
-  }
-
-  private void setDualPartitionTable(String baseTablePath, int index) throws Exception {
-    setPermission(baseTablePath, index);
-    setPermission(baseTablePath + "/part1=1", index);
-    setPermission(baseTablePath + "/part1=1/part2=1", index);
-
-    setPermission(baseTablePath + "/part1=2", index);
-    setPermission(baseTablePath + "/part1=2/part2=2", index);
-  }
-
-  private void verifyDualPartitionTable(String baseTablePath, int index) throws Exception {
-    verifyPermission(baseTablePath, index);
-    verifyPermission(baseTablePath + "/part1=1", index);
-    verifyPermission(baseTablePath + "/part1=1/part2=1", index);
-
-    verifyPermission(baseTablePath + "/part1=2", index);
-    verifyPermission(baseTablePath + "/part1=2/part2=2", index);
-
-    Assert.assertTrue(listStatus(baseTablePath + "/part1=1/part2=1").size() > 0);
-    for (String child : listStatus(baseTablePath + "/part1=1/part2=1")) {
-      verifyPermission(child, index);
-    }
-
-    Assert.assertTrue(listStatus(baseTablePath + "/part1=2/part2=2").size() > 0);
-    for (String child : listStatus(baseTablePath + "/part1=2/part2=2")) {
-      verifyPermission(child, index);
-    }
-  }
-
-  private void assertExistence(String locn) throws Exception {
-    Assert.assertTrue(fs.exists(new Path(locn)));
-  }
-
-  private List<String> listStatus(String locn) throws Exception {
-    List<String> results = new ArrayList<String>();
-    FileStatus[] listStatus = fs.listStatus(new Path(locn), hiddenFileFilter);
-    for (FileStatus status : listStatus) {
-      results.add(status.getPath().toString());
-    }
-    return results;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestFolderPermissions.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestFolderPermissions.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestFolderPermissions.java
deleted file mode 100644
index 6cc2d18..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestFolderPermissions.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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.hadoop.hive.ql.security;
-
-import junit.framework.Assert;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestFolderPermissions extends FolderPermissionBase {
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    conf = new HiveConf(TestFolderPermissions.class);
-    conf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
-    baseSetup();
-  }
-
-  public FsPermission[] expected = new FsPermission[] {
-     FsPermission.createImmutable((short) 0777),
-     FsPermission.createImmutable((short) 0766)
-  };
-
-  @Override
-  public void setPermission(String locn, int permIndex) throws Exception {
-    fs.setPermission(new Path(locn), expected[permIndex]);
-  }
-
-  @Override
-  public void verifyPermission(String locn, int permIndex) throws Exception {
-    FsPermission actual =  fs.getFileStatus(new Path(locn)).getPermission();
-    Assert.assertEquals(expected[permIndex], actual);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java
deleted file mode 100644
index bb65ee7..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/**
- * 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.hadoop.hive.ql.security;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
-import org.apache.hadoop.hive.shims.Utils;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases focusing on drop table permission checks
- */
-public class TestStorageBasedMetastoreAuthorizationDrops extends StorageBasedMetastoreTestBase {
-
-  protected static MiniDFSShim dfs = null;
-
-  @Override
-  protected HiveConf createHiveConf() throws Exception {
-    // Hadoop FS ACLs do not work with LocalFileSystem, so set up MiniDFS.
-    HiveConf conf = super.createHiveConf();
-
-    String currentUserName = Utils.getUGI().getShortUserName();
-    conf.set("hadoop.proxyuser." + currentUserName + ".groups", "*");
-    conf.set("hadoop.proxyuser." + currentUserName + ".hosts", "*");
-    dfs = ShimLoader.getHadoopShims().getMiniDfs(conf, 4, true, null);
-    FileSystem fs = dfs.getFileSystem();
-
-    Path warehouseDir = new Path(new Path(fs.getUri()), "/warehouse");
-    fs.mkdirs(warehouseDir);
-    conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, warehouseDir.toString());
-    conf.setBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS, true);
-
-    // Set up scratch directory
-    Path scratchDir = new Path(new Path(fs.getUri()), "/scratchdir");
-    conf.setVar(HiveConf.ConfVars.SCRATCHDIR, scratchDir.toString());
-
-    return conf;
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    super.tearDown();
-
-    if (dfs != null) {
-      dfs.shutdown();
-      dfs = null;
-    }
-  }
-
-  @Test
-  public void testDropDatabase() throws Exception {
-    dropDatabaseByOtherUser("-rwxrwxrwx", 0);
-    dropDatabaseByOtherUser("-rwxrwxrwt", 1);
-  }
-
-  /**
-   * Creates db and tries to drop as 'other' user
-   * @param perm - permission for warehouse dir
-   * @param expectedRet - expected return code for drop by other user
-   * @throws Exception
-   */
-  public void dropDatabaseByOtherUser(String perm, int expectedRet) throws Exception {
-    String dbName = getTestDbName();
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), perm);
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-
-    InjectableDummyAuthenticator.injectMode(true);
-
-
-    resp = driver.run("drop database " + dbName);
-    Assert.assertEquals(expectedRet, resp.getResponseCode());
-
-  }
-
-  @Test
-  public void testDropTable() throws Exception {
-    dropTableByOtherUser("-rwxrwxrwx", 0);
-    dropTableByOtherUser("-rwxrwxrwt", 1);
-  }
-
-  /**
-   * @param perm dir permission for database dir
-   * @param expectedRet expected return code on drop table
-   * @throws Exception
-   */
-  public void dropTableByOtherUser(String perm, int expectedRet) throws Exception {
-    String dbName = getTestDbName();
-    String tblName = getTestTableName();
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-
-    setPermissions(db.getLocationUri(), perm);
-
-    String dbDotTable = dbName + "." + tblName;
-    resp = driver.run("create table " + dbDotTable + "(i int)");
-    Assert.assertEquals(0, resp.getResponseCode());
-
-
-    InjectableDummyAuthenticator.injectMode(true);
-    resp = driver.run("drop table " + dbDotTable);
-    Assert.assertEquals(expectedRet, resp.getResponseCode());
-  }
-
-  /**
-   * Drop view should not be blocked by SBA. View will not have any location to drop.
-   * @throws Exception
-   */
-  @Test
-  public void testDropView() throws Exception {
-    String dbName = getTestDbName();
-    String tblName = getTestTableName();
-    String viewName = "view" + tblName;
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-
-    setPermissions(db.getLocationUri(), "-rwxrwxrwt");
-
-    String dbDotTable = dbName + "." + tblName;
-    resp = driver.run("create table " + dbDotTable + "(i int)");
-    Assert.assertEquals(0, resp.getResponseCode());
-
-    String dbDotView = dbName + "." + viewName;
-    resp = driver.run("create view " + dbDotView + " as select * from " +  dbDotTable);
-    Assert.assertEquals(0, resp.getResponseCode());
-
-    resp = driver.run("drop view " + dbDotView);
-    Assert.assertEquals(0, resp.getResponseCode());
-
-    resp = driver.run("drop table " + dbDotTable);
-    Assert.assertEquals(0, resp.getResponseCode());
-  }
-
-  @Test
-  public void testDropPartition() throws Exception {
-    dropPartitionByOtherUser("-rwxrwxrwx", 0);
-    dropPartitionByOtherUser("-rwxrwxrwt", 1);
-  }
-
-  /**
-   * @param perm permissions for table dir
-   * @param expectedRet expected return code
-   * @throws Exception
-   */
-  public void dropPartitionByOtherUser(String perm, int expectedRet) throws Exception {
-    String dbName = getTestDbName();
-    String tblName = getTestTableName();
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-    setPermissions(db.getLocationUri(), "-rwxrwxrwx");
-
-    String dbDotTable = dbName + "." + tblName;
-    resp = driver.run("create table " + dbDotTable + "(i int) partitioned by (b string)");
-    Assert.assertEquals(0, resp.getResponseCode());
-    Table tab = msc.getTable(dbName, tblName);
-    setPermissions(tab.getSd().getLocation(), perm);
-
-    resp = driver.run("alter table " + dbDotTable + " add partition (b='2011')");
-    Assert.assertEquals(0, resp.getResponseCode());
-
-    InjectableDummyAuthenticator.injectMode(true);
-    resp = driver.run("alter table " + dbDotTable + " drop partition (b='2011')");
-    Assert.assertEquals(expectedRet, resp.getResponseCode());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java
deleted file mode 100644
index ea631d2..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * 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.hadoop.hive.ql.security;
-
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.Driver;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test cases focusing on read table permission checks
- */
-public class TestStorageBasedMetastoreAuthorizationReads extends StorageBasedMetastoreTestBase {
-
-  @Test
-  public void testReadTableSuccess() throws Exception {
-    readTableByOtherUser("-rwxrwxrwx", true);
-  }
-
-  @Test
-  public void testReadTableSuccessWithReadOnly() throws Exception {
-    readTableByOtherUser("-r--r--r--", true);
-  }
-
-  @Test
-  public void testReadTableFailure() throws Exception {
-    readTableByOtherUser("-rwxrwx---", false);
-  }
-
-  /**
-   * @param perm dir permission for table dir
-   * @param isSuccess if command was successful
-   * @throws Exception
-   */
-  private void readTableByOtherUser(String perm, boolean isSuccess) throws Exception {
-    String dbName = getTestDbName();
-    String tblName = getTestTableName();
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-
-    setPermissions(db.getLocationUri(), "-rwxrwxrwx");
-
-    String dbDotTable = dbName + "." + tblName;
-    resp = driver.run("create table " + dbDotTable + "(i int) partitioned by (`date` string)");
-    Assert.assertEquals(0, resp.getResponseCode());
-    Table tab = msc.getTable(dbName, tblName);
-    setPermissions(tab.getSd().getLocation(), perm);
-
-    InjectableDummyAuthenticator.injectMode(true);
-
-    testCmd(driver, "DESCRIBE  " + dbDotTable, isSuccess);
-    testCmd(driver, "DESCRIBE EXTENDED  " + dbDotTable, isSuccess);
-    testCmd(driver, "SHOW PARTITIONS  " + dbDotTable, isSuccess);
-    testCmd(driver, "SHOW COLUMNS IN " + tblName + " IN " + dbName, isSuccess);
-    testCmd(driver, "use " + dbName, true);
-    testCmd(driver, "SHOW TABLE EXTENDED LIKE " + tblName, isSuccess);
-
-  }
-
-  @Test
-  public void testReadDbSuccess() throws Exception {
-    readDbByOtherUser("-rwxrwxrwx", true);
-  }
-
-  @Test
-  public void testReadDbFailure() throws Exception {
-    readDbByOtherUser("-rwxrwx---", false);
-  }
-
-
-  /**
-   * @param perm dir permission for database dir
-   * @param isSuccess if command was successful
-   * @throws Exception
-   */
-  private void readDbByOtherUser(String perm, boolean isSuccess) throws Exception {
-    String dbName = getTestDbName();
-    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), perm);
-
-    CommandProcessorResponse resp = driver.run("create database " + dbName);
-    Assert.assertEquals(0, resp.getResponseCode());
-    Database db = msc.getDatabase(dbName);
-    validateCreateDb(db, dbName);
-    setPermissions(db.getLocationUri(), perm);
-
-    InjectableDummyAuthenticator.injectMode(true);
-
-    testCmd(driver, "DESCRIBE DATABASE " + dbName, isSuccess);
-    testCmd(driver, "DESCRIBE DATABASE EXTENDED " + dbName, isSuccess);
-    testCmd(driver, "SHOW TABLES IN " + dbName, isSuccess);
-    driver.run("use " + dbName);
-    testCmd(driver, "SHOW TABLES ", isSuccess);
-
-  }
-
-  private void testCmd(Driver driver, String cmd, boolean isSuccess)
-      throws CommandNeedRetryException {
-    CommandProcessorResponse resp = driver.run(cmd);
-    Assert.assertEquals(isSuccess, resp.getResponseCode() == 0);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index d0511ad..de704e8 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -518,11 +518,11 @@ public class HiveAlterHandler implements AlterHandler {
           if (srcFs.exists(srcPath)) {
             //if destPath's parent path doesn't exist, we should mkdir it
             Path destParentPath = destPath.getParent();
-            if (!wh.mkdirs(destParentPath, true)) {
+            if (!wh.mkdirs(destParentPath)) {
                 throw new IOException("Unable to create path " + destParentPath);
             }
 
-            wh.renameDir(srcPath, destPath, true);
+            wh.renameDir(srcPath, destPath);
             LOG.info("Partition directory rename from " + srcPath + " to " + destPath + " done.");
           }
         } catch (IOException ex) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 217b341..a4693bd 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -875,7 +875,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         firePreEvent(new PreCreateDatabaseEvent(db, this));
         if (!wh.isDir(dbPath)) {
-          if (!wh.mkdirs(dbPath, true)) {
+          if (!wh.mkdirs(dbPath)) {
             throw new MetaException("Unable to create database path " + dbPath +
                 ", failed to create database " + db.getName());
           }
@@ -1432,7 +1432,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         if (tblPath != null) {
           if (!wh.isDir(tblPath)) {
-            if (!wh.mkdirs(tblPath, true)) {
+            if (!wh.mkdirs(tblPath)) {
               throw new MetaException(tblPath
                   + " is not a directory or unable to create one");
             }
@@ -2227,7 +2227,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         if (!wh.isDir(partLocation)) {
-          if (!wh.mkdirs(partLocation, true)) {
+          if (!wh.mkdirs(partLocation)) {
             throw new MetaException(partLocation
                 + " is not a directory or unable to create one");
           }
@@ -2778,7 +2778,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         // mkdirs() because if the file system is read-only, mkdirs will
         // throw an exception even if the directory already exists.
         if (!wh.isDir(partLocation)) {
-          if (!wh.mkdirs(partLocation, true)) {
+          if (!wh.mkdirs(partLocation)) {
             throw new MetaException(partLocation
                 + " is not a directory or unable to create one");
           }
@@ -2988,7 +2988,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
         Path destParentPath = destPath.getParent();
         if (!wh.isDir(destParentPath)) {
-          if (!wh.mkdirs(destParentPath, true)) {
+          if (!wh.mkdirs(destParentPath)) {
               throw new MetaException("Unable to create path " + destParentPath);
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
index a65a2e7..f8a98e8 100755
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
@@ -183,13 +183,11 @@ public class Warehouse {
     return partition.getDbName() + "." + partition.getTableName() + partition.getValues();
   }
 
-  public boolean mkdirs(Path f, boolean inheritPermCandidate) throws MetaException {
-    boolean inheritPerms = HiveConf.getBoolVar(conf,
-      HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS) && inheritPermCandidate;
+  public boolean mkdirs(Path f) throws MetaException {
     FileSystem fs = null;
     try {
       fs = getFs(f);
-      return FileUtils.mkdir(fs, f, inheritPerms, conf);
+      return FileUtils.mkdir(fs, f, conf);
     } catch (IOException e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -197,13 +195,9 @@ public class Warehouse {
   }
 
   public boolean renameDir(Path sourcePath, Path destPath) throws MetaException {
-    return renameDir(sourcePath, destPath, false);
-  }
-
-  public boolean renameDir(Path sourcePath, Path destPath, boolean inheritPerms) throws MetaException {
     try {
       FileSystem fs = getFs(sourcePath);
-      return FileUtils.renameWithPerms(fs, sourcePath, destPath, inheritPerms, conf);
+      return FileUtils.rename(fs, sourcePath, destPath, conf);
     } catch (Exception ex) {
       MetaStoreUtils.logAndThrowMetaException(ex);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
index f8eed18..2166c20 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
@@ -42,7 +42,6 @@ public class TestHiveMetaStoreTimeout {
   public static void setUp() throws Exception {
     HiveMetaStore.TEST_TIMEOUT_ENABLED = true;
     hiveConf = new HiveConf(TestHiveMetaStoreTimeout.class);
-    hiveConf.setBoolean(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS.varname, true);
     hiveConf.set(HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS.varname,
         MockPartitionExpressionForMetastore.class.getCanonicalName());
     hiveConf.setTimeVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 10 * 1000,

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index d1d2789..08bba3d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -355,9 +355,7 @@ public class Context {
 
       if (mkdir) {
         try {
-          boolean inheritPerms = HiveConf.getBoolVar(conf,
-              HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-          if (!FileUtils.mkdir(fs, dir, inheritPerms, conf)) {
+          if (!FileUtils.mkdir(fs, dir, conf)) {
             throw new IllegalStateException("Cannot create staging directory  '" + dir.toString() + "'");
           }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
index cbe0aca..2683f29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
@@ -70,8 +70,7 @@ public class CopyTask extends Task<CopyWork> implements Serializable {
         }
       }
 
-      boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-      if (!FileUtils.mkdir(dstFs, toPath, inheritPerms, conf)) {
+      if (!FileUtils.mkdir(dstFs, toPath, conf)) {
         console.printError("Cannot make target directory: " + toPath.toString());
         return 2;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 1802f37..f329b51 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -152,7 +152,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
         throw new HiveException("Target " + targetPath + " is not a local directory.");
       }
     } else {
-      if (!FileUtils.mkdir(dstFs, targetPath, false, conf)) {
+      if (!FileUtils.mkdir(dstFs, targetPath, conf)) {
         throw new HiveException("Failed to create local target directory " + targetPath);
       }
     }
@@ -181,9 +181,6 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
         actualPath = actualPath.getParent();
       }
       fs.mkdirs(mkDirPath);
-      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS)) {
-        HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, actualPath), fs, mkDirPath, true);
-      }
     }
     return deletePath;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
index 4686e2c..d2f9e79 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
@@ -106,8 +106,7 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
       srcFiles.addAll(Arrays.asList(srcs));
       LOG.debug("ReplCopyTask numFiles:" + (srcFiles == null ? "null" : srcFiles.size()));
 
-      boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-      if (!FileUtils.mkdir(dstFs, toPath, inheritPerms, conf)) {
+      if (!FileUtils.mkdir(dstFs, toPath, conf)) {
         console.printError("Cannot make target directory: " + toPath.toString());
         return 2;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index f64cfda..6deea96 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -2889,8 +2889,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
     if (!fullDestStatus.getFileStatus().isDirectory()) {
       throw new HiveException(destf + " is not a directory.");
     }
-    final boolean inheritPerms = HiveConf.getBoolVar(conf,
-        HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
     final List<Future<ObjectPair<Path, Path>>> futures = new LinkedList<>();
     final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ?
         Executors.newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25),
@@ -2938,9 +2936,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
               Path destPath = mvFile(conf, srcFs, srcP, destFs, destf, isSrcLocal, isRenameAllowed);
 
-              if (inheritPerms) {
-                HdfsUtils.setFullFileStatus(conf, fullDestStatus, srcGroup, destFs, destPath, false);
-              }
               if (null != newFiles) {
                 newFiles.add(destPath);
               }
@@ -2950,11 +2945,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         }
       }
     }
-    if (null == pool) {
-      if (inheritPerms) {
-        HdfsUtils.setFullFileStatus(conf, fullDestStatus, null, destFs, destf, true);
-      }
-    } else {
+    if (null != pool) {
       pool.shutdown();
       for (Future<ObjectPair<Path, Path>> future : futures) {
         try {
@@ -3104,9 +3095,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e.getMessage(), e);
     }
 
-    //needed for perm inheritance.
-    final boolean inheritPerms = HiveConf.getBoolVar(conf,
-        HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
     HdfsUtils.HadoopFileStatus destStatus = null;
 
     // If source path is a subdirectory of the destination path:
@@ -3117,7 +3105,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     // (3) Move individual files from scr dir to dest dir.
     boolean destIsSubDir = isSubDir(srcf, destf, srcFs, destFs, isSrcLocal);
     try {
-      if (inheritPerms || replace) {
+      if (replace) {
         try{
           destStatus = new HdfsUtils.HadoopFileStatus(conf, destFs, destf);
           //if destf is an existing directory:
@@ -3130,10 +3118,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
             LOG.debug("The path " + destf.toString() + " is deleted");
           }
         } catch (FileNotFoundException ignore) {
-          //if dest dir does not exist, any re
-          if (inheritPerms) {
-            destStatus = new HdfsUtils.HadoopFileStatus(conf, destFs, destf.getParent());
-          }
         }
       }
       final HdfsUtils.HadoopFileStatus desiredStatus = destStatus;
@@ -3141,9 +3125,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
       if (isSrcLocal) {
         // For local src file, copy to hdfs
         destFs.copyFromLocalFile(srcf, destf);
-        if (inheritPerms) {
-          HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
-        }
         return true;
       } else {
         if (needToCopy(srcf, destf, srcFs, destFs)) {
@@ -3176,11 +3157,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                   public Void call() throws Exception {
                     SessionState.setCurrentSessionState(parentSession);
                     final String group = srcStatus.getGroup();
-                    if(destFs.rename(srcStatus.getPath(), destFile)) {
-                      if (inheritPerms) {
-                        HdfsUtils.setFullFileStatus(conf, desiredStatus, group, destFs, destFile, false);
-                      }
-                    } else {
+                    if(!destFs.rename(srcStatus.getPath(), destFile)) {
                       throw new IOException("rename for src path: " + srcStatus.getPath() + " to dest path:"
                           + destFile + " returned false");
                     }
@@ -3189,11 +3166,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                 }));
               }
             }
-            if (null == pool) {
-              if (inheritPerms) {
-                HdfsUtils.setFullFileStatus(conf, desiredStatus, null, destFs, destf, true);
-              }
-            } else {
+            if (null != pool) {
               pool.shutdown();
               for (Future<Void> future : futures) {
                 try {
@@ -3208,9 +3181,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
             return true;
           } else {
             if (destFs.rename(srcf, destf)) {
-              if (inheritPerms) {
-                HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
-              }
               return true;
             }
             return false;
@@ -3261,12 +3231,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
    */
   static protected void copyFiles(HiveConf conf, Path srcf, Path destf,
       FileSystem fs, boolean isSrcLocal, boolean isAcid, List<Path> newFiles) throws HiveException {
-    boolean inheritPerms = HiveConf.getBoolVar(conf,
-        HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
     try {
       // create the destination if it does not exist
       if (!fs.exists(destf)) {
-        FileUtils.mkdir(fs, destf, inheritPerms, conf);
+        FileUtils.mkdir(fs, destf, conf);
       }
     } catch (IOException e) {
       throw new HiveException(
@@ -3459,10 +3427,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
       }
 
       // first call FileUtils.mkdir to make sure that destf directory exists, if not, it creates
-      // destf with inherited permissions
-      boolean inheritPerms = HiveConf.getBoolVar(conf, HiveConf.ConfVars
-          .HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
-      boolean destfExist = FileUtils.mkdir(destFs, destf, inheritPerms, conf);
+      // destf
+      boolean destfExist = FileUtils.mkdir(destFs, destf, conf);
       if(!destfExist) {
         throw new IOException("Directory " + destf.toString()
             + " does not exist and could not be created.");

http://git-wip-us.apache.org/repos/asf/hive/blob/ccc96b47/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 8eb011e..a319b88 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -391,7 +391,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       throw new MetaException("Temp table path not set for " + tbl.getTableName());
     } else {
       if (!wh.isDir(tblPath)) {
-        if (!wh.mkdirs(tblPath, true)) {
+        if (!wh.mkdirs(tblPath)) {
           throw new MetaException(tblPath
               + " is not a directory or unable to create one");
         }