|
@@ -38,13 +38,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
-import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
@@ -52,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
|
|
import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.SystemClock;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
@@ -118,8 +118,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
public static final String NATURAL_TERMINATION_FACTOR =
|
|
|
"yarn.resourcemanager.monitor.capacity.preemption.natural_termination_factor";
|
|
|
|
|
|
- // the dispatcher to send preempt and kill events
|
|
|
- public EventHandler<ContainerPreemptEvent> dispatcher;
|
|
|
+ private RMContext rmContext;
|
|
|
|
|
|
private final Clock clock;
|
|
|
private double maxIgnoredOverCapacity;
|
|
@@ -141,20 +140,17 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
}
|
|
|
|
|
|
public ProportionalCapacityPreemptionPolicy(Configuration config,
|
|
|
- EventHandler<ContainerPreemptEvent> dispatcher,
|
|
|
- CapacityScheduler scheduler) {
|
|
|
- this(config, dispatcher, scheduler, new SystemClock());
|
|
|
+ RMContext context, CapacityScheduler scheduler) {
|
|
|
+ this(config, context, scheduler, new SystemClock());
|
|
|
}
|
|
|
|
|
|
public ProportionalCapacityPreemptionPolicy(Configuration config,
|
|
|
- EventHandler<ContainerPreemptEvent> dispatcher,
|
|
|
- CapacityScheduler scheduler, Clock clock) {
|
|
|
- init(config, dispatcher, scheduler);
|
|
|
+ RMContext context, CapacityScheduler scheduler, Clock clock) {
|
|
|
+ init(config, context, scheduler);
|
|
|
this.clock = clock;
|
|
|
}
|
|
|
|
|
|
- public void init(Configuration config,
|
|
|
- EventHandler<ContainerPreemptEvent> disp,
|
|
|
+ public void init(Configuration config, RMContext context,
|
|
|
PreemptableResourceScheduler sched) {
|
|
|
LOG.info("Preemption monitor:" + this.getClass().getCanonicalName());
|
|
|
assert null == scheduler : "Unexpected duplicate call to init";
|
|
@@ -163,7 +159,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
sched.getClass().getCanonicalName() + " not instance of " +
|
|
|
CapacityScheduler.class.getCanonicalName());
|
|
|
}
|
|
|
- dispatcher = disp;
|
|
|
+ rmContext = context;
|
|
|
scheduler = (CapacityScheduler) sched;
|
|
|
maxIgnoredOverCapacity = config.getDouble(MAX_IGNORED_OVER_CAPACITY, 0.1);
|
|
|
naturalTerminationFactor =
|
|
@@ -196,6 +192,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
* @param root the root of the CapacityScheduler queue hierarchy
|
|
|
* @param clusterResources the total amount of resources in the cluster
|
|
|
*/
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
private void containerBasedPreemptOrKill(CSQueue root,
|
|
|
Resource clusterResources) {
|
|
|
// All partitions to look at
|
|
@@ -248,8 +245,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
// preempt (or kill) the selected containers
|
|
|
for (Map.Entry<ApplicationAttemptId,Set<RMContainer>> e
|
|
|
: toPreempt.entrySet()) {
|
|
|
+ ApplicationAttemptId appAttemptId = e.getKey();
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Send to scheduler: in app=" + e.getKey()
|
|
|
+ LOG.debug("Send to scheduler: in app=" + appAttemptId
|
|
|
+ " #containers-to-be-preempted=" + e.getValue().size());
|
|
|
}
|
|
|
for (RMContainer container : e.getValue()) {
|
|
@@ -257,13 +255,15 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
if (preempted.get(container) != null &&
|
|
|
preempted.get(container) + maxWaitTime < clock.getTime()) {
|
|
|
// kill it
|
|
|
- dispatcher.handle(new ContainerPreemptEvent(e.getKey(), container,
|
|
|
- ContainerPreemptEventType.KILL_CONTAINER));
|
|
|
+ rmContext.getDispatcher().getEventHandler().handle(
|
|
|
+ new ContainerPreemptEvent(appAttemptId, container,
|
|
|
+ SchedulerEventType.KILL_CONTAINER));
|
|
|
preempted.remove(container);
|
|
|
} else {
|
|
|
//otherwise just send preemption events
|
|
|
- dispatcher.handle(new ContainerPreemptEvent(e.getKey(), container,
|
|
|
- ContainerPreemptEventType.PREEMPT_CONTAINER));
|
|
|
+ rmContext.getDispatcher().getEventHandler().handle(
|
|
|
+ new ContainerPreemptEvent(appAttemptId, container,
|
|
|
+ SchedulerEventType.PREEMPT_CONTAINER));
|
|
|
if (preempted.get(container) == null) {
|
|
|
preempted.put(container, clock.getTime());
|
|
|
}
|
|
@@ -735,6 +735,7 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
* Given a target preemption for a specific application, select containers
|
|
|
* to preempt (after unreserving all reservation for that app).
|
|
|
*/
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
private void preemptFrom(FiCaSchedulerApp app,
|
|
|
Resource clusterResource, Map<String, Resource> resToObtainByPartition,
|
|
|
List<RMContainer> skippedAMContainerlist, Resource skippedAMSize,
|
|
@@ -758,8 +759,9 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
|
|
clusterResource, preemptMap);
|
|
|
|
|
|
if (!observeOnly) {
|
|
|
- dispatcher.handle(new ContainerPreemptEvent(appId, c,
|
|
|
- ContainerPreemptEventType.DROP_RESERVATION));
|
|
|
+ rmContext.getDispatcher().getEventHandler().handle(
|
|
|
+ new ContainerPreemptEvent(
|
|
|
+ appId, c, SchedulerEventType.DROP_RESERVATION));
|
|
|
}
|
|
|
}
|
|
|
|