From ae15ad6202c15f05d31e331decdbc352485b6671 Mon Sep 17 00:00:00 2001 From: Robert Joseph Evans Date: Fri, 19 Oct 2012 17:34:15 +0000 Subject: [PATCH] YARN-163. Retrieving container log via NM webapp can hang with multibyte characters in log (jlowe via bobby) git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1400189 13f79535-47bb-0310-9956-ffa450edef68 --- hadoop-yarn-project/CHANGES.txt | 3 ++ .../nodemanager/webapp/ContainerLogsPage.java | 29 +++++++++---------- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index 70f62a93fe..307ef6b385 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -158,6 +158,9 @@ Release 0.23.5 - UNRELEASED BUG FIXES + YARN-163. Retrieving container log via NM webapp can hang with multibyte + characters in log (jlowe via bobby) + Release 0.23.4 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java index 94c7f0e91d..4cfeae15b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java @@ -26,7 +26,7 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; import java.io.File; -import java.io.FileReader; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; import java.net.URI; @@ -37,6 +37,7 @@ import java.util.EnumSet; import java.util.List; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; @@ -54,6 +55,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.PRE; import org.apache.hadoop.yarn.webapp.view.HtmlBlock; import org.mortbay.log.Log; @@ -226,7 +229,7 @@ private void printLogs(Block html, ContainerId containerId, + ", end[" + end + "]"); return; } else { - InputStreamReader reader = null; + FileInputStream logByteStream = null; try { long toRead = end - start; if (toRead < logFile.length()) { @@ -237,38 +240,34 @@ private void printLogs(Block html, ContainerId containerId, } // TODO: Use secure IO Utils to avoid symlink attacks. // TODO Fix findBugs close warning along with IOUtils change - reader = new FileReader(logFile); + logByteStream = new FileInputStream(logFile); + IOUtils.skipFully(logByteStream, start); + + InputStreamReader reader = new InputStreamReader(logByteStream); int bufferSize = 65536; char[] cbuf = new char[bufferSize]; - long skipped = 0; - long totalSkipped = 0; - while (totalSkipped < start) { - skipped = reader.skip(start - totalSkipped); - totalSkipped += skipped; - } - int len = 0; int currentToRead = toRead > bufferSize ? bufferSize : (int) toRead; - writer().write("
");
+            PRE pre = html.pre();
 
             while ((len = reader.read(cbuf, 0, currentToRead)) > 0
                 && toRead > 0) {
-              writer().write(cbuf, 0, len); // TODO: HTMl Quoting?
+              pre._(new String(cbuf, 0, len));
               toRead = toRead - len;
               currentToRead = toRead > bufferSize ? bufferSize : (int) toRead;
             }
 
+            pre._();
             reader.close();
-            writer().write("
"); } catch (IOException e) { html.h1("Exception reading log-file. Log file was likely aggregated. " + StringUtils.stringifyException(e)); } finally { - if (reader != null) { + if (logByteStream != null) { try { - reader.close(); + logByteStream.close(); } catch (IOException e) { // Ignore }