瀏覽代碼

MAPREDUCE-7375 JobSubmissionFiles don't set right permission after mkdirs (#4237)

Signed-off-by: Chris Nauroth <cnauroth@apache.org>
skysiders 2 年之前
父節點
當前提交
36bf54aba0

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmissionFiles.java

@@ -159,7 +159,7 @@ public class JobSubmissionFiles {
         fs.setPermission(stagingArea, JOB_DIR_PERMISSION);
       }
     } catch (FileNotFoundException e) {
-      fs.mkdirs(stagingArea, new FsPermission(JOB_DIR_PERMISSION));
+      FileSystem.mkdirs(fs, stagingArea, new FsPermission(JOB_DIR_PERMISSION));
     }
     return stagingArea;
   }

+ 25 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobSubmissionFiles.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -33,6 +34,8 @@ import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 /**
  * Tests for JobSubmissionFiles Utility class.
  */
@@ -139,4 +142,26 @@ public class TestJobSubmissionFiles {
     assertEquals(stagingPath,
         JobSubmissionFiles.getStagingDir(cluster, conf, user));
   }
+
+  @Test
+  public void testDirPermission() throws Exception {
+    Cluster cluster = mock(Cluster.class);
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "700");
+    MiniDFSCluster dfsCluster = null;
+    try {
+      dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+      FileSystem fs = dfsCluster.getFileSystem();
+      UserGroupInformation user = UserGroupInformation
+          .createUserForTesting(USER_1_SHORT_NAME, GROUP_NAMES);
+      Path stagingPath = new Path(fs.getUri().toString() + "/testDirPermission");
+      when(cluster.getStagingAreaDir()).thenReturn(stagingPath);
+      Path res = JobSubmissionFiles.getStagingDir(cluster, conf, user);
+      assertEquals(new FsPermission(0700), fs.getFileStatus(res).getPermission());
+    } finally {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+      }
+    }
+  }
 }