You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sp...@apache.org on 2016/12/09 22:32:18 UTC

hive git commit: HIVE-15385: Failure to inherit permissions when running HdfsUtils.setFullFileStatus(..., false) causes queries to fail (Sahil Takiar, reviewed by Sergio Pena and Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 999dc553e -> e43861a10


HIVE-15385: Failure to inherit permissions when running HdfsUtils.setFullFileStatus(..., false) causes queries to fail (Sahil Takiar, reviewed by Sergio Pena and Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: e43861a1023920b23c126733ceca273bc766c58a
Parents: 999dc55
Author: Sahil Takir <ta...@gmail.com>
Authored: Fri Dec 9 16:31:11 2016 -0600
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri Dec 9 16:32:11 2016 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    |  28 +--
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   6 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   6 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  13 +-
 shims/common/pom.xml                            |   6 +
 .../org/apache/hadoop/hive/io/HdfsUtils.java    | 114 ++++++++----
 .../apache/hadoop/hive/io/TestHdfsUtils.java    | 184 +++++++++++++++++++
 7 files changed, 278 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/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 1d8c041..00fbc3b 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -545,15 +545,11 @@ public final class FileUtils {
       if (!success) {
         return false;
       } else {
-        try {
-          //set on the entire subtree
-          if (inheritPerms) {
-            HdfsUtils.setFullFileStatus(conf,
-                new HdfsUtils.HadoopFileStatus(conf, fs, lastExistingParent), fs,
-                firstNonExistentParent, true);
-          }
-        } catch (Exception e) {
-          LOG.warn("Error setting permissions of " + firstNonExistentParent, e);
+        //set on the entire subtree
+        if (inheritPerms) {
+          HdfsUtils.setFullFileStatus(conf,
+                  new HdfsUtils.HadoopFileStatus(conf, fs, lastExistingParent), fs,
+                  firstNonExistentParent, true);
         }
         return true;
       }
@@ -596,11 +592,7 @@ public final class FileUtils {
 
     boolean inheritPerms = conf.getBoolVar(HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS);
     if (copied && inheritPerms) {
-      try {
-        HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, dstFS, dst.getParent()), dstFS, dst, true);
-      } catch (Exception e) {
-        LOG.warn("Error setting permissions or group of " + dst, e);
-      }
+      HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, dstFS, dst.getParent()), dstFS, dst, true);
     }
     return copied;
   }
@@ -655,12 +647,8 @@ public final class FileUtils {
     } else {
       //rename the directory
       if (fs.rename(sourcePath, destPath)) {
-        try {
-          HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, destPath.getParent()), fs, destPath, true);
-        } catch (Exception e) {
-          LOG.warn("Error setting permissions or group of " + destPath, e);
-        }
-
+        HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, destPath.getParent()), fs, destPath,
+                true);
         return true;
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 493e1b3..8a1a2ac 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -4415,11 +4415,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           String targetGroup = targetStatus == null ? null : targetStatus.getGroup();
           FileUtils.moveToTrash(fs, location, conf);
           fs.mkdirs(location);
