|
@@ -23,10 +23,13 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
@@ -70,6 +73,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static final int MAXIMUM_DYNAMIC_QUEUE_DEPTH = 2;
|
|
|
private static final QueueHook NOOP = new QueueHook();
|
|
|
private CapacitySchedulerContext csContext;
|
|
|
private final YarnAuthorizationProvider authorizer;
|
|
@@ -437,6 +441,229 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|
|
return this.queueStateManager;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Removes an {@code AutoCreatedLeafQueue} from the manager collection and
|
|
|
+ * from its parent children collection.
|
|
|
+ *
|
|
|
+ * @param queueName queue to be removed
|
|
|
+ * @throws SchedulerDynamicEditException if queue is not eligible for deletion
|
|
|
+ */
|
|
|
+ public void removeLegacyDynamicQueue(String queueName)
|
|
|
+ throws SchedulerDynamicEditException {
|
|
|
+ LOG.info("Removing queue: " + queueName);
|
|
|
+ CSQueue q = this.getQueue(queueName);
|
|
|
+ if (q == null || !(AbstractAutoCreatedLeafQueue.class.isAssignableFrom(
|
|
|
+ q.getClass()))) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "The queue that we are asked " + "to remove (" + queueName
|
|
|
+ + ") is not a AutoCreatedLeafQueue or ReservationQueue");
|
|
|
+ }
|
|
|
+ AbstractAutoCreatedLeafQueue disposableLeafQueue =
|
|
|
+ (AbstractAutoCreatedLeafQueue) 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.getPendingApplications().size()
|
|
|
+ + " pending apps");
|
|
|
+ }
|
|
|
+
|
|
|
+ ((AbstractManagedParentQueue) disposableLeafQueue.getParent())
|
|
|
+ .removeChildQueue(q);
|
|
|
+ removeQueue(queueName);
|
|
|
+ LOG.info(
|
|
|
+ "Removal of AutoCreatedLeafQueue " + queueName + " has succeeded");
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Adds an {@code AutoCreatedLeafQueue} to the manager collection and extends
|
|
|
+ * the children collection of its parent.
|
|
|
+ *
|
|
|
+ * @param queue to be added
|
|
|
+ * @throws SchedulerDynamicEditException if queue is not eligible to be added
|
|
|
+ * @throws IOException if parent can not accept the queue
|
|
|
+ */
|
|
|
+ public void addLegacyDynamicQueue(Queue queue)
|
|
|
+ throws SchedulerDynamicEditException, IOException {
|
|
|
+ if (queue == null) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Queue specified is null. Should be an implementation of "
|
|
|
+ + "AbstractAutoCreatedLeafQueue");
|
|
|
+ } else if (!(AbstractAutoCreatedLeafQueue.class
|
|
|
+ .isAssignableFrom(queue.getClass()))) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Queue is not an implementation of "
|
|
|
+ + "AbstractAutoCreatedLeafQueue : " + queue.getClass());
|
|
|
+ }
|
|
|
+
|
|
|
+ AbstractAutoCreatedLeafQueue newQueue =
|
|
|
+ (AbstractAutoCreatedLeafQueue) queue;
|
|
|
+
|
|
|
+ if (newQueue.getParent() == null || !(AbstractManagedParentQueue.class.
|
|
|
+ isAssignableFrom(newQueue.getParent().getClass()))) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "ParentQueue for " + newQueue + " is not properly set"
|
|
|
+ + " (should be set and be a PlanQueue or ManagedParentQueue)");
|
|
|
+ }
|
|
|
+
|
|
|
+ AbstractManagedParentQueue parent =
|
|
|
+ (AbstractManagedParentQueue) newQueue.getParent();
|
|
|
+ String queuePath = newQueue.getQueuePath();
|
|
|
+ parent.addChildQueue(newQueue);
|
|
|
+ addQueue(queuePath, newQueue);
|
|
|
+
|
|
|
+ LOG.info("Creation of AutoCreatedLeafQueue " + newQueue + " succeeded");
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Auto creates a LeafQueue and its upper hierarchy given a path at runtime.
|
|
|
+ *
|
|
|
+ * @param queue the application placement information of the queue
|
|
|
+ * @return the auto created LeafQueue
|
|
|
+ * @throws YarnException if the given path is not eligible to be auto created
|
|
|
+ * @throws IOException if the given path can not be added to the parent
|
|
|
+ */
|
|
|
+ public LeafQueue createQueue(ApplicationPlacementContext queue)
|
|
|
+ throws YarnException, IOException {
|
|
|
+ String leafQueueName = queue.getQueue();
|
|
|
+ String parentQueueName = queue.getParentQueue();
|
|
|
+
|
|
|
+ if (!StringUtils.isEmpty(parentQueueName)) {
|
|
|
+ CSQueue parentQueue = getQueue(parentQueueName);
|
|
|
+
|
|
|
+ if (parentQueue != null && csContext.getConfiguration()
|
|
|
+ .isAutoCreateChildQueueEnabled(parentQueue.getQueuePath())) {
|
|
|
+ return createLegacyAutoQueue(queue);
|
|
|
+ } else {
|
|
|
+ return createAutoQueue(queue);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Could not auto-create leaf queue for " + leafQueueName
|
|
|
+ + ". Queue mapping does not specify"
|
|
|
+ + " which parent queue it needs to be created under.");
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Determines the missing parent paths of a potentially auto creatable queue.
|
|
|
+ * The missing parents are sorted in a way that the first item is the highest
|
|
|
+ * in the hierarchy.
|
|
|
+ * Example:
|
|
|
+ * root.a, root.a.b, root.a.b.c
|
|
|
+ *
|
|
|
+ * @param queue to be auto created
|
|
|
+ * @return missing parent paths
|
|
|
+ * @throws SchedulerDynamicEditException if the given queue is not eligible
|
|
|
+ * to be auto created
|
|
|
+ */
|
|
|
+ public List<String> determineMissingParents(
|
|
|
+ ApplicationPlacementContext queue) throws SchedulerDynamicEditException {
|
|
|
+ if (!queue.hasParentQueue()) {
|
|
|
+ throw new SchedulerDynamicEditException("Can not auto create queue "
|
|
|
+ + queue.getFullQueuePath() + " due to missing ParentQueue path.");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Start from the first parent
|
|
|
+ int firstStaticParentDistance = 1;
|
|
|
+
|
|
|
+ StringBuilder parentCandidate = new StringBuilder(queue.getParentQueue());
|
|
|
+ LinkedList<String> parentsToCreate = new LinkedList<>();
|
|
|
+
|
|
|
+ CSQueue firstExistingParent = getQueue(parentCandidate.toString());
|
|
|
+ CSQueue firstExistingStaticParent = firstExistingParent;
|
|
|
+
|
|
|
+ while (isNonStaticParent(firstExistingStaticParent)
|
|
|
+ && parentCandidate.length() != 0) {
|
|
|
+ ++firstStaticParentDistance;
|
|
|
+
|
|
|
+ if (firstStaticParentDistance > MAXIMUM_DYNAMIC_QUEUE_DEPTH) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Could not auto create queue " + queue.getFullQueuePath()
|
|
|
+ + ". The distance of the LeafQueue from the first static " +
|
|
|
+ "ParentQueue is " + firstStaticParentDistance + ", which is " +
|
|
|
+ "above the limit.");
|
|
|
+ }
|
|
|
+
|
|
|
+ if (firstExistingParent == null) {
|
|
|
+ parentsToCreate.addFirst(parentCandidate.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ int lastIndex = parentCandidate.lastIndexOf(".");
|
|
|
+ parentCandidate.setLength(Math.max(lastIndex, 0));
|
|
|
+
|
|
|
+ if (firstExistingParent == null) {
|
|
|
+ firstExistingParent = getQueue(parentCandidate.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ firstExistingStaticParent = getQueue(parentCandidate.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!(firstExistingParent instanceof ParentQueue)) {
|
|
|
+ throw new SchedulerDynamicEditException(
|
|
|
+ "Could not auto create hierarchy of "
|
|
|
+ + queue.getFullQueuePath() + ". Queue " + queue.getParentQueue() +
|
|
|
+ " is not a ParentQueue."
|
|
|
+ );
|
|
|
+ }
|
|
|
+
|
|
|
+ ParentQueue existingParentQueue = (ParentQueue) firstExistingParent;
|
|
|
+
|
|
|
+ if (!existingParentQueue.isEligibleForAutoQueueCreation()) {
|
|
|
+ throw new SchedulerDynamicEditException("Auto creation of queue " +
|
|
|
+ queue.getFullQueuePath() + " is not enabled under parent "
|
|
|
+ + existingParentQueue.getQueuePath());
|
|
|
+ }
|
|
|
+
|
|
|
+ return parentsToCreate;
|
|
|
+ }
|
|
|
+
|
|
|
+ private LeafQueue createAutoQueue(ApplicationPlacementContext queue)
|
|
|
+ throws SchedulerDynamicEditException {
|
|
|
+ List<String> parentsToCreate = determineMissingParents(queue);
|
|
|
+ // First existing parent is either the parent of the last missing parent
|
|
|
+ // or the parent of the given path
|
|
|
+ String existingParentName = queue.getParentQueue();
|
|
|
+ if (!parentsToCreate.isEmpty()) {
|
|
|
+ existingParentName = parentsToCreate.get(0).substring(
|
|
|
+ 0, parentsToCreate.get(0).lastIndexOf("."));
|
|
|
+ }
|
|
|
+
|
|
|
+ ParentQueue existingParentQueue = (ParentQueue) getQueue(
|
|
|
+ existingParentName);
|
|
|
+
|
|
|
+ for (String current : parentsToCreate) {
|
|
|
+ existingParentQueue = existingParentQueue.addDynamicParentQueue(current);
|
|
|
+ addQueue(existingParentQueue.getQueuePath(), existingParentQueue);
|
|
|
+ }
|
|
|
+
|
|
|
+ LeafQueue leafQueue = existingParentQueue.addDynamicLeafQueue(
|
|
|
+ queue.getFullQueuePath());
|
|
|
+ addQueue(leafQueue.getQueuePath(), leafQueue);
|
|
|
+
|
|
|
+ return leafQueue;
|
|
|
+ }
|
|
|
+
|
|
|
+ private LeafQueue createLegacyAutoQueue(ApplicationPlacementContext queue)
|
|
|
+ throws IOException, SchedulerDynamicEditException {
|
|
|
+ CSQueue parentQueue = getQueue(queue.getParentQueue());
|
|
|
+ // Case 1: Handle ManagedParentQueue
|
|
|
+ ManagedParentQueue autoCreateEnabledParentQueue =
|
|
|
+ (ManagedParentQueue) parentQueue;
|
|
|
+ AutoCreatedLeafQueue autoCreatedLeafQueue =
|
|
|
+ new AutoCreatedLeafQueue(
|
|
|
+ csContext, queue.getQueue(), autoCreateEnabledParentQueue);
|
|
|
+
|
|
|
+ addLegacyDynamicQueue(autoCreatedLeafQueue);
|
|
|
+ return autoCreatedLeafQueue;
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean isNonStaticParent(CSQueue queue) {
|
|
|
+ return (!(queue instanceof AbstractCSQueue)
|
|
|
+ || ((AbstractCSQueue) queue).isDynamicQueue());
|
|
|
+ }
|
|
|
+
|
|
|
private boolean isDynamicQueue(CSQueue queue) {
|
|
|
return (queue instanceof AbstractCSQueue) &&
|
|
|
((AbstractCSQueue) queue).isDynamicQueue();
|