You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2012/03/09 20:32:22 UTC

svn commit: r1299000 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/common/io/ ql/src/java/org/apache/hadoop/hive/ql/session/ ql/src/test/org/apache/hadoop/hive/ql/ ql/src/test/org/apache/hadoop/hive/q...

Author: namit
Date: Fri Mar  9 19:32:21 2012
New Revision: 1299000

URL: http://svn.apache.org/viewvc?rev=1299000&view=rev
Log:
HIVE-2832 Cache error messages for additional logging
(Kevin Wilfong via namit)


Added:
    hive/trunk/common/src/java/org/apache/hadoop/hive/common/io/CachingPrintStream.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyCachingPrintStreamHook.java
    hive/trunk/ql/src/test/queries/clientnegative/cachingprintstream.q
    hive/trunk/ql/src/test/results/clientnegative/cachingprintstream.q.out
Modified:
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1299000&r1=1298999&r2=1299000&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Fri Mar  9 19:32:21 2012
@@ -46,6 +46,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hive.common.HiveInterruptUtils;
 import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
+import org.apache.hadoop.hive.common.io.CachingPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
@@ -94,6 +95,8 @@ public class CliDriver {
 
   public int processCmd(String cmd) {
     CliSessionState ss = (CliSessionState) SessionState.get();
+    // Flush the print stream, so it doesn't include output from the last command
+    ss.err.flush();
     String cmd_trimmed = cmd.trim();
     String[] tokens = tokenizeCmd(cmd_trimmed);
     int ret = 0;
@@ -577,7 +580,8 @@ public class CliDriver {
     ss.in = System.in;
     try {
       ss.out = new PrintStream(System.out, true, "UTF-8");
-      ss.err = new PrintStream(System.err, true, "UTF-8");
+      ss.info = new PrintStream(System.err, true, "UTF-8");
+      ss.err = new CachingPrintStream(System.err, true, "UTF-8");
     } catch (UnsupportedEncodingException e) {
       return 3;
     }

Added: hive/trunk/common/src/java/org/apache/hadoop/hive/common/io/CachingPrintStream.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/common/io/CachingPrintStream.java?rev=1299000&view=auto
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/common/io/CachingPrintStream.java (added)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/common/io/CachingPrintStream.java Fri Mar  9 19:32:21 2012
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.common.io;
+
+import java.io.FileNotFoundException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.List;
+
+// A printStream that stores messages logged to it in a list.
+public class CachingPrintStream extends PrintStream {
+
+  List<String> output = new ArrayList<String>();
+
+  public CachingPrintStream(OutputStream out, boolean autoFlush, String encoding)
+      throws FileNotFoundException, UnsupportedEncodingException {
+
+    super(out, autoFlush, encoding);
+  }
+
+  @Override
+  public void println(String out) {
+    output.add(out);
+    super.println(out);
+  }
+
+  @Override
+  public void flush() {
+    output = new ArrayList<String>();
+    super.flush();
+  }
+
+  public List<String> getOutput() {
+    return output;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1299000&r1=1298999&r2=1299000&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Fri Mar  9 19:32:21 2012
@@ -84,6 +84,7 @@ public class SessionState {
    */
   public InputStream in;
   public PrintStream out;
+  public PrintStream info;
   public PrintStream err;
   /**
    * Standard output from any child process(es).
@@ -339,6 +340,11 @@ public class SessionState {
       return ((ss != null) && (ss.out != null)) ? ss.out : System.out;
     }
 
+    public PrintStream getInfoStream() {
+      SessionState ss = SessionState.get();
+      return ((ss != null) && (ss.info != null)) ? ss.info : getErrStream();
+    }
+
     public PrintStream getErrStream() {
       SessionState ss = SessionState.get();
       return ((ss != null) && (ss.err != null)) ? ss.err : System.err;
@@ -366,7 +372,7 @@ public class SessionState {
 
     public void printInfo(String info, String detail) {
       if (!getIsSilent()) {
-        getErrStream().println(info);
+        getInfoStream().println(info);
       }
       LOG.info(info + StringUtils.defaultString(detail));
     }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1299000&r1=1298999&r2=1299000&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Fri Mar  9 19:32:21 2012
@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hive.cli.CliDriver;
 import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.common.io.CachingPrintStream;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Index;
@@ -586,7 +587,7 @@ public class QTestUtil {
     outf = new File(outf, qf.getName().concat(".out"));
     FileOutputStream fo = new FileOutputStream(outf);
     ss.out = new PrintStream(fo, true, "UTF-8");
-    ss.err = ss.out;
+    ss.err = new CachingPrintStream(fo, true, "UTF-8");
     ss.setIsSilent(true);
     SessionState oldSs = SessionState.get();
     if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyCachingPrintStreamHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyCachingPrintStreamHook.java?rev=1299000&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyCachingPrintStreamHook.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyCachingPrintStreamHook.java Fri Mar  9 19:32:21 2012
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.hooks;
+
+import org.apache.hadoop.hive.common.io.CachingPrintStream;
+import org.apache.hadoop.hive.ql.hooks.HookContext.HookType;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+// If this is run as a pre or post execution hook, it writes a message to SessionState.err
+// (causing it to be cached if a CachingPrintStream is being used).  If it is run as a failure
+// hook, it will write what has been cached by the CachingPrintStream to SessionState.out for
+// verification.
+public class VerifyCachingPrintStreamHook implements ExecuteWithHookContext {
+
+  public void run(HookContext hookContext) {
+    SessionState ss = SessionState.get();
+
+    assert(ss.err instanceof CachingPrintStream);
+
+    if (hookContext.getHookType() == HookType.ON_FAILURE_HOOK) {
+      assert(ss.err instanceof CachingPrintStream);
+      ss.out.println("Begin cached logs.");
+      for (String output : ((CachingPrintStream)ss.err).getOutput()) {
+        ss.out.println(output);
+      }
+      ss.out.println("End cached logs.");
+    } else {
+      ss.err.println("TEST, this should only appear once in the log.");
+    }
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientnegative/cachingprintstream.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/cachingprintstream.q?rev=1299000&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/cachingprintstream.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/cachingprintstream.q Fri Mar  9 19:32:21 2012
@@ -0,0 +1,8 @@
+set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook;
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook;
+
+SELECT count(*) FROM src;
+FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value;
+
+set hive.exec.failure.hooks=;
+set hive.exec.post.hooks=;

Added: hive/trunk/ql/src/test/results/clientnegative/cachingprintstream.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/cachingprintstream.q.out?rev=1299000&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/cachingprintstream.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/cachingprintstream.q.out Fri Mar  9 19:32:21 2012
@@ -0,0 +1,37 @@
+PREHOOK: query: SELECT count(*) FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+TEST, this should only appear once in the log.
+500
+PREHOOK: query: FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+Execution failed with exit status: 2
+Obtaining error information
+
+Task failed!
+Task ID:
+  Stage-1
+
+Logs:
+
+#### A masked pattern was here ####
+Begin cached logs.
+PREHOOK: query: FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+Execution failed with exit status: 2
+Obtaining error information
+
+Task failed!
+Task ID:
+  Stage-1
+
+Logs:
+
+#### A masked pattern was here ####
+End cached logs.
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask