YARN-66. aggregated logs permissions not set properly (tgraves via bobby)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1379565 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
27cfde7f4d
commit
60b8c6e1e1
|
@ -76,3 +76,5 @@ Release 0.23.3 - Unreleased
|
||||||
|
|
||||||
YARN-60. Fixed a bug in ResourceManager which causes all NMs to get NPEs and
|
YARN-60. Fixed a bug in ResourceManager which causes all NMs to get NPEs and
|
||||||
thus causes all containers to be rejected. (vinodkv)
|
thus causes all containers to be rejected. (vinodkv)
|
||||||
|
|
||||||
|
YARN-66. aggregated logs permissions not set properly (tgraves via bobby)
|
||||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileContext;
|
import org.apache.hadoop.fs.FileContext;
|
||||||
import org.apache.hadoop.fs.Options;
|
import org.apache.hadoop.fs.Options;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
import org.apache.hadoop.io.file.tfile.TFile;
|
import org.apache.hadoop.io.file.tfile.TFile;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
@ -68,6 +69,13 @@ public class AggregatedLogFormat {
|
||||||
//Maybe write out a list of containerLogs skipped by the retention policy.
|
//Maybe write out a list of containerLogs skipped by the retention policy.
|
||||||
private static final int VERSION = 1;
|
private static final int VERSION = 1;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Umask for the log file.
|
||||||
|
*/
|
||||||
|
private static final FsPermission APP_LOG_FILE_UMASK = FsPermission
|
||||||
|
.createImmutable((short) (0640 ^ 0777));
|
||||||
|
|
||||||
|
|
||||||
static {
|
static {
|
||||||
RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
|
RESERVED_KEYS = new HashMap<String, AggregatedLogFormat.LogKey>();
|
||||||
RESERVED_KEYS.put(APPLICATION_ACL_KEY.toString(), APPLICATION_ACL_KEY);
|
RESERVED_KEYS.put(APPLICATION_ACL_KEY.toString(), APPLICATION_ACL_KEY);
|
||||||
|
@ -194,7 +202,9 @@ public class AggregatedLogFormat {
|
||||||
userUgi.doAs(new PrivilegedExceptionAction<FSDataOutputStream>() {
|
userUgi.doAs(new PrivilegedExceptionAction<FSDataOutputStream>() {
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream run() throws Exception {
|
public FSDataOutputStream run() throws Exception {
|
||||||
return FileContext.getFileContext(conf).create(
|
FileContext fc = FileContext.getFileContext(conf);
|
||||||
|
fc.setUMask(APP_LOG_FILE_UMASK);
|
||||||
|
return fc.create(
|
||||||
remoteAppLogFile,
|
remoteAppLogFile,
|
||||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
||||||
new Options.CreateOpts[] {});
|
new Options.CreateOpts[] {});
|
||||||
|
|
|
@ -32,7 +32,9 @@ import junit.framework.Assert;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
@ -100,6 +102,11 @@ public class TestAggregatedLogFormat {
|
||||||
logWriter.append(logKey, logValue);
|
logWriter.append(logKey, logValue);
|
||||||
logWriter.closeWriter();
|
logWriter.closeWriter();
|
||||||
|
|
||||||
|
// make sure permission are correct on the file
|
||||||
|
FileStatus fsStatus = fs.getFileStatus(remoteAppLogFile);
|
||||||
|
Assert.assertEquals("permissions on log aggregation file are wrong",
|
||||||
|
FsPermission.createImmutable((short) 0640), fsStatus.getPermission());
|
||||||
|
|
||||||
LogReader logReader = new LogReader(conf, remoteAppLogFile);
|
LogReader logReader = new LogReader(conf, remoteAppLogFile);
|
||||||
LogKey rLogKey = new LogKey();
|
LogKey rLogKey = new LogKey();
|
||||||
DataInputStream dis = logReader.next(rLogKey);
|
DataInputStream dis = logReader.next(rLogKey);
|
||||||
|
@ -123,6 +130,7 @@ public class TestAggregatedLogFormat {
|
||||||
|
|
||||||
Assert.assertEquals(expectedLength, s.length());
|
Assert.assertEquals(expectedLength, s.length());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void writeSrcFile(Path srcFilePath, String fileName, long length)
|
private void writeSrcFile(Path srcFilePath, String fileName, long length)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
Loading…
Reference in New Issue