|
@@ -24,6 +24,8 @@ import java.util.*;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
+import java.util.HashSet;
|
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -90,6 +92,11 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
|
|
|
+
|
|
@LimitedPrivate("yarn")
|
|
@LimitedPrivate("yarn")
|
|
@Evolving
|
|
@Evolving
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@@ -473,9 +480,12 @@ public class CapacityScheduler extends
|
|
private void validateExistingQueues(
|
|
private void validateExistingQueues(
|
|
Map<String, CSQueue> queues, Map<String, CSQueue> newQueues)
|
|
Map<String, CSQueue> queues, Map<String, CSQueue> newQueues)
|
|
throws IOException {
|
|
throws IOException {
|
|
- for (String queue : queues.keySet()) {
|
|
|
|
- if (!newQueues.containsKey(queue)) {
|
|
|
|
- throw new IOException(queue + " cannot be found during refresh!");
|
|
|
|
|
|
+ // check that all static queues are included in the newQueues list
|
|
|
|
+ for (Map.Entry<String, CSQueue> e : queues.entrySet()) {
|
|
|
|
+ if (!(e.getValue() instanceof ReservationQueue)) {
|
|
|
|
+ if (!newQueues.containsKey(e.getKey())) {
|
|
|
|
+ throw new IOException(e.getKey() + " cannot be found during refresh!");
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -507,26 +517,42 @@ public class CapacityScheduler extends
|
|
Map<String, CSQueue> oldQueues,
|
|
Map<String, CSQueue> oldQueues,
|
|
QueueHook hook) throws IOException {
|
|
QueueHook hook) throws IOException {
|
|
CSQueue queue;
|
|
CSQueue queue;
|
|
|
|
+ String fullQueueName =
|
|
|
|
+ (parent == null) ? queueName
|
|
|
|
+ : (parent.getQueuePath() + "." + queueName);
|
|
String[] childQueueNames =
|
|
String[] childQueueNames =
|
|
- conf.getQueues((parent == null) ?
|
|
|
|
- queueName : (parent.getQueuePath()+"."+queueName));
|
|
|
|
|
|
+ conf.getQueues(fullQueueName);
|
|
|
|
+ boolean isReservableQueue = conf.isReservableQueue(fullQueueName);
|
|
if (childQueueNames == null || childQueueNames.length == 0) {
|
|
if (childQueueNames == null || childQueueNames.length == 0) {
|
|
if (null == parent) {
|
|
if (null == parent) {
|
|
throw new IllegalStateException(
|
|
throw new IllegalStateException(
|
|
"Queue configuration missing child queue names for " + queueName);
|
|
"Queue configuration missing child queue names for " + queueName);
|
|
}
|
|
}
|
|
- queue =
|
|
|
|
- new LeafQueue(csContext, queueName, parent,oldQueues.get(queueName));
|
|
|
|
-
|
|
|
|
- // Used only for unit tests
|
|
|
|
- queue = hook.hook(queue);
|
|
|
|
|
|
+ // Check if the queue will be dynamically managed by the Reservation
|
|
|
|
+ // system
|
|
|
|
+ if (isReservableQueue) {
|
|
|
|
+ queue =
|
|
|
|
+ new PlanQueue(csContext, queueName, parent,
|
|
|
|
+ oldQueues.get(queueName));
|
|
|
|
+ } else {
|
|
|
|
+ queue =
|
|
|
|
+ new LeafQueue(csContext, queueName, parent,
|
|
|
|
+ oldQueues.get(queueName));
|
|
|
|
+
|
|
|
|
+ // Used only for unit tests
|
|
|
|
+ queue = hook.hook(queue);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
|
|
+ if (isReservableQueue) {
|
|
|
|
+ throw new IllegalStateException(
|
|
|
|
+ "Only Leaf Queues can be reservable for " + queueName);
|
|
|
|
+ }
|
|
ParentQueue parentQueue =
|
|
ParentQueue parentQueue =
|
|
new ParentQueue(csContext, queueName, parent,oldQueues.get(queueName));
|
|
new ParentQueue(csContext, queueName, parent,oldQueues.get(queueName));
|
|
|
|
|
|
// Used only for unit tests
|
|
// Used only for unit tests
|
|
queue = hook.hook(parentQueue);
|
|
queue = hook.hook(parentQueue);
|
|
-
|
|
|
|
|
|
+
|
|
List<CSQueue> childQueues = new ArrayList<CSQueue>();
|
|
List<CSQueue> childQueues = new ArrayList<CSQueue>();
|
|
for (String childQueueName : childQueueNames) {
|
|
for (String childQueueName : childQueueNames) {
|
|
CSQueue childQueue =
|
|
CSQueue childQueue =
|
|
@@ -548,7 +574,7 @@ public class CapacityScheduler extends
|
|
return queue;
|
|
return queue;
|
|
}
|
|
}
|
|
|
|
|
|
- synchronized CSQueue getQueue(String queueName) {
|
|
|
|
|
|
+ public synchronized CSQueue getQueue(String queueName) {
|
|
if (queueName == null) {
|
|
if (queueName == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -716,7 +742,7 @@ public class CapacityScheduler extends
|
|
ApplicationAttemptId applicationAttemptId,
|
|
ApplicationAttemptId applicationAttemptId,
|
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
|
LOG.info("Application Attempt " + applicationAttemptId + " is done." +
|
|
LOG.info("Application Attempt " + applicationAttemptId + " is done." +
|
|
- " finalState=" + rmAppAttemptFinalState);
|
|
|
|
|
|
+ " finalState=" + rmAppAttemptFinalState);
|
|
|
|
|
|
FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
|
|
FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
SchedulerApplication<FiCaSchedulerApp> application =
|
|
@@ -995,9 +1021,16 @@ public class CapacityScheduler extends
|
|
case APP_ADDED:
|
|
case APP_ADDED:
|
|
{
|
|
{
|
|
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
|
|
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
|
|
- addApplication(appAddedEvent.getApplicationId(),
|
|
|
|
- appAddedEvent.getQueue(),
|
|
|
|
- appAddedEvent.getUser(), appAddedEvent.getIsAppRecovering());
|
|
|
|
|
|
+ String queueName =
|
|
|
|
+ resolveReservationQueueName(appAddedEvent.getQueue(),
|
|
|
|
+ appAddedEvent.getApplicationId(),
|
|
|
|
+ appAddedEvent.getReservationID());
|
|
|
|
+ if (queueName != null) {
|
|
|
|
+ addApplication(appAddedEvent.getApplicationId(),
|
|
|
|
+ queueName,
|
|
|
|
+ appAddedEvent.getUser(),
|
|
|
|
+ appAddedEvent.getIsAppRecovering());
|
|
|
|
+ }
|
|
}
|
|
}
|
|
break;
|
|
break;
|
|
case APP_REMOVED:
|
|
case APP_REMOVED:
|
|
@@ -1230,6 +1263,123 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private synchronized String resolveReservationQueueName(String queueName,
|
|
|
|
+ ApplicationId applicationId, ReservationId reservationID) {
|
|
|
|
+ CSQueue queue = getQueue(queueName);
|
|
|
|
+ // Check if the queue is a plan queue
|
|
|
|
+ if ((queue == null) || !(queue instanceof PlanQueue)) {
|
|
|
|
+ return queueName;
|
|
|
|
+ }
|
|
|
|
+ if (reservationID != null) {
|
|
|
|
+ String resQName = reservationID.toString();
|
|
|
|
+ queue = getQueue(resQName);
|
|
|
|
+ if (queue == null) {
|
|
|
|
+ String message =
|
|
|
|
+ "Application "
|
|
|
|
+ + applicationId
|
|
|
|
+ + " submitted to a reservation which is not yet currently active: "
|
|
|
|
+ + resQName;
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler()
|
|
|
|
+ .handle(new RMAppRejectedEvent(applicationId, message));
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ // use the reservation queue to run the app
|
|
|
|
+ queueName = resQName;
|
|
|
|
+ } else {
|
|
|
|
+ // use the default child queue of the plan for unreserved apps
|
|
|
|
+ queueName = queueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
|
|
|
|
+ }
|
|
|
|
+ return queueName;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public synchronized void removeQueue(String queueName)
|
|
|
|
+ throws SchedulerDynamicEditException {
|
|
|
|
+ LOG.info("Removing queue: " + queueName);
|
|
|
|
+ CSQueue q = this.getQueue(queueName);
|
|
|
|
+ if (!(q instanceof ReservationQueue)) {
|
|
|
|
+ throw new SchedulerDynamicEditException("The queue that we are asked "
|
|
|
|
+ + "to remove (" + queueName + ") is not a ReservationQueue");
|
|
|
|
+ }
|
|
|
|
+ ReservationQueue disposableLeafQueue = (ReservationQueue) q;
|
|
|
|
+ // at this point we should have no more apps
|
|
|
|
+ if (disposableLeafQueue.getNumApplications() > 0) {
|
|
|
|
+ throw new SchedulerDynamicEditException("The queue " + queueName
|
|
|
|
+ + " is not empty " + disposableLeafQueue.getApplications().size()
|
|
|
|
+ + " active apps " + disposableLeafQueue.pendingApplications.size()
|
|
|
|
+ + " pending apps");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ ((PlanQueue) disposableLeafQueue.getParent()).removeChildQueue(q);
|
|
|
|
+ this.queues.remove(queueName);
|
|
|
|
+ LOG.info("Removal of ReservationQueue " + queueName + " has succeeded");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public synchronized void addQueue(Queue queue)
|
|
|
|
+ throws SchedulerDynamicEditException {
|
|
|
|
+
|
|
|
|
+ if (!(queue instanceof ReservationQueue)) {
|
|
|
|
+ throw new SchedulerDynamicEditException("Queue " + queue.getQueueName()
|
|
|
|
+ + " is not a ReservationQueue");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ ReservationQueue newQueue = (ReservationQueue) queue;
|
|
|
|
+
|
|
|
|
+ if (newQueue.getParent() == null
|
|
|
|
+ || !(newQueue.getParent() instanceof PlanQueue)) {
|
|
|
|
+ throw new SchedulerDynamicEditException("ParentQueue for "
|
|
|
|
+ + newQueue.getQueueName()
|
|
|
|
+ + " is not properly set (should be set and be a PlanQueue)");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ PlanQueue parentPlan = (PlanQueue) newQueue.getParent();
|
|
|
|
+ String queuename = newQueue.getQueueName();
|
|
|
|
+ parentPlan.addChildQueue(newQueue);
|
|
|
|
+ this.queues.put(queuename, newQueue);
|
|
|
|
+ LOG.info("Creation of ReservationQueue " + newQueue + " succeeded");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public synchronized void setEntitlement(String inQueue,
|
|
|
|
+ QueueEntitlement entitlement) throws SchedulerDynamicEditException,
|
|
|
|
+ YarnException {
|
|
|
|
+ LeafQueue queue = getAndCheckLeafQueue(inQueue);
|
|
|
|
+ ParentQueue parent = (ParentQueue) queue.getParent();
|
|
|
|
+
|
|
|
|
+ if (!(queue instanceof ReservationQueue)) {
|
|
|
|
+ throw new SchedulerDynamicEditException("Entitlement can not be"
|
|
|
|
+ + " modified dynamically since queue " + inQueue
|
|
|
|
+ + " is not a ReservationQueue");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (!(parent instanceof PlanQueue)) {
|
|
|
|
+ throw new SchedulerDynamicEditException("The parent of ReservationQueue "
|
|
|
|
+ + inQueue + " must be an PlanQueue");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ ReservationQueue newQueue = (ReservationQueue) queue;
|
|
|
|
+
|
|
|
|
+ float sumChilds = ((PlanQueue) parent).sumOfChildCapacities();
|
|
|
|
+ float newChildCap = sumChilds - queue.getCapacity() + entitlement.getCapacity();
|
|
|
|
+
|
|
|
|
+ if (newChildCap >= 0 && newChildCap < 1.0f + CSQueueUtils.EPSILON) {
|
|
|
|
+ // note: epsilon checks here are not ok, as the epsilons might accumulate
|
|
|
|
+ // and become a problem in aggregate
|
|
|
|
+ if (Math.abs(entitlement.getCapacity() - queue.getCapacity()) == 0
|
|
|
|
+ && Math.abs(entitlement.getMaxCapacity() - queue.getMaximumCapacity()) == 0) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ newQueue.setEntitlement(entitlement);
|
|
|
|
+ } else {
|
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
|
+ "Sum of child queues would exceed 100% for PlanQueue: "
|
|
|
|
+ + parent.getQueueName());
|
|
|
|
+ }
|
|
|
|
+ LOG.info("Set entitlement for ReservationQueue " + inQueue + " to "
|
|
|
|
+ + queue.getCapacity() + " request was (" + entitlement.getCapacity() + ")");
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public synchronized String moveApplication(ApplicationId appId,
|
|
public synchronized String moveApplication(ApplicationId appId,
|
|
String targetQueueName) throws YarnException {
|
|
String targetQueueName) throws YarnException {
|
|
@@ -1237,11 +1387,12 @@ public class CapacityScheduler extends
|
|
getApplicationAttempt(ApplicationAttemptId.newInstance(appId, 0));
|
|
getApplicationAttempt(ApplicationAttemptId.newInstance(appId, 0));
|
|
String sourceQueueName = app.getQueue().getQueueName();
|
|
String sourceQueueName = app.getQueue().getQueueName();
|
|
LeafQueue source = getAndCheckLeafQueue(sourceQueueName);
|
|
LeafQueue source = getAndCheckLeafQueue(sourceQueueName);
|
|
- LeafQueue dest = getAndCheckLeafQueue(targetQueueName);
|
|
|
|
|
|
+ String destQueueName = handleMoveToPlanQueue(targetQueueName);
|
|
|
|
+ LeafQueue dest = getAndCheckLeafQueue(destQueueName);
|
|
// Validation check - ACLs, submission limits for user & queue
|
|
// Validation check - ACLs, submission limits for user & queue
|
|
String user = app.getUser();
|
|
String user = app.getUser();
|
|
try {
|
|
try {
|
|
- dest.submitApplication(appId, user, targetQueueName);
|
|
|
|
|
|
+ dest.submitApplication(appId, user, destQueueName);
|
|
} catch (AccessControlException e) {
|
|
} catch (AccessControlException e) {
|
|
throw new YarnException(e);
|
|
throw new YarnException(e);
|
|
}
|
|
}
|
|
@@ -1260,7 +1411,7 @@ public class CapacityScheduler extends
|
|
dest.submitApplicationAttempt(app, user);
|
|
dest.submitApplicationAttempt(app, user);
|
|
applications.get(appId).setQueue(dest);
|
|
applications.get(appId).setQueue(dest);
|
|
LOG.info("App: " + app.getApplicationId() + " successfully moved from "
|
|
LOG.info("App: " + app.getApplicationId() + " successfully moved from "
|
|
- + sourceQueueName + " to: " + targetQueueName);
|
|
|
|
|
|
+ + sourceQueueName + " to: " + destQueueName);
|
|
return targetQueueName;
|
|
return targetQueueName;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1295,4 +1446,24 @@ public class CapacityScheduler extends
|
|
return EnumSet
|
|
return EnumSet
|
|
.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU);
|
|
.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private String handleMoveToPlanQueue(String targetQueueName) {
|
|
|
|
+ CSQueue dest = getQueue(targetQueueName);
|
|
|
|
+ if (dest != null && dest instanceof PlanQueue) {
|
|
|
|
+ // use the default child reservation queue of the plan
|
|
|
|
+ targetQueueName = targetQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
|
|
|
|
+ }
|
|
|
|
+ return targetQueueName;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public Set<String> getPlanQueues() {
|
|
|
|
+ Set<String> ret = new HashSet<String>();
|
|
|
|
+ for (Map.Entry<String, CSQueue> l : queues.entrySet()) {
|
|
|
|
+ if (l.getValue() instanceof PlanQueue) {
|
|
|
|
+ ret.add(l.getKey());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return ret;
|
|
|
|
+ }
|
|
}
|
|
}
|