YARN-6288. Exceptions during aggregated log writes are mishandled. Contributed by Akira Ajisaka

This commit is contained in:
Jason Lowe 2017-04-06 16:26:24 -05:00
parent ab91ca7338
commit 6e41aec109
5 changed files with 147 additions and 140 deletions

View File

@ -390,18 +390,18 @@ public class TestLogsCLI {
Path path =
new Path(appDir, LogAggregationUtils.getNodeString(nodeId)
+ System.currentTimeMillis());
AggregatedLogFormat.LogWriter writer =
new AggregatedLogFormat.LogWriter(configuration, path, ugi);
try (AggregatedLogFormat.LogWriter writer =
new AggregatedLogFormat.LogWriter()) {
writer.initialize(configuration, path, ugi);
writer.writeApplicationOwner(ugi.getUserName());
Map<ApplicationAccessType, String> appAcls =
new HashMap<ApplicationAccessType, String>();
Map<ApplicationAccessType, String> appAcls = new HashMap<>();
appAcls.put(ApplicationAccessType.VIEW_APP, ugi.getUserName());
writer.writeApplicationACLs(appAcls);
writer.append(new AggregatedLogFormat.LogKey(containerId),
new AggregatedLogFormat.LogValue(rootLogDirs, containerId,
UserGroupInformation.getCurrentUser().getShortUserName()));
writer.close();
}
}
private static void uploadEmptyContainerLogIntoRemoteDir(UserGroupInformation ugi,
@ -410,12 +410,12 @@ public class TestLogsCLI {
Path path =
new Path(appDir, LogAggregationUtils.getNodeString(nodeId)
+ System.currentTimeMillis());
AggregatedLogFormat.LogWriter writer =
new AggregatedLogFormat.LogWriter(configuration, path, ugi);
try (AggregatedLogFormat.LogWriter writer =
new AggregatedLogFormat.LogWriter()) {
writer.initialize(configuration, path, ugi);
writer.writeApplicationOwner(ugi.getUserName());
Map<ApplicationAccessType, String> appAcls =
new HashMap<ApplicationAccessType, String>();
Map<ApplicationAccessType, String> appAcls = new HashMap<>();
appAcls.put(ApplicationAccessType.VIEW_APP, ugi.getUserName());
writer.writeApplicationACLs(appAcls);
DataOutputStream out = writer.getWriter().prepareAppendKey(-1);
@ -426,7 +426,7 @@ public class TestLogsCLI {
UserGroupInformation.getCurrentUser().getShortUserName()).write(out,
new HashSet<File>());
out.close();
writer.close();
}
}
private YarnClient createMockYarnClient(YarnApplicationState appState)

View File

@ -70,7 +70,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LogAggregationContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Times;
import com.google.common.annotations.VisibleForTesting;
@ -378,13 +377,22 @@ public class AggregatedLogFormat {
* The writer that writes out the aggregated logs.
*/
@Private
public static class LogWriter {
public static class LogWriter implements AutoCloseable {
private final FSDataOutputStream fsDataOStream;
private final TFile.Writer writer;
private FSDataOutputStream fsDataOStream;
private TFile.Writer writer;
private FileContext fc;
public LogWriter(final Configuration conf, final Path remoteAppLogFile,
/**
* Initialize the LogWriter.
* Must be called just after the instance is created.
* @param conf Configuration
* @param remoteAppLogFile remote log file path
* @param userUgi Ugi of the user
* @throws IOException Failed to initialize
*/
public void initialize(final Configuration conf,
final Path remoteAppLogFile,
UserGroupInformation userUgi) throws IOException {
try {
this.fsDataOStream =
@ -463,12 +471,15 @@ public class AggregatedLogFormat {
}
}
@Override
public void close() {
if (writer != null) {
try {
this.writer.close();
} catch (IOException e) {
LOG.warn("Exception closing writer", e);
}
}
IOUtils.closeStream(fsDataOStream);
}
}

View File

@ -140,7 +140,8 @@ public class TestAggregatedLogFormat {
final int ch = filler;
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
LogWriter logWriter = new LogWriter(conf, remoteAppLogFile, ugi);
try (LogWriter logWriter = new LogWriter()) {
logWriter.initialize(conf, remoteAppLogFile, ugi);
LogKey logKey = new LogKey(testContainerId);
LogValue logValue =
@ -176,7 +177,7 @@ public class TestAggregatedLogFormat {
//Aggregate The Logs
logWriter.append(logKey, logValue);
logWriter.close();
}
}
@Test
@ -215,22 +216,22 @@ public class TestAggregatedLogFormat {
writeSrcFile(srcFilePath, "stdout", numChars);
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
LogWriter logWriter = new LogWriter(conf, remoteAppLogFile, ugi);
try (LogWriter logWriter = new LogWriter()) {
logWriter.initialize(conf, remoteAppLogFile, ugi);
LogKey logKey = new LogKey(testContainerId);
LogValue logValue =
new LogValue(Collections.singletonList(srcFileRoot.toString()),
testContainerId, ugi.getShortUserName());
// When we try to open FileInputStream for stderr, it will throw out an IOException.
// Skip the log aggregation for stderr.
// When we try to open FileInputStream for stderr, it will throw out an
// IOException. Skip the log aggregation for stderr.
LogValue spyLogValue = spy(logValue);
File errorFile = new File((new Path(srcFilePath, "stderr")).toString());
doThrow(new IOException("Mock can not open FileInputStream")).when(
spyLogValue).secureOpenFile(errorFile);
logWriter.append(logKey, spyLogValue);
logWriter.close();
}
// make sure permission are correct on the file
FileStatus fsStatus = fs.getFileStatus(remoteAppLogFile);
@ -310,8 +311,8 @@ public class TestAggregatedLogFormat {
UserGroupInformation ugi =
UserGroupInformation.getCurrentUser();
LogWriter logWriter = new LogWriter(conf, remoteAppLogFile, ugi);
try (LogWriter logWriter = new LogWriter()) {
logWriter.initialize(conf, remoteAppLogFile, ugi);
LogKey logKey = new LogKey(testContainerId1);
String randomUser = "randomUser";
LogValue logValue =
@ -320,13 +321,12 @@ public class TestAggregatedLogFormat {
// It is trying simulate a situation where first log file is owned by
// different user (probably symlink) and second one by the user itself.
// The first file should not be aggregated. Because this log file has the invalid
// user name.
// The first file should not be aggregated. Because this log file has the
// invalid user name.
when(logValue.getUser()).thenReturn(randomUser).thenReturn(
ugi.getShortUserName());
logWriter.append(logKey, logValue);
logWriter.close();
}
BufferedReader in =
new BufferedReader(new FileReader(new File(remoteAppLogFile

View File

@ -275,8 +275,9 @@ public class TestAggregatedLogsBlock {
List<String> rootLogDirs = Arrays.asList("target/logs/logs");
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
AggregatedLogFormat.LogWriter writer = new AggregatedLogFormat.LogWriter(
configuration, new Path(path), ugi);
try (AggregatedLogFormat.LogWriter writer =
new AggregatedLogFormat.LogWriter()) {
writer.initialize(configuration, new Path(path), ugi);
writer.writeApplicationOwner(ugi.getUserName());
Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
@ -284,8 +285,9 @@ public class TestAggregatedLogsBlock {
writer.writeApplicationACLs(appAcls);
writer.append(new AggregatedLogFormat.LogKey("container_0_0001_01_000001"),
new AggregatedLogFormat.LogValue(rootLogDirs, containerId,UserGroupInformation.getCurrentUser().getShortUserName()));
writer.close();
new AggregatedLogFormat.LogValue(rootLogDirs, containerId,
UserGroupInformation.getCurrentUser().getShortUserName()));
}
}
private void writeLogs(String dirName) throws Exception {

View File

@ -68,7 +68,6 @@ import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.Times;
@ -313,24 +312,21 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
}
}
LogWriter writer = null;
String diagnosticMessage = "";
boolean logAggregationSucceedInThisCycle = true;
try {
if (pendingContainerInThisCycle.isEmpty()) {
sendLogAggregationReport(true, "", appFinished);
return;
}
logAggregationTimes++;
String diagnosticMessage = "";
boolean logAggregationSucceedInThisCycle = true;
try (LogWriter writer = new LogWriter()){
try {
writer =
new LogWriter(this.conf, this.remoteNodeTmpLogFileForApp,
writer.initialize(this.conf, this.remoteNodeTmpLogFileForApp,
this.userUgi);
// Write ACLs once when the writer is created.
writer.writeApplicationACLs(appAcls);
writer.writeApplicationOwner(this.userUgi.getShortUserName());
} catch (IOException e1) {
logAggregationSucceedInThisCycle = false;
LOG.error("Cannot create writer for app " + this.applicationId
@ -371,11 +367,6 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
cleanupOldLogTimes++;
}
if (writer != null) {
writer.close();
writer = null;
}
long currentTime = System.currentTimeMillis();
final Path renamedPath = this.rollingMonitorInterval <= 0
? remoteNodeLogFileForApp : new Path(
@ -416,11 +407,19 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
logAggregationSucceedInThisCycle = false;
}
} finally {
sendLogAggregationReport(logAggregationSucceedInThisCycle,
diagnosticMessage, appFinished);
}
}
private void sendLogAggregationReport(
boolean logAggregationSucceedInThisCycle, String diagnosticMessage,
boolean appFinished) {
LogAggregationStatus logAggregationStatus =
logAggregationSucceedInThisCycle
? LogAggregationStatus.RUNNING
: LogAggregationStatus.RUNNING_WITH_FAILURE;
sendLogAggregationReport(logAggregationStatus, diagnosticMessage);
sendLogAggregationReportInternal(logAggregationStatus, diagnosticMessage);
if (appFinished) {
// If the app is finished, one extra final report with log aggregation
// status SUCCEEDED/FAILED will be sent to RM to inform the RM
@ -429,16 +428,11 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
renameTemporaryLogFileFailed || !logAggregationSucceedInThisCycle
? LogAggregationStatus.FAILED
: LogAggregationStatus.SUCCEEDED;
sendLogAggregationReport(finalLogAggregationStatus, "");
}
if (writer != null) {
writer.close();
}
sendLogAggregationReportInternal(finalLogAggregationStatus, "");
}
}
private void sendLogAggregationReport(
private void sendLogAggregationReportInternal(
LogAggregationStatus logAggregationStatus, String diagnosticMessage) {
LogAggregationReport report =
Records.newRecord(LogAggregationReport.class);