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

Haibo Chen edited comment on MAPREDUCE-6815 at 12/2/16 12:39 AM:
-----------------------------------------------------------------

bq. 
2016-11-23 10:08:07,725 ERROR [AsyncDispatcher event handler] impl.JobImpl 
(JobImpl.java:handle(1004)) - Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: 
JOB_TASK_COMPLETED at SETUP
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.handle(JobImpl.java:1002)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.handle(JobImpl.java:140)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$JobEventDispatcher.handle(MRAppMaster.java:1465)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$JobEventDispatcher.handle(MRAppMaster.java:1461)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:187)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:113)
        at java.lang.Thread.run(Thread.java:745)

In TestKill.testKillTask(), we do  app.waitForState(job, JobState.RUNNING) to 
wait for the job to be running. But when the job is in running state 
externally, it can be in either SetUp or Running internally. If the Job is 
still in SetUp state, sending task_kill event will eventually cause the job to 
fail as show above. We could wait on the internal state instead of external 
state.



was (Author: haibochen):
bq. 
2016-11-23 10:08:07,725 ERROR [AsyncDispatcher event handler] impl.JobImpl 
(JobImpl.java:handle(1004)) - Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: 
JOB_TASK_COMPLETED at SETUP
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.handle(JobImpl.java:1002)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl.handle(JobImpl.java:140)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$JobEventDispatcher.handle(MRAppMaster.java:1465)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$JobEventDispatcher.handle(MRAppMaster.java:1461)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:187)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:113)
        at java.lang.Thread.run(Thread.java:745)

In TestKill.testKillTask(), we do  app.waitForState(job, JobState.RUNNING) to 
wait for the job to be running. But when the job is in running state 
externally, it can be in either SetUp or Running internally. If the Job is 
still in SetUp state, sending task_kill event will eventually cause the job to 
fail as show above


> Fix flaky TestKill.testKillTask()
> ---------------------------------
>
>                 Key: MAPREDUCE-6815
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-6815
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: mrv2
>    Affects Versions: 3.0.0-alpha1
>            Reporter: Haibo Chen
>            Assignee: Haibo Chen
>
> Error Message
> Job state is not correct (timedout) expected:<SUCCEEDED> but was:<ERROR>
> Stacktrace
> java.lang.AssertionError: Job state is not correct (timedout) 
> expected:<SUCCEEDED> but was:<ERROR>
> at org.junit.Assert.fail(Assert.java:88)
> at org.junit.Assert.failNotEquals(Assert.java:743)
> at org.junit.Assert.assertEquals(Assert.java:118)
> at org.apache.hadoop.mapreduce.v2.app.MRApp.waitForState(MRApp.java:416)
> at org.apache.hadoop.mapreduce.v2.app.TestKill.testKillTask(TestKill.java:124)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org

Reply via email to