diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java index 25f493e0aaf..1e57cddc795 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java @@ -182,76 +182,69 @@ protected void render(Block html) { Decompressor decompressor = compressName.getDecompressor(); FileContext fileContext = FileContext.getFileContext( thisNodeFile.getPath().toUri(), conf); - FSDataInputStream fsin = fileContext.open(thisNodeFile.getPath()); - int bufferSize = 65536; - for (IndexedFileLogMeta candidate : candidates) { - byte[] cbuf = new byte[bufferSize]; - InputStream in = null; - try { - in = compressName.createDecompressionStream( - new BoundedRangeFileInputStream(fsin, - candidate.getStartIndex(), - candidate.getFileCompressedSize()), - decompressor, - LogAggregationIndexedFileController.getFSInputBufferSize( - conf)); - long logLength = candidate.getFileSize(); - html.pre()._("\n\n")._(); - html.p()._("Log Type: " + candidate.getFileName())._(); - html.p()._("Log Upload Time: " + Times.format( - candidate.getLastModificatedTime()))._(); - html.p()._("Log Length: " + Long.toString( - logLength))._(); - long startIndex = start < 0 - ? logLength + start : start; - startIndex = startIndex < 0 ? 0 : startIndex; - startIndex = startIndex > logLength ? logLength : startIndex; - long endLogIndex = end < 0 - ? logLength + end : end; - endLogIndex = endLogIndex < 0 ? 0 : endLogIndex; - endLogIndex = endLogIndex > logLength ? logLength : endLogIndex; - endLogIndex = endLogIndex < startIndex ? - startIndex : endLogIndex; - long toRead = endLogIndex - startIndex; - if (toRead < logLength) { - html.p()._("Showing " + toRead + " bytes of " + logLength - + " total. Click ").a(url("logs", $(NM_NODENAME), - $(CONTAINER_ID), $(ENTITY_STRING), $(APP_OWNER), - candidate.getFileName(), "?start=0"), "here"). - _(" for the full log.")._(); - } - long totalSkipped = 0; - while (totalSkipped < startIndex) { - long ret = in.skip(startIndex - totalSkipped); - if (ret == 0) { - //Read one byte - int nextByte = in.read(); - // Check if we have reached EOF - if (nextByte == -1) { - throw new IOException("Premature EOF from container log"); + try(FSDataInputStream fsin = fileContext.open(thisNodeFile.getPath())) { + int bufferSize = 65536; + for (IndexedFileLogMeta candidate : candidates) { + byte[] cbuf = new byte[bufferSize]; + InputStream in = null; + try { + in = compressName.createDecompressionStream( + new BoundedRangeFileInputStream(fsin,candidate.getStartIndex(), + candidate.getFileCompressedSize()), decompressor, + LogAggregationIndexedFileController.getFSInputBufferSize(conf)); + long logLength = candidate.getFileSize(); + html.pre()._("\n\n")._(); + html.p()._("Log Type: " + candidate.getFileName())._(); + html.p()._("Log Upload Time: " + + Times.format(candidate.getLastModificatedTime()))._(); + html.p()._("Log Length: " + Long.toString(logLength))._(); + long startIndex = start < 0 ? logLength + start : start; + startIndex = startIndex < 0 ? 0 : startIndex; + startIndex = startIndex > logLength ? logLength : startIndex; + long endLogIndex = end < 0 ? logLength + end : end; + endLogIndex = endLogIndex < 0 ? 0 : endLogIndex; + endLogIndex = endLogIndex > logLength ? logLength : endLogIndex; + endLogIndex = endLogIndex < startIndex ? startIndex : endLogIndex; + long toRead = endLogIndex - startIndex; + if (toRead < logLength) { + html.p()._("Showing " + toRead + " bytes of " + logLength + " total. Click ") + .a(url("logs", $(NM_NODENAME), $(CONTAINER_ID), $(ENTITY_STRING), $(APP_OWNER), + candidate.getFileName(), "?start=0"), "here"). + _(" for the full log.")._(); + } + long totalSkipped = 0; + while (totalSkipped < startIndex) { + long ret = in.skip(startIndex - totalSkipped); + if (ret == 0) { + //Read one byte + int nextByte = in.read(); + // Check if we have reached EOF + if (nextByte == -1) { + throw new IOException("Premature EOF from container log"); + } + ret = 1; } - ret = 1; + totalSkipped += ret; } - totalSkipped += ret; - } - int len = 0; - int currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; - PRE pre = html.pre(); + int len = 0; + int currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; + PRE pre = html.pre(); - while (toRead > 0 - && (len = in.read(cbuf, 0, currentToRead)) > 0) { - pre._(new String(cbuf, 0, len, Charset.forName("UTF-8"))); - toRead = toRead - len; - currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; - } + while (toRead > 0 && + (len = in.read(cbuf, 0, currentToRead)) > 0) { + pre._(new String(cbuf, 0, len, Charset.forName("UTF-8"))); + toRead = toRead - len; + currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; + } - pre._(); - foundLog = true; - } catch (Exception ex) { - LOG.error("Error getting logs for " + logEntity, ex); - continue; - } finally { - IOUtils.closeQuietly(in); + pre._(); + foundLog = true; + } catch (Exception ex) { + LOG.error("Error getting logs for " + logEntity, ex); + continue; + } finally { + IOUtils.closeQuietly(in); + } } } }