You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2017/04/28 15:26:38 UTC

hive git commit: Revert "HIVE-16346: inheritPerms should be conditional based on the target filesystem (Sahil Takiar, reviewed by Aihua Xu)"

Repository: hive
Updated Branches:
  refs/heads/branch-2 ab3a24bf6 -> 2ba121757


Revert "HIVE-16346: inheritPerms should be conditional based on the target filesystem (Sahil Takiar, reviewed by Aihua Xu)"

This reverts commit cce4d5e78582c8744972d265147d39a345e082db.


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

Branch: refs/heads/branch-2
Commit: 2ba121757699c551a7688f1e42c0d4ecedc826f8
Parents: ab3a24b
Author: Aihua Xu <ai...@apache.org>
Authored: Fri Apr 28 11:22:22 2017 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Fri Apr 28 11:22:22 2017 -0400

----------------------------------------------------------------------
 common/pom.xml                                  |  8 +--
 .../hadoop/hive/common/BlobStorageUtils.java    | 10 +--
 .../apache/hadoop/hive/common/FileUtils.java    | 39 ++---------
 .../apache/hadoop/hive/common/StorageUtils.java | 40 -----------
 .../hive/common/TestBlobStorageUtils.java       |  8 +--
 .../hadoop/hive/common/TestStorageUtils.java    | 57 ---------------
 .../apache/hadoop/hive/metastore/Warehouse.java |  5 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |  4 +-
 .../apache/hadoop/hive/ql/exec/CopyTask.java    |  3 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |  4 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |  3 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    | 29 ++++----
 .../org/apache/hadoop/hive/io/HdfsUtils.java    | 73 +++++++++-----------
 .../apache/hadoop/hive/io/TestHdfsUtils.java    | 19 -----
 14 files changed, 72 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 84bb1e5..e1c15ee 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -157,13 +157,7 @@
           <artifactId>commons-logging</artifactId>
         </exclusion>
       </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <version>${hadoop.version}</version>
-      <optional>true</optional>
-    </dependency>
+  </dependency>
     <!-- test inter-project -->
     <dependency>
       <groupId>com.google.code.tempus-fugit</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/common/src/java/org/apache/hadoop/hive/common/BlobStorageUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/BlobStorageUtils.java b/common/src/java/org/apache/hadoop/hive/common/BlobStorageUtils.java
index b7f1359..e6a17cb 100644
--- a/common/src/java/org/apache/hadoop/hive/common/BlobStorageUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/BlobStorageUtils.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hive.common;
 
 import org.apache.hadoop.conf.Configuration;
@@ -25,23 +24,21 @@ import org.apache.hadoop.hive.conf.HiveConf;
 
 import java.util.Collection;
 
