|
@@ -27,7 +27,10 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
@@ -97,6 +100,12 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
|
|
|
private final boolean allowZeroCapacitySum;
|
|
|
|
|
|
+ // effective min ratio per resource, it is used during updateClusterResource,
|
|
|
+ // leaf queue can use this to calculate effective resources.
|
|
|
+ // This field will not be edited, reference will point to a new immutable map
|
|
|
+ // after every time recalculation
|
|
|
+ private volatile Map<String, Float> effectiveMinRatioPerResource;
|
|
|
+
|
|
|
public ParentQueue(CapacitySchedulerContext cs,
|
|
|
String queueName, CSQueue parent, CSQueue old) throws IOException {
|
|
|
super(cs, queueName, parent, old);
|
|
@@ -172,117 +181,199 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
|
|
|
private static float PRECISION = 0.0005f; // 0.05% precision
|
|
|
|
|
|
- void setChildQueues(Collection<CSQueue> childQueues) {
|
|
|
- writeLock.lock();
|
|
|
- try {
|
|
|
- // Validate
|
|
|
- float childCapacities = 0;
|
|
|
- Resource minResDefaultLabel = Resources.createResource(0, 0);
|
|
|
- for (CSQueue queue : childQueues) {
|
|
|
- childCapacities += queue.getCapacity();
|
|
|
- Resources.addTo(minResDefaultLabel, queue.getQueueResourceQuotas()
|
|
|
- .getConfiguredMinResource());
|
|
|
-
|
|
|
- // If any child queue is using percentage based capacity model vs parent
|
|
|
- // queues' absolute configuration or vice versa, throw back an
|
|
|
- // exception.
|
|
|
- if (!queueName.equals("root") && getCapacity() != 0f
|
|
|
- && !queue.getQueueResourceQuotas().getConfiguredMinResource()
|
|
|
- .equals(Resources.none())) {
|
|
|
- throw new IllegalArgumentException("Parent queue '" + getQueuePath()
|
|
|
- + "' and child queue '" + queue.getQueuePath()
|
|
|
- + "' should use either percentage based capacity"
|
|
|
- + " configuration or absolute resource together.");
|
|
|
- }
|
|
|
- }
|
|
|
+ // Check weight configuration, throw exception when configuration is invalid
|
|
|
+ // return true when all children use weight mode.
|
|
|
+ private QueueCapacityType getCapacityConfigurationTypeForQueues(
|
|
|
+ Collection<CSQueue> queues) throws IOException {
|
|
|
+ // Do we have ANY queue set capacity in any labels?
|
|
|
+ boolean percentageIsSet = false;
|
|
|
|
|
|
- float delta = Math.abs(1.0f - childCapacities); // crude way to check
|
|
|
-
|
|
|
- if (allowZeroCapacitySum) {
|
|
|
- // If we allow zero capacity for children, only fail if:
|
|
|
- // Σ(childCapacities) != 1.0f OR Σ(childCapacities) != 0.0f
|
|
|
- //
|
|
|
- // Therefore, child queues either add up to 0% or 100%.
|
|
|
- //
|
|
|
- // Current capacity doesn't matter, because we apply this logic
|
|
|
- // regardless of whether the current capacity is zero or not.
|
|
|
- if (minResDefaultLabel.equals(Resources.none())
|
|
|
- && (delta > PRECISION && childCapacities > PRECISION)) {
|
|
|
- LOG.error("Capacity validation check is relaxed for"
|
|
|
- + " queue {}, but the capacity must be either 0% or 100%",
|
|
|
- getQueuePath());
|
|
|
- throw new IllegalArgumentException("Illegal" + " capacity of "
|
|
|
- + childCapacities + " for children of queue " + queueName);
|
|
|
+ // Do we have ANY queue set weight in any labels?
|
|
|
+ boolean weightIsSet = false;
|
|
|
+
|
|
|
+ // Do we have ANY queue set absolute in any labels?
|
|
|
+ boolean absoluteMinResSet = false;
|
|
|
+
|
|
|
+ StringBuilder diagMsg = new StringBuilder();
|
|
|
+
|
|
|
+ for (CSQueue queue : queues) {
|
|
|
+ for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
+ float capacityByLabel = queue.getQueueCapacities().getCapacity(nodeLabel);
|
|
|
+ if (capacityByLabel > 0) {
|
|
|
+ percentageIsSet = true;
|
|
|
+ }
|
|
|
+ float weightByLabel = queue.getQueueCapacities().getWeight(nodeLabel);
|
|
|
+ // By default weight is set to -1, so >= 0 is enough.
|
|
|
+ if (weightByLabel >= 0) {
|
|
|
+ weightIsSet = true;
|
|
|
+ diagMsg.append(
|
|
|
+ "{Queue=" + queue.getQueuePath() + ", label=" + nodeLabel
|
|
|
+ + " uses weight mode}. ");
|
|
|
+ }
|
|
|
+ if (!queue.getQueueResourceQuotas().getConfiguredMinResource(nodeLabel)
|
|
|
+ .equals(Resources.none())) {
|
|
|
+ absoluteMinResSet = true;
|
|
|
+ // There's a special handling: when absolute resource is configured,
|
|
|
+ // capacity will be calculated (and set) for UI/metrics purposes, so
|
|
|
+ // when asboluteMinResource is set, unset percentage
|
|
|
+ percentageIsSet = false;
|
|
|
+ diagMsg.append(
|
|
|
+ "{Queue=" + queue.getQueuePath() + ", label=" + nodeLabel
|
|
|
+ + " uses absolute mode}. ");
|
|
|
+ }
|
|
|
+ if (percentageIsSet) {
|
|
|
+ diagMsg.append(
|
|
|
+ "{Queue=" + queue.getQueuePath() + ", label=" + nodeLabel
|
|
|
+ + " uses percentage mode}. ");
|
|
|
}
|
|
|
- } else if ((minResDefaultLabel.equals(Resources.none())
|
|
|
- && (queueCapacities.getCapacity() > 0) && (delta > PRECISION))
|
|
|
- || ((queueCapacities.getCapacity() == 0) && (childCapacities > 0))) {
|
|
|
- // allow capacities being set to 0, and enforce child 0 if parent is 0
|
|
|
- throw new IllegalArgumentException("Illegal" + " capacity of "
|
|
|
- + childCapacities + " for children of queue " + queueName);
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- // check label capacities
|
|
|
- for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
- float capacityByLabel = queueCapacities.getCapacity(nodeLabel);
|
|
|
- // check children's labels
|
|
|
- float sum = 0;
|
|
|
- Resource minRes = Resources.createResource(0, 0);
|
|
|
- Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel,
|
|
|
- scheduler.getClusterResource());
|
|
|
- for (CSQueue queue : childQueues) {
|
|
|
- sum += queue.getQueueCapacities().getCapacity(nodeLabel);
|
|
|
-
|
|
|
- // If any child queue of a label is using percentage based capacity
|
|
|
- // model vs parent queues' absolute configuration or vice versa, throw
|
|
|
- // back an exception
|
|
|
- if (!queueName.equals("root") && !this.capacityConfigType
|
|
|
- .equals(queue.getCapacityConfigType())) {
|
|
|
- throw new IllegalArgumentException("Parent queue '" + getQueuePath()
|
|
|
- + "' and child queue '" + queue.getQueuePath()
|
|
|
- + "' should use either percentage based capacity"
|
|
|
- + "configuration or absolute resource together for label:"
|
|
|
- + nodeLabel);
|
|
|
- }
|
|
|
+ // If we have mixed capacity, weight or absolute resource (any of the two)
|
|
|
+ // We will throw exception
|
|
|
+ // Root queue is an exception here, because by default root queue returns
|
|
|
+ // 100 as capacity no matter what. We should look into this case in the
|
|
|
+ // future. To avoid impact too many code paths, we don;t check root queue's
|
|
|
+ // config.
|
|
|
+ if (queues.iterator().hasNext() &&
|
|
|
+ !queues.iterator().next().getQueuePath().equals(
|
|
|
+ CapacitySchedulerConfiguration.ROOT) &&
|
|
|
+ (percentageIsSet ? 1 : 0) + (weightIsSet ? 1 : 0) + (absoluteMinResSet ?
|
|
|
+ 1 :
|
|
|
+ 0) > 1) {
|
|
|
+ throw new IOException("Parent queue '" + getQueuePath()
|
|
|
+ + "' have children queue used mixed of "
|
|
|
+ + " weight mode, percentage and absolute mode, it is not allowed, please "
|
|
|
+ + "double check, details:" + diagMsg.toString());
|
|
|
+ }
|
|
|
|
|
|
- // Accumulate all min/max resource configured for all child queues.
|
|
|
- Resources.addTo(minRes, queue.getQueueResourceQuotas()
|
|
|
- .getConfiguredMinResource(nodeLabel));
|
|
|
- }
|
|
|
+ if (weightIsSet) {
|
|
|
+ return QueueCapacityType.WEIGHT;
|
|
|
+ } else if (absoluteMinResSet) {
|
|
|
+ return QueueCapacityType.ABSOLUTE_RESOURCE;
|
|
|
+ } else if (percentageIsSet) {
|
|
|
+ return QueueCapacityType.PERCENT;
|
|
|
+ } else {
|
|
|
+ // When all values equals to 0, consider it is a percent mode.
|
|
|
+ return QueueCapacityType.PERCENT;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- float labelDelta = Math.abs(1.0f - sum);
|
|
|
-
|
|
|
- if (allowZeroCapacitySum) {
|
|
|
- // Similar to above, we only throw exception if
|
|
|
- // Σ(childCapacities) != 1.0f OR Σ(childCapacities) != 0.0f
|
|
|
- if (minResDefaultLabel.equals(Resources.none())
|
|
|
- && capacityByLabel > 0
|
|
|
- && (labelDelta > PRECISION && sum > PRECISION)) {
|
|
|
- LOG.error("Capacity validation check is relaxed for"
|
|
|
- + " queue {}, but the capacity must be either 0% or 100%",
|
|
|
- getQueuePath());
|
|
|
- throw new IllegalArgumentException(
|
|
|
- "Illegal" + " capacity of " + sum + " for children of queue "
|
|
|
- + queueName + " for label=" + nodeLabel);
|
|
|
- }
|
|
|
- } else if ((minResDefaultLabel.equals(Resources.none())
|
|
|
- && capacityByLabel > 0
|
|
|
- && Math.abs(1.0f - sum) > PRECISION)
|
|
|
- || (capacityByLabel == 0) && (sum > 0)) {
|
|
|
- throw new IllegalArgumentException(
|
|
|
- "Illegal" + " capacity of " + sum + " for children of queue "
|
|
|
- + queueName + " for label=" + nodeLabel);
|
|
|
+ private enum QueueCapacityType {
|
|
|
+ WEIGHT, ABSOLUTE_RESOURCE, PERCENT;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Set child queue and verify capacities
|
|
|
+ * +--------------+---------------------------+-------------------------------------+------------------------+
|
|
|
+ * | | parent-weight | parent-pct | parent-abs |
|
|
|
+ * +--------------+---------------------------+-------------------------------------+------------------------+
|
|
|
+ * | child-weight | No specific check | No specific check | X |
|
|
|
+ * +--------------+---------------------------+-------------------------------------+------------------------+
|
|
|
+ * | child-pct | Sum(children.capacity) = | When: | X |
|
|
|
+ * | | 0 OR 100 | parent.capacity>0 | |
|
|
|
+ * | | | sum(children.capacity)=100 OR 0 | |
|
|
|
+ * | | | parent.capacity=0 | |
|
|
|
+ * | | | sum(children.capacity)=0 | |
|
|
|
+ * +--------------+---------------------------+-------------------------------------+------------------------+
|
|
|
+ * | child-abs | X | X | Sum(children.minRes)<= |
|
|
|
+ * | | | | parent.minRes |
|
|
|
+ * +--------------+---------------------------+-------------------------------------+------------------------+
|
|
|
+ * @param childQueues
|
|
|
+ */
|
|
|
+ void setChildQueues(Collection<CSQueue> childQueues) throws IOException {
|
|
|
+ writeLock.lock();
|
|
|
+ try {
|
|
|
+ QueueCapacityType childrenCapacityType =
|
|
|
+ getCapacityConfigurationTypeForQueues(childQueues);
|
|
|
+ QueueCapacityType parentCapacityType =
|
|
|
+ getCapacityConfigurationTypeForQueues(ImmutableList.of(this));
|
|
|
+
|
|
|
+ if (childrenCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE
|
|
|
+ || parentCapacityType == QueueCapacityType.ABSOLUTE_RESOURCE) {
|
|
|
+ // We don't allow any mixed absolute + {weight, percentage} between
|
|
|
+ // children and parent
|
|
|
+ if (childrenCapacityType != parentCapacityType && !this.getQueuePath()
|
|
|
+ .equals(CapacitySchedulerConfiguration.ROOT)) {
|
|
|
+ throw new IOException("Parent=" + this.getQueuePath()
|
|
|
+ + ": When absolute minResource is used, we must make sure both "
|
|
|
+ + "parent and child all use absolute minResource");
|
|
|
}
|
|
|
|
|
|
// Ensure that for each parent queue: parent.min-resource >=
|
|
|
// Σ(child.min-resource).
|
|
|
- Resource parentMinResource = queueResourceQuotas
|
|
|
- .getConfiguredMinResource(nodeLabel);
|
|
|
- if (!parentMinResource.equals(Resources.none()) && Resources.lessThan(
|
|
|
- resourceCalculator, resourceByLabel, parentMinResource, minRes)) {
|
|
|
- throw new IllegalArgumentException("Parent Queues" + " capacity: "
|
|
|
- + parentMinResource + " is less than" + " to its children:"
|
|
|
- + minRes + " for queue:" + queueName);
|
|
|
+ for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
+ Resource minRes = Resources.createResource(0, 0);
|
|
|
+ for (CSQueue queue : childQueues) {
|
|
|
+ // Accumulate all min/max resource configured for all child queues.
|
|
|
+ Resources.addTo(minRes, queue.getQueueResourceQuotas()
|
|
|
+ .getConfiguredMinResource(nodeLabel));
|
|
|
+ }
|
|
|
+ Resource resourceByLabel = labelManager.getResourceByLabel(nodeLabel,
|
|
|
+ scheduler.getClusterResource());
|
|
|
+ Resource parentMinResource =
|
|
|
+ queueResourceQuotas.getConfiguredMinResource(nodeLabel);
|
|
|
+ if (!parentMinResource.equals(Resources.none()) && Resources.lessThan(
|
|
|
+ resourceCalculator, resourceByLabel, parentMinResource, minRes)) {
|
|
|
+ throw new IOException(
|
|
|
+ "Parent Queues" + " capacity: " + parentMinResource
|
|
|
+ + " is less than" + " to its children:" + minRes
|
|
|
+ + " for queue:" + queueName);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // When child uses percent
|
|
|
+ if (childrenCapacityType == QueueCapacityType.PERCENT) {
|
|
|
+ float childrenPctSum = 0;
|
|
|
+ // check label capacities
|
|
|
+ for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
+ // check children's labels
|
|
|
+ childrenPctSum = 0;
|
|
|
+ for (CSQueue queue : childQueues) {
|
|
|
+ childrenPctSum += queue.getQueueCapacities().getCapacity(nodeLabel);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (Math.abs(1 - childrenPctSum) > PRECISION) {
|
|
|
+ // When children's percent sum != 100%
|
|
|
+ if (Math.abs(childrenPctSum) > PRECISION) {
|
|
|
+ // It is wrong when percent sum != {0, 1}
|
|
|
+ throw new IOException(
|
|
|
+ "Illegal" + " capacity sum of " + childrenPctSum
|
|
|
+ + " for children of queue " + queueName + " for label="
|
|
|
+ + nodeLabel + ". It should be either 0 or 1.0");
|
|
|
+ } else{
|
|
|
+ // We also allow children's percent sum = 0 under the following
|
|
|
+ // conditions
|
|
|
+ // - Parent uses weight mode
|
|
|
+ // - Parent uses percent mode, and parent has
|
|
|
+ // (capacity=0 OR allowZero)
|
|
|
+ if (parentCapacityType == QueueCapacityType.PERCENT) {
|
|
|
+ if ((Math.abs(queueCapacities.getCapacity(nodeLabel))
|
|
|
+ > PRECISION) && (!allowZeroCapacitySum)) {
|
|
|
+ throw new IOException(
|
|
|
+ "Illegal" + " capacity sum of " + childrenPctSum
|
|
|
+ + " for children of queue " + queueName
|
|
|
+ + " for label=" + nodeLabel
|
|
|
+ + ". It is set to 0, but parent percent != 0, and "
|
|
|
+ + "doesn't allow children capacity to set to 0");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // Even if child pct sum == 1.0, we will make sure parent has
|
|
|
+ // positive percent.
|
|
|
+ if (parentCapacityType == QueueCapacityType.PERCENT && Math.abs(
|
|
|
+ queueCapacities.getCapacity(nodeLabel)) <= 0f
|
|
|
+ && !allowZeroCapacitySum) {
|
|
|
+ throw new IOException(
|
|
|
+ "Illegal" + " capacity sum of " + childrenPctSum
|
|
|
+ + " for children of queue " + queueName + " for label="
|
|
|
+ + nodeLabel + ". queue=" + queueName
|
|
|
+ + " has zero capacity, but child"
|
|
|
+ + "queues have positive capacities");
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -451,8 +542,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
|
|
|
// Re-sort all queues
|
|
|
- childQueues.clear();
|
|
|
- childQueues.addAll(currentChildQueues.values());
|
|
|
+ setChildQueues(currentChildQueues.values());
|
|
|
|
|
|
// Make sure we notifies QueueOrderingPolicy
|
|
|
queueOrderingPolicy.setQueues(childQueues);
|
|
@@ -788,14 +878,24 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
|
|
|
private ResourceLimits getResourceLimitsOfChild(CSQueue child,
|
|
|
- Resource clusterResource, Resource parentLimits,
|
|
|
- String nodePartition) {
|
|
|
+ Resource clusterResource, ResourceLimits parentLimits,
|
|
|
+ String nodePartition, boolean netLimit) {
|
|
|
// Set resource-limit of a given child, child.limit =
|
|
|
// min(my.limit - my.used + child.used, child.max)
|
|
|
|
|
|
+ // First, cap parent limit by parent's max
|
|
|
+ parentLimits.setLimit(Resources.min(resourceCalculator, clusterResource,
|
|
|
+ parentLimits.getLimit(),
|
|
|
+ queueResourceQuotas.getEffectiveMaxResource(nodePartition)));
|
|
|
+
|
|
|
// Parent available resource = parent-limit - parent-used-resource
|
|
|
+ Resource limit = parentLimits.getLimit();
|
|
|
+ if (netLimit) {
|
|
|
+ limit = parentLimits.getNetLimit();
|
|
|
+ }
|
|
|
Resource parentMaxAvailableResource = Resources.subtract(
|
|
|
- parentLimits, queueUsage.getUsed(nodePartition));
|
|
|
+ limit, queueUsage.getUsed(nodePartition));
|
|
|
+
|
|
|
// Deduct killable from used
|
|
|
Resources.addTo(parentMaxAvailableResource,
|
|
|
getTotalKillableResource(nodePartition));
|
|
@@ -804,15 +904,6 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
Resource childLimit = Resources.add(parentMaxAvailableResource,
|
|
|
child.getQueueResourceUsage().getUsed(nodePartition));
|
|
|
|
|
|
- // Get child's max resource
|
|
|
- Resource childConfiguredMaxResource = child
|
|
|
- .getEffectiveMaxCapacityDown(nodePartition, minimumAllocation);
|
|
|
-
|
|
|
- // Child's limit should be capped by child configured max resource
|
|
|
- childLimit =
|
|
|
- Resources.min(resourceCalculator, clusterResource, childLimit,
|
|
|
- childConfiguredMaxResource);
|
|
|
-
|
|
|
// Normalize before return
|
|
|
childLimit =
|
|
|
Resources.roundDown(resourceCalculator, childLimit, minimumAllocation);
|
|
@@ -841,8 +932,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
|
|
|
// Get ResourceLimits of child queue before assign containers
|
|
|
ResourceLimits childLimits =
|
|
|
- getResourceLimitsOfChild(childQueue, cluster, limits.getNetLimit(),
|
|
|
- candidates.getPartition());
|
|
|
+ getResourceLimitsOfChild(childQueue, cluster, limits,
|
|
|
+ candidates.getPartition(), true);
|
|
|
|
|
|
CSAssignment childAssignment = childQueue.assignContainers(cluster,
|
|
|
candidates, childLimits, schedulingMode);
|
|
@@ -941,6 +1032,40 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
ResourceLimits resourceLimits) {
|
|
|
writeLock.lock();
|
|
|
try {
|
|
|
+ // Special handle root queue
|
|
|
+ if (rootQueue) {
|
|
|
+ for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
+ if (queueCapacities.getWeight(nodeLabel) > 0) {
|
|
|
+ queueCapacities.setNormalizedWeight(nodeLabel, 1f);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Update absolute capacities of this queue, this need to happen before
|
|
|
+ // below calculation for effective capacities
|
|
|
+ updateAbsoluteCapacities();
|
|
|
+
|
|
|
+ // Normalize weight of children
|
|
|
+ if (getCapacityConfigurationTypeForQueues(childQueues)
|
|
|
+ == QueueCapacityType.WEIGHT) {
|
|
|
+ for (String nodeLabel : queueCapacities.getExistingNodeLabels()) {
|
|
|
+ float sumOfWeight = 0;
|
|
|
+ for (CSQueue queue : childQueues) {
|
|
|
+ float weight = Math.max(0,
|
|
|
+ queue.getQueueCapacities().getWeight(nodeLabel));
|
|
|
+ sumOfWeight += weight;
|
|
|
+ }
|
|
|
+ // When sum of weight == 0, skip setting normalized_weight (so
|
|
|
+ // normalized weight will be 0).
|
|
|
+ if (Math.abs(sumOfWeight) > 1e-6) {
|
|
|
+ for (CSQueue queue : childQueues) {
|
|
|
+ queue.getQueueCapacities().setNormalizedWeight(nodeLabel,
|
|
|
+ queue.getQueueCapacities().getWeight(nodeLabel) / sumOfWeight);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
// Update effective capacity in all parent queue.
|
|
|
Set<String> configuredNodelabels = csContext.getConfiguration()
|
|
|
.getConfiguredNodeLabels(getQueuePath());
|
|
@@ -952,8 +1077,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
for (CSQueue childQueue : childQueues) {
|
|
|
// Get ResourceLimits of child queue before assign containers
|
|
|
ResourceLimits childLimits = getResourceLimitsOfChild(childQueue,
|
|
|
- clusterResource, resourceLimits.getLimit(),
|
|
|
- RMNodeLabelsManager.NO_LABEL);
|
|
|
+ clusterResource, resourceLimits,
|
|
|
+ RMNodeLabelsManager.NO_LABEL, false);
|
|
|
childQueue.updateClusterResource(clusterResource, childLimits);
|
|
|
}
|
|
|
|
|
@@ -963,6 +1088,9 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
CSQueueUtils.updateConfiguredCapacityMetrics(resourceCalculator,
|
|
|
labelManager.getResourceByLabel(null, clusterResource),
|
|
|
RMNodeLabelsManager.NO_LABEL, this);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Fatal issue found: e", e);
|
|
|
+ throw new YarnRuntimeException("Fatal issue during scheduling", e);
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
@@ -979,16 +1107,13 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
// cluster resource.
|
|
|
Resource resourceByLabel = labelManager.getResourceByLabel(label,
|
|
|
clusterResource);
|
|
|
- if (getQueuePath().equals("root")) {
|
|
|
- queueResourceQuotas.setConfiguredMinResource(label, resourceByLabel);
|
|
|
- queueResourceQuotas.setConfiguredMaxResource(label, resourceByLabel);
|
|
|
- queueResourceQuotas.setEffectiveMinResource(label, resourceByLabel);
|
|
|
- queueResourceQuotas.setEffectiveMaxResource(label, resourceByLabel);
|
|
|
- queueCapacities.setAbsoluteCapacity(label, 1.0f);
|
|
|
- }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * == Below logic are added to calculate effectiveMinRatioPerResource ==
|
|
|
+ */
|
|
|
|
|
|
// Total configured min resources of direct children of this given parent
|
|
|
- // queue.
|
|
|
+ // queue
|
|
|
Resource configuredMinResources = Resource.newInstance(0L, 0);
|
|
|
for (CSQueue childQueue : getChildQueues()) {
|
|
|
Resources.addTo(configuredMinResources,
|
|
@@ -1014,90 +1139,16 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Map<String, Float> effectiveMinRatioPerResource = getEffectiveMinRatioPerResource(
|
|
|
+ effectiveMinRatioPerResource = getEffectiveMinRatioPerResource(
|
|
|
configuredMinResources, numeratorForMinRatio);
|
|
|
|
|
|
- // loop and do this for all child queues
|
|
|
- for (CSQueue childQueue : getChildQueues()) {
|
|
|
- Resource minResource = childQueue.getQueueResourceQuotas()
|
|
|
- .getConfiguredMinResource(label);
|
|
|
-
|
|
|
- // Update effective resource (min/max) to each child queue.
|
|
|
- if (childQueue.getCapacityConfigType()
|
|
|
- .equals(CapacityConfigType.ABSOLUTE_RESOURCE)) {
|
|
|
- childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
|
|
|
- getMinResourceNormalized(
|
|
|
- childQueue.getQueuePath(),
|
|
|
- effectiveMinRatioPerResource,
|
|
|
- minResource));
|
|
|
-
|
|
|
- // Max resource of a queue should be a minimum of {configuredMaxRes,
|
|
|
- // parentMaxRes}. parentMaxRes could be configured value. But if not
|
|
|
- // present could also be taken from effective max resource of parent.
|
|
|
- Resource parentMaxRes = queueResourceQuotas
|
|
|
- .getConfiguredMaxResource(label);
|
|
|
- if (parent != null && parentMaxRes.equals(Resources.none())) {
|
|
|
- parentMaxRes = parent.getQueueResourceQuotas()
|
|
|
- .getEffectiveMaxResource(label);
|
|
|
- }
|
|
|
-
|
|
|
- // Minimum of {childMaxResource, parentMaxRes}. However if
|
|
|
- // childMaxResource is empty, consider parent's max resource alone.
|
|
|
- Resource childMaxResource = childQueue.getQueueResourceQuotas()
|
|
|
- .getConfiguredMaxResource(label);
|
|
|
- Resource effMaxResource = Resources.min(resourceCalculator,
|
|
|
- resourceByLabel, childMaxResource.equals(Resources.none())
|
|
|
- ? parentMaxRes
|
|
|
- : childMaxResource,
|
|
|
- parentMaxRes);
|
|
|
- childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
|
|
|
- Resources.clone(effMaxResource));
|
|
|
-
|
|
|
- // In cases where we still need to update some units based on
|
|
|
- // percentage, we have to calculate percentage and update.
|
|
|
- deriveCapacityFromAbsoluteConfigurations(label, clusterResource, rc,
|
|
|
- childQueue);
|
|
|
- } else {
|
|
|
- childQueue.getQueueResourceQuotas().setEffectiveMinResource(label,
|
|
|
- Resources.multiply(resourceByLabel,
|
|
|
- childQueue.getQueueCapacities().getAbsoluteCapacity(label)));
|
|
|
- childQueue.getQueueResourceQuotas().setEffectiveMaxResource(label,
|
|
|
- Resources.multiply(resourceByLabel, childQueue.getQueueCapacities()
|
|
|
- .getAbsoluteMaximumCapacity(label)));
|
|
|
- }
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Updating effective min resource for queue:"
|
|
|
- + childQueue.getQueuePath() + " as effMinResource="
|
|
|
- + childQueue.getQueueResourceQuotas().getEffectiveMinResource(label)
|
|
|
- + "and Updating effective max resource as effMaxResource="
|
|
|
- + childQueue.getQueueResourceQuotas()
|
|
|
- .getEffectiveMaxResource(label));
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Resource getMinResourceNormalized(String name, Map<String, Float> effectiveMinRatio,
|
|
|
- Resource minResource) {
|
|
|
- Resource ret = Resource.newInstance(minResource);
|
|
|
- int maxLength = ResourceUtils.getNumberOfCountableResourceTypes();
|
|
|
- for (int i = 0; i < maxLength; i++) {
|
|
|
- ResourceInformation nResourceInformation = minResource
|
|
|
- .getResourceInformation(i);
|
|
|
-
|
|
|
- Float ratio = effectiveMinRatio.get(nResourceInformation.getName());
|
|
|
- if (ratio != null) {
|
|
|
- ret.setResourceValue(i,
|
|
|
- (long) (nResourceInformation.getValue() * ratio.floatValue()));
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Updating min resource for Queue: " + name + " as "
|
|
|
- + ret.getResourceInformation(i) + ", Actual resource: "
|
|
|
- + nResourceInformation.getValue() + ", ratio: "
|
|
|
- + ratio.floatValue());
|
|
|
- }
|
|
|
- }
|
|
|
+ // Update effective resources for my self;
|
|
|
+ if (rootQueue) {
|
|
|
+ queueResourceQuotas.setEffectiveMinResource(label, resourceByLabel);
|
|
|
+ queueResourceQuotas.setEffectiveMaxResource(label, resourceByLabel);
|
|
|
+ } else{
|
|
|
+ super.updateEffectiveResources(clusterResource);
|
|
|
}
|
|
|
- return ret;
|
|
|
}
|
|
|
|
|
|
private Map<String, Float> getEffectiveMinRatioPerResource(
|
|
@@ -1121,74 +1172,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- return effectiveMinRatioPerResource;
|
|
|
- }
|
|
|
-
|
|
|
- private void deriveCapacityFromAbsoluteConfigurations(String label,
|
|
|
- Resource clusterResource, ResourceCalculator rc, CSQueue childQueue) {
|
|
|
-
|
|
|
- /*
|
|
|
- * In case when queues are configured with absolute resources, it is better
|
|
|
- * to update capacity/max-capacity etc w.r.t absolute resource as well. In
|
|
|
- * case of computation, these values wont be used any more. However for
|
|
|
- * metrics and UI, its better these values are pre-computed here itself.
|
|
|
- */
|
|
|
-
|
|
|
- // 1. Update capacity as a float based on parent's minResource
|
|
|
- childQueue.getQueueCapacities().setCapacity(label,
|
|
|
- rc.divide(clusterResource,
|
|
|
- childQueue.getQueueResourceQuotas().getEffectiveMinResource(label),
|
|
|
- getQueueResourceQuotas().getEffectiveMinResource(label)));
|
|
|
-
|
|
|
- // 2. Update max-capacity as a float based on parent's maxResource
|
|
|
- childQueue.getQueueCapacities().setMaximumCapacity(label,
|
|
|
- rc.divide(clusterResource,
|
|
|
- childQueue.getQueueResourceQuotas().getEffectiveMaxResource(label),
|
|
|
- getQueueResourceQuotas().getEffectiveMaxResource(label)));
|
|
|
-
|
|
|
- // 3. Update absolute capacity as a float based on parent's minResource and
|
|
|
- // cluster resource.
|
|
|
- childQueue.getQueueCapacities().setAbsoluteCapacity(label,
|
|
|
- childQueue.getQueueCapacities().getCapacity(label)
|
|
|
- * getQueueCapacities().getAbsoluteCapacity(label));
|
|
|
-
|
|
|
- // 4. Update absolute max-capacity as a float based on parent's maxResource
|
|
|
- // and cluster resource.
|
|
|
- childQueue.getQueueCapacities().setAbsoluteMaximumCapacity(label,
|
|
|
- childQueue.getQueueCapacities().getMaximumCapacity(label)
|
|
|
- * getQueueCapacities().getAbsoluteMaximumCapacity(label));
|
|
|
-
|
|
|
- // Re-visit max applications for a queue based on absolute capacity if
|
|
|
- // needed.
|
|
|
- if (childQueue instanceof LeafQueue) {
|
|
|
- LeafQueue leafQueue = (LeafQueue) childQueue;
|
|
|
- CapacitySchedulerConfiguration conf = csContext.getConfiguration();
|
|
|
- int maxApplications =
|
|
|
- conf.getMaximumApplicationsPerQueue(childQueue.getQueuePath());
|
|
|
- if (maxApplications < 0) {
|
|
|
- int maxGlobalPerQueueApps = conf.getGlobalMaximumApplicationsPerQueue();
|
|
|
- if (maxGlobalPerQueueApps > 0) {
|
|
|
- maxApplications = (int) (maxGlobalPerQueueApps *
|
|
|
- childQueue.getQueueCapacities().getAbsoluteCapacity(label));
|
|
|
- } else {
|
|
|
- maxApplications = (int) (conf.getMaximumSystemApplications()
|
|
|
- * childQueue.getQueueCapacities().getAbsoluteCapacity(label));
|
|
|
- }
|
|
|
- }
|
|
|
- leafQueue.setMaxApplications(maxApplications);
|
|
|
-
|
|
|
- int maxApplicationsPerUser = Math.min(maxApplications,
|
|
|
- (int) (maxApplications
|
|
|
- * (leafQueue.getUsersManager().getUserLimit() / 100.0f)
|
|
|
- * leafQueue.getUsersManager().getUserLimitFactor()));
|
|
|
- leafQueue.setMaxApplicationsPerUser(maxApplicationsPerUser);
|
|
|
- LOG.info("LeafQueue:" + leafQueue.getQueuePath() + ", maxApplications="
|
|
|
- + maxApplications + ", maxApplicationsPerUser="
|
|
|
- + maxApplicationsPerUser + ", Abs Cap:"
|
|
|
- + childQueue.getQueueCapacities().getAbsoluteCapacity(label) + ", Cap: "
|
|
|
- + childQueue.getQueueCapacities().getCapacity(label) + ", MaxCap : "
|
|
|
- + childQueue.getQueueCapacities().getMaximumCapacity(label));
|
|
|
- }
|
|
|
+ return ImmutableMap.copyOf(effectiveMinRatioPerResource);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1463,4 +1447,9 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ // This is a locking free method
|
|
|
+ Map<String, Float> getEffectiveMinRatioPerResource() {
|
|
|
+ return effectiveMinRatioPerResource;
|
|
|
+ }
|
|
|
}
|