AM fails to restart when first AM is killed
-------------------------------------------

                 Key: MAPREDUCE-3463
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-3463
             Project: Hadoop Map/Reduce
          Issue Type: Bug
          Components: applicationmaster, mrv2
    Affects Versions: 0.23.1
            Reporter: Karam Singh
            Priority: Blocker


Set yarn.resourcemanager.am.max-retries=5 in yarn-site.xml. Started yarn 4 Node 
cluster.
First Ran Randowriter/Sort/Sort-validate successfully
Then again sort, when job was 50% complete
Login node running AppMaster, and killed AppMaster with kill -9
On Client side failed with following:
{code}
11/11/23 10:57:27 INFO mapreduce.Job:  map 58% reduce 8%
11/11/23 10:57:27 INFO mapred.ClientServiceDelegate: Failed to contact 
AM/History for job job_1322040898409_0005 retrying..
11/11/23 10:57:28 INFO mapreduce.Job:  map 0% reduce 0%
11/11/23 10:57:37 INFO mapred.ClientServiceDelegate: Application state is 
completed. FinalApplicationStatus=UNDEFINED. Redirecting to job history server
11/11/23 10:57:37 INFO client.ClientTokenSelector: Looking for a token with 
service <RM Host>:Port
11/11/23 10:57:37 INFO client.ClientTokenSelector: Token kind is 
YARN_CLIENT_TOKEN and the token's service name is <New AM Host>:Port
11/11/23 10:57:38 WARN mapred.ClientServiceDelegate: Error from remote end: 
Unknown job job_1322040898409_0005
RemoteTrace: 
 at Local Trace: 
        org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl: 
Unknown job job_1322040898409_0005
        at 
org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine$Invoker.invoke(ProtoOverHadoopRpcEngine.java:151)
        at $Proxy10.getTaskAttemptCompletionEvents(Unknown Source)
        at 
org.apache.hadoop.mapreduce.v2.api.impl.pb.client.MRClientProtocolPBClientImpl.getTaskAttemptCompletionEvents(MRClientProtocolPBClientImpl.java:172)
        at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at 
org.apache.hadoop.mapred.ClientServiceDelegate.invoke(ClientServiceDelegate.java:273)
        at 
org.apache.hadoop.mapred.ClientServiceDelegate.getTaskCompletionEvents(ClientServiceDelegate.java:320)
        at 
org.apache.hadoop.mapred.YARNRunner.getTaskCompletionEvents(YARNRunner.java:438)
        at org.apache.hadoop.mapreduce.Job.getTaskCompletionEvents(Job.java:621)
        at org.apache.hadoop.mapreduce.Job.monitorAndPrintJob(Job.java:1231)
        at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1179)
        at org.apache.hadoop.examples.Sort.run(Sort.java:181)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:69)
        at org.apache.hadoop.examples.Sort.main(Sort.java:192)
        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.ProgramDriver$ProgramDescription.invoke(ProgramDriver.java:72)
        at org.apache.hadoop.util.ProgramDriver.driver(ProgramDriver.java:144)
        at org.apache.hadoop.examples.ExampleDriver.main(ExampleDriver.java:68)
        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)
{code}

On lookig RM logs found second AM was also lauched, it was saying -:
{code}
011-11-23 10:57:37,737 INFO 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: 
appattempt_1322040898409_0005_000002 State change from RUNNING to FINISHED
2011-11-23 10:57:37,737 INFO 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Processing event 
for application_1322040898409_0005 of type ATTEMPT_FINISHED
2011-11-23 10:57:37,737 INFO 
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: 
application_1322040898409_0005 State change from RUNNING to FINISHED
2011-11-23 10:57:37,737 INFO 
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
 Application appattempt_1322040898409_0005_000002 is done. finalState=FINISHED
{code}

Now looking at AM logs and found Second AM was shutdown gracefully due to :-
{code}
2011-11-23 10:57:37,640 INFO [AsyncDispatcher event handler] 
org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService: Sending assigned 
event to attempt_1322040898409_0005_m_000000_0
2011-11-23 10:57:37,641 FATAL [AsyncDispatcher event handler] 
org.apache.hadoop.yarn.event.AsyncDispatcher: Error in dispatcher thread. 
Exiting..
java.lang.IllegalArgumentException: Invalid NodeId [<NMHostName>]. Expected 
host:port
        at 
org.apache.hadoop.yarn.util.ConverterUtils.toNodeId(ConverterUtils.java:144)
        at 
org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService$InterceptingEventHandler.sendAssignedEvent(RecoveryService.java:410)
        at 
org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService$InterceptingEventHandler.handle(RecoveryService.java:314)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$RequestContainerTransition.transition(TaskAttemptImpl.java:1010)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$RequestContainerTransition.transition(TaskAttemptImpl.java:985)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:357)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:298)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:43)
        at 
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:443)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:851)
        at 
org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:128)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:853)
        at 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:845)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:116)
        at 
org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService$RecoveryDispatcher.dispatch(RecoveryService.java:270)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:75)
        at java.lang.Thread.run(Thread.java:619)
2011-11-23 10:57:37,642 INFO [CompositeServiceShutdownHook for 
org.apache.hadoop.mapreduce.v2.app.MRAppMaster] 
org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler: Stopping 
JobHistoryEventHandler
{code}

--
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