MAPREDUCE-7159. FrameworkUploader: ensure proper permissions of generated framework tar.gz if restrictive umask is used. Contributed by Peter Bacsko
This commit is contained in:
parent
343aaea2d1
commit
9886c27c71
|
@ -38,6 +38,12 @@
|
|||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs-client</artifactId>
|
||||
|
|
|
@ -26,12 +26,16 @@ import org.apache.commons.compress.archivers.ArchiveEntry;
|
|||
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.util.GenericOptionsParser;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
@ -71,6 +75,10 @@ public class FrameworkUploader implements Runnable {
|
|||
LoggerFactory.getLogger(FrameworkUploader.class);
|
||||
private Configuration conf = new Configuration();
|
||||
|
||||
// Minimal required permissions for the uploaded framework
|
||||
private static final FsPermission FRAMEWORK_PERMISSION =
|
||||
new FsPermission(0644);
|
||||
|
||||
@VisibleForTesting
|
||||
String input = null;
|
||||
@VisibleForTesting
|
||||
|
@ -99,6 +107,7 @@ public class FrameworkUploader implements Runnable {
|
|||
List<Pattern> blacklistedFiles = new LinkedList<>();
|
||||
|
||||
private OutputStream targetStream = null;
|
||||
private FSDataOutputStream fsDataStream = null;
|
||||
private String alias = null;
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -203,11 +212,50 @@ public class FrameworkUploader implements Runnable {
|
|||
targetStream = fileSystem.create(targetPath, true);
|
||||
}
|
||||
|
||||
if (!FRAMEWORK_PERMISSION.equals(
|
||||
FRAMEWORK_PERMISSION.applyUMask(FsPermission.getUMask(conf)))) {
|
||||
LOG.info("Modifying permissions to " + FRAMEWORK_PERMISSION);
|
||||
fileSystem.setPermission(targetPath, FRAMEWORK_PERMISSION);
|
||||
}
|
||||
|
||||
fsDataStream = (FSDataOutputStream) targetStream;
|
||||
if (targetPath.getName().endsWith("gz") ||
|
||||
targetPath.getName().endsWith("tgz")) {
|
||||
LOG.info("Creating GZip");
|
||||
targetStream = new GZIPOutputStream(targetStream);
|
||||
}
|
||||
|
||||
Path current = targetPath.getParent();
|
||||
// Walk the path backwards to verify that the uploaded
|
||||
// framework is accessible for all users
|
||||
while (current != null) {
|
||||
try {
|
||||
FileStatus fstat = fileSystem.getFileStatus(current);
|
||||
FsPermission perm = fstat.getPermission();
|
||||
|
||||
// Note: READ is not necessary to enter the directory.
|
||||
// We need to check only the EXECUTE flag
|
||||
boolean userCanEnter = perm.getUserAction()
|
||||
.implies(FsAction.EXECUTE);
|
||||
boolean groupCanEnter = perm.getGroupAction()
|
||||
.implies(FsAction.EXECUTE);
|
||||
boolean othersCanEnter = perm.getOtherAction()
|
||||
.implies(FsAction.EXECUTE);
|
||||
|
||||
if (!userCanEnter || !groupCanEnter || !othersCanEnter) {
|
||||
LOG.warn("Path " + current + " is not accessible"
|
||||
+ " for all users. Current permissions are: " + perm);
|
||||
LOG.warn("Please set EXECUTE permissions on this directory");
|
||||
}
|
||||
current = current.getParent();
|
||||
} catch (AccessControlException e) {
|
||||
LOG.warn("Path " + current + " is not accessible,"
|
||||
+ " cannot retrieve permissions");
|
||||
LOG.warn("Please set EXECUTE permissions on this directory");
|
||||
LOG.debug("Stack trace", e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -290,6 +338,10 @@ public class FrameworkUploader implements Runnable {
|
|||
out.closeArchiveEntry();
|
||||
}
|
||||
}
|
||||
|
||||
// Necessary to see proper replication counts in endUpload()
|
||||
fsDataStream.hflush();
|
||||
|
||||
endUpload();
|
||||
} finally {
|
||||
if (targetStream != null) {
|
||||
|
|
|
@ -24,6 +24,14 @@ import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
|
|||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
|
@ -470,4 +478,54 @@ public class TestFrameworkUploader {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPermissionSettingsOnRestrictiveUmask()
|
||||
throws Exception {
|
||||
File parent = new File(testDir);
|
||||
parent.deleteOnExit();
|
||||
MiniDFSCluster cluster = null;
|
||||
|
||||
try {
|
||||
Assert.assertTrue("Directory creation failed", parent.mkdirs());
|
||||
Configuration hdfsConf = new HdfsConfiguration();
|
||||
String namenodeDir = new File(MiniDFSCluster.getBaseDirectory(),
|
||||
"name").getAbsolutePath();
|
||||
hdfsConf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, namenodeDir);
|
||||
hdfsConf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, namenodeDir);
|
||||
hdfsConf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "027");
|
||||
cluster = new MiniDFSCluster.Builder(hdfsConf)
|
||||
.numDataNodes(1).build();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
cluster.waitActive();
|
||||
|
||||
File file1 = new File(parent, "a.jar");
|
||||
file1.createNewFile();
|
||||
File file2 = new File(parent, "b.jar");
|
||||
file2.createNewFile();
|
||||
File file3 = new File(parent, "c.jar");
|
||||
file3.createNewFile();
|
||||
|
||||
FrameworkUploader uploader = new FrameworkUploader();
|
||||
uploader.whitelist = "";
|
||||
uploader.blacklist = "";
|
||||
uploader.input = parent.getAbsolutePath() + File.separatorChar + "*";
|
||||
String hdfsUri = hdfsConf.get(FS_DEFAULT_NAME_KEY);
|
||||
String targetPath = "/test.tar.gz";
|
||||
uploader.target = hdfsUri + targetPath;
|
||||
uploader.acceptableReplication = 1;
|
||||
uploader.setConf(hdfsConf);
|
||||
|
||||
uploader.collectPackages();
|
||||
uploader.buildPackage();
|
||||
|
||||
FileStatus fileStatus = dfs.getFileStatus(new Path(targetPath));
|
||||
FsPermission perm = fileStatus.getPermission();
|
||||
Assert.assertEquals("Permissions", new FsPermission(0644), perm);
|
||||
} finally {
|
||||
if (cluster != null) {
|
||||
cluster.close();
|
||||
}
|
||||
FileUtils.deleteDirectory(parent);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue