Index: shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
===================================================================
--- shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (revision 1979)
+++ shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (working copy)
@@ -27,6 +27,9 @@
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
+
import java.io.IOException;
import java.lang.reflect.Constructor;
@@ -306,4 +309,9 @@
return "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat";
}
+ @Override
+ public Object getTaskAttemptId(TaskCompletionEvent t) {
+ return t.getTaskAttemptId();
+ }
+
}
Index: shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java
===================================================================
--- shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java (revision 1979)
+++ shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java (working copy)
@@ -26,6 +26,8 @@
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
+
import java.io.IOException;
/**
@@ -103,4 +105,9 @@
public String getInputFormatClassName() {
return "org.apache.hadoop.hive.ql.io.HiveInputFormat";
}
+
+ @Override
+ public Object getTaskAttemptId(TaskCompletionEvent t) {
+ return null;
+ }
}
Index: shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java
===================================================================
--- shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java (revision 1979)
+++ shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java (working copy)
@@ -26,6 +26,9 @@
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
+
import java.io.IOException;
/**
@@ -99,8 +102,13 @@
public String getInputFormatClassName() {
return "org.apache.hadoop.hive.ql.io.HiveInputFormat";
}
-
+
@Override
+ public Object getTaskAttemptId(TaskCompletionEvent t) {
+ return t.getTaskAttemptId();
+ }
+
+ @Override
public long getAccessTime(FileStatus file) {
return -1;
}
Index: shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java
===================================================================
--- shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java (revision 1979)
+++ shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java (working copy)
@@ -26,6 +26,8 @@
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
+
import java.io.IOException;
/**
@@ -104,4 +106,9 @@
public long getAccessTime(FileStatus file) {
return file.getAccessTime();
}
+
+ @Override
+ public Object getTaskAttemptId(TaskCompletionEvent t) {
+ return t.getTaskAttemptId();
+ }
}
Index: shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java
===================================================================
--- shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java (revision 1979)
+++ shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java (working copy)
@@ -29,6 +29,7 @@
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import java.io.IOException;
import java.io.DataInput;
@@ -108,6 +109,11 @@
public CombineFileInputFormatShim getCombineFileInputFormat();
public String getInputFormatClassName();
+
+ /**
+ * getTaskAtemptId is supported starting hadoop 0.18
+ */
+ public Object getTaskAttemptId(TaskCompletionEvent t);
public interface InputSplitShim extends InputSplit {
public JobConf getJob();
Index: conf/hive-default.xml
===================================================================
--- conf/hive-default.xml (revision 1979)
+++ conf/hive-default.xml (working copy)
@@ -299,6 +299,12 @@
+ hive.exec.fail.debug
+ true
+ Shows a links to the most failed tasks when a query fails.
+
+
+
hive.hwi.listen.host
0.0.0.0
This is the host address the Hive Web Interface will listen on
Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
===================================================================
--- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1979)
+++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy)
@@ -70,6 +70,7 @@
MAXREDUCERS("hive.exec.reducers.max", 999),
PREEXECHOOKS("hive.exec.pre.hooks", ""),
POSTEXECHOOKS("hive.exec.post.hooks", ""),
+ JOBFAILUREDEBUG("hive.exec.fail.debug", true),
// hadoop stuff
HADOOPBIN("hadoop.bin.path", System.getenv("HADOOP_HOME") + "/bin/hadoop"),
Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (revision 1979)
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (working copy)
@@ -36,6 +36,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -524,6 +525,8 @@
statusMesg += " with errors";
returnVal = 2;
console.printError(statusMesg);
+ if(HiveConf.getBoolVar(job, HiveConf.ConfVars.JOBFAILUREDEBUG))
+ showJobFailDebugInfo(job, rj);
} else {
console.printInfo(statusMesg);
}
@@ -581,6 +584,64 @@
return (returnVal);
}
+ private void showJobFailDebugInfo(JobConf conf, RunningJob rj) throws IOException {
+
+ Map failures = new HashMap();
+ Set successes = new HashSet ();
+
+ TaskCompletionEvent[] taskCompletions = rj.getTaskCompletionEvents(0);
+
+ for(TaskCompletionEvent t : taskCompletions) {
+ TaskAttemptID attemptId = (TaskAttemptID)ShimLoader.getHadoopShims().getTaskAttemptId(t);
+ if(attemptId == null) {
+ console.printError("Task attempt info is unavailable in this hadoop version");
+ break;
+ }
+
+ if(t.getTaskStatus() != TaskCompletionEvent.Status.SUCCEEDED) {
+ Integer failAttempts = failures.get(attemptId.getTaskID());
+ if(failAttempts == null) {
+ failAttempts = new Integer(0);
+ }
+ failAttempts = failAttempts.intValue() + 1;
+ failures.put(attemptId.getTaskID(), failAttempts);
+ } else {
+ successes.add(attemptId.getTaskID());
+ }
+ }
+
+ // Remove failures for tasks that succeeded
+ for(TaskID task : successes) {
+ failures.remove(task);
+ }
+
+ if(failures.keySet().size() == 0) {
+ return;
+ }
+
+ // Find the tasks with the most failures
+ int maxFailures = 0;
+ for(Integer failCount : failures.values()) {
+ if(maxFailures < failCount.intValue())
+ maxFailures = failCount.intValue();
+ }
+
+ // Display Error Message for tasks with the highest failure count
+ console.printError("\nFailed tasks with most" + "(" + maxFailures + ")" + " failures " + ": ");
+ String jtUrl = rj.getTrackingURL();
+ jtUrl = jtUrl.substring(0, jtUrl.lastIndexOf("/"));
+
+ for(TaskID task : failures.keySet()) {
+ if(failures.get(task).intValue() == maxFailures) {
+ String jobId = task.getJobID().toString();
+ String taskUrl = jtUrl + "/taskdetails.jsp?jobid=" + jobId + "&tipid=" + task.toString();
+ console.printError("Task URL: " + taskUrl +"\n");
+ }
+ }
+ return;
+
+ }
+
private static void printUsage() {
System.out
.println("ExecDriver -plan [-jobconf k1=v1 [-jobconf k2=v2] ...] "