[ 
https://issues.apache.org/jira/browse/HIVE-2708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13185448#comment-13185448
 ] 

Amareshwari Sriramadasu commented on HIVE-2708:
-----------------------------------------------

Here is code flow in Hive:

RunningJob rj = jobClient.submitJob(jobConf);

while (!rj.complete()) {
  // sleep for a second
  // print the number of mappers and reducers of the job

  RunningJob newRj = jc.getJob(jobId);
  if (newRj == null) { //fail with exception }
  else { rj = newRj }
}

I tried removing the sleep and running the same job, now the log looks like:

{noformat}
2012-01-13 11:13:14,004 INFO  exec.ExecDriver 
(SessionState.java:printInfo(371)) - Job running in-process (local Hadoop)
2012-01-13 11:13:14,021 INFO  exec.HadoopJobExecHelper 
(HadoopJobExecHelper.java:progress(234)) - job id:job_local_0001
2012-01-13 11:13:14,054 INFO  exec.ExecDriver 
(SessionState.java:printInfo(371)) - Hadoop job information for null: number of 
mappers: 0; number of reducers: 0
2012-01-13 11:13:14,060 ERROR exec.ExecDriver 
(SessionState.java:printError(380)) - Ended Job = job_local_0001 with exception 
'java.io.IOException(Could not find status of job:job_local_0001)'
java.io.IOException: Could not find status of job:job_local_0001
  at 
org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper.progress(HadoopJobExecHelper.java:291)
  at 
org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper.progress(HadoopJobExecHelper.java:685)
  at org.apache.hadoop.hive.ql.exec.ExecDriver.execute(ExecDriver.java:458)
  at org.apache.hadoop.hive.ql.exec.ExecDriver.main(ExecDriver.java:710)
  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
  at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
  at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
  at java.lang.reflect.Method.invoke(Method.java:597)
  at org.apache.hadoop.util.RunJar.main(RunJar.java:189)

2012-01-13 11:13:14,068 INFO  mapred.LocalJobRunner 
(LocalJobRunner.java:run(344)) - Waiting for map tasks
2012-01-13 11:13:14,076 INFO  mapred.LocalJobRunner 
(LocalJobRunner.java:run(207)) - Starting task: attempt_local_0001_m_000000_0
2012-01-13 11:13:14,128 WARN  mapred.LocalJobRunner 
(LocalJobRunner.java:run(437)) - job_local_0001
java.lang.InterruptedException
  at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:1899)
  at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1976)
  at 
java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1245)
  at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:345)
{noformat}

Looks like the api JobClient.getJob(jobId) is broken in 0.23. Vinod, Can you 
confirm this?
                
> Hive MR local jobs fail on Hadoop 0.23
> --------------------------------------
>
>                 Key: HIVE-2708
>                 URL: https://issues.apache.org/jira/browse/HIVE-2708
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Amareshwari Sriramadasu
>            Assignee: Amareshwari Sriramadasu
>             Fix For: 0.8.1
>
>         Attachments: localjob-hive-mr23.txt
>
>
> Hive MR local jobs fail on 0.23 with following exception:
> Job running in-process (local Hadoop)
> Hadoop job information for null: number of mappers: 0; number of reducers: 0
> java.io.IOException: Could not find status of job:job_local_0001
>       at 
> org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper.progress(HadoopJobExecHelper.java:291)
>       at 
> org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper.progress(HadoopJobExecHelper.java:685)
>       at 
> org.apache.hadoop.hive.ql.exec.ExecDriver.execute(ExecDriver.java:458)
>       at org.apache.hadoop.hive.ql.exec.ExecDriver.main(ExecDriver.java:710)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>       at java.lang.reflect.Method.invoke(Method.java:597)
>       at org.apache.hadoop.util.RunJar.main(RunJar.java:189)
> Ended Job = job_local_0001 with exception 'java.io.IOException(Could not find 
> status of job:job_local_0001)'
> Execution

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to