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

Jeff Zhang commented on TEZ-2379:
---------------------------------

I think the following the code may cause the issue. There may be already some 
event causing TA killed in the event queue.    E.g. 
TA_CONTAINER_TERMINATED_BY_SYSTEM
May need to ignore T_ATTEMPT_KILLED at state of KILLED.

{code}
    if (attempt != null && !attempt.isFinished()) {
      eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(), 
logMsg, errorCause));
    }
{code}

> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event: 
> T_ATTEMPT_KILLED at KILLED
> ------------------------------------------------------------------------------------------------------
>
>                 Key: TEZ-2379
>                 URL: https://issues.apache.org/jira/browse/TEZ-2379
>             Project: Apache Tez
>          Issue Type: Bug
>            Reporter: Rajesh Balamohan
>         Attachments: application_1429683757595_0479.log
>
>
> {noformat}
> 2015-04-28 04:49:32,455 ERROR [Dispatcher thread: Central] impl.TaskImpl: 
> Can't handle this event at current state for 
> task_1429683757595_0479_1_03_000013
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event: 
> T_ATTEMPT_KILLED at KILLED
>         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.tez.state.StateMachineTez.doTransition(StateMachineTez.java:57)
>         at org.apache.tez.dag.app.dag.impl.TaskImpl.handle(TaskImpl.java:853)
>         at org.apache.tez.dag.app.dag.impl.TaskImpl.handle(TaskImpl.java:106)
>         at 
> org.apache.tez.dag.app.DAGAppMaster$TaskEventDispatcher.handle(DAGAppMaster.java:1874)
>         at 
> org.apache.tez.dag.app.DAGAppMaster$TaskEventDispatcher.handle(DAGAppMaster.java:1860)
>         at 
> org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:182)
>         at 
> org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:113)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Additional notes:
> ============
> Hive - latest build 
> Tez - master
> tpch-200 gb scale q_17 (kill the job in the middle of execution)



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

Reply via email to