|
@@ -41,20 +41,26 @@ class FSPreemptionThread extends Thread {
|
|
|
protected final FSContext context;
|
|
|
private final FairScheduler scheduler;
|
|
|
private final long warnTimeBeforeKill;
|
|
|
+ private final long delayBeforeNextStarvationCheck;
|
|
|
private final Timer preemptionTimer;
|
|
|
|
|
|
FSPreemptionThread(FairScheduler scheduler) {
|
|
|
+ setDaemon(true);
|
|
|
+ setName("FSPreemptionThread");
|
|
|
this.scheduler = scheduler;
|
|
|
this.context = scheduler.getContext();
|
|
|
FairSchedulerConfiguration fsConf = scheduler.getConf();
|
|
|
context.setPreemptionEnabled();
|
|
|
context.setPreemptionUtilizationThreshold(
|
|
|
fsConf.getPreemptionUtilizationThreshold());
|
|
|
- warnTimeBeforeKill = fsConf.getWaitTimeBeforeKill();
|
|
|
preemptionTimer = new Timer("Preemption Timer", true);
|
|
|
|
|
|
- setDaemon(true);
|
|
|
- setName("FSPreemptionThread");
|
|
|
+ warnTimeBeforeKill = fsConf.getWaitTimeBeforeKill();
|
|
|
+ long allocDelay = (fsConf.isContinuousSchedulingEnabled()
|
|
|
+ ? 10 * fsConf.getContinuousSchedulingSleepMs() // 10 runs
|
|
|
+ : 4 * scheduler.getNMHeartbeatInterval()); // 4 heartbeats
|
|
|
+ delayBeforeNextStarvationCheck = warnTimeBeforeKill + allocDelay +
|
|
|
+ fsConf.getWaitTimeBeforeNextStarvationCheck();
|
|
|
}
|
|
|
|
|
|
public void run() {
|
|
@@ -62,13 +68,8 @@ class FSPreemptionThread extends Thread {
|
|
|
FSAppAttempt starvedApp;
|
|
|
try{
|
|
|
starvedApp = context.getStarvedApps().take();
|
|
|
- if (!Resources.isNone(starvedApp.getStarvation())) {
|
|
|
- PreemptableContainers containers =
|
|
|
- identifyContainersToPreempt(starvedApp);
|
|
|
- if (containers != null) {
|
|
|
- preemptContainers(containers.containers);
|
|
|
- }
|
|
|
- }
|
|
|
+ preemptContainers(identifyContainersToPreempt(starvedApp));
|
|
|
+ starvedApp.preemptionTriggered(delayBeforeNextStarvationCheck);
|
|
|
} catch (InterruptedException e) {
|
|
|
LOG.info("Preemption thread interrupted! Exiting.");
|
|
|
return;
|
|
@@ -77,55 +78,57 @@ class FSPreemptionThread extends Thread {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Given an app, identify containers to preempt to satisfy the app's next
|
|
|
- * resource request.
|
|
|
+ * Given an app, identify containers to preempt to satisfy the app's
|
|
|
+ * starvation.
|
|
|
+ *
|
|
|
+ * Mechanics:
|
|
|
+ * 1. Fetch all {@link ResourceRequest}s corresponding to the amount of
|
|
|
+ * starvation.
|
|
|
+ * 2. For each {@link ResourceRequest}, iterate through matching
|
|
|
+ * nodes and identify containers to preempt all on one node, also
|
|
|
+ * optimizing for least number of AM container preemptions.
|
|
|
*
|
|
|
* @param starvedApp starved application for which we are identifying
|
|
|
* preemption targets
|
|
|
- * @return list of containers to preempt to satisfy starvedApp, null if the
|
|
|
- * app cannot be satisfied by preempting any running containers
|
|
|
+ * @return list of containers to preempt to satisfy starvedApp
|
|
|
*/
|
|
|
- private PreemptableContainers identifyContainersToPreempt(
|
|
|
+ private List<RMContainer> identifyContainersToPreempt(
|
|
|
FSAppAttempt starvedApp) {
|
|
|
- PreemptableContainers bestContainers = null;
|
|
|
+ List<RMContainer> containersToPreempt = new ArrayList<>();
|
|
|
|
|
|
- // Find the nodes that match the next resource request
|
|
|
- ResourceRequest request = starvedApp.getNextResourceRequest();
|
|
|
- // TODO (KK): Should we check other resource requests if we can't match
|
|
|
- // the first one?
|
|
|
-
|
|
|
- Resource requestCapability = request.getCapability();
|
|
|
- List<FSSchedulerNode> potentialNodes =
|
|
|
- scheduler.getNodeTracker().getNodesByResourceName(
|
|
|
- request.getResourceName());
|
|
|
+ // Iterate through enough RRs to address app's starvation
|
|
|
+ for (ResourceRequest rr : starvedApp.getStarvedResourceRequests()) {
|
|
|
+ for (int i = 0; i < rr.getNumContainers(); i++) {
|
|
|
+ PreemptableContainers bestContainers = null;
|
|
|
+ List<FSSchedulerNode> potentialNodes = scheduler.getNodeTracker()
|
|
|
+ .getNodesByResourceName(rr.getResourceName());
|
|
|
+ for (FSSchedulerNode node : potentialNodes) {
|
|
|
+ // TODO (YARN-5829): Attempt to reserve the node for starved app.
|
|
|
+ if (isNodeAlreadyReserved(node, starvedApp)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
|
|
|
- // From the potential nodes, pick a node that has enough containers
|
|
|
- // from apps over their fairshare
|
|
|
- for (FSSchedulerNode node : potentialNodes) {
|
|
|
- // TODO (YARN-5829): Attempt to reserve the node for starved app. The
|
|
|
- // subsequent if-check needs to be reworked accordingly.
|
|
|
- FSAppAttempt nodeReservedApp = node.getReservedAppSchedulable();
|
|
|
- if (nodeReservedApp != null && !nodeReservedApp.equals(starvedApp)) {
|
|
|
- // This node is already reserved by another app. Let us not consider
|
|
|
- // this for preemption.
|
|
|
- continue;
|
|
|
- }
|
|
|
+ int maxAMContainers = bestContainers == null ?
|
|
|
+ Integer.MAX_VALUE : bestContainers.numAMContainers;
|
|
|
+ PreemptableContainers preemptableContainers =
|
|
|
+ identifyContainersToPreemptOnNode(
|
|
|
+ rr.getCapability(), node, maxAMContainers);
|
|
|
+ if (preemptableContainers != null) {
|
|
|
+ // This set is better than any previously identified set.
|
|
|
+ bestContainers = preemptableContainers;
|
|
|
+ if (preemptableContainers.numAMContainers == 0) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } // End of iteration through nodes for one RR
|
|
|
|
|
|
- int maxAMContainers = bestContainers == null ?
|
|
|
- Integer.MAX_VALUE : bestContainers.numAMContainers;
|
|
|
- PreemptableContainers preemptableContainers =
|
|
|
- identifyContainersToPreemptOnNode(requestCapability, node,
|
|
|
- maxAMContainers);
|
|
|
- if (preemptableContainers != null) {
|
|
|
- if (preemptableContainers.numAMContainers == 0) {
|
|
|
- return preemptableContainers;
|
|
|
- } else {
|
|
|
- bestContainers = preemptableContainers;
|
|
|
+ if (bestContainers != null && bestContainers.containers.size() > 0) {
|
|
|
+ containersToPreempt.addAll(bestContainers.containers);
|
|
|
+ trackPreemptionsAgainstNode(bestContainers.containers);
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- return bestContainers;
|
|
|
+ } // End of iteration over RRs
|
|
|
+ return containersToPreempt;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -176,23 +179,25 @@ class FSPreemptionThread extends Thread {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- private void preemptContainers(List<RMContainer> containers) {
|
|
|
- // Mark the containers as being considered for preemption on the node.
|
|
|
- // Make sure the containers are subsequently removed by calling
|
|
|
- // FSSchedulerNode#removeContainerForPreemption.
|
|
|
- if (containers.size() > 0) {
|
|
|
- FSSchedulerNode node = (FSSchedulerNode) scheduler.getNodeTracker()
|
|
|
- .getNode(containers.get(0).getNodeId());
|
|
|
- node.addContainersForPreemption(containers);
|
|
|
- }
|
|
|
+ private boolean isNodeAlreadyReserved(
|
|
|
+ FSSchedulerNode node, FSAppAttempt app) {
|
|
|
+ FSAppAttempt nodeReservedApp = node.getReservedAppSchedulable();
|
|
|
+ return nodeReservedApp != null && !nodeReservedApp.equals(app);
|
|
|
+ }
|
|
|
|
|
|
+ private void trackPreemptionsAgainstNode(List<RMContainer> containers) {
|
|
|
+ FSSchedulerNode node = (FSSchedulerNode) scheduler.getNodeTracker()
|
|
|
+ .getNode(containers.get(0).getNodeId());
|
|
|
+ node.addContainersForPreemption(containers);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void preemptContainers(List<RMContainer> containers) {
|
|
|
// Warn application about containers to be killed
|
|
|
for (RMContainer container : containers) {
|
|
|
ApplicationAttemptId appAttemptId = container.getApplicationAttemptId();
|
|
|
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
|
|
- FSLeafQueue queue = app.getQueue();
|
|
|
LOG.info("Preempting container " + container +
|
|
|
- " from queue " + queue.getName());
|
|
|
+ " from queue " + app.getQueueName());
|
|
|
app.trackContainerForPreemption(container);
|
|
|
}
|
|
|
|