|
@@ -43,13 +43,11 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.security.Permission;
|
|
|
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueStateManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerQueueManager;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
|
|
|
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
@@ -231,99 +229,62 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
static CSQueue parseQueue(
|
|
|
- CapacitySchedulerQueueContext queueContext,
|
|
|
- CapacitySchedulerConfiguration conf,
|
|
|
- CSQueue parent, String queueName,
|
|
|
- CSQueueStore newQueues,
|
|
|
- CSQueueStore oldQueues,
|
|
|
+ CapacitySchedulerQueueContext queueContext, CapacitySchedulerConfiguration conf,
|
|
|
+ CSQueue parent, String queueName, CSQueueStore newQueues, CSQueueStore oldQueues,
|
|
|
QueueHook hook) throws IOException {
|
|
|
CSQueue queue;
|
|
|
- String fullQueueName = (parent == null) ?
|
|
|
- queueName :
|
|
|
- (parent.getQueuePath() + "." + queueName);
|
|
|
+ String fullQueueName = (parent == null) ? queueName :
|
|
|
+ (QueuePath.createFromQueues(parent.getQueuePath(), queueName).getFullPath());
|
|
|
String[] staticChildQueueNames = conf.getQueues(fullQueueName);
|
|
|
List<String> childQueueNames = staticChildQueueNames != null ?
|
|
|
Arrays.asList(staticChildQueueNames) : Collections.emptyList();
|
|
|
-
|
|
|
- boolean isReservableQueue = conf.isReservable(fullQueueName);
|
|
|
- boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(
|
|
|
- fullQueueName);
|
|
|
- // if a queue is eligible for auto queue creation v2
|
|
|
- // it must be a ParentQueue (even if it is empty)
|
|
|
- boolean isAutoQueueCreationV2Enabled = conf.isAutoQueueCreationV2Enabled(
|
|
|
- fullQueueName);
|
|
|
- boolean isDynamicParent = false;
|
|
|
-
|
|
|
- // Auto created parent queues might not have static children, but they
|
|
|
- // must be kept as a ParentQueue
|
|
|
CSQueue oldQueue = oldQueues.get(fullQueueName);
|
|
|
- if (oldQueue instanceof ParentQueue) {
|
|
|
- isDynamicParent = ((ParentQueue) oldQueue).isDynamicQueue();
|
|
|
- }
|
|
|
|
|
|
- if (childQueueNames.size() == 0 && !isDynamicParent &&
|
|
|
- !isAutoQueueCreationV2Enabled) {
|
|
|
- if (null == parent) {
|
|
|
- throw new IllegalStateException(
|
|
|
- "Queue configuration missing child queue names for " + queueName);
|
|
|
- }
|
|
|
- // Check if the queue will be dynamically managed by the Reservation
|
|
|
- // system
|
|
|
+ boolean isReservableQueue = conf.isReservable(fullQueueName);
|
|
|
+ boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(fullQueueName);
|
|
|
+ // if a queue is eligible for auto queue creation v2 it must be a ParentQueue
|
|
|
+ // (even if it is empty)
|
|
|
+ final boolean isDynamicParent = oldQueue instanceof ParentQueue && oldQueue.isDynamicQueue();
|
|
|
+ boolean isAutoQueueCreationEnabledParent = isDynamicParent || conf.isAutoQueueCreationV2Enabled(
|
|
|
+ fullQueueName) || isAutoCreateEnabled;
|
|
|
+
|
|
|
+ if (childQueueNames.size() == 0 && !isAutoQueueCreationEnabledParent) {
|
|
|
+ validateParent(parent, queueName);
|
|
|
+ // Check if the queue will be dynamically managed by the Reservation system
|
|
|
if (isReservableQueue) {
|
|
|
- queue = new PlanQueue(queueContext, queueName, parent,
|
|
|
- oldQueues.get(fullQueueName));
|
|
|
-
|
|
|
- //initializing the "internal" default queue, for SLS compatibility
|
|
|
- String defReservationId =
|
|
|
- queueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
|
|
|
-
|
|
|
- List<CSQueue> childQueues = new ArrayList<>();
|
|
|
- ReservationQueue resQueue = new ReservationQueue(queueContext,
|
|
|
- defReservationId, (PlanQueue) queue);
|
|
|
- try {
|
|
|
- resQueue.setEntitlement(new QueueEntitlement(1.0f, 1.0f));
|
|
|
- } catch (SchedulerDynamicEditException e) {
|
|
|
- throw new IllegalStateException(e);
|
|
|
- }
|
|
|
- childQueues.add(resQueue);
|
|
|
- ((PlanQueue) queue).setChildQueues(childQueues);
|
|
|
- newQueues.add(resQueue);
|
|
|
-
|
|
|
- } else if (isAutoCreateEnabled) {
|
|
|
- queue = new ManagedParentQueue(queueContext, queueName, parent,
|
|
|
- oldQueues.get(fullQueueName));
|
|
|
-
|
|
|
- } else{
|
|
|
- queue = new LeafQueue(queueContext, queueName, parent,
|
|
|
- oldQueues.get(fullQueueName));
|
|
|
- // Used only for unit tests
|
|
|
- queue = hook.hook(queue);
|
|
|
+ queue = new PlanQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName));
|
|
|
+ ReservationQueue defaultResQueue = ((PlanQueue) queue).initializeDefaultInternalQueue();
|
|
|
+ newQueues.add(defaultResQueue);
|
|
|
+ } else {
|
|
|
+ queue = new LeafQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName));
|
|
|
}
|
|
|
- } else{
|
|
|
+
|
|
|
+ queue = hook.hook(queue);
|
|
|
+ } else {
|
|
|
if (isReservableQueue) {
|
|
|
- throw new IllegalStateException(
|
|
|
- "Only Leaf Queues can be reservable for " + fullQueueName);
|
|
|
+ throw new IllegalStateException("Only Leaf Queues can be reservable for " + fullQueueName);
|
|
|
}
|
|
|
|
|
|
ParentQueue parentQueue;
|
|
|
if (isAutoCreateEnabled) {
|
|
|
- parentQueue = new ManagedParentQueue(queueContext, queueName, parent,
|
|
|
- oldQueues.get(fullQueueName));
|
|
|
- } else{
|
|
|
- parentQueue = new ParentQueue(queueContext, queueName, parent,
|
|
|
- oldQueues.get(fullQueueName));
|
|
|
+ parentQueue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get(
|
|
|
+ fullQueueName));
|
|
|
+ } else {
|
|
|
+ parentQueue = new ParentQueue(queueContext, queueName, parent, oldQueues.get(
|
|
|
+ fullQueueName));
|
|
|
}
|
|
|
|
|
|
- // Used only for unit tests
|
|
|
queue = hook.hook(parentQueue);
|
|
|
-
|
|
|
List<CSQueue> childQueues = new ArrayList<>();
|
|
|
for (String childQueueName : childQueueNames) {
|
|
|
- CSQueue childQueue = parseQueue(queueContext, conf, queue, childQueueName,
|
|
|
- newQueues, oldQueues, hook);
|
|
|
+ CSQueue childQueue = parseQueue(queueContext, conf, queue, childQueueName, newQueues,
|
|
|
+ oldQueues, hook);
|
|
|
childQueues.add(childQueue);
|
|
|
}
|
|
|
- parentQueue.setChildQueues(childQueues);
|
|
|
+
|
|
|
+ if (!childQueues.isEmpty()) {
|
|
|
+ parentQueue.setChildQueues(childQueues);
|
|
|
+ }
|
|
|
|
|
|
}
|
|
|
|
|
@@ -721,4 +682,11 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
|
|
|
// that existingQueues contain valid dynamic queues.
|
|
|
return !isDynamicQueue(parent);
|
|
|
}
|
|
|
+
|
|
|
+ private static void validateParent(CSQueue parent, String queueName) {
|
|
|
+ if (parent == null) {
|
|
|
+ throw new IllegalStateException("Queue configuration missing child queue names for "
|
|
|
+ + queueName);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|