HIVE-13183 : Adds MapredLocalTasks logs in operation log (Rajat Khandelwal, reviwed by Amareshwari)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/14bcbab1 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/14bcbab1 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/14bcbab1 Branch: refs/heads/llap Commit: 14bcbab13a1dd925943ffa860138e89a0521e888 Parents: dc010a3 Author: Rajat Khandelwal <pro...@apache.org> Authored: Tue Apr 12 17:20:04 2016 +0530 Committer: Amareshwari Sriramadasu <amareshw...@apache.org> Committed: Tue Apr 12 17:20:04 2016 +0530 ---------------------------------------------------------------------- .../apache/hive/common/util/StreamPrinter.java | 18 +++++++++++------- .../apache/hadoop/hive/ql/exec/TaskRunner.java | 3 +-- .../apache/hadoop/hive/ql/exec/mr/MapRedTask.java | 5 ++--- .../hadoop/hive/ql/exec/mr/MapredLocalTask.java | 9 ++++++--- .../hadoop/hive/ql/session/OperationLog.java | 10 +++++++++- .../clientpositive/auto_sortmerge_join_8.q.out | 2 ++ .../clientpositive/llap/tez_join_hash.q.out | 4 ++++ .../clientpositive/tez/tez_join_hash.q.out | 4 ++++ 8 files changed, 39 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/common/src/java/org/apache/hive/common/util/StreamPrinter.java ---------------------------------------------------------------------- diff --git a/common/src/java/org/apache/hive/common/util/StreamPrinter.java b/common/src/java/org/apache/hive/common/util/StreamPrinter.java index 72638fd..1517751 100644 --- a/common/src/java/org/apache/hive/common/util/StreamPrinter.java +++ b/common/src/java/org/apache/hive/common/util/StreamPrinter.java @@ -33,12 +33,12 @@ import org.apache.hadoop.io.IOUtils; public class StreamPrinter extends Thread { InputStream is; String type; - PrintStream os; + PrintStream[] outputStreams; - public StreamPrinter(InputStream is, String type, PrintStream os) { + public StreamPrinter(InputStream is, String type, PrintStream... outputStreams) { this.is = is; this.type = type; - this.os = os; + this.outputStreams = outputStreams; } @Override @@ -50,18 +50,22 @@ public class StreamPrinter extends Thread { String line = null; if (type != null) { while ((line = br.readLine()) != null) { - os.println(type + ">" + line); + for (PrintStream os: outputStreams) { + os.println(type + ">" + line); + } } } else { while ((line = br.readLine()) != null) { - os.println(line); + for (PrintStream os: outputStreams) { + os.println(line); + } } } br.close(); - br=null; + br = null; } catch (IOException ioe) { ioe.printStackTrace(); - }finally{ + } finally { IOUtils.closeStream(br); } } http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java index f6fd081..81f6db0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskRunner.java @@ -32,7 +32,6 @@ import org.slf4j.LoggerFactory; **/ public class TaskRunner extends Thread { - protected Task<? extends Serializable> tsk; protected TaskResult result; protected SessionState ss; @@ -103,7 +102,7 @@ public class TaskRunner extends Thread { if (tsk.getException() == null) { tsk.setException(t); } - t.printStackTrace(); + LOG.error("Error in executeTask", t); } result.setExitVal(exitVal, tsk.getException()); } http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java index 310356c..a42c2e9 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java @@ -301,8 +301,7 @@ public class MapRedTask extends ExecDriver implements Serializable { return exitVal; } catch (Exception e) { - e.printStackTrace(); - LOG.error("Exception: " + e.getMessage()); + LOG.error("Got exception", e); return (1); } finally { try { @@ -313,7 +312,7 @@ public class MapRedTask extends ExecDriver implements Serializable { } } catch (Exception e) { - LOG.error("Exception: " + e.getMessage()); + LOG.error("Exception: ", e); } } } http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java index f5500a4..c81b14c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java @@ -65,6 +65,7 @@ import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.api.StageType; +import org.apache.hadoop.hive.ql.session.OperationLog; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; @@ -317,8 +318,10 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab CachingPrintStream errPrintStream = new CachingPrintStream(System.err); - StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out); - StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, errPrintStream); + StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out, + OperationLog.getCurrentOperationLog().getPrintStream()); + StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, errPrintStream, + OperationLog.getCurrentOperationLog().getPrintStream()); outPrinter.start(); errPrinter.start(); @@ -340,7 +343,7 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab return exitVal; } catch (Exception e) { - LOG.error("Exception: " + e, e); + LOG.error("Exception: ", e); return (1); } finally { if (secureDoAs != null) { http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java b/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java index 2ecdde9..6d0f14a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/OperationLog.java @@ -39,7 +39,11 @@ public class OperationLog { private final LogFile logFile; private LoggingLevel opLoggingLevel = LoggingLevel.UNKNOWN; - public static enum LoggingLevel { + public PrintStream getPrintStream() { + return logFile.getPrintStream(); + } + + public enum LoggingLevel { NONE, EXECUTION, PERFORMANCE, VERBOSE, UNKNOWN } @@ -221,5 +225,9 @@ public class OperationLog { } return logs; } + + public PrintStream getPrintStream() { + return out; + } } } http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out index 38996a7..d22ba3b 100644 --- a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out +++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out @@ -1507,6 +1507,8 @@ PREHOOK: Input: default@bucket_small PREHOOK: Input: default@bucket_small@ds=2008-04-08 PREHOOK: Input: default@bucket_small@ds=2008-04-09 #### A masked pattern was here #### +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@bucket_big http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out index 54ca9d2..1fd45aa 100644 --- a/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out +++ b/ql/src/test/results/clientpositive/llap/tez_join_hash.q.out @@ -652,6 +652,10 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 #### A masked pattern was here #### +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: select key, count(*) from (select x.key as key, y.value as value from srcpart x join srcpart y on (x.key = y.key) union all http://git-wip-us.apache.org/repos/asf/hive/blob/14bcbab1/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out b/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out index 8d0aba1..2f51094 100644 --- a/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out +++ b/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out @@ -638,6 +638,10 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 #### A masked pattern was here #### +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: select key, count(*) from (select x.key as key, y.value as value from srcpart x join srcpart y on (x.key = y.key) union all