-          try {
-            HdfsUtils.setFullFileStatus(conf, status, targetGroup, fs, location, false);
-          } catch (Exception e) {
-            LOG.warn("Error setting permissions of " + location, e);
-          }
+          HdfsUtils.setFullFileStatus(conf, status, targetGroup, fs, location, false);
         } else {
           FileStatus[] statuses = fs.listStatus(location, FileUtils.HIDDEN_FILES_PATH_FILTER);
           if (statuses == null || statuses.length == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/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 e1381be..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
@@ -182,11 +182,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
       }
       fs.mkdirs(mkDirPath);
       if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_WAREHOUSE_SUBDIR_INHERIT_PERMS)) {
-        try {
-          HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, actualPath), fs, mkDirPath, true);
-        } catch (Exception e) {
-          LOG.warn("Error setting permissions or group of " + actualPath, e);
-        }
+        HdfsUtils.setFullFileStatus(conf, new HdfsUtils.HadoopFileStatus(conf, fs, actualPath), fs, mkDirPath, true);
       }
     }
     return deletePath;

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/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 e477f24..750fdef 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
@@ -2867,12 +2867,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
     if (null == pool) {
       if (inheritPerms) {
-        try {
-          HdfsUtils.setFullFileStatus(conf, fullDestStatus, null, destFs, destf, true);
-        } catch (IOException e) {
-          LOG.error("Failed to move: {}", e.getMessage());
-          throw new HiveException(e.getCause());
-        }
+        HdfsUtils.setFullFileStatus(conf, fullDestStatus, null, destFs, destf, true);
       }
     } else {
       pool.shutdown();
@@ -3062,11 +3057,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         // For local src file, copy to hdfs
         destFs.copyFromLocalFile(srcf, destf);
         if (inheritPerms) {
-          try {
-            HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
-          } catch (IOException e) {
-            LOG.warn("Error setting permission of file " + destf + ": "+ e.getMessage(), e);
-          }
+          HdfsUtils.setFullFileStatus(conf, destStatus, destFs, destf, true);
         }
         return true;
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/shims/common/pom.xml
----------------------------------------------------------------------
diff --git a/shims/common/pom.xml b/shims/common/pom.xml
index 700bd22..19821cd 100644
--- a/shims/common/pom.xml
+++ b/shims/common/pom.xml
@@ -87,4 +87,10 @@
       </exclusions>
     </dependency>
   </dependencies>
+
+  <build>
+    <sourceDirectory>${basedir}/src/main/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/main/test</testSourceDirectory>
+  </build>
+
 </project>

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/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 7b6a9bd..7270426 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
@@ -23,7 +23,10 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.lang.ArrayUtils;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -36,6 +39,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,37 +62,63 @@ public class HdfsUtils {
         ? new Path(HDFS_ID_PATH_PREFIX + fileId) : path;
   }
 
+  /**
+   * Copy the permissions, group, and ACLs from a source {@link HadoopFileStatus} to a target {@link Path}. This method
+   * will only log a warning if permissions cannot be set, no exception will be thrown.
+   *
+   * @param conf the {@link Configuration} used when setting permissions and ACLs
+   * @param sourceStatus the source {@link HadoopFileStatus} to copy permissions and ACLs from
+   * @param fs the {@link FileSystem} that contains the target {@link Path}
+   * @param target the {@link Path} to copy permissions, group, and ACLs to
+   * @param recursion recursively set permissions and ACLs on the target {@link Path}
+   */
   public static void setFullFileStatus(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus,
-      FileSystem fs, Path target, boolean recursion) throws IOException {
+      FileSystem fs, Path target, boolean recursion) {
     setFullFileStatus(conf, sourceStatus, null, fs, target, recursion);
   }
 
+  /**
+   * Copy the permissions, group, and ACLs from a source {@link HadoopFileStatus} to a target {@link Path}. This method
+   * will only log a warning if permissions cannot be set, no exception will be thrown.
+   *
+   * @param conf the {@link Configuration} used when setting permissions and ACLs
+   * @param sourceStatus the source {@link HadoopFileStatus} to copy permissions and ACLs from
+   * @param targetGroup the group of the target {@link Path}, if this is set and it is equal to the source group, an
+   *                    extra set group operation is avoided
+   * @param fs the {@link FileSystem} that contains the target {@link Path}
+   * @param target the {@link Path} to copy permissions, group, and ACLs to
+   * @param recursion recursively set permissions and ACLs on the target {@link Path}
+   */
   public static void setFullFileStatus(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus,
-    String targetGroup, FileSystem fs, Path target, boolean recursion) throws IOException {
-    FileStatus fStatus= sourceStatus.getFileStatus();
-    String group = fStatus.getGroup();
-    boolean aclEnabled = Objects.equal(conf.get("dfs.namenode.acls.enabled"), "true");
-    FsPermission sourcePerm = fStatus.getPermission();
-    List<AclEntry> aclEntries = null;
-    if (aclEnabled) {
-      if (sourceStatus.getAclEntries() != null) {
-        LOG.trace(sourceStatus.aclStatus.toString());
-        aclEntries = new ArrayList<>(sourceStatus.getAclEntries());
-        removeBaseAclEntries(aclEntries);
-
-        //the ACL api's also expect the tradition user/group/other permission in the form of ACL
-        aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.USER, sourcePerm.getUserAction()));
-        aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.GROUP, sourcePerm.getGroupAction()));
-        aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.OTHER, sourcePerm.getOtherAction()));
-      }
-    }
+      String targetGroup, FileSystem fs, Path target, boolean recursion) {
+    setFullFileStatus(conf, sourceStatus, targetGroup, fs, target, recursion, recursion ? new FsShell() : null);
+  }
 
