|
@@ -18,8 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
|
|
|
|
|
|
-import java.io.DataInputStream;
|
|
|
-import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.nio.charset.Charset;
|
|
@@ -43,12 +41,10 @@ import javax.ws.rs.core.Response;
|
|
|
import javax.ws.rs.core.StreamingOutput;
|
|
|
import javax.ws.rs.core.Response.ResponseBuilder;
|
|
|
import javax.ws.rs.core.Response.Status;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.FileContext;
|
|
|
-import org.apache.hadoop.fs.FileStatus;
|
|
|
-import org.apache.hadoop.fs.RemoteIterator;
|
|
|
import org.apache.hadoop.http.JettyUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -56,13 +52,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
|
import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
|
|
|
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
-import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
|
|
|
import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
|
|
|
import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
|
|
|
-import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
|
|
|
import org.apache.hadoop.yarn.logaggregation.LogToolUtils;
|
|
|
-import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
|
import org.apache.hadoop.yarn.server.webapp.WebServices;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo;
|
|
@@ -71,11 +63,11 @@ import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainerLogsInfo;
|
|
|
import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
|
|
|
-import org.apache.hadoop.yarn.util.Times;
|
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
|
import org.apache.hadoop.yarn.webapp.BadRequestException;
|
|
|
import org.apache.hadoop.yarn.webapp.NotFoundException;
|
|
|
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Joiner;
|
|
|
import com.google.inject.Inject;
|
|
|
import com.google.inject.Singleton;
|
|
@@ -360,27 +352,27 @@ public class AHSWebServices extends WebServices {
|
|
|
} catch (Exception ex) {
|
|
|
// directly find logs from HDFS.
|
|
|
return sendStreamOutputResponse(appId, null, null, containerIdStr,
|
|
|
- filename, format, length);
|
|
|
+ filename, format, length, false);
|
|
|
}
|
|
|
String appOwner = appInfo.getUser();
|
|
|
+ if (isFinishedState(appInfo.getAppState())) {
|
|
|
+ // directly find logs from HDFS.
|
|
|
+ return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
|
|
|
+ filename, format, length, false);
|
|
|
+ }
|
|
|
|
|
|
- ContainerInfo containerInfo;
|
|
|
- try {
|
|
|
- containerInfo = super.getContainer(
|
|
|
- req, res, appId.toString(),
|
|
|
- containerId.getApplicationAttemptId().toString(),
|
|
|
- containerId.toString());
|
|
|
- } catch (Exception ex) {
|
|
|
- if (isFinishedState(appInfo.getAppState())) {
|
|
|
- // directly find logs from HDFS.
|
|
|
+ if (isRunningState(appInfo.getAppState())) {
|
|
|
+ ContainerInfo containerInfo;
|
|
|
+ try {
|
|
|
+ containerInfo = super.getContainer(
|
|
|
+ req, res, appId.toString(),
|
|
|
+ containerId.getApplicationAttemptId().toString(),
|
|
|
+ containerId.toString());
|
|
|
+ } catch (Exception ex) {
|
|
|
+ // output the aggregated logs
|
|
|
return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
|
|
|
- filename, format, length);
|
|
|
+ filename, format, length, true);
|
|
|
}
|
|
|
- return createBadResponse(Status.INTERNAL_SERVER_ERROR,
|
|
|
- "Can not get ContainerInfo for the container: " + containerId);
|
|
|
- }
|
|
|
- String nodeId = containerInfo.getNodeId();
|
|
|
- if (isRunningState(appInfo.getAppState())) {
|
|
|
String nodeHttpAddress = containerInfo.getNodeHttpAddress();
|
|
|
String uri = "/" + containerId.toString() + "/logs/" + filename;
|
|
|
String resURI = JOINER.join(nodeHttpAddress, NM_DOWNLOAD_URI_STR, uri);
|
|
@@ -392,9 +384,6 @@ public class AHSWebServices extends WebServices {
|
|
|
HttpServletResponse.SC_TEMPORARY_REDIRECT);
|
|
|
response.header("Location", resURI);
|
|
|
return response.build();
|
|
|
- } else if (isFinishedState(appInfo.getAppState())) {
|
|
|
- return sendStreamOutputResponse(appId, appOwner, nodeId,
|
|
|
- containerIdStr, filename, format, length);
|
|
|
} else {
|
|
|
return createBadResponse(Status.NOT_FOUND,
|
|
|
"The application is not at Running or Finished State.");
|
|
@@ -419,7 +408,8 @@ public class AHSWebServices extends WebServices {
|
|
|
|
|
|
private Response sendStreamOutputResponse(ApplicationId appId,
|
|
|
String appOwner, String nodeId, String containerIdStr,
|
|
|
- String fileName, String format, long bytes) {
|
|
|
+ String fileName, String format, long bytes,
|
|
|
+ boolean printEmptyLocalContainerLog) {
|
|
|
String contentType = WebAppUtils.getDefaultLogContentType();
|
|
|
if (format != null && !format.isEmpty()) {
|
|
|
contentType = WebAppUtils.getSupportedLogContentType(format);
|
|
@@ -433,15 +423,11 @@ public class AHSWebServices extends WebServices {
|
|
|
StreamingOutput stream = null;
|
|
|
try {
|
|
|
stream = getStreamingOutput(appId, appOwner, nodeId,
|
|
|
- containerIdStr, fileName, bytes);
|
|
|
+ containerIdStr, fileName, bytes, printEmptyLocalContainerLog);
|
|
|
} catch (Exception ex) {
|
|
|
return createBadResponse(Status.INTERNAL_SERVER_ERROR,
|
|
|
ex.getMessage());
|
|
|
}
|
|
|
- if (stream == null) {
|
|
|
- return createBadResponse(Status.INTERNAL_SERVER_ERROR,
|
|
|
- "Can not get log for container: " + containerIdStr);
|
|
|
- }
|
|
|
ResponseBuilder response = Response.ok(stream);
|
|
|
response.header("Content-Type", contentType);
|
|
|
// Sending the X-Content-Type-Options response header with the value
|
|
@@ -451,146 +437,30 @@ public class AHSWebServices extends WebServices {
|
|
|
return response.build();
|
|
|
}
|
|
|
|
|
|
- private StreamingOutput getStreamingOutput(ApplicationId appId,
|
|
|
- String appOwner, final String nodeId, final String containerIdStr,
|
|
|
- final String logFile, final long bytes) throws IOException{
|
|
|
- String suffix = LogAggregationUtils.getRemoteNodeLogDirSuffix(conf);
|
|
|
- org.apache.hadoop.fs.Path remoteRootLogDir = new org.apache.hadoop.fs.Path(
|
|
|
- conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
- YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
|
|
|
- org.apache.hadoop.fs.Path qualifiedRemoteRootLogDir =
|
|
|
- FileContext.getFileContext(conf).makeQualified(remoteRootLogDir);
|
|
|
- FileContext fc = FileContext.getFileContext(
|
|
|
- qualifiedRemoteRootLogDir.toUri(), conf);
|
|
|
- org.apache.hadoop.fs.Path remoteAppDir = null;
|
|
|
- if (appOwner == null) {
|
|
|
- org.apache.hadoop.fs.Path toMatch = LogAggregationUtils
|
|
|
- .getRemoteAppLogDir(remoteRootLogDir, appId, "*", suffix);
|
|
|
- FileStatus[] matching = fc.util().globStatus(toMatch);
|
|
|
- if (matching == null || matching.length != 1) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- remoteAppDir = matching[0].getPath();
|
|
|
- } else {
|
|
|
- remoteAppDir = LogAggregationUtils
|
|
|
- .getRemoteAppLogDir(remoteRootLogDir, appId, appOwner, suffix);
|
|
|
- }
|
|
|
- final RemoteIterator<FileStatus> nodeFiles;
|
|
|
- nodeFiles = fc.listStatus(remoteAppDir);
|
|
|
- if (!nodeFiles.hasNext()) {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
+ private StreamingOutput getStreamingOutput(final ApplicationId appId,
|
|
|
+ final String appOwner, final String nodeId, final String containerIdStr,
|
|
|
+ final String logFile, final long bytes,
|
|
|
+ final boolean printEmptyLocalContainerLog) throws IOException{
|
|
|
StreamingOutput stream = new StreamingOutput() {
|
|
|
|
|
|
@Override
|
|
|
public void write(OutputStream os) throws IOException,
|
|
|
WebApplicationException {
|
|
|
byte[] buf = new byte[65535];
|
|
|
- boolean findLogs = false;
|
|
|
- while (nodeFiles.hasNext()) {
|
|
|
- final FileStatus thisNodeFile = nodeFiles.next();
|
|
|
- String nodeName = thisNodeFile.getPath().getName();
|
|
|
- if ((nodeId == null || nodeName.contains(LogAggregationUtils
|
|
|
- .getNodeString(nodeId))) && !nodeName.endsWith(
|
|
|
- LogAggregationUtils.TMP_FILE_SUFFIX)) {
|
|
|
- AggregatedLogFormat.LogReader reader = null;
|
|
|
- try {
|
|
|
- reader = new AggregatedLogFormat.LogReader(conf,
|
|
|
- thisNodeFile.getPath());
|
|
|
- DataInputStream valueStream;
|
|
|
- LogKey key = new LogKey();
|
|
|
- valueStream = reader.next(key);
|
|
|
- while (valueStream != null && !key.toString()
|
|
|
- .equals(containerIdStr)) {
|
|
|
- // Next container
|
|
|
- key = new LogKey();
|
|
|
- valueStream = reader.next(key);
|
|
|
- }
|
|
|
- if (valueStream == null) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- while (true) {
|
|
|
- try {
|
|
|
- String fileType = valueStream.readUTF();
|
|
|
- String fileLengthStr = valueStream.readUTF();
|
|
|
- long fileLength = Long.parseLong(fileLengthStr);
|
|
|
- if (fileType.equalsIgnoreCase(logFile)) {
|
|
|
- StringBuilder sb = new StringBuilder();
|
|
|
- sb.append("LogType:");
|
|
|
- sb.append(fileType + "\n");
|
|
|
- sb.append("Log Upload Time:");
|
|
|
- sb.append(Times.format(System.currentTimeMillis()) + "\n");
|
|
|
- sb.append("LogLength:");
|
|
|
- sb.append(fileLengthStr + "\n");
|
|
|
- sb.append("Log Contents:\n");
|
|
|
- byte[] b = sb.toString().getBytes(
|
|
|
- Charset.forName("UTF-8"));
|
|
|
- os.write(b, 0, b.length);
|
|
|
-
|
|
|
- long toSkip = 0;
|
|
|
- long totalBytesToRead = fileLength;
|
|
|
- long skipAfterRead = 0;
|
|
|
- if (bytes < 0) {
|
|
|
- long absBytes = Math.abs(bytes);
|
|
|
- if (absBytes < fileLength) {
|
|
|
- toSkip = fileLength - absBytes;
|
|
|
- totalBytesToRead = absBytes;
|
|
|
- }
|
|
|
- org.apache.hadoop.io.IOUtils.skipFully(
|
|
|
- valueStream, toSkip);
|
|
|
- } else {
|
|
|
- if (bytes < fileLength) {
|
|
|
- totalBytesToRead = bytes;
|
|
|
- skipAfterRead = fileLength - bytes;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- long curRead = 0;
|
|
|
- long pendingRead = totalBytesToRead - curRead;
|
|
|
- int toRead = pendingRead > buf.length ? buf.length
|
|
|
- : (int) pendingRead;
|
|
|
- int len = valueStream.read(buf, 0, toRead);
|
|
|
- while (len != -1 && curRead < totalBytesToRead) {
|
|
|
- os.write(buf, 0, len);
|
|
|
- curRead += len;
|
|
|
-
|
|
|
- pendingRead = totalBytesToRead - curRead;
|
|
|
- toRead = pendingRead > buf.length ? buf.length
|
|
|
- : (int) pendingRead;
|
|
|
- len = valueStream.read(buf, 0, toRead);
|
|
|
- }
|
|
|
- org.apache.hadoop.io.IOUtils.skipFully(
|
|
|
- valueStream, skipAfterRead);
|
|
|
- sb = new StringBuilder();
|
|
|
- sb.append("\nEnd of LogType:" + fileType + "\n");
|
|
|
- b = sb.toString().getBytes(Charset.forName("UTF-8"));
|
|
|
- os.write(b, 0, b.length);
|
|
|
- findLogs = true;
|
|
|
- } else {
|
|
|
- long totalSkipped = 0;
|
|
|
- long currSkipped = 0;
|
|
|
- while (currSkipped != -1 && totalSkipped < fileLength) {
|
|
|
- currSkipped = valueStream.skip(
|
|
|
- fileLength - totalSkipped);
|
|
|
- totalSkipped += currSkipped;
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (EOFException eof) {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (reader != null) {
|
|
|
- reader.close();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- os.flush();
|
|
|
+ boolean findLogs = LogToolUtils.outputAggregatedContainerLog(conf,
|
|
|
+ appId, appOwner, containerIdStr, nodeId, logFile, bytes, os, buf);
|
|
|
if (!findLogs) {
|
|
|
throw new IOException("Can not find logs for container:"
|
|
|
+ containerIdStr);
|
|
|
+ } else {
|
|
|
+ if (printEmptyLocalContainerLog) {
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append(containerIdStr + "\n");
|
|
|
+ sb.append("LogType: " + ContainerLogType.LOCAL + "\n");
|
|
|
+ sb.append("LogContents:\n");
|
|
|
+ sb.append(getNoRedirectWarning() + "\n");
|
|
|
+ os.write(sb.toString().getBytes(Charset.forName("UTF-8")));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
};
|
|
@@ -640,4 +510,12 @@ public class AHSWebServices extends WebServices {
|
|
|
throw new WebApplicationException(ex);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Private
|
|
|
+ @VisibleForTesting
|
|
|
+ public static String getNoRedirectWarning() {
|
|
|
+ return "We do not have NodeManager web address, so we can not "
|
|
|
+ + "re-direct the request to related NodeManager "
|
|
|
+ + "for local container logs.";
|
|
|
+ }
|
|
|
}
|