From 33ee912e0ce9b96115c69f79d43468ed3390c454 Mon Sep 17 00:00:00 2001 From: Syed Albiz Date: Tue, 2 Aug 2011 16:59:55 -0700 Subject: [PATCH 1/1] HIVE-2156-code Summary: Task ID: # Blame Rev: Reviewers: CC: Test Plan: Revert Plan: Tags: - begin *PUBLIC* platform impact section - Bugzilla: # - end platform impact - diff --git build-common.xml build-common.xml index 4856c5f..2696556 100644 --- build-common.xml +++ build-common.xml @@ -57,6 +57,7 @@ + diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index b46976f..0492b2b 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -136,6 +136,8 @@ public class HiveConf extends Configuration { DEFAULT_ZOOKEEPER_PARTITION_NAME("hive.lockmgr.zookeeper.default.partition.name", "__HIVE_DEFAULT_ZOOKEEPER_PARTITION__"), // Whether to show a link to the most failed task + debugging tips SHOW_JOB_FAIL_DEBUG_INFO("hive.exec.show.job.failure.debug.info", true), + JOB_DEBUG_TIMEOUT("hive.exec.job.debug.timeout", 30000), + TASKLOG_DEBUG_TIMEOUT("hive.exec.tasklog.debug.timeout", 20000), // should hive determine whether to run in local mode automatically ? LOCALMODEAUTO("hive.exec.mode.local.auto", false), @@ -263,8 +265,8 @@ public class HiveConf extends Configuration { HIVESCRIPTIDENVVAR("hive.script.operator.id.env.var", "HIVE_SCRIPT_OPERATOR_ID"), HIVEMAPREDMODE("hive.mapred.mode", "nonstrict"), HIVEALIAS("hive.alias", ""), - HIVEMAPSIDEAGGREGATE("hive.map.aggr", "true"), - HIVEGROUPBYSKEW("hive.groupby.skewindata", "false"), + HIVEMAPSIDEAGGREGATE("hive.map.aggr", true), + HIVEGROUPBYSKEW("hive.groupby.skewindata", false), HIVEJOINEMITINTERVAL("hive.join.emit.interval", 1000), HIVEJOINCACHESIZE("hive.join.cache.size", 25000), HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100), diff --git conf/hive-default.xml conf/hive-default.xml index 3a4f833..d0b2e5b 100644 --- conf/hive-default.xml +++ conf/hive-default.xml @@ -930,7 +930,7 @@ hive.exec.show.job.failure.debug.info - false + true If a job fails, whether to provide a link in the CLI to the task with the most failures, along with debugging hints if applicable. diff --git contrib/src/test/results/clientnegative/case_with_row_sequence.q.out contrib/src/test/results/clientnegative/case_with_row_sequence.q.out index 4447c65..b1931ca 100644 --- contrib/src/test/results/clientnegative/case_with_row_sequence.q.out +++ contrib/src/test/results/clientnegative/case_with_row_sequence.q.out @@ -14,5 +14,15 @@ SELECT CASE WHEN 3 > 2 THEN 10 WHEN row_sequence() > 5 THEN 20 ELSE 30 END FROM src LIMIT 1 PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/var/folders/7P/7PeC14kXFIWq0PIYyexGbmKuXUk/-Tmp-/jsichi/hive_2011-02-22_23-14-39_576_5692614556807208481/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-05_12-14-46_686_3062563290565430667/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/build.xml ql/build.xml index 449b47a..ec8cdbc 100644 --- ql/build.xml +++ ql/build.xml @@ -119,6 +119,19 @@ logDirectory="${test.log.dir}/clientpositive" hadoopVersion="${hadoopVersion}" /> + + diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java index 1c6f092..80727d9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java @@ -28,6 +28,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.Enumeration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.MapRedStats; @@ -45,6 +46,11 @@ import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.TaskCompletionEvent; import org.apache.hadoop.mapred.TaskReport; import org.apache.hadoop.mapred.Counters.Counter; +import org.apache.log4j.Appender; +import org.apache.log4j.BasicConfigurator; +import org.apache.log4j.FileAppender; +import org.apache.log4j.LogManager; +import org.apache.log4j.PropertyConfigurator; public class HadoopJobExecHelper { @@ -512,6 +518,7 @@ public class HadoopJobExecHelper { int startIndex = 0; + console.printError("Error during job, obtaining debugging information..."); // Loop to get all task completion events because getTaskCompletionEvents // only returns a subset per call while (true) { @@ -537,6 +544,7 @@ public class HadoopJobExecHelper { // and the logs String taskId = taskJobIds[0]; String jobId = taskJobIds[1]; + console.printError("Examining task ID: " + taskId + " from job " + jobId); TaskInfo ti = taskIdToInfo.get(taskId); if (ti == null) { @@ -544,7 +552,7 @@ public class HadoopJobExecHelper { taskIdToInfo.put(taskId, ti); } // These tasks should have come from the same job. - assert (ti.getJobId() == jobId); + assert (ti.getJobId() != null && ti.getJobId().equals(jobId)); ti.getLogUrls().add(getTaskAttemptLogUrl(t.getTaskTrackerHttp(), t.getTaskId())); // If a task failed, then keep track of the total number of failures @@ -619,7 +627,6 @@ public class HadoopJobExecHelper { sb.append("-----\n"); console.printError(sb.toString()); - // Only print out one task because that's good enough for debugging. break; } @@ -628,6 +635,45 @@ public class HadoopJobExecHelper { } + public void localJobDebugger(int exitVal, String taskId) { + StringBuilder sb = new StringBuilder(); + sb.append("\n"); + sb.append("Task failed!\n"); + sb.append("Task ID:\n " + taskId + "\n\n"); + sb.append("Logs:\n"); + console.printError(sb.toString()); + + for (Appender a : Collections.list((Enumeration) + LogManager.getRootLogger().getAllAppenders())) { + if (a instanceof FileAppender) { + console.printError(((FileAppender)a).getFile()); + } + } + } + + public int progressLocal(Process runningJob, String taskId) { + int exitVal = -101; + try { + exitVal = runningJob.waitFor(); //TODO: poll periodically + } catch (InterruptedException e) { + } + + if (exitVal != 0) { + console.printError("Execution failed with exit status: " + exitVal); + console.printError("Obtaining error information"); + if (HiveConf.getBoolVar(job, HiveConf.ConfVars.SHOW_JOB_FAIL_DEBUG_INFO)) { + // Since local jobs are run sequentially, all relevant information is already available + // Therefore, no need to fetch job debug info asynchronously + localJobDebugger(exitVal, taskId); + } + } else { + console.printInfo("Execution completed successfully"); + console.printInfo("Mapred Local Task Succeeded . Convert the Join into MapJoin"); + } + return exitVal; + } + + public int progress(RunningJob rj, JobClient jc) throws IOException { jobId = rj.getJobID(); @@ -666,7 +712,15 @@ public class HadoopJobExecHelper { returnVal = 2; console.printError(statusMesg); if (HiveConf.getBoolVar(job, HiveConf.ConfVars.SHOW_JOB_FAIL_DEBUG_INFO)) { - showJobFailDebugInfo(job, rj); + try { + JobDebugger jd = new JobDebugger(job, rj, console); + Thread t = new Thread(jd); + t.start(); + t.join(HiveConf.getIntVar(job, HiveConf.ConfVars.JOB_DEBUG_TIMEOUT)); + } catch (InterruptedException e) { + console.printError("Timed out trying to grab more detailed job failure" + + " information, please check jobtracker for more info"); + } } } else { console.printInfo(statusMesg); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java new file mode 100644 index 0000000..1d21924 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java @@ -0,0 +1,245 @@ +/** + * 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.io.Serializable; +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.Operator.ProgressCounter; +import org.apache.hadoop.hive.ql.exec.errors.ErrorAndSolution; +import org.apache.hadoop.hive.ql.exec.errors.TaskLogProcessor; +import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.mapred.Counters; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RunningJob; +import org.apache.hadoop.mapred.TaskCompletionEvent; +import org.apache.hadoop.mapred.TaskReport; + +/** + * JobDebugger takes a RunningJob that has failed and grabs the top 4 failing + * tasks and outputs this information to the Hive CLI. + */ +public class JobDebugger implements Runnable { + private JobConf conf; + private RunningJob rj; + private LogHelper console; + private Map failures = new HashMap(); // Mapping from task ID to the number of failures + private Set successes = new HashSet(); // Successful task ID's + private Map taskIdToInfo = new HashMap(); + + // Used for showJobFailDebugInfo + private static class TaskInfo { + String jobId; + Set logUrls; + + public TaskInfo(String jobId) { + this.jobId = jobId; + logUrls = new HashSet(); + } + + public void addLogUrl(String logUrl) { + logUrls.add(logUrl); + } + + public Set getLogUrls() { + return logUrls; + } + + public String getJobId() { + return jobId; + } + } + + public JobDebugger(JobConf conf, RunningJob rj, LogHelper console) { + this.conf = conf; + this.rj = rj; + this.console = console; + } + + public void run() { + try { + showJobFailDebugInfo(); + } catch (IOException e) { + console.printError(e.getMessage()); + } + } + private String getTaskAttemptLogUrl(String taskTrackerHttpAddress, String taskAttemptId) { + return taskTrackerHttpAddress + "/tasklog?taskid=" + taskAttemptId + "&start=-8193"; + } + + class TaskLogGrabber implements Runnable { + + public void run() { + try { + getTaskLogs(); + } catch (IOException e) { + console.printError(e.getMessage()); + } + } + + private void getTaskLogs() throws IOException { + 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 returns Strings for compatibility with Hadoop versions + // 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; + } + + // For each task completion event, get the associated task id, job id + // and the logs + String taskId = taskJobIds[0]; + String jobId = taskJobIds[1]; + console.printError("Examining task ID: " + taskId + " from job " + jobId); + + TaskInfo ti = taskIdToInfo.get(taskId); + if (ti == null) { + ti = new TaskInfo(jobId); + taskIdToInfo.put(taskId, ti); + } + // These tasks should have come from the same job. + assert (ti.getJobId() != null && ti.getJobId().equals(jobId)); + ti.getLogUrls().add(getTaskAttemptLogUrl(t.getTaskTrackerHttp(), t.getTaskId())); + + // If a task failed, then keep track of the total number of failures + // for that task (typically, a task gets re-run up to 4 times if it + // fails + + 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; + } + } + } + + @SuppressWarnings("deprecation") + private void showJobFailDebugInfo() throws IOException { + + + console.printError("Error during job, obtaining debugging information..."); + // Loop to get all task completion events because getTaskCompletionEvents + // only returns a subset per call + TaskLogGrabber tlg = new TaskLogGrabber(); + Thread t = new Thread(tlg); + try { + t.start(); + t.join(HiveConf.getIntVar(conf, HiveConf.ConfVars.TASKLOG_DEBUG_TIMEOUT)); + } catch (InterruptedException e) { + console.printError("Timed out trying to finish grabbing task log URLs, " + + "some task info may be missing"); + } + + // 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 + String jtUrl = JobTrackerURLResolver.getURL(conf); + + for (String task : failures.keySet()) { + if (failures.get(task).intValue() == maxFailures) { + TaskInfo ti = taskIdToInfo.get(task); + String jobId = ti.getJobId(); + String taskUrl = jtUrl + "/taskdetails.jsp?jobid=" + jobId + "&tipid=" + task.toString(); + + TaskLogProcessor tlp = new TaskLogProcessor(conf); + for (String logUrl : ti.getLogUrls()) { + tlp.addTaskAttemptLogUrl(logUrl); + } + + List errors = tlp.getErrors(); + + StringBuilder sb = new StringBuilder(); + // We use a StringBuilder and then call printError only once as + // printError will write to both stderr and the error log file. In + // situations where both the stderr and the log file output is + // simultaneously output to a single stream, this will look cleaner. + sb.append("\n"); + sb.append("Task with the most failures(" + maxFailures + "): \n"); + sb.append("-----\n"); + sb.append("Task ID:\n " + task + "\n\n"); + sb.append("URL:\n " + taskUrl + "\n"); + + for (ErrorAndSolution e : errors) { + sb.append("\n"); + sb.append("Possible error:\n " + e.getError() + "\n\n"); + sb.append("Solution:\n " + e.getSolution() + "\n"); + } + sb.append("-----\n"); + + console.printError(sb.toString()); + + // Only print out one task because that's good enough for debugging. + break; + } + } + return; + + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java index e687b1a..6cf4bc6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java @@ -267,7 +267,7 @@ public class MapRedTask extends ExecDriver implements Serializable { outPrinter.start(); errPrinter.start(); - int exitVal = executor.waitFor(); + int exitVal = jobExecHelper.progressLocal(executor, getId()); if (exitVal != 0) { LOG.error("Execution failed with exit status: " + exitVal); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java index 3d5e95d..511dd45 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java @@ -63,6 +63,7 @@ import org.apache.hadoop.util.ReflectionUtils; public class MapredLocalTask extends Task implements Serializable { private Map fetchOperators; + protected HadoopJobExecHelper jobExecHelper; private JobConf job; public static transient final Log l4j = LogFactory.getLog(MapredLocalTask.class); static final String HADOOP_MEM_KEY = "HADOOP_HEAPSIZE"; @@ -89,6 +90,8 @@ public class MapredLocalTask extends Task implements Serializab public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { super.initialize(conf, queryPlan, driverContext); job = new JobConf(conf, ExecDriver.class); + //we don't use the HadoopJobExecHooks for local tasks + this.jobExecHelper = new HadoopJobExecHelper(job, console, this, null); } public static String now() { @@ -213,7 +216,7 @@ public class MapredLocalTask extends Task implements Serializab outPrinter.start(); errPrinter.start(); - int exitVal = executor.waitFor(); + int exitVal = jobExecHelper.progressLocal(executor, getId()); if (exitVal != 0) { LOG.error("Execution failed with exit status: " + exitVal); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index cec0d46..5a14030 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -5812,8 +5812,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { if (qbp.getAggregationExprsForClause(dest).size() != 0 || getGroupByForClause(qbp, dest).size() > 0) { //multiple distincts is not supported with skew in data - if (conf.getVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) - .equalsIgnoreCase("true") && + if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) && qbp.getDistinctFuncExprsForClause(dest).size() > 1) { throw new SemanticException(ErrorMsg.UNSUPPORTED_MULTIPLE_DISTINCTS. getMsg()); @@ -5821,16 +5820,13 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { // insert a select operator here used by the ColumnPruner to reduce // the data to shuffle curr = insertSelectAllPlanForGroupBy(dest, curr); - if (conf.getVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE) - .equalsIgnoreCase("true")) { - if (conf.getVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) - .equalsIgnoreCase("false")) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE)) { + if (!conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { curr = genGroupByPlanMapAggr1MR(dest, qb, curr); } else { curr = genGroupByPlanMapAggr2MR(dest, qb, curr); } - } else if (conf.getVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) - .equalsIgnoreCase("true")) { + } else if (conf.getBoolVar(HiveConf.ConfVars.HIVEGROUPBYSKEW)) { curr = genGroupByPlan2MR(dest, qb, curr); } else { curr = genGroupByPlan1MR(dest, qb, curr); diff --git ql/src/test/queries/clientnegative/minimr_broken_pipe.q ql/src/test/queries/clientnegative/minimr_broken_pipe.q new file mode 100644 index 0000000..88a89d9 --- /dev/null +++ ql/src/test/queries/clientnegative/minimr_broken_pipe.q @@ -0,0 +1,4 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.exec.script.allow.partial.consumption = false; +-- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data +SELECT TRANSFORM(*) USING 'true' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp; diff --git ql/src/test/results/clientnegative/dyn_part3.q.out ql/src/test/results/clientnegative/dyn_part3.q.out index 5f4df65..577f623 100644 --- ql/src/test/results/clientnegative/dyn_part3.q.out +++ ql/src/test/results/clientnegative/dyn_part3.q.out @@ -7,4 +7,14 @@ PREHOOK: query: insert overwrite table nzhang_part partition(value) select key, PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_part +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/index_compact_entry_limit.q.out ql/src/test/results/clientnegative/index_compact_entry_limit.q.out index 06437c7..4788f85 100644 --- ql/src/test/results/clientnegative/index_compact_entry_limit.q.out +++ ql/src/test/results/clientnegative/index_compact_entry_limit.q.out @@ -31,5 +31,15 @@ POSTHOOK: Lineage: default__src_src_index__.key SIMPLE [(src)src.FieldSchema(nam PREHOOK: query: SELECT key, value FROM src WHERE key=100 ORDER BY key PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-01_16-15-31_393_2323708709856396072/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-05_12-04-13_955_1009283972520120333/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/index_compact_size_limit.q.out ql/src/test/results/clientnegative/index_compact_size_limit.q.out index 54f1262..623086b 100644 --- ql/src/test/results/clientnegative/index_compact_size_limit.q.out +++ ql/src/test/results/clientnegative/index_compact_size_limit.q.out @@ -31,5 +31,15 @@ POSTHOOK: Lineage: default__src_src_index__.key SIMPLE [(src)src.FieldSchema(nam PREHOOK: query: SELECT key, value FROM src WHERE key=100 ORDER BY key PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-01_16-15-40_648_4958727540603697272/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-05_12-04-23_821_264262594564060016/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/minimr_broken_pipe.q.out ql/src/test/results/clientnegative/minimr_broken_pipe.q.out new file mode 100644 index 0000000..39ec145 --- /dev/null +++ ql/src/test/results/clientnegative/minimr_broken_pipe.q.out @@ -0,0 +1,7 @@ +PREHOOK: query: -- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data +SELECT TRANSFORM(*) USING 'true' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: hdfs://localhost.localdomain:60804/data/users/salbiz/apache-hive/build/ql/scratchdir/hive_2011-08-08_15-08-01_102_678085189648474024/-mr-10000 +Ended Job = job_20110808150746814_0001 with errors +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/script_broken_pipe1.q.out ql/src/test/results/clientnegative/script_broken_pipe1.q.out index d33d2cc..bd79fb7 100644 --- ql/src/test/results/clientnegative/script_broken_pipe1.q.out +++ ql/src/test/results/clientnegative/script_broken_pipe1.q.out @@ -2,5 +2,15 @@ PREHOOK: query: -- Tests exception in ScriptOperator.close() by passing to the o SELECT TRANSFORM(*) USING 'true' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/data/users/pyang/trunk/VENDOR.hive/trunk/build/ql/tmp/1832401066/10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-22_660_8303431302792802567/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/script_broken_pipe2.q.out ql/src/test/results/clientnegative/script_broken_pipe2.q.out index afbaa44..12df810 100644 --- ql/src/test/results/clientnegative/script_broken_pipe2.q.out +++ ql/src/test/results/clientnegative/script_broken_pipe2.q.out @@ -2,5 +2,15 @@ PREHOOK: query: -- Tests exception in ScriptOperator.processOp() by passing extr SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value) USING 'true' as a,b,c,d FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/data/users/pyang/script/trunk/VENDOR.hive/trunk/build/ql/tmp/1650258494/10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-25_737_1025187653570997701/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/script_broken_pipe3.q.out ql/src/test/results/clientnegative/script_broken_pipe3.q.out index fe8f757..3932886 100644 --- ql/src/test/results/clientnegative/script_broken_pipe3.q.out +++ ql/src/test/results/clientnegative/script_broken_pipe3.q.out @@ -2,5 +2,15 @@ PREHOOK: query: -- Test to ensure that a script with a bad error code still fail SELECT TRANSFORM(*) USING 'false' AS a, b, c FROM (SELECT * FROM src LIMIT 1) tmp PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/data/users/pyang/trunk/VENDOR.hive/trunk/build/ql/tmp/1937270363/10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-28_813_6316382116761729323/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/script_error.q.out ql/src/test/results/clientnegative/script_error.q.out index c72d780..895acc4 100644 --- ql/src/test/results/clientnegative/script_error.q.out +++ ql/src/test/results/clientnegative/script_error.q.out @@ -48,5 +48,15 @@ PREHOOK: query: SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/erro FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/tmp/sdong/hive_2011-02-10_21-03-24_190_21878950587359648/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-31_979_3869579274390938020/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/udf_reflect_neg.q.out ql/src/test/results/clientnegative/udf_reflect_neg.q.out index f2082a3..db59af9 100644 --- ql/src/test/results/clientnegative/udf_reflect_neg.q.out +++ ql/src/test/results/clientnegative/udf_reflect_neg.q.out @@ -8,5 +8,15 @@ PREHOOK: query: SELECT reflect("java.lang.StringClassThatDoesNotExist", "valueOf FROM src LIMIT 1 PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/tmp/edward/hive_2010-08-28_18-06-12_525_3180708990156012812/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-49_514_931146053236968163/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/udf_test_error.q.out ql/src/test/results/clientnegative/udf_test_error.q.out index 5fd9a00..238268b 100644 --- ql/src/test/results/clientnegative/udf_test_error.q.out +++ ql/src/test/results/clientnegative/udf_test_error.q.out @@ -5,5 +5,15 @@ POSTHOOK: type: CREATEFUNCTION PREHOOK: query: SELECT test_error(key < 125 OR key > 130) FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/data/users/zshao/hadoop_hive_trunk/build/ql/scratchdir/hive_2010-03-06_00-58-40_004_2624763517220611615/10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-52_827_5964651401121748786/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientnegative/udf_test_error_reduce.q.out ql/src/test/results/clientnegative/udf_test_error_reduce.q.out index ddc5e5b..2db994f 100644 --- ql/src/test/results/clientnegative/udf_test_error_reduce.q.out +++ ql/src/test/results/clientnegative/udf_test_error_reduce.q.out @@ -10,5 +10,15 @@ FROM ( ) map_output PREHOOK: type: QUERY PREHOOK: Input: default@src -PREHOOK: Output: file:/data/users/zshao/hadoop_hive_trunk/build/ql/scratchdir/hive_2010-03-05_23-12-16_809_4809554819212794550/10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-02_17-12-55_772_7446403141126843360/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask diff --git ql/src/test/results/clientpositive/auto_join25.q.out ql/src/test/results/clientpositive/auto_join25.q.out index 362b2fa..0dfbd80 100644 --- ql/src/test/results/clientpositive/auto_join25.q.out +++ ql/src/test/results/clientpositive/auto_join25.q.out @@ -13,6 +13,16 @@ 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 PREHOOK: Output: default@dest1 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-7 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask POSTHOOK: query: FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) @@ -30,11 +40,11 @@ POSTHOOK: Lineage: dest1.value SIMPLE [(src)src2.FieldSchema(name:value, type:st PREHOOK: query: SELECT sum(hash(dest1.key,dest1.value)) FROM dest1 PREHOOK: type: QUERY PREHOOK: Input: default@dest1 -PREHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-12_062_2731833788874193660/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-26-47_118_1954071308745703162/-mr-10000 POSTHOOK: query: SELECT sum(hash(dest1.key,dest1.value)) FROM dest1 POSTHOOK: type: QUERY POSTHOOK: Input: default@dest1 -POSTHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-12_062_2731833788874193660/-mr-10000 +POSTHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-26-47_118_1954071308745703162/-mr-10000 POSTHOOK: Lineage: dest1.key EXPRESSION [(srcpart)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ] 407444119660 @@ -50,8 +60,28 @@ INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j2 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-14 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-12 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) @@ -66,11 +96,11 @@ POSTHOOK: Lineage: dest_j2.value SIMPLE [(src)src3.FieldSchema(name:value, type: PREHOOK: query: SELECT sum(hash(dest_j2.key,dest_j2.value)) FROM dest_j2 PREHOOK: type: QUERY PREHOOK: Input: default@dest_j2 -PREHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-36_524_5308749215651001089/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-27-02_282_7965605157514278016/-mr-10000 POSTHOOK: query: SELECT sum(hash(dest_j2.key,dest_j2.value)) FROM dest_j2 POSTHOOK: type: QUERY POSTHOOK: Input: default@dest_j2 -POSTHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-36_524_5308749215651001089/-mr-10000 +POSTHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-27-02_282_7965605157514278016/-mr-10000 POSTHOOK: Lineage: dest1.key EXPRESSION [(srcpart)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: dest_j2.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] @@ -90,6 +120,16 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-7 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) @@ -106,11 +146,11 @@ POSTHOOK: Lineage: dest_j2.value SIMPLE [(src)src3.FieldSchema(name:value, type: PREHOOK: query: SELECT sum(hash(dest_j1.key,dest_j1.value)) FROM dest_j1 PREHOOK: type: QUERY PREHOOK: Input: default@dest_j1 -PREHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-49_448_5881123257419888652/-mr-10000 +PREHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-27-11_787_504951263993420939/-mr-10000 POSTHOOK: query: SELECT sum(hash(dest_j1.key,dest_j1.value)) FROM dest_j1 POSTHOOK: type: QUERY POSTHOOK: Input: default@dest_j1 -POSTHOOK: Output: file:/tmp/liyintang/hive_2010-11-23_13-09-49_448_5881123257419888652/-mr-10000 +POSTHOOK: Output: file:/tmp/salbiz/hive_2011-08-03_11-27-11_787_504951263993420939/-mr-10000 POSTHOOK: Lineage: dest1.key EXPRESSION [(srcpart)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ] POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] diff --git ql/src/test/results/clientpositive/mapjoin_hook.q.out ql/src/test/results/clientpositive/mapjoin_hook.q.out index acdeefa..e8c30e5 100644 --- ql/src/test/results/clientpositive/mapjoin_hook.q.out +++ ql/src/test/results/clientpositive/mapjoin_hook.q.out @@ -25,6 +25,16 @@ 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 PREHOOK: Output: default@dest1 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-7 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask [MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 1 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 1 @@ -33,8 +43,28 @@ INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-14 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-12 + +Logs: + +/data/users/salbiz/apache-hive/build/ql/tmp//hive.log FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapredLocalTask ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask [MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 2 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 2 diff --git ql/src/test/templates/TestNegativeCliDriver.vm ql/src/test/templates/TestNegativeCliDriver.vm index ec13f79..48283c4 100644 --- ql/src/test/templates/TestNegativeCliDriver.vm +++ ql/src/test/templates/TestNegativeCliDriver.vm @@ -19,7 +19,12 @@ public class $className extends TestCase { static { try { - qt = new QTestUtil("$resultsDir.getCanonicalPath()", "$logDir.getCanonicalPath()", false, "0.20"); + boolean miniMR = false; + String hadoopVer; + if ("$clusterMode".equals("miniMR")) + miniMR = true; + hadoopVer = "$hadoopVersion"; + qt = new QTestUtil("$resultsDir.getCanonicalPath()", "$logDir.getCanonicalPath()", miniMR, hadoopVer); // do a one time initialization qt.cleanUp(); qt.createSources(); -- 1.7.4