|
@@ -24,8 +24,11 @@ import java.util.Collection;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -155,10 +158,16 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
private Resource clusterCapacity =
|
|
|
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class);
|
|
|
|
|
|
- // How often tasks are preempted (must be longer than a couple
|
|
|
+ // How often tasks are preempted
|
|
|
+ protected long preemptionInterval;
|
|
|
+
|
|
|
+ // ms to wait before force killing stuff (must be longer than a couple
|
|
|
// of heartbeats to give task-kill commands a chance to act).
|
|
|
- protected long preemptionInterval = 15000;
|
|
|
-
|
|
|
+ protected long waitTimeBeforeKill;
|
|
|
+
|
|
|
+ // Containers whose AMs have been warned that they will be preempted soon.
|
|
|
+ private List<RMContainer> warnedContainers = new ArrayList<RMContainer>();
|
|
|
+
|
|
|
protected boolean preemptionEnabled;
|
|
|
protected boolean sizeBasedWeight; // Give larger weights to larger jobs
|
|
|
protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster
|
|
@@ -331,34 +340,78 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
// Sort containers into reverse order of priority
|
|
|
Collections.sort(runningContainers, new Comparator<RMContainer>() {
|
|
|
public int compare(RMContainer c1, RMContainer c2) {
|
|
|
- return c2.getContainer().getPriority().compareTo(
|
|
|
+ int ret = c2.getContainer().getPriority().compareTo(
|
|
|
c1.getContainer().getPriority());
|
|
|
+ if (ret == 0) {
|
|
|
+ return c2.getContainerId().compareTo(c1.getContainerId());
|
|
|
+ }
|
|
|
+ return ret;
|
|
|
}
|
|
|
});
|
|
|
+
|
|
|
+ // Scan down the list of containers we've already warned and kill them
|
|
|
+ // if we need to. Remove any containers from the list that we don't need
|
|
|
+ // or that are no longer running.
|
|
|
+ Iterator<RMContainer> warnedIter = warnedContainers.iterator();
|
|
|
+ Set<RMContainer> preemptedThisRound = new HashSet<RMContainer>();
|
|
|
+ while (warnedIter.hasNext()) {
|
|
|
+ RMContainer container = warnedIter.next();
|
|
|
+ if (container.getState() == RMContainerState.RUNNING &&
|
|
|
+ Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
|
|
|
+ toPreempt, Resources.none())) {
|
|
|
+ warnOrKillContainer(container, apps.get(container), queues.get(container));
|
|
|
+ preemptedThisRound.add(container);
|
|
|
+ Resources.subtractFrom(toPreempt, container.getContainer().getResource());
|
|
|
+ } else {
|
|
|
+ warnedIter.remove();
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // Scan down the sorted list of task statuses until we've killed enough
|
|
|
- // tasks, making sure we don't kill too many from any queue
|
|
|
- for (RMContainer container : runningContainers) {
|
|
|
+ // Scan down the rest of the containers until we've preempted enough, making
|
|
|
+ // sure we don't preempt too many from any queue
|
|
|
+ Iterator<RMContainer> runningIter = runningContainers.iterator();
|
|
|
+ while (runningIter.hasNext() &&
|
|
|
+ Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
|
|
|
+ toPreempt, Resources.none())) {
|
|
|
+ RMContainer container = runningIter.next();
|
|
|
FSLeafQueue sched = queues.get(container);
|
|
|
- if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
|
|
|
- sched.getResourceUsage(), sched.getFairShare())) {
|
|
|
- LOG.info("Preempting container (prio=" + container.getContainer().getPriority() +
|
|
|
- "res=" + container.getContainer().getResource() +
|
|
|
- ") from queue " + sched.getName());
|
|
|
- ContainerStatus status = SchedulerUtils.createAbnormalContainerStatus(
|
|
|
+ if (!preemptedThisRound.contains(container) &&
|
|
|
+ Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
|
|
|
+ sched.getResourceUsage(), sched.getFairShare())) {
|
|
|
+ warnOrKillContainer(container, apps.get(container), sched);
|
|
|
+
|
|
|
+ warnedContainers.add(container);
|
|
|
+ Resources.subtractFrom(toPreempt, container.getContainer().getResource());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void warnOrKillContainer(RMContainer container, FSSchedulerApp app,
|
|
|
+ FSLeafQueue queue) {
|
|
|
+ LOG.info("Preempting container (prio=" + container.getContainer().getPriority() +
|
|
|
+ "res=" + container.getContainer().getResource() +
|
|
|
+ ") from queue " + queue.getName());
|
|
|
+
|
|
|
+ Long time = app.getContainerPreemptionTime(container);
|
|
|
+
|
|
|
+ if (time != null) {
|
|
|
+ // if we asked for preemption more than maxWaitTimeBeforeKill ms ago,
|
|
|
+ // proceed with kill
|
|
|
+ if (time + waitTimeBeforeKill < clock.getTime()) {
|
|
|
+ ContainerStatus status =
|
|
|
+ SchedulerUtils.createAbnormalContainerStatus(
|
|
|
container.getContainerId(), SchedulerUtils.PREEMPTED_CONTAINER);
|
|
|
|
|
|
// TODO: Not sure if this ever actually adds this to the list of cleanup
|
|
|
// containers on the RMNode (see SchedulerNode.releaseContainer()).
|
|
|
completedContainer(container, status, RMContainerEventType.KILL);
|
|
|
-
|
|
|
- toPreempt = Resources.subtract(toPreempt,
|
|
|
- container.getContainer().getResource());
|
|
|
- if (Resources.lessThanOrEqual(RESOURCE_CALCULATOR, clusterCapacity,
|
|
|
- toPreempt, Resources.none())) {
|
|
|
- break;
|
|
|
- }
|
|
|
+ LOG.info("Killing container" + container +
|
|
|
+ " (after waiting for premption for " +
|
|
|
+ (clock.getTime() - time) + "ms)");
|
|
|
}
|
|
|
+ } else {
|
|
|
+ // track the request in the FSSchedulerApp itself
|
|
|
+ app.addPreemption(container, clock.getTime());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -483,11 +536,11 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
return clusterCapacity;
|
|
|
}
|
|
|
|
|
|
- public Clock getClock() {
|
|
|
+ public synchronized Clock getClock() {
|
|
|
return clock;
|
|
|
}
|
|
|
|
|
|
- protected void setClock(Clock clock) {
|
|
|
+ protected synchronized void setClock(Clock clock) {
|
|
|
this.clock = clock;
|
|
|
}
|
|
|
|
|
@@ -745,10 +798,18 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
LOG.debug("allocate:" +
|
|
|
" applicationAttemptId=" + appAttemptId +
|
|
|
" #ask=" + ask.size());
|
|
|
- }
|
|
|
|
|
|
+ LOG.debug("Preempting " + application.getPreemptionContainers().size()
|
|
|
+ + " container(s)");
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<ContainerId> preemptionContainerIds = new HashSet<ContainerId>();
|
|
|
+ for (RMContainer container : application.getPreemptionContainers()) {
|
|
|
+ preemptionContainerIds.add(container.getContainerId());
|
|
|
+ }
|
|
|
+
|
|
|
return new Allocation(application.pullNewlyAllocatedContainers(),
|
|
|
- application.getHeadroom());
|
|
|
+ application.getHeadroom(), preemptionContainerIds);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -963,7 +1024,9 @@ public class FairScheduler implements ResourceScheduler {
|
|
|
assignMultiple = this.conf.getAssignMultiple();
|
|
|
maxAssign = this.conf.getMaxAssign();
|
|
|
sizeBasedWeight = this.conf.getSizeBasedWeight();
|
|
|
-
|
|
|
+ preemptionInterval = this.conf.getPreemptionInterval();
|
|
|
+ waitTimeBeforeKill = this.conf.getWaitTimeBeforeKill();
|
|
|
+
|
|
|
if (!initialized) {
|
|
|
rootMetrics = QueueMetrics.forQueue("root", null, true, conf);
|
|
|
this.rmContext = rmContext;
|