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

Jason Lowe commented on YARN-2171:
----------------------------------

When the CapacityScheduler scheduler thread is running full-time due to a 
constant stream of events (e.g.: large number of running applications with a 
large number of cluster nodes) then the CapacityScheduler lock is held by that 
scheduler loop most of the time.  As AMs heartbeat into the RM to try to get 
their resources, the capacity scheduler code goes out of its way to try to 
avoid having the AMs grab the scheduler lock.  Unfortunately this one was 
missed to get this one integer value.  Therefore they end up piling up on the 
scheduler lock, filling all of the IPC handlers of the ApplicationMasterService 
and the others back up on the call queue.  Once the scheduler releases the lock 
it will quickly try to grab it again, so only a few AMs end up getting through 
the "gate" and the IPC handlers fill again with the next batch of AMs blocking 
on the scheduler lock.  This causes the average RPC response times to skyrocket 
for AMs.  AMs experience large delays getting their allocations which in turn 
leads to lower cluster utilization and increased application runtimes.

> AMs block on the CapacityScheduler lock during allocate()
> ---------------------------------------------------------
>
>                 Key: YARN-2171
>                 URL: https://issues.apache.org/jira/browse/YARN-2171
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: capacityscheduler
>    Affects Versions: 0.23.10, 2.4.0
>            Reporter: Jason Lowe
>            Assignee: Jason Lowe
>            Priority: Critical
>
> When AMs heartbeat into the RM via the allocate() call they are blocking on 
> the CapacityScheduler lock when trying to get the number of nodes in the 
> cluster via getNumClusterNodes.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to