You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2009/10/23 06:18:30 UTC
svn commit: r828929 - in /hadoop/hive/trunk: ./
ql/src/java/org/apache/hadoop/hive/ql/exec/
shims/src/0.17/java/org/apache/hadoop/hive/shims/
shims/src/0.18/java/org/apache/hadoop/hive/shims/
shims/src/0.19/java/org/apache/hadoop/hive/shims/ shims/src/...
Author: zshao
Date: Fri Oct 23 04:18:30 2009
New Revision: 828929
URL: http://svn.apache.org/viewvc?rev=828929&view=rev
Log:
HIVE-873. Better error messages for Hive cmdline. (Paul Yang via zshao)
Added:
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java
Modified:
hadoop/hive/trunk/CHANGES.txt
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java
hadoop/hive/trunk/shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java
hadoop/hive/trunk/shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java
hadoop/hive/trunk/shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java
hadoop/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
hadoop/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java
Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Fri Oct 23 04:18:30 2009
@@ -71,6 +71,8 @@
HIVE-874. Add 'repair' option to mack. (Cyrus Katrak via prasadc).
+ HIVE-873. Better error messages for Hive cmdline. (Paul Yang via zshao)
+
IMPROVEMENTS
HIVE-760. Add version info to META-INF/MANIFEST.MF.
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java Fri Oct 23 04:18:30 2009
@@ -32,10 +32,15 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.RunningJob;
+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;
@@ -526,6 +531,7 @@
statusMesg += " with errors";
returnVal = 2;
console.printError(statusMesg);
+ showJobFailDebugInfo(job, rj);
} else {
console.printInfo(statusMesg);
}
@@ -583,6 +589,84 @@
return (returnVal);
}
+ private void showJobFailDebugInfo(JobConf conf, RunningJob rj) throws IOException {
+
+ Map<String, Integer> failures = new HashMap<String, Integer>();
+ Set<String> successes = new HashSet<String> ();
+ Map<String, String> taskToJob = new HashMap<String,String>();
+
+ int startIndex = 0;
+
+ while(true) {
+ TaskCompletionEvent[] taskCompletions = rj.getTaskCompletionEvents(startIndex);
+
+ if(taskCompletions == null || taskCompletions.length == 0) {
+ break;
+ }
+
+ boolean more = true;
+ for(TaskCompletionEvent t : taskCompletions) {
+ // getTaskJobIDs return Strings for compatibility with Hadoop version without
+ // TaskID or TaskAttemptID
+ String [] taskJobIds = ShimLoader.getHadoopShims().getTaskJobIDs(t);
+
+ if(taskJobIds == null) {
+ console.printError("Task attempt info is unavailable in this Hadoop version");
+ more = false;
+ break;
+ }
+
+ String taskId = taskJobIds[0];
+ String jobId = taskJobIds[1];
+ taskToJob.put(taskId, jobId);
+
+ if(t.getTaskStatus() != TaskCompletionEvent.Status.SUCCEEDED) {
+ Integer failAttempts = failures.get(taskId);
+ if(failAttempts == null) {
+ failAttempts = Integer.valueOf(0);
+ }
+ failAttempts = Integer.valueOf(failAttempts.intValue() + 1);
+ failures.put(taskId, failAttempts);
+ } else {
+ successes.add(taskId);
+ }
+ }
+ if(!more) {
+ break;
+ }
+ startIndex += taskCompletions.length;
+ }
+ // Remove failures for tasks that succeeded
+ for(String task : successes) {
+ failures.remove(task);
+ }
+
+ if(failures.keySet().size() == 0) {
+ return;
+ }
+
+ // Find the highest failure count
+ 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 = JobTrackerURLResolver.getURL(conf);
+
+ for(String task : failures.keySet()) {
+ if(failures.get(task).intValue() == maxFailures) {
+ String jobId = taskToJob.get(task);
+ 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 <plan-file> [-jobconf k1=v1 [-jobconf k2=v2] ...] "
Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java?rev=828929&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java Fri Oct 23 04:18:30 2009
@@ -0,0 +1,43 @@
+/**
+ * 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.exec;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.net.NetUtils;
+
+public class JobTrackerURLResolver {
+ public static String getURL(JobConf conf) throws IOException {
+ String infoAddr = conf.get("mapred.job.tracker.http.address");
+ if (infoAddr == null) {
+ throw new IOException("Unable to find job tracker info port.");
+ }
+ InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
+ int infoPort = infoSocAddr.getPort();
+
+ String tracker = "http://" +
+ JobTracker.getAddress(conf).getHostName() + ":" +
+ infoPort;
+
+ return tracker;
+ }
+}
Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java Fri Oct 23 04:18:30 2009
@@ -65,18 +65,8 @@
return;
}
- // find the http port for the jobtracker
- String infoAddr = conf.get("mapred.job.tracker.http.address");
- if (infoAddr == null) {
- throw new IOException("Throttle: Unable to find job tracker info port.");
- }
- InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
- int infoPort = infoSocAddr.getPort();
-
// This is the Job Tracker URL
- String tracker = "http://" +
- JobTracker.getAddress(conf).getHostName() + ":" +
- infoPort +
+ String tracker = JobTrackerURLResolver.getURL(conf) +
"/gc.jsp?threshold=" + threshold;
while (true) {
Modified: hadoop/hive/trunk/shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java (original)
+++ hadoop/hive/trunk/shims/src/0.17/java/org/apache/hadoop/hive/shims/Hadoop17Shims.java Fri Oct 23 04:18:30 2009
@@ -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 String [] getTaskJobIDs(TaskCompletionEvent t) {
+ return null;
+ }
}
Modified: hadoop/hive/trunk/shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java (original)
+++ hadoop/hive/trunk/shims/src/0.18/java/org/apache/hadoop/hive/shims/Hadoop18Shims.java Fri Oct 23 04:18:30 2009
@@ -26,6 +26,10 @@
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
+
import java.io.IOException;
/**
@@ -99,7 +103,16 @@
public String getInputFormatClassName() {
return "org.apache.hadoop.hive.ql.io.HiveInputFormat";
}
-
+
+ String [] ret = new String[2];
+ @Override
+ public String [] getTaskJobIDs(TaskCompletionEvent t) {
+ TaskID tid = t.getTaskAttemptId().getTaskID();
+ ret[0] = tid.toString();
+ ret[1] = tid.getJobID().toString();
+ return ret;
+ }
+
@Override
public long getAccessTime(FileStatus file) {
return -1;
Modified: hadoop/hive/trunk/shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java (original)
+++ hadoop/hive/trunk/shims/src/0.19/java/org/apache/hadoop/hive/shims/Hadoop19Shims.java Fri Oct 23 04:18:30 2009
@@ -26,6 +26,10 @@
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 org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+
import java.io.IOException;
/**
@@ -104,4 +108,14 @@
public long getAccessTime(FileStatus file) {
return file.getAccessTime();
}
+
+
+ String [] ret = new String[2];
+ @Override
+ public String [] getTaskJobIDs(TaskCompletionEvent t) {
+ TaskID tid = t.getTaskAttemptId().getTaskID();
+ ret[0] = tid.toString();
+ ret[1] = tid.getJobID().toString();
+ return ret;
+ }
}
Modified: hadoop/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (original)
+++ hadoop/hive/trunk/shims/src/0.20/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java Fri Oct 23 04:18:30 2009
@@ -27,6 +27,10 @@
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 org.apache.hadoop.mapred.TaskID;
+
import java.io.IOException;
import java.lang.reflect.Constructor;
@@ -306,4 +310,13 @@
return "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat";
}
+ String [] ret = new String[2];
+ @Override
+ public String [] getTaskJobIDs(TaskCompletionEvent t) {
+ TaskID tid = t.getTaskAttemptId().getTaskID();
+ ret[0] = tid.toString();
+ ret[1] = tid.getJobID().toString();
+ return ret;
+ }
+
}
Modified: hadoop/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java?rev=828929&r1=828928&r2=828929&view=diff
==============================================================================
--- hadoop/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java (original)
+++ hadoop/hive/trunk/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java Fri Oct 23 04:18:30 2009
@@ -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,14 @@
public CombineFileInputFormatShim getCombineFileInputFormat();
public String getInputFormatClassName();
+
+ /**
+ * getTaskJobIDs returns an array of String with two elements. The first
+ * element is a string representing the task id and the second is a string
+ * representing the job id. This is necessary as TaskID and TaskAttemptID
+ * are not supported in Haddop 0.17
+ */
+ public String [] getTaskJobIDs(TaskCompletionEvent t);
public interface InputSplitShim extends InputSplit {
public JobConf getJob();