[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-04-11 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

- Comments of attached Graphs --
The attached graph highlights the need for preemption by means of an example 
designed to highlights this. We run 2 sort jobs over 128GB of data on a 10 
nodes cluster, starting the first job in queue B (20% guaranteed capacity) and 
the second job 400sec later in queue A (80% guaranteed capacity).

We compare three scenarios:
# Default CapacityScheduler with A and B having maximum capacity set to 100%: 
the cluster utilization is high, B runs fast since it can use the entire 
cluster when A is not around, but A needs to wait for very long (almost 20 min) 
before obtaining access to its all of its guaranteed capacity (and over 250 
secs to get any container beside the AM).
# Default CapacityScheduler with A and B have maximum capacity set to 80 and 
20% respectively, A obtains its guaranteed resources immediately, but the 
cluster utilization is very low and jobs in B take over 2X longer since they 
cannot use spare overcapacity.
# CapacityScheduler + preemption: A and B are configured as in 1) but we 
preempt containers. We obtain both high-utilization, short runtimes for B 
(comparable to scenario 1), and prompt resources to A (within 30 sec). 

The second attached graph shows a scenario with 3 queues A, B, C with 40%, 20%, 
40% capacity guaranteed. We show more "internals" of the policy by plotting, 
instantaneous resource utilization as above, total pending request, guaranteed 
capacity, ideal assignment of memory, ideal preemption, actual preemption.
 
Things to note:
# The idealized memory assignment and instaneous resource utilization are very 
close to each other, i.e., the combination of CapacityScheduler+Preemption 
tightly follows the the ideal distribution of resources
# When only one job is running it gets 100% of the cluster, when B, A are 
running they get 33% and 66% each (which is a fair overcapacity assignment from 
their 20%, 40% guaranteed capacity), when all three jobs are running (and they 
want at least their capacity worth of resources) they obtain their guaranteed 
capacity.
#actual preemption is a fraction of ideal preemption, this is because we 
account for natural completion of tasks (with a configurable parameter)
#in this experiment we do not bound the total amount of preemption per round 
(i.e., parameter set to 1.0)
 




> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Improvement
>  Components: capacityscheduler
>Reporter: Carlo Curino
> Attachments: 3queues.pdf, capacity.patch, 
> CapScheduler_with_preemption.pdf
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, th

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-04-22 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

Regenerated patches based on conversations on YARN-45. We introduced the 
"priority-first, containerid-second" order of containers selection as per 
feedback we received, however I have some doubts. 

Looking at the 
[documentation|https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html#Writing_an_ApplicationMaster]
 and at the MR use of priority in RMContainerAllocater.java (around line 98). I 
find the choice of priorities in MR seems not very amenable to preemption. In 
particular, MR assign the following priorities (where bigger value means less 
priority): 
# PRIORITY_MAP = 20 
# PRIORITY_REDUCE = 10
# PRIORITY_FAILED_MAP = 5  

I believe this is needed for slow_start parameter to make the desired effect, 
and to make sure the FAILED_MAPs to start before REDUCERs. 
However, if we use the (reverse) priority to instruct the choice of containers 
to preempt, we will preempt REDUCERs after MAPs which is I think wrong. 
I think this might unveil some issues with the semantics of Priority, as they 
do not capture a long-running "value of this container for the AM", but 
rather "value at time of allocation". To be sorted-out before commit. 


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one t

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-04 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

Thank Bikas, we will look into it, and see whether we can integrate your ideas 
straight into the patch, 
or at least set things up to prepare the ground for a future version of this 
that leverages your work.


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how frequently to run the policy
> # how long to wait between preemption and kill of a container
> # which fraction o

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-06 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

Hi Bikas, I noticed that your patch and ours share a common architectural 
style, i.e., the preemption policy runs in a separate thread on a timer. 
Moreover, they also seem to mostly agree on the I/O to/from the policy, we both 
grab state from the CapacityScheduler (e.g., the root of the queues) as in 
input, and both trigger actions that affect the CapacityScheduler. In our 
design we tried to put the actions behind an event handler, but I think the 
ideas are very similar. 
In fact, I would guess that a good portion of your patch could be placed behind 
the ScheduleEditPolicy interface we defined. As I mentioned in some of our 
conversations, this is nice because the ScheduleEditPolicy API I think can be 
used also for other purposes (e.g., for a deadline-monitor, or an IO-starvation 
monitor, etc..). Basically to implement monitors that focus on specific (even 
orthogonal) properties of the schedule, and that can observe the cluster state 
through the CapacityScheduler viewpoint, and try to affect it somehow (via 
events in our design). As an example, imagine a deadline monitor trying to 
affect jobs' completion times, by tweaking capacity of the queues, or ordering 
of job in the queue etc. 
While I am not sure this API will see a broad public :-) it would be nice to 
agree on it. 

As for the specifics of what you do with all the enforcements stuff, I haven't 
read the code carefully enough to follow the details. Actually, if you have 
time to write a high-level summary of it and post it here, it would be useful 
to orient us through your patch. 

While I think it would be too convoluted to try to merge the two approaches, I 
would like to see whether, other than the SchedulerEditPolicy, there is more we 
can factor out to make your version of the policy easy to write. I know it 
wouldn't be hard to  evolve this later on, as the code is rather isolated, but 
if we can do something that make it easier now I think is worth considering.


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (acco

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-17 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

We post an improved version of the patch, that reflects: 
- the committed versions of YARN-45, and YARN-567
- uses the resource-based version of YARN-45, and 
- handles hierarchies of queues 

The key change to handle hierarchies is to:
- roll up pending requests from the leaf to parents
- compute the "ideal" capacity assignment (same algo as before) for level of 
the three from the top down
- determine preemption as (current - ideal) in the leafs and select containers 

This covers nicely the use case brought up by Bikas, Arun, Hitish, Sid, and 
Vinod where a (even heavily) over-capacity 
leaf queue should not be preempted if its parent is within capacity. We 
included this specific test as part of 
our unit tests. 

Note: my previous [comment | 
https://issues.apache.org/jira/browse/YARN-569?focusedCommentId=13638825&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13638825]
 about having doubts on the priority-first still stands. Priorities capture the 
"order" in which the application wants containers, but they are not updated 
after containers are granted to capture the relative relevance of containers at 
runtime. This is way using a resource-based PreemptionMessage is important, 
since it allows the underlying app to pick a different set of containers. This 
is what we do in the implementation of this for mapreduce (MAPREDUCE-5196 and 
friends), where we preempt reducers instead of maps whenever possible.



> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological 

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-17 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12583710/YARN-569.2.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:red}-1 findbugs{color}.  The patch appears to introduce 2 new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/953//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-YARN-Build/953//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-yarn-server-resourcemanager.html
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/953//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more re

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-18 Thread Carlo Curino (JIRA)

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

Carlo Curino commented on YARN-569:
---

The findbugs warnings are on accesses of a ResourceCalculator and 
minAllocation, so not really concerning.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how frequently to run the policy
> # how long to wait between preemption and kill of a container
> # which fraction of the containers I would like to obtain should I preempt 
> (has

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-05-31 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12585682/YARN-569.3.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:red}-1 findbugs{color}.  The patch appears to introduce 2 new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1058//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-YARN-Build/1058//artifact/trunk/patchprocess/newPatchFindbugsWarningshadoop-yarn-server-resourcemanager.html
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1058//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, o

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-03 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12586057/YARN-569.4.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1106//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1106//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological or

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-10 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

Sorry for the delayed response.

This doesnt seem to affect the fair scheduler or does it? If not, then it can 
be misleading for users.
{code}
+  public static final String RM_SCHEDULER_ENABLE_PREEMPTION =
{code}

Missing default?
{code}
+  /** List of ScheduleEditPolicy classes affecting scheduler preemption. */
+  public static final String RM_SCHEDULER_PREEMPTION_POLICIES =
+RM_PREFIX + "scheduler.preemption.policies";
{code}

Why cast when one has generic T?
{code}
+public RMContainerPreemptEventDispatcher(ResourceScheduler scheduler) {
+  this.scheduler = (T) scheduler;
+}
{code}

How do we envisage multiple policies working together without stepping on each 
other? Better off limiting to 1?
{code}
+for (ScheduleEditPolicy policy : policies) {
+  LOG.info("LOADING ScheduleEditPolicy:" + policy.toString());
+  policy.init(conf, this.rmContext.getDispatcher().getEventHandler(),
+  (PreemptableResourceScheduler) scheduler);
+  // preemption service, periodically check whether we need to
+  // preempt to guarantee capacity constraints
+  ScheduleMonitor mon = new ScheduleMonitor(policy);
+  addService(mon);
+
+}
{code}

Might be a personal choice but ScheduleMonitor or ScheduleEditPolicy would 
sound better if they used Scheduling instead of Schedule.

Why would we want to get this from the policy (which seems natural) as well as 
be able to set it. If it needs to be configurable then it can be done via the 
policy config right?
{code}
+  protected void setMonitorInterval(int monitorInterval) {
+this.monitorInterval = monitorInterval;
+  }
{code}

Having multiple threads named "Preemption Checker" will probably not help 
debugging.

Not joining the thread to make sure its cleaned up?
{code}
+  public void stop() {
+stopped = true;
+if (checkerThread != null) {
+  checkerThread.interrupt();
+}
{code}

Nothing else other than this seems to be synchronized. Then why this?
{code}
+  private class PreepmtionChecker implements Runnable {
+@Override
+public void run() {+  while (!stopped && 
!Thread.currentThread().isInterrupted()) {
+synchronized (ScheduleMonitor.this) {
{code}

Couldnt quite grok this. What is delta? What is 0.5? A percentage? Whats the 
math behind the calculation? Should it be "even absent preemption" instead of 
"even absent natural termination"? Is this applied before or after 
TOTAL_PREEMPTION_PER_ROUND?
{code}
+  /**
+   * Given a computed preemption target, account for containers naturally
+   * expiring and preempt only this percentage of the delta. This determines
+   * the rate of geometric convergence into the deadzone ({@link
+   * #MAX_IGNORED_OVER_CAPACITY}). For example, a termination factor of 0.5
+   * will reclaim almost 95% of resources within 5 * {@link
+   * #WAIT_TIME_BEFORE_KILL}, even absent natural termination. */
+  public static final String NATURAL_TERMINATION_FACTOR =
{code}

In which config file do these above configurations go when defined by the 
admin? Shouldnt they be defined in the config defaults of that file? e.g. 
capacity-scheduler.xml? If they get it from the scheduler config then we 
probably shouldnt pass it a configuration object during init.

RMContainer already has the ApplicationAttemptId inside it. No need for extra 
args.
{code}
+  void preemptContainer(ApplicationAttemptId aid, RMContainer container);
{code}

Why no lock here when the other new methods have a lock? Do we not care that 
the app remains in applications during the duration of the operations?
{code}
+  @Override
+  @Lock(Lock.NoLock.class)
+  public void preemptContainer(ApplicationAttemptId aid, RMContainer cont) {
+if(LOG.isDebugEnabled()){
+  LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() + 
+  " container: " + cont.toString());
+}
+FiCaSchedulerApp app = applications.get(aid);
+if (app != null) {
+  app.addPreemptContainer(cont.getContainerId());
+}
+  }
{code}

UnmodifiableSet?
{code}
+  // need to access the list of apps from the preemption monitor
+  public Set getApplications() {
+return activeApplications;
+  }
{code}

containersToPreempt?
{code}
+  private final Set containerToPreempt =
{code}

There is one critical difference between old and new behavior. The new code 
will not send the finish event to the container if its not part of the 
liveContainers. This probably is wrong. Secondly, the parent/queue metrics etc 
are not updated also. I am not sure if this book-keeping is actually designed 
to be in sync with liveContainers - which is what the new code enforces it to 
be. Same comment for the hierarchical callers of this method who now use the 
n

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-12 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12587548/YARN-569.5.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1210//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1210//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (revers

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-12 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12587557/YARN-569.6.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1212//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1212//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
>

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-13 Thread Chris Douglas (JIRA)

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

Chris Douglas commented on YARN-569:


Thanks for the feedback; we revised the patch. We comment below on questions 
that required explanation, while all the small ones are addressed directly in 
the code following your suggestions.

bq. This doesnt seem to affect the fair scheduler or does it? If not, then it 
can be misleading for users.
bq. How do we envisage multiple policies working together without stepping on 
each other? Better off limiting to 1?

The intent was for orthogonal policies to interact with the scheduler, or- if 
conflicting- be coordinated by a composite policy. Though you're right, the 
naming toward preemption is confusing; the patch renames the properties to 
refer to monitors, only. Because the only example is the 
{{ProportionalCapacityPreemptionPolicy}}, {{null}} seemed like the correct 
default. As for limiting to 1 monitor or not, we are experiencing with other 
policies that focus on different aspect of the schedule (e.g., deadlines and 
automatic tuning of queue capacity) and it seems possible to play nice with 
other policies (e.g., ProportionalCapacityPreemptionPolicy), so we would prefer 
to have the mechanism to remain capable of loading multiple monitors.

bq. Not joining the thread to make sure its cleaned up?

The contract for shutting down a monitor is not baked into the API, yet. While 
the proportional policy runs quickly, it's not obvious whether other policies 
would be both long running and respond to interrupts. By way of illustration, 
other monitors we've experimented with call into third party code for 
CPU-intensive calculation. Since YARN-117 went in a few hours ago, that might 
be a chance to define that more crisply. Thoughts?

bq. Why no lock here when the other new methods have a lock? Do we not care 
that the app remains in applications during the duration of the operations?

The semantics of the {{\@Lock}} annotation were not entirely clear from the 
examples in the code, so it's possible the inconsistency is our application of 
it. We're probably making the situation worse, so we omitted the annotations in 
the updated patch. To answer your question: we don't care, because the selected 
container already exited (part of the natural termination factor in the policy).

bq. There is one critical difference between old and new behavior. The new code 
will not send the finish event to the container if its not part of the 
liveContainers. This probably is wrong.
bq. FicaSchedulerNode.unreserveResource(). Checks have been added for the 
reserved container but will the code reach that point if there was no 
reservation actually left on that node? In the same vein, can it happen that 
the node has a new reservation that was made out of band of the preemption 
logic cycle. Hence, the reserved container on the node would exist but could be 
from a different application. 

Good catch, these are related. The change to boolean was necessary because 
we're calling the {{unreserve}} logic from a new context. Since only one 
application can have a single reservation on a node, and because we're freeing 
it through that application, we won't accidentally free another application's 
reservation. However, calling {{unreserve}} on a reservation that converted to 
a container will fail, so we need to know whether the state changed before 
updating the metric.

bq. Couldnt quite grok this. What is delta? What is 0.5? A percentage? Whats 
the math behind the calculation? Should it be "even absent preemption" instead 
of "even absent natural termination"? Is this applied before or after 
TOTAL_PREEMPTION_PER_ROUND?

The delta is the difference between the computed ideal capacity and the actual. 
A value of 0.5 would preempt only 50% of the containers the policy thinks 
should be preempted, as the rest are expected to exit "naturally". The comment 
is saying that- even without any containers exiting on their own- the policy 
will geometrically push capacity into the deadzone. In this case, 50% per 
round, in 5 rounds the policy will be within a 5% deadzone of the ideal 
capacity. It's applied before the total preemption per round; the latter 
proportionally affects all preemption targets.

Because some containers will complete while the policy runs, it may make sense 
to tune it aggressively (or affect it with observed completion rates), but 
we'll want to get some experience running with this.


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>  

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-17 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

bq. so we would prefer to have the mechanism to remain capable of loading 
multiple monitors.
Ok. We can keep that for now. Though we will probably may end up needing 
ordering among them. Shuffling queue capacity probably needs to happen before 
preemption enforces it right?

bq. YARN-117 went in a few hours ago, that might be a chance to define that 
more crisply. Thoughts?
In the case you describe, the RM JVM would essentially hang upon stopping until 
that thread actually exits. Its not daemon right?

The notes on the new boolean return value and its usage make sense logically. 
Hopefully things will work as expected :P

All the new configs like natural wait factor etc would need to be specified in 
yarn-site.xml?

Lets define this value unambiguously in the Priority interface class instead of 
using literal -1.
{code}
+ResourceRequest rr = ResourceRequest.newInstance(
+Priority.newInstance(-1), ResourceRequest.ANY,
{code}

There are references to ScheduleEditPolicy in the code after it has been 
renamed to SchedulingEditPolicy.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among t

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-18 Thread Chris Douglas (JIRA)

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

Chris Douglas commented on YARN-569:


bq. Though we will probably may end up needing ordering among them. Shuffling 
queue capacity probably needs to happen before preemption enforces it right?

This is where we waved our hands and hoped a "composite policy" might appear. 
The lifecycle for monitors is limited to instantiating them. Given the 
complexity of the scheduler, we didn't want to add a lot of scaffolding. If 
this doesn't turn out to be a useful abstraction and we decide to inline 
preemption, tearing out an obscure hook for "monitors" is cheap.

We _expect_ it's useful, but we leaned toward unobtrusive. Adding metadata to 
affect the ordering of monitors' effects... one could imagine scenarios where 
this is necessary, but that's an exercise to the implementer now. Combining 
their effects is something else we have looked at, but not formalized.

bq. In the case you describe, the RM JVM would essentially hang upon stopping 
until that thread actually exits. Its not daemon right?

On reflection, making these daemon threads is correct. Will add this to the 
patch.

bq. The notes on the new boolean return value and its usage make sense 
logically. Hopefully things will work as expected

Heh; fingers crossed.

bq. All the new configs like natural wait factor etc would need to be specified 
in yarn-site.xml?

We could add some of these to the capacity scheduler config (since the 
{{ProportionalCapacityPreemptionPolicy}} is only valid for the CS), but it's a 
little esoteric. The monitor probably should appear in yarn-site, though. Maybe 
we could add some documentation for the policy to YARN-650?

bq. Lets define this value unambiguously in the Priority interface class 
instead of using literal -1.
bq. There are references to ScheduleEditPolicy in the code after it has been 
renamed to SchedulingEditPolicy

Will fix both of these, also

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural compl

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-18 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

The correct place for the configs would depend on which config is being used to 
read them. So it could be a mix of yarn-site.xml and capacity-scheduler.xml. So 
we need to check the code to see which config file is being used to read them.

On a related note, I saw the following in the ApplicationMasterService code.

We are setting values on the allocateresponse after replacing lastResponse in 
the responseMap. This entire section is guarded by the lastResponse value 
obtained from this map (questionable effectiveness perhaps but orthogonal). So 
we should probably be setting everything in the new response (the preemption 
stuff) before the new response replaces the lastResponse in the responseMap.
{code}
  AllocateResponse oldResponse =
  responseMap.put(appAttemptId, allocateResponse);
  if (oldResponse == null) {
// appAttempt got unregistered, remove it back out
responseMap.remove(appAttemptId);
String message = "App Attempt removed from the cache during allocate"
+ appAttemptId;
LOG.error(message);
return resync;
  }
  
  allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
   
  // add preemption to the allocateResponse message (if any)
  
allocateResponse.setPreemptionMessage(generatePreemptionMessage(allocation));
  
  // Adding NMTokens for allocated containers.
  if (!allocation.getContainers().isEmpty()) {
allocateResponse.setNMTokens(rmContext.getNMTokenSecretManager()
.getNMTokens(app.getUser(), appAttemptId,
allocation.getContainers()));
  }
  return allocateResponse;
{code}

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity q

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-18 Thread Chris Douglas (JIRA)

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

Chris Douglas commented on YARN-569:


Updated patch, rebased on YARN-117, etc. On configuration, we didn't include 
the knobs for the proportional policy, but left it as a default with a warning 
to look at the config for the policy. Does that seem reasonable? We can add a 
section on it as part of YARN-650.

bq. We are setting values on the allocateresponse after replacing lastResponse 
in the responseMap. This entire section is guarded by the lastResponse value 
obtained from this map (questionable effectiveness perhaps but orthogonal). So 
we should probably be setting everything in the new response (the preemption 
stuff) before the new response replaces the lastResponse in the responseMap.

You're saying the block updating the {{responseMap}} probably belongs just 
before the return? That makes sense, though I haven't traced it explicitly.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we 

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-18 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12588513/YARN-569.8.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1340//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1340//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> #

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-19 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

bq. You're saying the block updating the responseMap probably belongs just 
before the return? That makes sense, though I haven't traced it explicitly.
Yes.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how frequently to run the policy
> # how lo

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-19 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

Since configuring this involves more than 1 config file, as long as its clear 
which file to change for which config, its all good. 

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how frequently to run the policy
> # how long to wait between preem

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-19 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

One other thing to check would be if the preemption policy will use refreshed 
values when the capacity scheduler config is refreshed on the fly. Looks like 
cloneQueues() will take the absolute used and guaranteed numbers on every 
clone. So we should be good wrt that. Would be good to check other values the 
policy looks at.
Noticed formatting issues with spaces in the patch. eg. cloneQueues()

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a 

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-19 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12588685/YARN-569.9.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1352//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1352//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.9.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more rese

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-22 Thread Bikas Saha (JIRA)

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

Bikas Saha commented on YARN-569:
-

Changes look good overall. Didnt look deeply at the preemption heuristics since 
we shall probably be working on them as we do more experimentation on real 
workloads. +1.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.1.patch, YARN-569.2.patch, YARN-569.3.patch, 
> YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, YARN-569.8.patch, 
> YARN-569.9.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how freque

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-24 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12589506/YARN-569.10.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:red}-1 core tests{color}.  The patch failed these unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager:

  
org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1391//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1391//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.10.patch, YARN-569.1.patch, YARN-569.2.patch, 
> YARN-569.3.patch, YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, 
> YARN-569.8.patch, YARN-569.9.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from 

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-06-24 Thread Chris Douglas (JIRA)

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

Chris Douglas commented on YARN-569:


{{TestAMAuthorization}} also fails on trunk, YARN-878

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.10.patch, YARN-569.1.patch, YARN-569.2.patch, 
> YARN-569.3.patch, YARN-569.4.patch, YARN-569.5.patch, YARN-569.6.patch, 
> YARN-569.8.patch, YARN-569.9.patch, YARN-569.patch, YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to reclaim is obtained, or until no more reservations exits
> # (if not enough) it issues preemptions for containers from the same 
> applications (reverse chronological order, last assigned container first) 
> again until necessary or until no containers except the AM container are left,
> # (if not enough) it moves onto unreserve and preempt from the next 
> application. 
> # containers that have been asked to preempt are tracked across executions. 
> If a containers is among the one to be preempted for more than a certain 
> time, the container is moved in a the list of containers to be forcibly 
> killed. 
> Notes:
> (*) at the moment, in order to avoid double-counting of the requests, we only 
> look at the "ANY" part of pending resource requests, which means we might not 
> preempt on behalf of AMs that ask only for specific locations but not any. 
> (**) The ideal balance state is one in which each queue has at least its 
> guaranteed capacity, and the spare capacity is distributed among queues (that 
> wants some) as a weighted fair share. Where the weighting is based on the 
> guaranteed capacity of a queue, and the function runs to a fix point.  
> Tunables of the ProportionalCapacityPreemptionPolicy:
> # observe-only mode (i.e., log the actions it would take, but behave as 
> read-only)
> # how frequently to run the policy
> # how long to wait between preemption and kill of a container
> # wh

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-07-10 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on YARN-569:


{color:green}+1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12591768/YARN-569.11.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-YARN-Build/1453//testReport/
Console output: https://builds.apache.org/job/PreCommit-YARN-Build/1453//console

This message is automatically generated.

> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Project: Hadoop YARN
>  Issue Type: Sub-task
>  Components: capacityscheduler
>Reporter: Carlo Curino
>Assignee: Carlo Curino
> Attachments: 3queues.pdf, CapScheduler_with_preemption.pdf, 
> preemption.2.patch, YARN-569.10.patch, YARN-569.11.patch, YARN-569.1.patch, 
> YARN-569.2.patch, YARN-569.3.patch, YARN-569.4.patch, YARN-569.5.patch, 
> YARN-569.6.patch, YARN-569.8.patch, YARN-569.9.patch, YARN-569.patch, 
> YARN-569.patch
>
>
> There is a tension between the fast-pace reactive role of the 
> CapacityScheduler, which needs to respond quickly to 
> applications resource requests, and node updates, and the more introspective, 
> time-based considerations 
> needed to observe and correct for capacity balance. To this purpose we opted 
> instead of hacking the delicate
> mechanisms of the CapacityScheduler directly to add support for preemption by 
> means of a "Capacity Monitor",
> which can be run optionally as a separate service (much like the 
> NMLivelinessMonitor).
> The capacity monitor (similarly to equivalent functionalities in the fairness 
> scheduler) operates running on intervals 
> (e.g., every 3 seconds), observe the state of the assignment of resources to 
> queues from the capacity scheduler, 
> performs off-line computation to determine if preemption is needed, and how 
> best to "edit" the current schedule to 
> improve capacity, and generates events that produce four possible actions:
> # Container de-reservations
> # Resource-based preemptions
> # Container-based preemptions
> # Container killing
> The actions listed above are progressively more costly, and it is up to the 
> policy to use them as desired to achieve the rebalancing goals. 
> Note that due to the "lag" in the effect of these actions the policy should 
> operate at the macroscopic level (e.g., preempt tens of containers
> from a queue) and not trying to tightly and consistently micromanage 
> container allocations. 
> - Preemption policy  (ProportionalCapacityPreemptionPolicy): 
> - 
> Preemption policies are by design pluggable, in the following we present an 
> initial policy (ProportionalCapacityPreemptionPolicy) we have been 
> experimenting with.  The ProportionalCapacityPreemptionPolicy behaves as 
> follows:
> # it gathers from the scheduler the state of the queues, in particular, their 
> current capacity, guaranteed capacity and pending requests (*)
> # if there are pending requests from queues that are under capacity it 
> computes a new ideal balanced state (**)
> # it computes the set of preemptions needed to repair the current schedule 
> and achieve capacity balance (accounting for natural completion rates, and 
> respecting bounds on the amount of preemption we allow for each round)
> # it selects which applications to preempt from each over-capacity queue (the 
> last one in the FIFO order)
> # it remove reservations from the most recently assigned app until the amount 
> of resource to 

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-07-10 Thread Hudson (JIRA)

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

Hudson commented on YARN-569:
-

Integrated in Hadoop-trunk-Commit #4065 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/4065/])
YARN-569. Add support for requesting and enforcing preemption requests via
a capacity monitor. Contributed by Carlo Curino, Chris Douglas (Revision 
1502083)

 Result = SUCCESS
cdouglas : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1502083
Files : 
* /hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEvent.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEventType.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
>   

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-07-11 Thread Hudson (JIRA)

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

Hudson commented on YARN-569:
-

Integrated in Hadoop-Yarn-trunk #267 (See 
[https://builds.apache.org/job/Hadoop-Yarn-trunk/267/])
YARN-569. Add support for requesting and enforcing preemption requests via
a capacity monitor. Contributed by Carlo Curino, Chris Douglas (Revision 
1502083)

 Result = SUCCESS
cdouglas : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1502083
Files : 
* /hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEvent.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEventType.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Proj

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-07-11 Thread Hudson (JIRA)

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

Hudson commented on YARN-569:
-

Integrated in Hadoop-Hdfs-trunk #1457 (See 
[https://builds.apache.org/job/Hadoop-Hdfs-trunk/1457/])
YARN-569. Add support for requesting and enforcing preemption requests via
a capacity monitor. Contributed by Carlo Curino, Chris Douglas (Revision 
1502083)

 Result = FAILURE
cdouglas : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1502083
Files : 
* /hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEvent.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEventType.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
> Pr

[jira] [Commented] (YARN-569) CapacityScheduler: support for preemption (using a capacity monitor)

2013-07-11 Thread Hudson (JIRA)

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

Hudson commented on YARN-569:
-

Integrated in Hadoop-Mapreduce-trunk #1484 (See 
[https://builds.apache.org/job/Hadoop-Mapreduce-trunk/1484/])
YARN-569. Add support for requesting and enforcing preemption requests via
a capacity monitor. Contributed by Carlo Curino, Chris Douglas (Revision 
1502083)

 Result = SUCCESS
cdouglas : 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1502083
Files : 
* /hadoop/common/trunk/hadoop-yarn-project/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEvent.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerPreemptEventType.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java


> CapacityScheduler: support for preemption (using a capacity monitor)
> 
>
> Key: YARN-569
> URL: https://issues.apache.org/jira/browse/YARN-569
>