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

Vinod Kumar Vavilapalli commented on YARN-5333:
-----------------------------------------------

bq. However the new active RM will reject recovered apps because it might have 
not loaded the new fair-scheduler.xml. We need call initScheduler before start 
active services or bring refreshAll() in front of rm.transitionToActive(). It 
seems it is aslo important for other scheduler.
Unfortunately, today, unless you are using the not-commonly-used 
FileSystemBasedConfigurationProvider stuff, it is expected that the admin 
update the xml files on both the RMs *and* issue a refreshQueues on both the 
RMs. I understand that in the fair-scheduler's case, there is an auto-refresh - 
this means FairScheduler needs special handling to auto-refresh on standby RM 
becoming active.

> Recovered apps are rejected when RM HA
> --------------------------------------
>
>                 Key: YARN-5333
>                 URL: https://issues.apache.org/jira/browse/YARN-5333
>             Project: Hadoop YARN
>          Issue Type: Bug
>            Reporter: Jun Gong
>            Assignee: Jun Gong
>
> Enable RM HA and use FairScheduler, 
> {{yarn.scheduler.fair.allow-undeclared-pools}} is set to false, 
> {{yarn.scheduler.fair.user-as-default-queue}} is set to false.
> Reproduce steps:
> 1. Start two RMs.
> 2. After RMs are running, change both RM's file 
> {{etc/hadoop/fair-scheduler.xml}}, then add some queues.
> 3. Submit some apps to the new added queues.
> 4. Stop the active RM, then the standby RM will transit to active and recover 
> apps.
> However the new active RM will reject recovered apps because it might have 
> not loaded the new {{fair-scheduler.xml}}. We need call {{initScheduler}} 
> before start active services or bring {{refreshAll()}} in front of 
> {{rm.transitionToActive()}}. *It seems it is aslo important for other 
> scheduler*.
> Related logs are as following:
> {quote}
> 2016-07-07 16:55:34,756 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Recover ended
> ...
> 2016-07-07 16:55:34,824 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationFileLoaderService:
>  Loading allocation file /gaia/hadoop/etc/hadoop/fair-scheduler.xml
> 2016-07-07 16:55:34,826 ERROR 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Application rejected by queue placement policy
> 2016-07-07 16:55:34,828 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Application appattempt_1467803586002_0006_000001 is done. finalState=FAILED
> 2016-07-07 16:55:34,828 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Unknown application appattempt_1467803586002_0006_000001 has completed!
> 2016-07-07 16:55:34,828 ERROR 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Application rejected by queue placement policy
> 2016-07-07 16:55:34,828 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Application appattempt_1467803586002_0004_000001 is done. finalState=FAILED
> 2016-07-07 16:55:34,828 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler: 
> Unknown application appattempt_1467803586002_0004_000001 has completed!
> 2016-07-07 16:55:34,828 ERROR 
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Can't handle 
> this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event: 
> APP_REJECTED at ACCEPTED
>       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.yarn.server.resourcemanager.rmapp.RMAppImpl.handle(RMAppImpl.java:697)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl.handle(RMAppImpl.java:88)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationEventDispatcher.handle(ResourceManager.java:718)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationEventDispatcher.handle(ResourceManager.java:702)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:191)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:124)
>       at java.lang.Thread.run(Thread.java:745)
> {quote}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to