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

Signed-off-by: Chris Nauroth <cnauroth@apache.org>
This commit is contained in:
skysiders 2023-01-13 05:48:29 +08:00 committed by GitHub
parent a90e424d9f
commit 36bf54aba0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 26 additions and 1 deletions

View File

@ -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;
}

View File

@ -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();
}
}
}
}