[
https://issues.apache.org/jira/browse/YARN-4108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15147605#comment-15147605
]
Sunil G commented on YARN-4108:
-------------------------------
Hi [~leftnoteasy]
Thanks for sharing full version of updated patch, really appreciate the
efforts. I have few dbts/comments in this patch. Pls help to check the same. :-)
1.
{{updateResToObtainAndKillableContainers}} tries to raise cancel preemption
back to scheduler if container resource is empty. So could we also remove from
{{resToObtainByPartition}} here?
{code}
+ if (!partitionsHasResToObtain.contains(partition)) {
+ // When we don't need to preempt any resources from the
queue/partition
+ // , cancel all killable containers from the queue
+ rmContext.getDispatcher().getEventHandler().handle(
+ new ContainerPreemptEvent(
+ killableContainer.getApplicationAttemptId(),
+ killableContainer,
+ SchedulerEventType.MARK_CONTAINER_FOR_NONKILLABLE))
{code}
2. If {{conf.getLazyPreemptionEnabled()}} is disabled, do we need to handle
events like MARK_CONTAINER_FOR_NONKILLABLE in scheduler?
3. {{cleanupCompletedKillableContainers()}} in
ProportionalCapacityPreemptionPolicy operates on full live containers in
cluster. And this can be a very big number, so we may hold the concurrent map's
access on each iteration of preemption policy. Will this be a pblm? Can we
optimize this further. For eg, could we set a boolean state within RMContainer
to indicate its not a live container?
4. {{killToPreemptContainers}} in LeafQueue invokes completedContainer on
application's leafQueue. I am thinking on some corner case where one
{{LeafQueue#killToPreemptContainers}} invocation results in a call to
completedContainer of another leafQueue. Do you see some chance for same, may
be a moveQueue operation?
> CapacityScheduler: Improve preemption to preempt only those containers that
> would satisfy the incoming request
> --------------------------------------------------------------------------------------------------------------
>
> Key: YARN-4108
> URL: https://issues.apache.org/jira/browse/YARN-4108
> Project: Hadoop YARN
> Issue Type: Bug
> Components: capacity scheduler
> Reporter: Wangda Tan
> Assignee: Wangda Tan
> Attachments: YARN-4108-design-doc-V3.pdf,
> YARN-4108-design-doc-v1.pdf, YARN-4108-design-doc-v2.pdf, YARN-4108.1.patch,
> YARN-4108.2.patch, YARN-4108.poc.1.patch, YARN-4108.poc.2-WIP.patch,
> YARN-4108.poc.3-WIP.patch, YARN-4108.poc.4-WIP.patch
>
>
> This is sibling JIRA for YARN-2154. We should make sure container preemption
> is more effective.
> *Requirements:*:
> 1) Can handle case of user-limit preemption
> 2) Can handle case of resource placement requirements, such as: hard-locality
> (I only want to use rack-1) / node-constraints (YARN-3409) / black-list (I
> don't want to use rack1 and host\[1-3\])
> 3) Can handle preemption within a queue: cross user preemption (YARN-2113),
> cross applicaiton preemption (such as priority-based (YARN-1963) /
> fairness-based (YARN-3319)).
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)