Skip to content

Commit

Permalink
YARN-6100. Improve YARN webservice to output aggregated container log…
Browse files Browse the repository at this point in the history
…s. Contributed by Xuan Gong.
  • Loading branch information
JunpingDu committed Feb 2, 2017
1 parent 2a942ee commit 327c998
Show file tree
Hide file tree
Showing 6 changed files with 332 additions and 234 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@
import org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest;
import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
Expand Down Expand Up @@ -509,17 +508,9 @@ public int printContainerLogsFromRunningApplication(Configuration conf,
newOptions.setLogTypes(matchedFiles);

Client webServiceClient = Client.create();
String containerString = String.format(
LogCLIHelpers.CONTAINER_ON_NODE_PATTERN, containerIdStr, nodeId);
out.println(containerString);
out.println(StringUtils.repeat("=", containerString.length()));
boolean foundAnyLogs = false;
byte[] buffer = new byte[65536];
for (String logFile : newOptions.getLogTypes()) {
out.println("LogType:" + logFile);
out.println("Log Upload Time:"
+ Times.format(System.currentTimeMillis()));
out.println("Log Contents:");
InputStream is = null;
try {
ClientResponse response = getResponeFromNMWebService(conf,
Expand All @@ -541,14 +532,6 @@ public int printContainerLogsFromRunningApplication(Configuration conf,
response.getEntity(String.class));
out.println(msg);
}
StringBuilder sb = new StringBuilder();
sb.append("End of LogType:" + logFile + ".");
if (request.getContainerState() == ContainerState.RUNNING) {
sb.append(" This log file belongs"
+ " to a running container (" + containerIdStr + ") and so may"
+ " not be complete.");
}
out.println(sb.toString());
out.flush();
foundAnyLogs = true;
} catch (ClientHandlerException | UniformInterfaceException ex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,17 @@
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.math3.util.Pair;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.HarFs;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
Expand All @@ -40,6 +46,9 @@ public final class LogToolUtils {

private LogToolUtils() {}

public static final String CONTAINER_ON_NODE_PATTERN =
"Container: %s on %s";

/**
* Return a list of {@link ContainerLogMeta} for a container
* from Remote FileSystem.
Expand Down Expand Up @@ -114,4 +123,153 @@ public static List<ContainerLogMeta> getContainerLogMetaFromRemoteFS(
}
return containersLogMeta;
}

/**
* Output container log.
* @param containerId the containerId
* @param nodeId the nodeId
* @param fileName the log file name
* @param fileLength the log file length
* @param outputSize the output size
* @param lastModifiedTime the log file last modified time
* @param fis the log file input stream
* @param os the output stream
* @param buf the buffer
* @param logType the log type.
* @throws IOException if we can not access the log file.
*/
public static void outputContainerLog(String containerId, String nodeId,
String fileName, long fileLength, long outputSize,
String lastModifiedTime, InputStream fis, OutputStream os,
byte[] buf, ContainerLogType logType) throws IOException {
long toSkip = 0;
long totalBytesToRead = fileLength;
long skipAfterRead = 0;
if (outputSize < 0) {
long absBytes = Math.abs(outputSize);
if (absBytes < fileLength) {
toSkip = fileLength - absBytes;
totalBytesToRead = absBytes;
}
org.apache.hadoop.io.IOUtils.skipFully(fis, toSkip);
} else {
if (outputSize < fileLength) {
totalBytesToRead = outputSize;
skipAfterRead = fileLength - outputSize;
}
}

long curRead = 0;
long pendingRead = totalBytesToRead - curRead;
int toRead = pendingRead > buf.length ? buf.length
: (int) pendingRead;
int len = fis.read(buf, 0, toRead);
boolean keepGoing = (len != -1 && curRead < totalBytesToRead);
if (keepGoing) {
StringBuilder sb = new StringBuilder();
String containerStr = String.format(
LogToolUtils.CONTAINER_ON_NODE_PATTERN,
containerId, nodeId);
sb.append(containerStr + "\n");
sb.append("LogType: " + logType + "\n");
sb.append(StringUtils.repeat("=", containerStr.length()) + "\n");
sb.append("FileName:" + fileName + "\n");
sb.append("LogLastModifiedTime:" + lastModifiedTime + "\n");
sb.append("LogLength:" + Long.toString(fileLength) + "\n");
sb.append("LogContents:\n");
byte[] b = sb.toString().getBytes(
Charset.forName("UTF-8"));
os.write(b, 0, b.length);
}
while (keepGoing) {
os.write(buf, 0, len);
curRead += len;

pendingRead = totalBytesToRead - curRead;
toRead = pendingRead > buf.length ? buf.length
: (int) pendingRead;
len = fis.read(buf, 0, toRead);
keepGoing = (len != -1 && curRead < totalBytesToRead);
}
org.apache.hadoop.io.IOUtils.skipFully(fis, skipAfterRead);
os.flush();
}

public static boolean outputAggregatedContainerLog(Configuration conf,
ApplicationId appId, String appOwner,
String containerId, String nodeId,
String logFileName, long outputSize, OutputStream os,
byte[] buf) throws IOException {
boolean findLogs = false;
RemoteIterator<FileStatus> nodeFiles = LogAggregationUtils
.getRemoteNodeFileDir(conf, appId, appOwner);
while (nodeFiles != null && nodeFiles.hasNext()) {
final FileStatus thisNodeFile = nodeFiles.next();
String nodeName = thisNodeFile.getPath().getName();
if (nodeName.equals(appId + ".har")) {
Path p = new Path("har:///"
+ thisNodeFile.getPath().toUri().getRawPath());
nodeFiles = HarFs.get(p.toUri(), conf).listStatusIterator(p);
continue;
}
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(containerId)) {
// 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(logFileName)) {
LogToolUtils.outputContainerLog(containerId,
nodeId, fileType, fileLength, outputSize,
Times.format(thisNodeFile.getModificationTime()),
valueStream, os, buf, ContainerLogType.AGGREGATED);
StringBuilder sb = new StringBuilder();
String endOfFile = "End of LogFile:" + fileType;
sb.append("\n" + endOfFile + "\n");
sb.append(StringUtils.repeat("*", endOfFile.length() + 50)
+ "\n\n");
byte[] 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();
return findLogs;
}
}
Loading

0 comments on commit 327c998

Please sign in to comment.