From 10e8116d3886a63f4c042fd04c96920544190af7 Mon Sep 17 00:00:00 2001 From: Syed Albiz Date: Fri, 13 May 2011 18:09:05 -0700 Subject: [PATCH 1/1] Execution error improvements - unify local error messages -- display logs -- display display task id - grab debugging info asynchronously diff --git build-common.xml build-common.xml index 00c3680..6901efc 100644 --- build-common.xml +++ build-common.xml @@ -55,6 +55,7 @@ + diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index dc96a1f..553be16 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -136,6 +136,7 @@ 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), // should hive determine whether to run in local mode automatically ? LOCALMODEAUTO("hive.exec.mode.local.auto", false), @@ -262,8 +263,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 159d825..ed8a716 100644 --- conf/hive-default.xml +++ conf/hive-default.xml @@ -911,7 +911,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 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 4717c25..35a4f64 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.exec.Operator.ProgressCounter; @@ -43,12 +44,17 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.TaskCompletionEvent; import org.apache.hadoop.mapred.TaskReport; +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 { - + protected transient JobConf job; protected Task task; - + protected transient int mapProgress = 0; protected transient int reduceProgress = 0; public transient String jobId; @@ -69,10 +75,10 @@ public class HadoopJobExecHelper { return; } if(callBackObj != null) { - callBackObj.updateCounters(ctrs, rj); + callBackObj.updateCounters(ctrs, rj); } } - + /** * This msg pattern is used to track when a job is started. * @@ -113,7 +119,7 @@ public class HadoopJobExecHelper { return reduceProgress == 100; } - + public String getJobId() { return jobId; } @@ -122,10 +128,10 @@ public class HadoopJobExecHelper { this.jobId = jobId; } - + public HadoopJobExecHelper() { } - + public HadoopJobExecHelper(JobConf job, LogHelper console, Task task, HadoopJobExecHook hookCallBack) { this.job = job; @@ -134,7 +140,7 @@ public class HadoopJobExecHelper { this.callBackObj = hookCallBack; } - + /** * A list of the currently running jobs spawned in this Hive instance that is used to kill all * running jobs in the event of an unexpected shutdown - i.e., the JVM shuts down while there are @@ -143,7 +149,7 @@ public class HadoopJobExecHelper { public static Map runningJobKillURIs = Collections .synchronizedMap(new HashMap()); - + /** * In Hive, when the user control-c's the command line, any running jobs spawned from that command * line are best-effort killed. @@ -180,7 +186,7 @@ public class HadoopJobExecHelper { }); } } - + public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { if (ctrs == null) { // hadoop might return null if it cannot locate the job. @@ -196,7 +202,7 @@ public class HadoopJobExecHelper { } return this.callBackObj.checkFatalErrors(ctrs, errMsg); } - + private boolean progress(ExecDriverTaskHandle th) throws IOException { JobClient jc = th.getJobClient(); RunningJob rj = th.getRunningJob(); @@ -229,7 +235,7 @@ public class HadoopJobExecHelper { String logMapper; String logReducer; - + TaskReport[] mappers = jc.getMapTaskReports(rj.getJobID()); if (mappers == null) { logMapper = "no information for number of mappers; "; @@ -241,7 +247,7 @@ public class HadoopJobExecHelper { } logMapper = "number of mappers: " + numMap + "; "; } - + TaskReport[] reducers = jc.getReduceTaskReports(rj.getJobID()); if (reducers == null) { logReducer = "no information for number of reducers. "; @@ -338,7 +344,7 @@ public class HadoopJobExecHelper { // LOG.info(queryPlan); return (success); } - + private String getId() { return this.task.getId(); } @@ -392,7 +398,7 @@ public class HadoopJobExecHelper { return rj.getCounters(); } } - + // Used for showJobFailDebugInfo private static class TaskInfo { String jobId; @@ -415,7 +421,7 @@ public class HadoopJobExecHelper { return jobId; } } - + @SuppressWarnings("deprecation") private void showJobFailDebugInfo(JobConf conf, RunningJob rj) throws IOException { // Mapping from task ID to the number of failures @@ -427,6 +433,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) { @@ -452,6 +459,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) { @@ -459,7 +467,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 @@ -534,7 +542,6 @@ public class HadoopJobExecHelper { sb.append("-----\n"); console.printError(sb.toString()); - // Only print out one task because that's good enough for debugging. break; } @@ -543,11 +550,50 @@ public class HadoopJobExecHelper { } + public void localJobDebugger(int exitVal, String queryId, 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 queryId, 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, queryId, 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(); - + int returnVal = 0; - + // remove the pwd from conf file so that job tracker doesn't show this // logs String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); @@ -574,12 +620,20 @@ 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); } - + return returnVal; } } 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..431a5fd --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java @@ -0,0 +1,224 @@ +/** + * 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 job; + private RunningJob rj; + private LogHelper console; + + // Used for showJobFailDebugInfo + private static class TaskInfo { + String jobId; + HashSet logUrls; + + public TaskInfo(String jobId) { + this.jobId = jobId; + logUrls = new HashSet(); + } + + public void addLogUrl(String logUrl) { + logUrls.add(logUrl); + } + + public HashSet getLogUrls() { + return logUrls; + } + + public String getJobId() { + return jobId; + } + } + + public JobDebugger(JobConf job, RunningJob rj, LogHelper console) { + this.job = job; + this.rj = rj; + this.console = console; + } + + public void run() { + try { + showJobFailDebugInfo(job, rj); + } catch (IOException e) { + } + } + + private String getTaskAttemptLogUrl(String taskTrackerHttpAddress, String taskAttemptId) { + return taskTrackerHttpAddress + "/tasklog?taskid=" + taskAttemptId + "&all=true"; + } + + @SuppressWarnings("deprecation") + private void showJobFailDebugInfo(JobConf conf, RunningJob rj) throws IOException { + // Mapping from task ID to the number of failures + Map failures = new HashMap(); + // Successful task ID's + Set successes = new HashSet(); + + Map taskIdToInfo = new HashMap(); + + 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) { + 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; + } + // 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 53769a0..9f52373 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java @@ -258,7 +258,7 @@ public class MapRedTask extends ExecDriver implements Serializable { outPrinter.start(); errPrinter.start(); - int exitVal = executor.waitFor(); + int exitVal = jobExecHelper.progressLocal(executor, queryPlan.getQueryId(), 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 691f038..be83adc 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 final Log l4j = LogFactory.getLog("MapredLocalTask"); static final String HADOOP_MEM_KEY = "HADOOP_HEAPSIZE"; @@ -83,6 +84,8 @@ public class MapredLocalTask extends Task implements Serializab this.job = job; LOG = LogFactory.getLog(this.getClass().getName()); console = new LogHelper(LOG, isSilent); + //we don't use the HadoopJobExecHooks for local tasks + this.jobExecHelper = new HadoopJobExecHelper(job, console, this, null); } @Override @@ -212,7 +215,7 @@ public class MapredLocalTask extends Task implements Serializab outPrinter.start(); errPrinter.start(); - int exitVal = executor.waitFor(); + int exitVal = jobExecHelper.progressLocal(executor, queryPlan.getQueryId(), 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 9cb407c..cfd4ddc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -5497,8 +5497,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()); @@ -5506,16 +5505,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..cc749ad 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: + +/Users/salbiz/dev/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..9eb66f7 --- /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:49998/Users/salbiz/dev/hive/build/ql/scratchdir/hive_2011-05-23_21-25-50_640_1427907292542075938/-mr-10000 +Ended Job = job_20110523212538630_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..1b190f4 100644 --- ql/src/test/results/clientnegative/script_broken_pipe1.q.out +++ ql/src/test/results/clientnegative/script_broken_pipe1.q.out @@ -2,5 +2,6 @@ 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: hdfs://localhost:65079/Users/salbiz/dev/hive/build/ql/scratchdir/hive_2011-05-23_20-55-25_441_4326857197218351909/-mr-10000 +Ended Job = job_20110523205513261_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_pipe2.q.out ql/src/test/results/clientnegative/script_broken_pipe2.q.out index afbaa44..4f05eb1 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_15-32-40_009_1491119319689979479/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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..7d68932 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_14-24-55_065_1643917106463193684/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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..431abb8 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_15-32-51_805_3565859554438379158/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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..8e873a8 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_15-33-20_912_4794317650344882829/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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..d9e3921 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_15-33-26_526_2470834091016783125/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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..bc3dd24 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:/var/folders/5V/5V4Zq77qGD4aSK9m8V3frVsFdRU/-Tmp-/salbiz/hive_2011-05-23_15-33-32_577_9220331356425746018/-mr-10000 +Execution failed with exit status: 2 +Obtaining error information + +Task failed! +Task ID: + Stage-1 + +Logs: + +/Users/salbiz/dev/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/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.4