You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2016/05/23 17:51:23 UTC

hive git commit: HIVE-13628: Support for permanent functions - error handling if no restart (Vikram Dixit K, reviewed by Gunther Hagleitner)

Repository: hive
Updated Branches:
  refs/heads/master b84154be3 -> 999f0607b


HIVE-13628: Support for permanent functions - error handling if no restart (Vikram Dixit K, reviewed by Gunther Hagleitner)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/999f0607
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/999f0607
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/999f0607

Branch: refs/heads/master
Commit: 999f0607bdefd34028bbbff64268fb16f399a0cf
Parents: b84154b
Author: vikram <vi...@hortonworks.com>
Authored: Mon May 23 10:46:45 2016 -0700
Committer: vikram <vi...@hortonworks.com>
Committed: Mon May 23 10:47:48 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Operator.java    |  6 ++++-
 .../apache/hadoop/hive/ql/exec/Utilities.java   | 24 ++++++++++++++++++++
 .../hadoop/hive/ql/exec/tez/TezProcessor.java   |  3 +++
 3 files changed, 32 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/999f0607/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index 636f079..00552a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -406,7 +406,11 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
       }
     }
     if (asyncEx != null) {
-      throw new HiveException("Async initialization failed", asyncEx);
+      if (asyncEx instanceof Exception) {
+        throw new HiveException("Async initialization failed", asyncEx);
+      } else {
+        throw (Error) asyncEx;
+      }
     }
     completeInitializationOp(os);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/999f0607/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 449bef8..2ab9ed2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec;
 import java.util.ArrayList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
 import java.beans.DefaultPersistenceDelegate;
 import java.beans.Encoder;
 import java.beans.Expression;
@@ -96,6 +97,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -194,6 +196,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoException;
 import com.google.common.base.Preconditions;
 
 /**
@@ -436,6 +439,7 @@ public final class Utilities {
             throw new RuntimeException("Unknown work type: " + name);
           }
         }
+
         gWorkMap.get(conf).put(path, gWork);
       } else if (LOG.isDebugEnabled()) {
         LOG.debug("Found plan in cache for name: " + name);
@@ -446,6 +450,16 @@ public final class Utilities {
       LOG.debug("File not found: " + fnf.getMessage());
       LOG.info("No plan file found: "+path);
       return null;
+    } catch (KryoException ke) {
+      Throwable cnfThrowable = findClassNotFoundException(ke);
+      if (LlapProxy.isDaemon() && (cnfThrowable != null)) {
+        LOG.error("Missing class \"" + cnfThrowable.getMessage() + "\". If this is a UDF and you " +
+            "are running LLAP, you may need to regenerate the llap startup script and restart " +
+            "llap with jars for your udf.", cnfThrowable);
+        throw new RuntimeException("Cannot find \"" + cnfThrowable.getMessage() + "\" You may" +
+           " need to regenerate the LLAP startup script and restart llap daemons.", cnfThrowable);
+      }
+      throw new RuntimeException(ke);
     } catch (Exception e) {
       String msg = "Failed to load plan: " + path + ": " + e;
       LOG.error(msg, e);
@@ -460,6 +474,16 @@ public final class Utilities {
     }
   }
 
+  private static Throwable findClassNotFoundException(Throwable ke) {
+    while (ke != null) {
+      if (ke instanceof ClassNotFoundException) {
+        return ke;
+      }
+      ke = ke.getCause();
+    }
+    return null;
+  }
+
   public static void setWorkflowAdjacencies(Configuration conf, QueryPlan plan) {
     try {
       Graph stageGraph = plan.getQueryPlan().getStageGraph();

http://git-wip-us.apache.org/repos/asf/hive/blob/999f0607/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
index c560f37..a33b6e2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
@@ -39,6 +39,7 @@ import org.apache.tez.runtime.api.ExecutionContext;
 import org.apache.tez.runtime.api.LogicalInput;
 import org.apache.tez.runtime.api.LogicalOutput;
 import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.TaskFailureType;
 import org.apache.tez.runtime.library.api.KeyValueWriter;
 
 /**
@@ -178,6 +179,8 @@ public class TezProcessor extends AbstractLogicalIOProcessor {
     } finally {
       if (originalThrowable != null && originalThrowable instanceof Error) {
         LOG.error(StringUtils.stringifyException(originalThrowable));
+        getContext().reportFailure(TaskFailureType.FATAL, originalThrowable,
+            "Cannot recover from this error");
         throw new RuntimeException(originalThrowable);
       }