-    if (recursion) {
-      //use FsShell to change group, permissions, and extended ACL's recursively
-      FsShell fsShell = new FsShell();
-      fsShell.setConf(conf);
+  @VisibleForTesting
+  static void setFullFileStatus(Configuration conf, HdfsUtils.HadoopFileStatus sourceStatus,
+    String targetGroup, FileSystem fs, Path target, boolean recursion, FsShell fsShell) {
+    try {
+      FileStatus fStatus = sourceStatus.getFileStatus();
+      String group = fStatus.getGroup();
+      boolean aclEnabled = Objects.equal(conf.get("dfs.namenode.acls.enabled"), "true");
+      FsPermission sourcePerm = fStatus.getPermission();
+      List<AclEntry> aclEntries = null;
+      if (aclEnabled) {
+        if (sourceStatus.getAclEntries() != null) {
+          LOG.trace(sourceStatus.getAclStatus().toString());
+          aclEntries = new ArrayList<>(sourceStatus.getAclEntries());
+          removeBaseAclEntries(aclEntries);
+
+          //the ACL api's also expect the tradition user/group/other permission in the form of ACL
+          aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.USER, sourcePerm.getUserAction()));
+          aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.GROUP, sourcePerm.getGroupAction()));
+          aclEntries.add(newAclEntry(AclEntryScope.ACCESS, AclEntryType.OTHER, sourcePerm.getOtherAction()));
+        }
+      }
 
