|
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.*;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
@@ -93,7 +94,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private int nodeLocalityDelay;
|
|
|
|
|
|
- Set<FiCaSchedulerApp> activeApplications;
|
|
|
Map<ApplicationAttemptId, FiCaSchedulerApp> applicationAttemptMap =
|
|
|
new HashMap<ApplicationAttemptId, FiCaSchedulerApp>();
|
|
|
|
|
@@ -121,6 +121,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
private volatile ResourceLimits currentResourceLimits = null;
|
|
|
|
|
|
+ private OrderingPolicy<FiCaSchedulerApp>
|
|
|
+ orderingPolicy = new FifoOrderingPolicy<FiCaSchedulerApp>();
|
|
|
+
|
|
|
public LeafQueue(CapacitySchedulerContext cs,
|
|
|
String queueName, CSQueue parent, CSQueue old) throws IOException {
|
|
|
super(cs, queueName, parent, old);
|
|
@@ -137,7 +140,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
cs.getApplicationComparator();
|
|
|
this.pendingApplications =
|
|
|
new TreeSet<FiCaSchedulerApp>(applicationComparator);
|
|
|
- this.activeApplications = new TreeSet<FiCaSchedulerApp>(applicationComparator);
|
|
|
|
|
|
setupQueueConfigs(cs.getClusterResource());
|
|
|
}
|
|
@@ -159,6 +161,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
setQueueResourceLimitsInfo(clusterResource);
|
|
|
|
|
|
CapacitySchedulerConfiguration conf = csContext.getConfiguration();
|
|
|
+
|
|
|
+ setOrderingPolicy(conf.<FiCaSchedulerApp>getOrderingPolicy(getQueuePath()));
|
|
|
+
|
|
|
userLimit = conf.getUserLimit(getQueuePath());
|
|
|
userLimitFactor = conf.getUserLimitFactor(getQueuePath());
|
|
|
|
|
@@ -322,7 +327,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
|
|
|
public synchronized int getNumActiveApplications() {
|
|
|
- return activeApplications.size();
|
|
|
+ return orderingPolicy.getNumSchedulableEntities();
|
|
|
}
|
|
|
|
|
|
@Private
|
|
@@ -637,7 +642,7 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
}
|
|
|
user.activateApplication();
|
|
|
- activeApplications.add(application);
|
|
|
+ orderingPolicy.addSchedulableEntity(application);
|
|
|
queueUsage.incAMUsed(application.getAMResource());
|
|
|
user.getResourceUsage().incAMUsed(application.getAMResource());
|
|
|
i.remove();
|
|
@@ -686,7 +691,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
public synchronized void removeApplicationAttempt(
|
|
|
FiCaSchedulerApp application, User user) {
|
|
|
- boolean wasActive = activeApplications.remove(application);
|
|
|
+ boolean wasActive =
|
|
|
+ orderingPolicy.removeSchedulableEntity(application);
|
|
|
if (!wasActive) {
|
|
|
pendingApplications.remove(application);
|
|
|
} else {
|
|
@@ -727,7 +733,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("assignContainers: node=" + node.getNodeName()
|
|
|
- + " #applications=" + activeApplications.size());
|
|
|
+ + " #applications=" +
|
|
|
+ orderingPolicy.getNumSchedulableEntities());
|
|
|
}
|
|
|
|
|
|
// Check for reserved resources
|
|
@@ -759,9 +766,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
return NULL_ASSIGNMENT;
|
|
|
}
|
|
|
|
|
|
- // Try to assign containers to applications in order
|
|
|
- for (FiCaSchedulerApp application : activeApplications) {
|
|
|
-
|
|
|
+ for (Iterator<FiCaSchedulerApp> assignmentIterator =
|
|
|
+ orderingPolicy.getAssignmentIterator();
|
|
|
+ assignmentIterator.hasNext();) {
|
|
|
+ FiCaSchedulerApp application = assignmentIterator.next();
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("pre-assignContainers for application "
|
|
|
+ application.getApplicationId());
|
|
@@ -1606,6 +1614,9 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
|
|
|
// Inform the node
|
|
|
node.allocateContainer(allocatedContainer);
|
|
|
+
|
|
|
+ // Inform the ordering policy
|
|
|
+ orderingPolicy.containerAllocated(application, allocatedContainer);
|
|
|
|
|
|
LOG.info("assignedContainer" +
|
|
|
" application attempt=" + application.getApplicationAttemptId() +
|
|
@@ -1715,11 +1726,16 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
removed =
|
|
|
application.containerCompleted(rmContainer, containerStatus,
|
|
|
event, node.getPartition());
|
|
|
+
|
|
|
node.releaseContainer(container);
|
|
|
}
|
|
|
|
|
|
// Book-keeping
|
|
|
if (removed) {
|
|
|
+
|
|
|
+ // Inform the ordering policy
|
|
|
+ orderingPolicy.containerReleased(application, rmContainer);
|
|
|
+
|
|
|
releaseResource(clusterResource, application,
|
|
|
container.getResource(), node.getPartition());
|
|
|
LOG.info("completedContainer" +
|
|
@@ -1822,7 +1838,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
activateApplications();
|
|
|
|
|
|
// Update application properties
|
|
|
- for (FiCaSchedulerApp application : activeApplications) {
|
|
|
+ for (FiCaSchedulerApp application :
|
|
|
+ orderingPolicy.getSchedulableEntities()) {
|
|
|
synchronized (application) {
|
|
|
computeUserLimitAndSetHeadroom(application, clusterResource,
|
|
|
Resources.none(), RMNodeLabelsManager.NO_LABEL,
|
|
@@ -1916,19 +1933,19 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
getParent().recoverContainer(clusterResource, attempt, rmContainer);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Obtain (read-only) collection of active applications.
|
|
|
*/
|
|
|
- public Set<FiCaSchedulerApp> getApplications() {
|
|
|
- // need to access the list of apps from the preemption monitor
|
|
|
- return activeApplications;
|
|
|
+ public Collection<FiCaSchedulerApp> getApplications() {
|
|
|
+ return orderingPolicy.getSchedulableEntities();
|
|
|
}
|
|
|
|
|
|
// return a single Resource capturing the overal amount of pending resources
|
|
|
public synchronized Resource getTotalResourcePending() {
|
|
|
Resource ret = BuilderUtils.newResource(0, 0);
|
|
|
- for (FiCaSchedulerApp f : activeApplications) {
|
|
|
+ for (FiCaSchedulerApp f :
|
|
|
+ orderingPolicy.getSchedulableEntities()) {
|
|
|
Resources.addTo(ret, f.getTotalPendingRequests());
|
|
|
}
|
|
|
return ret;
|
|
@@ -1940,7 +1957,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
for (FiCaSchedulerApp pendingApp : pendingApplications) {
|
|
|
apps.add(pendingApp.getApplicationAttemptId());
|
|
|
}
|
|
|
- for (FiCaSchedulerApp app : activeApplications) {
|
|
|
+ for (FiCaSchedulerApp app :
|
|
|
+ orderingPolicy.getSchedulableEntities()) {
|
|
|
apps.add(app.getApplicationAttemptId());
|
|
|
}
|
|
|
}
|
|
@@ -1993,6 +2011,19 @@ public class LeafQueue extends AbstractCSQueue {
|
|
|
this.maxApplications = maxApplications;
|
|
|
}
|
|
|
|
|
|
+ public synchronized OrderingPolicy<FiCaSchedulerApp>
|
|
|
+ getOrderingPolicy() {
|
|
|
+ return orderingPolicy;
|
|
|
+ }
|
|
|
+
|
|
|
+ public synchronized void setOrderingPolicy(
|
|
|
+ OrderingPolicy<FiCaSchedulerApp> orderingPolicy) {
|
|
|
+ orderingPolicy.addAllSchedulableEntities(
|
|
|
+ this.orderingPolicy.getSchedulableEntities()
|
|
|
+ );
|
|
|
+ this.orderingPolicy = orderingPolicy;
|
|
|
+ }
|
|
|
+
|
|
|
/*
|
|
|
* Holds shared values used by all applications in
|
|
|
* the queue to calculate headroom on demand
|