-
 /**
  * Utilities for different blob (object) storage systems
  */
 public class BlobStorageUtils {
-
     private static final boolean DISABLE_BLOBSTORAGE_AS_SCRATCHDIR = false;
 
     public static boolean isBlobStoragePath(final Configuration conf, final Path path) {
         return path != null && isBlobStorageScheme(conf, path.toUri().getScheme());
     }
 
-    static boolean isBlobStorageFileSystem(final Configuration conf, final FileSystem fs) {
-        return fs != null && fs.getUri() != null && isBlobStorageScheme(conf, fs.getUri().getScheme());
+    public static boolean isBlobStorageFileSystem(final Configuration conf, final FileSystem fs) {
+        return fs != null && isBlobStorageScheme(conf, fs.getScheme());
     }
 
-    static boolean isBlobStorageScheme(final Configuration conf, final String scheme) {
+    public static boolean isBlobStorageScheme(final Configuration conf, final String scheme) {
         Collection<String> supportedBlobStoreSchemes =
                 conf.getStringCollection(HiveConf.ConfVars.HIVE_BLOBSTORE_SUPPORTED_SCHEMES.varname);
 
@@ -64,5 +61,4 @@ public class BlobStorageUtils {
                 HiveConf.ConfVars.HIVE_BLOBSTORE_OPTIMIZATIONS_ENABLED.defaultBoolVal
         );
     }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/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 8ed8cc4..e586015 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -46,11 +46,13 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConfUtil;
 import org.apache.hadoop.hive.io.HdfsUtils;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.slf4j.Logger;
@@ -507,29 +509,11 @@ public final class FileUtils {
 
   /**
    * Creates the directory and all necessary parent directories.
-   *
-   * @param fs FileSystem to use
-   * @param f path to create.
-   * @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, Configuration conf) throws IOException {
-    return mkdir(fs, f, shouldInheritPerms(conf, fs), conf);
-  }
-
-  /**
-   * 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 {
@@ -617,10 +601,9 @@ public final class FileUtils {
       copied = FileUtil.copy(srcFS, src, dstFS, dst, deleteSource, overwrite, conf);
     }
 
-    boolean inheritPerms = shouldInheritPerms(conf, dstFS);
+    boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
     if (copied && inheritPerms) {
-      inheritPerms(conf, new HdfsUtils.HadoopFileStatus(conf, dstFS, dst.getParent()), dstFS, dst,
-              true);
+      HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, dstFS, dst.getParent()), dstFS, dst, true);
     }
     return copied;
   }
@@ -1011,18 +994,4 @@ public final class FileUtils {
     return result;
   }
 
-  public static boolean shouldInheritPerms(Configuration conf, FileSystem fs) {
-    return HiveConf.getBoolVar(conf,
-            HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS) && StorageUtils.shouldSetPerms(conf, fs);
-  }
-
-  public static void inheritPerms(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus, FileSystem fs, Path target,
-                                  boolean recursive) {
-    inheritPerms(conf, sourceStatus, null, fs, target, recursive);
-  }
-
-  public static void inheritPerms(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus, String targetGroup,
-                                  FileSystem fs, Path target, boolean recursive) {
-    HdfsUtils.setFullFileStatus(conf, sourceStatus, targetGroup, fs, target, recursive);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/common/src/java/org/apache/hadoop/hive/common/StorageUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/StorageUtils.java b/common/src/java/org/apache/hadoop/hive/common/StorageUtils.java
deleted file mode 100644
index 900e2b1..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/StorageUtils.java
+++ /dev/null
@@ -1,40 +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.common;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-
-
-public class StorageUtils {
-
-  /**
-   * Returns true if permissions should be set on the given filesystem, false otherwise. Certain implementations of
-   * {@link FileSystem} don't have a concept of permissions, such as S3. This method checks to determine if the given
-   * {@link FileSystem} falls into that category.
-   *
-   * @param conf the {@link Configuration} to use when checking if permissions should be set on the {@link FileSystem}
-   * @param fs the {@link FileSystem} to check to see if permission should be set or not
-   *
-   * @return true if permissions should be set on the given {@link FileSystem}, false otherwise
-   */
-  public static boolean shouldSetPerms(Configuration conf, FileSystem fs) {
-      return !(BlobStorageUtils.areOptimizationsEnabled(conf) && BlobStorageUtils.isBlobStorageFileSystem(conf, fs));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/common/src/test/org/apache/hadoop/hive/common/TestBlobStorageUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestBlobStorageUtils.java b/common/src/test/org/apache/hadoop/hive/common/TestBlobStorageUtils.java
index 918ec95..84a0d86 100644
--- a/common/src/test/org/apache/hadoop/hive/common/TestBlobStorageUtils.java
+++ b/common/src/test/org/apache/hadoop/hive/common/TestBlobStorageUtils.java
@@ -64,18 +64,18 @@ public class TestBlobStorageUtils {
 
     /* Valid FileSystem schemes */
 
-    doReturn(URI.create("s3a:///")).when(fs).getUri();
+    doReturn("s3a").when(fs).getScheme();
     assertTrue(isBlobStorageFileSystem(conf, fs));
 
-    doReturn(URI.create("swift:///")).when(fs).getUri();
+    doReturn("swift").when(fs).getScheme();
     assertTrue(isBlobStorageFileSystem(conf, fs));
 
     /* Invalid FileSystem schemes */
 
-    doReturn(URI.create("hdfs:///")).when(fs).getUri();
+    doReturn("hdfs").when(fs).getScheme();
     assertFalse(isBlobStorageFileSystem(conf, fs));
 
-    doReturn(URI.create("")).when(fs).getUri();
+    doReturn("").when(fs).getScheme();
     assertFalse(isBlobStorageFileSystem(conf, fs));
 
     assertFalse(isBlobStorageFileSystem(conf, null));

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/common/src/test/org/apache/hadoop/hive/common/TestStorageUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestStorageUtils.java b/common/src/test/org/apache/hadoop/hive/common/TestStorageUtils.java
deleted file mode 100644
index 638f48c..0000000
--- a/common/src/test/org/apache/hadoop/hive/common/TestStorageUtils.java
+++ /dev/null
@@ -1,57 +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.common;
-
-import java.net.URI;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hive.conf.HiveConf;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestStorageUtils {
-
-  private final HiveConf conf = new HiveConf();
-
-  @Before
-  public void setUp() {
-    conf.set(HiveConf.ConfVars.HIVE_BLOBSTORE_SUPPORTED_SCHEMES.varname, HiveConf.ConfVars.HIVE_BLOBSTORE_SUPPORTED_SCHEMES.getDefaultValue());
-  }
-
-  @Test
-  public void testShouldInheritPerms() {
-    conf.set(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS.varname, "true");
-    FileSystem fs = mock(FileSystem.class);
-    for (String blobScheme : conf.getStringCollection(HiveConf.ConfVars.HIVE_BLOBSTORE_SUPPORTED_SCHEMES.varname)) {
-      when(fs.getUri()).thenReturn(URI.create(blobScheme + ":///"));
-      assertFalse(FileUtils.shouldInheritPerms(conf, fs));
-    }
-
-    when(fs.getUri()).thenReturn(URI.create("hdfs:///"));
-    assertTrue(FileUtils.shouldInheritPerms(conf, fs));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/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 c94ef07..27283d8 100755
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
@@ -33,10 +33,8 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
@@ -188,10 +186,11 @@ public class Warehouse {
   }
 
   public boolean mkdirs(Path f, boolean inheritPermCandidate) throws MetaException {
+    boolean inheritPerms = HiveConf.getBoolVar(conf,
+      HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS) && inheritPermCandidate;
     FileSystem fs = null;
     try {
       fs = getFs(f);
-      boolean inheritPerms = FileUtils.shouldInheritPerms(conf, fs) && inheritPermCandidate;
       return FileUtils.mkdir(fs, f, inheritPerms, conf);
     } catch (IOException e) {
       MetaStoreUtils.logAndThrowMetaException(e);

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

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/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 2683f29..cbe0aca 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,7 +70,8 @@ public class CopyTask extends Task<CopyWork> implements Serializable {
         }
       }
 
-      if (!FileUtils.mkdir(dstFs, toPath, conf)) {
+      boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
+      if (!FileUtils.mkdir(dstFs, toPath, inheritPerms, conf)) {
         console.printError("Cannot make target directory: " + toPath.toString());
         return 2;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/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 5cf2c2b..1802f37 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
@@ -181,8 +181,8 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
         actualPath = actualPath.getParent();
       }
       fs.mkdirs(mkDirPath);
-      if (FileUtils.shouldInheritPerms(conf, fs)) {
-        FileUtils.inheritPerms(conf, new HdfsUtils.HadoopFileStatus(conf, fs, actualPath), fs, mkDirPath, true);
+      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/2ba12175/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 d2f9e79..4686e2c 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,7 +106,8 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
       srcFiles.addAll(Arrays.asList(srcs));
       LOG.debug("ReplCopyTask numFiles:" + (srcFiles == null ? "null" : srcFiles.size()));
 
-      if (!FileUtils.mkdir(dstFs, toPath, conf)) {
+      boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
+      if (!FileUtils.mkdir(dstFs, toPath, inheritPerms, conf)) {
         console.printError("Cannot make target directory: " + toPath.toString());
         return 2;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/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 f11feb7..f64cfda 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,7 +2889,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
     if (!fullDestStatus.getFileStatus().isDirectory()) {
       throw new HiveException(destf + " is not a directory.");
     }
-    final boolean inheritPerms = FileUtils.shouldInheritPerms(conf, destFs);
+    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 +2939,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
               Path destPath = mvFile(conf, srcFs, srcP, destFs, destf, isSrcLocal, isRenameAllowed);
 
               if (inheritPerms) {
-                FileUtils.inheritPerms(conf, fullDestStatus, srcGroup, destFs, destPath, false);
+                HdfsUtils.setFullFileStatus(conf, fullDestStatus, srcGroup, destFs, destPath, false);
               }
-
               if (null != newFiles) {
                 newFiles.add(destPath);
               }
@@ -2952,7 +2952,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
     if (null == pool) {
       if (inheritPerms) {
-        FileUtils.inheritPerms(conf, fullDestStatus, destFs, destf, true);
+        HdfsUtils.setFullFileStatus(conf, fullDestStatus, null, destFs, destf, true);
       }
     } else {
       pool.shutdown();
@@ -3105,7 +3105,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
 
     //needed for perm inheritance.
-    final boolean inheritPerms = FileUtils.shouldInheritPerms(conf, destFs);
+    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 +3118,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     boolean destIsSubDir = isSubDir(srcf, destf, srcFs, destFs, isSrcLocal);
     try {
       if (inheritPerms || replace) {
-        try {
+        try{
           destStatus = new HdfsUtils.HadoopFileStatus(conf, destFs, destf);
           //if destf is an existing directory:
           //if replace is true, delete followed by rename(mv) is equivalent to replace
@@ -3141,7 +3142,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         // For local src file, copy to hdfs
         destFs.copyFromLocalFile(srcf, destf);
         if (inheritPerms) {
-          FileUtils.inheritPerms(conf, destStatus, destFs, destf, true);
+          HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
         }
         return true;
       } else {
@@ -3177,7 +3178,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                     final String group = srcStatus.getGroup();
                     if(destFs.rename(srcStatus.getPath(), destFile)) {
                       if (inheritPerms) {
-                        FileUtils.inheritPerms(conf, desiredStatus, group, destFs, destFile, false);
+                        HdfsUtils.setFullFileStatus(conf, desiredStatus, group, destFs, destFile, false);
                       }
                     } else {
                       throw new IOException("rename for src path: " + srcStatus.getPath() + " to dest path:"
@@ -3190,7 +3191,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
             }
             if (null == pool) {
               if (inheritPerms) {
-                FileUtils.inheritPerms(conf, desiredStatus, destFs, destf, true);
+                HdfsUtils.setFullFileStatus(conf, desiredStatus, null, destFs, destf, true);
               }
             } else {
               pool.shutdown();
@@ -3208,7 +3209,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
           } else {
             if (destFs.rename(srcf, destf)) {
               if (inheritPerms) {
-                FileUtils.inheritPerms(conf, destStatus, destFs, destf, true);
+                HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
               }
               return true;
             }
@@ -3260,10 +3261,12 @@ 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, conf);
+        FileUtils.mkdir(fs, destf, inheritPerms, conf);
       }
     } catch (IOException e) {
       throw new HiveException(
@@ -3457,7 +3460,9 @@ 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 destfExist = FileUtils.mkdir(destFs, destf, conf);
+      boolean inheritPerms = HiveConf.getBoolVar(conf, HiveConf.ConfVars
+          .HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
+      boolean destfExist = FileUtils.mkdir(destFs, destf, inheritPerms, 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/2ba12175/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
index 1b57184..277738f 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 
-import org.apache.hadoop.hive.common.StorageUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,7 +49,6 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
 public class HdfsUtils {
-
   private static final Logger LOG = LoggerFactory.getLogger("shims.HdfsUtils");
 
   // TODO: this relies on HDFS not changing the format; we assume if we could get inode ID, this
@@ -76,9 +74,7 @@ public class HdfsUtils {
    */
   public static void setFullFileStatus(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus,
       FileSystem fs, Path target, boolean recursion) {
-    if (StorageUtils.shouldSetPerms(conf, fs)) {
-      setFullFileStatus(conf, sourceStatus, null, fs, target, recursion);
-    }
+    setFullFileStatus(conf, sourceStatus, null, fs, target, recursion);
   }
 
   /**
@@ -95,9 +91,7 @@ public class HdfsUtils {
    */
   public static void setFullFileStatus(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus,
       String targetGroup, FileSystem fs, Path target, boolean recursion) {
-    if (StorageUtils.shouldSetPerms(conf, fs)) {
-      setFullFileStatus(conf, sourceStatus, targetGroup, fs, target, recursion, recursion ? new FsShell() : null);
-    }
+    setFullFileStatus(conf, sourceStatus, targetGroup, fs, target, recursion, recursion ? new FsShell() : null);
   }
 
   @VisibleForTesting
@@ -186,7 +180,6 @@ public class HdfsUtils {
     return new AclEntry.Builder().setScope(scope).setType(type)
         .setPermission(permission).build();
   }
-
   /**
    * Removes basic permission acls (unamed acls) from the list of acl entries
    * @param entries acl entries to remove from.
@@ -208,41 +201,39 @@ public class HdfsUtils {
     int retval = shell.run(command);
     LOG.debug("Return value is :" + retval);
   }
-
-  public static class HadoopFileStatus {
-
-    private final FileStatus fileStatus;
-    private final AclStatus aclStatus;
-
-    public HadoopFileStatus(Configuration conf, FileSystem fs, Path file) throws IOException {
-
-      FileStatus fileStatus = fs.getFileStatus(file);
-      AclStatus aclStatus = null;
-      if (Objects.equal(conf.get("dfs.namenode.acls.enabled"), "true")) {
-        //Attempt extended Acl operations only if its enabled, but don't fail the operation regardless.
-        try {
-          aclStatus = fs.getAclStatus(file);
-        } catch (Exception e) {
-          LOG.info("Skipping ACL inheritance: File system for path " + file + " " +
-                  "does not support ACLs but dfs.namenode.acls.enabled is set to true. ");
-          LOG.debug("The details are: " + e, e);
-        }
+public static class HadoopFileStatus {
+
+  private final FileStatus fileStatus;
+  private final AclStatus aclStatus;
+
+  public HadoopFileStatus(Configuration conf, FileSystem fs, Path file) throws IOException {
+
+    FileStatus fileStatus = fs.getFileStatus(file);
+    AclStatus aclStatus = null;
+    if (Objects.equal(conf.get("dfs.namenode.acls.enabled"), "true")) {
+      //Attempt extended Acl operations only if its enabled, but don't fail the operation regardless.
+      try {
+        aclStatus = fs.getAclStatus(file);
+      } catch (Exception e) {
+        LOG.info("Skipping ACL inheritance: File system for path " + file + " " +
+                "does not support ACLs but dfs.namenode.acls.enabled is set to true. ");
+        LOG.debug("The details are: " + e, e);
       }
-      this.fileStatus = fileStatus;
-      this.aclStatus = aclStatus;
-    }
+    }this.fileStatus = fileStatus;
+    this.aclStatus = aclStatus;
+  }
 
-    public FileStatus getFileStatus() {
-      return fileStatus;
-    }
+  public FileStatus getFileStatus() {
+    return fileStatus;
+  }
 
-    public List<AclEntry> getAclEntries() {
-      return aclStatus == null ? null : Collections.unmodifiableList(aclStatus.getEntries());
-    }
+  public List<AclEntry> getAclEntries() {
+    return aclStatus == null ? null : Collections.unmodifiableList(aclStatus.getEntries());
+  }
 
-    @VisibleForTesting
-    AclStatus getAclStatus() {
-      return this.aclStatus;
-    }
+  @VisibleForTesting
+  AclStatus getAclStatus() {
+    return this.aclStatus;
   }
 }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2ba12175/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java b/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java
index 79507e4..86a132c 100644
--- a/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java
+++ b/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.io;
 
 import java.io.IOException;
-import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -32,14 +31,11 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.junit.Test;
 
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyList;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -185,19 +181,4 @@ public class TestHdfsUtils {
             true, mockFsShell);
     verify(mockFsShell).run(new String[]{"-chmod", "-R", any(String.class), fakeTarget.toString()});
   }
-
-  @Test
-  public void testSkipSetFullFileStatusIfBlobStore() throws IOException {
-    Configuration conf = new Configuration();
-    conf.set(HiveConf.ConfVars.HIVE_BLOBSTORE_SUPPORTED_SCHEMES.varname, "s3a");
-    FileSystem fs = mock(FileSystem.class);
-    when(fs.getUri()).thenReturn(URI.create("s3a:///"));
-    HdfsUtils.setFullFileStatus(conf, null, null, fs, null, false);
-
-    verify(fs, never()).getFileStatus(any(Path.class));
-    verify(fs, never()).listStatus(any(Path[].class));
-    verify(fs, never()).setPermission(any(Path.class), any(FsPermission.class));
-    verify(fs, never()).setAcl(any(Path.class), anyList());
-    verify(fs, never()).setOwner(any(Path.class), any(String.class), any(String.class));
-  }
 }