-      try {
+      if (recursion) {
+        //use FsShell to change group, permissions, and extended ACL's recursively
+        fsShell.setConf(conf);
         //If there is no group of a file, no need to call chgrp
         if (group != null && !group.isEmpty()) {
           run(fsShell, new String[]{"-chgrp", "-R", group, target.toString()});
@@ -111,23 +141,26 @@ public class HdfsUtils {
           String permission = Integer.toString(sourcePerm.toShort(), 8);
           run(fsShell, new String[]{"-chmod", "-R", permission, target.toString()});
         }
-      } catch (Exception e) {
-        throw new IOException("Unable to set permissions of " + target, e);
-      }
-    } else {
-      if (group != null && !group.isEmpty()) {
-        if (targetGroup == null ||
-            !group.equals(targetGroup)) {
-          fs.setOwner(target, null, group);
+      } else {
+        if (group != null && !group.isEmpty()) {
+          if (targetGroup == null ||
+              !group.equals(targetGroup)) {
+            fs.setOwner(target, null, group);
+          }
         }
-      }
-      if (aclEnabled) {
-        if (null != aclEntries) {
-          fs.setAcl(target, aclEntries);
+        if (aclEnabled) {
+          if (null != aclEntries) {
+            fs.setAcl(target, aclEntries);
+          }
+        } else {
+          fs.setPermission(target, sourcePerm);
         }
-      } else {
-        fs.setPermission(target, sourcePerm);
       }
+    } catch (Exception e) {
+      LOG.warn(
+              "Unable to inherit permissions for file " + target + " from file " + sourceStatus.getFileStatus().getPath(),
+              e.getMessage());
+      LOG.debug("Exception while inheriting permissions", e);
     }
   }
 
@@ -197,5 +230,10 @@ public static class HadoopFileStatus {
   public List<AclEntry> getAclEntries() {
     return aclStatus == null ? null : Collections.unmodifiableList(aclStatus.getEntries());
   }
+
+  @VisibleForTesting
+  AclStatus getAclStatus() {
+    return this.aclStatus;
+  }
 }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e43861a1/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
new file mode 100644
index 0000000..86a132c
--- /dev/null
+++ b/shims/common/src/main/test/org/apache/hadoop/hive/io/TestHdfsUtils.java
@@ -0,0 +1,184 @@
+/**
+ * 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.io;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import org.junit.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestHdfsUtils {
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, String, FileSystem, Path, boolean)}
+   * does not throw an exception when setting the group and without recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritGroup() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "false");
+
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    FileSystem fs = mock(FileSystem.class);
+
+    when(mockSourceStatus.getGroup()).thenReturn("fakeGroup1");
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    doThrow(RuntimeException.class).when(fs).setOwner(any(Path.class), any(String.class), any(String.class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, "fakeGroup2", fs, new Path("fakePath"), false);
+    verify(fs).setOwner(any(Path.class), any(String.class), any(String.class));
+  }
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, FileSystem, Path, boolean)}
+   * does not thrown an exception when setting ACLs and without recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritAcls() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "true");
+
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    AclStatus mockAclStatus = mock(AclStatus.class);
+    FileSystem mockFs = mock(FileSystem.class);
+
+    when(mockSourceStatus.getPermission()).thenReturn(new FsPermission((short) 777));
+    when(mockAclStatus.toString()).thenReturn("");
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    when(mockHadoopFileStatus.getAclEntries()).thenReturn(new ArrayList<AclEntry>());
+    when(mockHadoopFileStatus.getAclStatus()).thenReturn(mockAclStatus);
+    doThrow(RuntimeException.class).when(mockFs).setAcl(any(Path.class), any(List.class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, mockFs, new Path("fakePath"), false);
+    verify(mockFs).setAcl(any(Path.class), any(List.class));
+  }
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, FileSystem, Path, boolean)}
+   * does not thrown an exception when setting permissions and without recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritPerms() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "false");
+
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    FileSystem mockFs = mock(FileSystem.class);
+
+    when(mockSourceStatus.getPermission()).thenReturn(new FsPermission((short) 777));
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    doThrow(RuntimeException.class).when(mockFs).setPermission(any(Path.class), any(FsPermission.class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, mockFs, new Path("fakePath"), false);
+    verify(mockFs).setPermission(any(Path.class), any(FsPermission.class));
+  }
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, String, FileSystem, Path, boolean)}
+   * does not throw an exception when setting the group and with recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritGroupRecursive() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "false");
+
+    String fakeSourceGroup = "fakeGroup1";
+    String fakeTargetGroup = "fakeGroup2";
+    Path fakeTarget = new Path("fakePath");
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    FsShell mockFsShell = mock(FsShell.class);
+
+    when(mockSourceStatus.getGroup()).thenReturn(fakeSourceGroup);
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    doThrow(RuntimeException.class).when(mockFsShell).run(any(String[].class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, fakeTargetGroup, mock(FileSystem.class), fakeTarget,
+            true, mockFsShell);
+    verify(mockFsShell).run(new String[]{"-chgrp", "-R", fakeSourceGroup, fakeTarget.toString()});
+  }
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, FileSystem, Path, boolean)}
+   * does not thrown an exception when setting ACLs and with recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritAclsRecursive() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "true");
+
+    Path fakeTarget = new Path("fakePath");
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    FsShell mockFsShell = mock(FsShell.class);
+    AclStatus mockAclStatus = mock(AclStatus.class);
+
+    when(mockSourceStatus.getPermission()).thenReturn(new FsPermission((short) 777));
+    when(mockAclStatus.toString()).thenReturn("");
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    when(mockHadoopFileStatus.getAclEntries()).thenReturn(new ArrayList<AclEntry>());
+    when(mockHadoopFileStatus.getAclStatus()).thenReturn(mockAclStatus);
+    doThrow(RuntimeException.class).when(mockFsShell).run(any(String[].class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, "", mock(FileSystem.class), fakeTarget, true, mockFsShell);
+    verify(mockFsShell).run(new String[]{"-setfacl", "-R", "--set", any(String.class), fakeTarget.toString()});
+  }
+
+  /**
+   * Tests that {@link HdfsUtils#setFullFileStatus(Configuration, HdfsUtils.HadoopFileStatus, FileSystem, Path, boolean)}
+   * does not thrown an exception when setting permissions and with recursion.
+   */
+  @Test
+  public void testSetFullFileStatusFailInheritPermsRecursive() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("dfs.namenode.acls.enabled", "false");
+
+    Path fakeTarget = new Path("fakePath");
+    HdfsUtils.HadoopFileStatus mockHadoopFileStatus = mock(HdfsUtils.HadoopFileStatus.class);
+    FileStatus mockSourceStatus = mock(FileStatus.class);
+    FsShell mockFsShell = mock(FsShell.class);
+
+    when(mockSourceStatus.getPermission()).thenReturn(new FsPermission((short) 777));
+    when(mockHadoopFileStatus.getFileStatus()).thenReturn(mockSourceStatus);
+    doThrow(RuntimeException.class).when(mockFsShell).run(any(String[].class));
+
+    HdfsUtils.setFullFileStatus(conf, mockHadoopFileStatus, "", mock(FileSystem.class), fakeTarget,
+            true, mockFsShell);
+    verify(mockFsShell).run(new String[]{"-chmod", "-R", any(String.class), fakeTarget.toString()});
+  }
+}