|
@@ -18,7 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|
|
|
|
|
-import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
@@ -37,7 +36,6 @@ import org.apache.hadoop.yarn.api.records.QueueStatistics;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.security.AccessRequest;
|
|
@@ -77,28 +75,24 @@ import java.util.Set;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.UNDEFINED;
|
|
|
|
|
|
public abstract class AbstractCSQueue implements CSQueue {
|
|
|
-
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(AbstractCSQueue.class);
|
|
|
+ protected final QueueAllocationSettings queueAllocationSettings;
|
|
|
volatile CSQueue parent;
|
|
|
+ protected final QueuePath queuePath;
|
|
|
final String queueName;
|
|
|
- private final String queuePath;
|
|
|
+ protected QueueNodeLabelsSettings queueNodeLabelsSettings;
|
|
|
+ private volatile QueueAppLifetimeAndLimitSettings queueAppLifetimeSettings;
|
|
|
+ private CSQueuePreemptionSettings preemptionSettings;
|
|
|
|
|
|
- final Resource minimumAllocation;
|
|
|
- volatile Resource maximumAllocation;
|
|
|
private volatile QueueState state = null;
|
|
|
protected final PrivilegedEntity queueEntity;
|
|
|
|
|
|
final ResourceCalculator resourceCalculator;
|
|
|
- Set<String> accessibleLabels;
|
|
|
- protected Set<String> configuredNodeLabels;
|
|
|
Set<String> resourceTypes;
|
|
|
final RMNodeLabelsManager labelManager;
|
|
|
- String defaultLabelExpression;
|
|
|
private String multiNodeSortingPolicyName = null;
|
|
|
|
|
|
Map<AccessType, AccessControlList> acls =
|
|
@@ -109,17 +103,6 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
// used-capacity/abs-used-capacity/capacity/abs-capacity,
|
|
|
// etc.
|
|
|
QueueCapacities queueCapacities;
|
|
|
-
|
|
|
- // -1 indicates lifetime is disabled
|
|
|
- private volatile long maxApplicationLifetime = -1;
|
|
|
-
|
|
|
- private volatile long defaultApplicationLifetime = -1;
|
|
|
-
|
|
|
- // Indicates if this queue's default lifetime was set by a config property,
|
|
|
- // either at this level or anywhere in the queue's hierarchy.
|
|
|
- private volatile boolean defaultAppLifetimeWasSpecifiedInConfig = false;
|
|
|
- private CSQueuePreemption preemptionSettings;
|
|
|
-
|
|
|
CSQueueUsageTracker usageTracker;
|
|
|
|
|
|
public enum CapacityConfigType {
|
|
@@ -145,7 +128,6 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
volatile Priority priority = Priority.newInstance(0);
|
|
|
private UserWeights userWeights = UserWeights.createEmpty();
|
|
|
- private int maxParallelApps;
|
|
|
|
|
|
// is it a dynamic queue?
|
|
|
private boolean dynamicQueue = false;
|
|
@@ -158,12 +140,10 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
public AbstractCSQueue(CapacitySchedulerContext cs,
|
|
|
CapacitySchedulerConfiguration configuration, String queueName,
|
|
|
CSQueue parent, CSQueue old) {
|
|
|
-
|
|
|
this.labelManager = cs.getRMContext().getNodeLabelManager();
|
|
|
this.parent = parent;
|
|
|
- this.queueName = queueName;
|
|
|
- this.queuePath = ((parent == null) ? "" : (parent.getQueuePath() + "."))
|
|
|
- + this.queueName;
|
|
|
+ this.queuePath = createQueuePath(parent, queueName);
|
|
|
+ this.queueName = queuePath.getLeafName();
|
|
|
this.resourceCalculator = cs.getResourceCalculator();
|
|
|
this.activitiesManager = cs.getActivitiesManager();
|
|
|
|
|
@@ -174,7 +154,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
cs.getConfiguration().getEnableUserMetrics(), configuration);
|
|
|
usageTracker = new CSQueueUsageTracker(metrics);
|
|
|
this.csContext = cs;
|
|
|
- this.minimumAllocation = csContext.getMinimumResourceCapability();
|
|
|
+ this.queueAllocationSettings = new QueueAllocationSettings(csContext);
|
|
|
queueEntity = new PrivilegedEntity(EntityType.QUEUE, getQueuePath());
|
|
|
queueCapacities = new QueueCapacities(parent == null);
|
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
@@ -182,6 +162,13 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
writeLock = lock.writeLock();
|
|
|
}
|
|
|
|
|
|
+ private static QueuePath createQueuePath(CSQueue parent, String queueName) {
|
|
|
+ if (parent == null) {
|
|
|
+ return new QueuePath(null, queueName);
|
|
|
+ }
|
|
|
+ return new QueuePath(parent.getQueuePath(), queueName);
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
protected void setupConfigurableCapacities() {
|
|
|
setupConfigurableCapacities(csContext.getConfiguration());
|
|
@@ -190,12 +177,12 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
protected void setupConfigurableCapacities(
|
|
|
CapacitySchedulerConfiguration configuration) {
|
|
|
CSQueueUtils.loadCapacitiesByLabelsFromConf(getQueuePath(), queueCapacities,
|
|
|
- configuration, configuredNodeLabels);
|
|
|
+ configuration, this.queueNodeLabelsSettings.getConfiguredNodeLabels());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public String getQueuePath() {
|
|
|
- return queuePath;
|
|
|
+ return queuePath.getFullPath();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -249,7 +236,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
@Override
|
|
|
public String getQueueShortName() {
|
|
|
- return queueName;
|
|
|
+ return queuePath.getLeafName();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -257,11 +244,6 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
return queueName;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public PrivilegedEntity getPrivilegedEntity() {
|
|
|
- return queueEntity;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public CSQueue getParent() {
|
|
|
return parent;
|
|
@@ -272,8 +254,13 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
this.parent = newParentQueue;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public PrivilegedEntity getPrivilegedEntity() {
|
|
|
+ return queueEntity;
|
|
|
+ }
|
|
|
+
|
|
|
public Set<String> getAccessibleNodeLabels() {
|
|
|
- return accessibleLabels;
|
|
|
+ return queueNodeLabelsSettings.getAccessibleNodeLabels();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -331,7 +318,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
@Override
|
|
|
public String getDefaultNodeLabelExpression() {
|
|
|
- return defaultLabelExpression;
|
|
|
+ return this.queueNodeLabelsSettings.getDefaultLabelExpression();
|
|
|
}
|
|
|
|
|
|
protected void setupQueueConfigs(Resource clusterResource,
|
|
@@ -345,7 +332,8 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
// Collect and set the Node label configuration
|
|
|
- initializeNodeLabels(configuration);
|
|
|
+ this.queueNodeLabelsSettings = new QueueNodeLabelsSettings(configuration, parent,
|
|
|
+ getQueuePath(), csContext);
|
|
|
|
|
|
// Initialize the queue capacities
|
|
|
setupConfigurableCapacities(configuration);
|
|
@@ -363,7 +351,8 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
// Setup queue's maximumAllocation respecting the global
|
|
|
// and the queue settings
|
|
|
- setupMaximumAllocation(configuration);
|
|
|
+ this.queueAllocationSettings.setupMaximumAllocation(configuration, getQueuePath(),
|
|
|
+ parent, csContext);
|
|
|
|
|
|
// Initialize the queue state based on previous state, configured state
|
|
|
// and its parent state
|
|
@@ -378,14 +367,15 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
this.reservationsContinueLooking =
|
|
|
configuration.getReservationContinueLook();
|
|
|
this.configuredCapacityVectors = csContext.getConfiguration()
|
|
|
- .parseConfiguredResourceVector(queuePath, configuredNodeLabels);
|
|
|
+ .parseConfiguredResourceVector(queuePath.getFullPath(),
|
|
|
+ this.queueNodeLabelsSettings.getConfiguredNodeLabels());
|
|
|
|
|
|
// Update metrics
|
|
|
CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
|
|
|
this, labelManager, null);
|
|
|
|
|
|
// Store preemption settings
|
|
|
- this.preemptionSettings = new CSQueuePreemption(this, csContext, configuration);
|
|
|
+ this.preemptionSettings = new CSQueuePreemptionSettings(this, csContext, configuration);
|
|
|
this.priority = configuration.getQueuePriority(
|
|
|
getQueuePath());
|
|
|
|
|
@@ -394,7 +384,8 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
configuration.getMultiNodesSortingAlgorithmPolicy(getQueuePath()));
|
|
|
|
|
|
// Setup application related limits
|
|
|
- setupApplicationLimits(configuration);
|
|
|
+ this.queueAppLifetimeSettings = new QueueAppLifetimeAndLimitSettings(configuration,
|
|
|
+ this, getQueuePath());
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
@@ -407,11 +398,11 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
protected void setDynamicQueueProperties(
|
|
|
CapacitySchedulerConfiguration configuration) {
|
|
|
// Set properties from parent template
|
|
|
- if (getParent() instanceof ParentQueue) {
|
|
|
- ((ParentQueue) getParent()).getAutoCreatedQueueTemplate()
|
|
|
+ if (parent instanceof ParentQueue) {
|
|
|
+ ((ParentQueue) parent).getAutoCreatedQueueTemplate()
|
|
|
.setTemplateEntriesForChild(configuration, getQueuePath());
|
|
|
|
|
|
- String parentTemplate = String.format("%s.%s", getParent().getQueuePath(),
|
|
|
+ String parentTemplate = String.format("%s.%s", parent.getQueuePath(),
|
|
|
AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX);
|
|
|
parentTemplate = parentTemplate.substring(0, parentTemplate.lastIndexOf(
|
|
|
DOT));
|
|
@@ -421,135 +412,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
if (parentNodeLabels != null && parentNodeLabels.size() > 1) {
|
|
|
csContext.getCapacitySchedulerQueueManager().getConfiguredNodeLabels()
|
|
|
- .setLabelsByQueue(queuePath, new HashSet<>(parentNodeLabels));
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void initializeNodeLabels(
|
|
|
- CapacitySchedulerConfiguration configuration) throws IOException {
|
|
|
- // Collect and store labels
|
|
|
- this.accessibleLabels =
|
|
|
- configuration.getAccessibleNodeLabels(getQueuePath());
|
|
|
- this.defaultLabelExpression =
|
|
|
- configuration.getDefaultNodeLabelExpression(
|
|
|
- getQueuePath());
|
|
|
-
|
|
|
- // Inherit labels from parent if not set
|
|
|
- if (this.accessibleLabels == null && parent != null) {
|
|
|
- this.accessibleLabels = parent.getAccessibleNodeLabels();
|
|
|
- }
|
|
|
-
|
|
|
- // If the accessible labels is not null and the queue has a parent with a
|
|
|
- // similar set of labels copy the defaultNodeLabelExpression from the parent
|
|
|
- if (this.accessibleLabels != null && parent != null
|
|
|
- && this.defaultLabelExpression == null &&
|
|
|
- this.accessibleLabels.containsAll(parent.getAccessibleNodeLabels())) {
|
|
|
- this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
|
|
|
- }
|
|
|
-
|
|
|
- if (csContext.getCapacitySchedulerQueueManager() != null
|
|
|
- && csContext.getCapacitySchedulerQueueManager()
|
|
|
- .getConfiguredNodeLabels() != null) {
|
|
|
- if (getQueuePath().equals(ROOT)) {
|
|
|
- this.configuredNodeLabels = csContext.getCapacitySchedulerQueueManager()
|
|
|
- .getConfiguredNodeLabels().getAllConfiguredLabels();
|
|
|
- } else {
|
|
|
- this.configuredNodeLabels = csContext.getCapacitySchedulerQueueManager()
|
|
|
- .getConfiguredNodeLabels().getLabelsByQueue(getQueuePath());
|
|
|
- }
|
|
|
- } else {
|
|
|
- // Fallback to suboptimal but correct logic
|
|
|
- this.configuredNodeLabels = csContext.getConfiguration()
|
|
|
- .getConfiguredNodeLabels(queuePath);
|
|
|
- }
|
|
|
-
|
|
|
- // Validate the initialized settings
|
|
|
- validateNodeLabels();
|
|
|
- }
|
|
|
-
|
|
|
- private void validateNodeLabels() throws IOException {
|
|
|
- // Check if labels of this queue is a subset of parent queue, only do this
|
|
|
- // when the queue in question is not root
|
|
|
- if (parent != null && parent.getParent() != null) {
|
|
|
- if (parent.getAccessibleNodeLabels() != null && !parent
|
|
|
- .getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
|
|
|
- // if parent isn't "*", child shouldn't be "*" too
|
|
|
- if (this.getAccessibleNodeLabels().contains(
|
|
|
- RMNodeLabelsManager.ANY)) {
|
|
|
- throw new IOException("Parent's accessible queue is not ANY(*), "
|
|
|
- + "but child's accessible queue is *");
|
|
|
- } else{
|
|
|
- Set<String> diff = Sets.difference(this.getAccessibleNodeLabels(),
|
|
|
- parent.getAccessibleNodeLabels());
|
|
|
- if (!diff.isEmpty()) {
|
|
|
- throw new IOException(
|
|
|
- "Some labels of child queue is not a subset "
|
|
|
- + "of parent queue, these labels=[" + StringUtils
|
|
|
- .join(diff, ",") + "]");
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void setupApplicationLimits(CapacitySchedulerConfiguration configuration) {
|
|
|
- // Store max parallel apps property
|
|
|
- this.maxParallelApps = configuration.getMaxParallelAppsForQueue(getQueuePath());
|
|
|
-
|
|
|
- maxApplicationLifetime = getInheritedMaxAppLifetime(this, configuration);
|
|
|
- defaultApplicationLifetime =
|
|
|
- getInheritedDefaultAppLifetime(this, configuration,
|
|
|
- maxApplicationLifetime);
|
|
|
- }
|
|
|
-
|
|
|
- private void setupMaximumAllocation(CapacitySchedulerConfiguration csConf) {
|
|
|
- String myQueuePath = getQueuePath();
|
|
|
- /* YARN-10869: When using AutoCreatedLeafQueues, the passed configuration
|
|
|
- * object is a cloned one containing only the template configs
|
|
|
- * (see ManagedParentQueue#getLeafQueueConfigs). To ensure that the actual
|
|
|
- * cluster maximum allocation is fetched the original config object should
|
|
|
- * be used.
|
|
|
- */
|
|
|
- Resource clusterMax = ResourceUtils
|
|
|
- .fetchMaximumAllocationFromConfig(this.csContext.getConfiguration());
|
|
|
- Resource queueMax = csConf.getQueueMaximumAllocation(myQueuePath);
|
|
|
-
|
|
|
- maximumAllocation = Resources.clone(
|
|
|
- parent == null ? clusterMax : parent.getMaximumAllocation());
|
|
|
-
|
|
|
- String errMsg =
|
|
|
- "Queue maximum allocation cannot be larger than the cluster setting"
|
|
|
- + " for queue " + myQueuePath
|
|
|
- + " max allocation per queue: %s"
|
|
|
- + " cluster setting: " + clusterMax;
|
|
|
-
|
|
|
- if (queueMax == Resources.none()) {
|
|
|
- // Handle backward compatibility
|
|
|
- long queueMemory = csConf.getQueueMaximumAllocationMb(myQueuePath);
|
|
|
- int queueVcores = csConf.getQueueMaximumAllocationVcores(myQueuePath);
|
|
|
- if (queueMemory != UNDEFINED) {
|
|
|
- maximumAllocation.setMemorySize(queueMemory);
|
|
|
- }
|
|
|
-
|
|
|
- if (queueVcores != UNDEFINED) {
|
|
|
- maximumAllocation.setVirtualCores(queueVcores);
|
|
|
- }
|
|
|
-
|
|
|
- if ((queueMemory != UNDEFINED && queueMemory > clusterMax.getMemorySize()
|
|
|
- || (queueVcores != UNDEFINED
|
|
|
- && queueVcores > clusterMax.getVirtualCores()))) {
|
|
|
- throw new IllegalArgumentException(
|
|
|
- String.format(errMsg, maximumAllocation));
|
|
|
- }
|
|
|
- } else {
|
|
|
- // Queue level maximum-allocation can't be larger than cluster setting
|
|
|
- for (ResourceInformation ri : queueMax.getResources()) {
|
|
|
- if (ri.compareTo(clusterMax.getResourceInformation(ri.getName())) > 0) {
|
|
|
- throw new IllegalArgumentException(String.format(errMsg, queueMax));
|
|
|
- }
|
|
|
-
|
|
|
- maximumAllocation.setResourceInformation(ri.getName(), ri);
|
|
|
+ .setLabelsByQueue(getQueuePath(), new HashSet<>(parentNodeLabels));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -557,7 +420,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
private UserWeights getUserWeightsFromHierarchy(
|
|
|
CapacitySchedulerConfiguration configuration) {
|
|
|
UserWeights unionInheritedWeights = UserWeights.createEmpty();
|
|
|
- CSQueue parentQ = getParent();
|
|
|
+ CSQueue parentQ = parent;
|
|
|
if (parentQ != null) {
|
|
|
// Inherit all of parent's userWeights
|
|
|
unionInheritedWeights.addFrom(parentQ.getUserWeights());
|
|
@@ -589,12 +452,12 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
protected void updateCapacityConfigType() {
|
|
|
this.capacityConfigType = CapacityConfigType.NONE;
|
|
|
- for (String label : configuredNodeLabels) {
|
|
|
+ for (String label : queueNodeLabelsSettings.getConfiguredNodeLabels()) {
|
|
|
LOG.debug("capacityConfigType is '{}' for queue {}",
|
|
|
capacityConfigType, getQueuePath());
|
|
|
|
|
|
CapacityConfigType localType = checkConfigTypeIsAbsoluteResource(
|
|
|
- queuePath, label) ? CapacityConfigType.ABSOLUTE_RESOURCE
|
|
|
+ getQueuePath(), label) ? CapacityConfigType.ABSOLUTE_RESOURCE
|
|
|
: CapacityConfigType.PERCENTAGE;
|
|
|
|
|
|
if (this.capacityConfigType.equals(CapacityConfigType.NONE)) {
|
|
@@ -608,12 +471,13 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
protected void updateConfigurableResourceLimits(Resource clusterResource) {
|
|
|
- for (String label : configuredNodeLabels) {
|
|
|
- final Resource minResource = getMinimumAbsoluteResource(queuePath, label);
|
|
|
- Resource maxResource = getMaximumAbsoluteResource(queuePath, label);
|
|
|
+ for (String label : queueNodeLabelsSettings.getConfiguredNodeLabels()) {
|
|
|
+ final Resource minResource = getMinimumAbsoluteResource(getQueuePath(), label);
|
|
|
+ Resource maxResource = getMaximumAbsoluteResource(getQueuePath(), label);
|
|
|
|
|
|
if (parent != null) {
|
|
|
- final Resource parentMax = parent.getQueueResourceQuotas().getConfiguredMaxResource(label);
|
|
|
+ final Resource parentMax = parent.getQueueResourceQuotas()
|
|
|
+ .getConfiguredMaxResource(label);
|
|
|
validateMinResourceIsNotGreaterThanMaxResource(maxResource, parentMax, clusterResource,
|
|
|
"Max resource configuration "
|
|
|
+ maxResource + " is greater than parents max value:"
|
|
@@ -654,7 +518,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
private void validateAbsoluteVsPercentageCapacityConfig(
|
|
|
CapacityConfigType localType) {
|
|
|
- if (!queuePath.equals("root")
|
|
|
+ if (!getQueuePath().equals("root")
|
|
|
&& !this.capacityConfigType.equals(localType)) {
|
|
|
throw new IllegalArgumentException("Queue '" + getQueuePath()
|
|
|
+ "' should use either percentage based capacity"
|
|
@@ -677,7 +541,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
public Resource getEffectiveCapacityDown(String label, Resource factor) {
|
|
|
return Resources.normalizeDown(resourceCalculator,
|
|
|
getQueueResourceQuotas().getEffectiveMinResource(label),
|
|
|
- minimumAllocation);
|
|
|
+ queueAllocationSettings.getMinimumAllocation());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -690,7 +554,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
public Resource getEffectiveMaxCapacityDown(String label, Resource factor) {
|
|
|
return Resources.normalizeDown(resourceCalculator,
|
|
|
getQueueResourceQuotas().getEffectiveMaxResource(label),
|
|
|
- minimumAllocation);
|
|
|
+ queueAllocationSettings.getMinimumAllocation());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -729,7 +593,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
&& parentState != QueueState.RUNNING) {
|
|
|
throw new IllegalArgumentException(
|
|
|
"The parent queue:" + parent.getQueuePath()
|
|
|
- + " cannot be STOPPED as the child queue:" + queuePath
|
|
|
+ + " cannot be STOPPED as the child queue:" + getQueuePath()
|
|
|
+ " is in RUNNING state.");
|
|
|
} else {
|
|
|
updateQueueState(configuredState);
|
|
@@ -760,20 +624,20 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
// TODO, improve this
|
|
|
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
|
|
queueInfo.setQueueName(queueName);
|
|
|
- queueInfo.setQueuePath(queuePath);
|
|
|
- queueInfo.setAccessibleNodeLabels(accessibleLabels);
|
|
|
+ queueInfo.setQueuePath(getQueuePath());
|
|
|
+ queueInfo.setAccessibleNodeLabels(queueNodeLabelsSettings.getAccessibleNodeLabels());
|
|
|
queueInfo.setCapacity(queueCapacities.getCapacity());
|
|
|
queueInfo.setMaximumCapacity(queueCapacities.getMaximumCapacity());
|
|
|
queueInfo.setQueueState(getState());
|
|
|
- queueInfo.setDefaultNodeLabelExpression(defaultLabelExpression);
|
|
|
+ queueInfo.setDefaultNodeLabelExpression(queueNodeLabelsSettings.getDefaultLabelExpression());
|
|
|
queueInfo.setCurrentCapacity(getUsedCapacity());
|
|
|
queueInfo.setQueueStatistics(getQueueStatistics());
|
|
|
- queueInfo.setPreemptionDisabled(getPreemptionDisabled());
|
|
|
+ queueInfo.setPreemptionDisabled(preemptionSettings.isPreemptionDisabled());
|
|
|
queueInfo.setIntraQueuePreemptionDisabled(
|
|
|
getIntraQueuePreemptionDisabled());
|
|
|
queueInfo.setQueueConfigurations(getQueueConfigurations());
|
|
|
queueInfo.setWeight(queueCapacities.getWeight());
|
|
|
- queueInfo.setMaxParallelApps(maxParallelApps);
|
|
|
+ queueInfo.setMaxParallelApps(queueAppLifetimeSettings.getMaxParallelApps());
|
|
|
return queueInfo;
|
|
|
}
|
|
|
|
|
@@ -839,12 +703,12 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
@Private
|
|
|
public Resource getMaximumAllocation() {
|
|
|
- return maximumAllocation;
|
|
|
+ return queueAllocationSettings.getMaximumAllocation();
|
|
|
}
|
|
|
|
|
|
@Private
|
|
|
public Resource getMinimumAllocation() {
|
|
|
- return minimumAllocation;
|
|
|
+ return queueAllocationSettings.getMinimumAllocation();
|
|
|
}
|
|
|
|
|
|
void allocateResource(Resource clusterResource,
|
|
@@ -897,8 +761,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
@Private
|
|
|
public boolean getIntraQueuePreemptionDisabled() {
|
|
|
- return preemptionSettings.isIntraQueuePreemptionDisabledInHierarchy() ||
|
|
|
- preemptionSettings.isPreemptionDisabled();
|
|
|
+ return preemptionSettings.getIntraQueuePreemptionDisabled();
|
|
|
}
|
|
|
|
|
|
@Private
|
|
@@ -926,76 +789,6 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
return readLock;
|
|
|
}
|
|
|
|
|
|
- private long getInheritedMaxAppLifetime(CSQueue q,
|
|
|
- CapacitySchedulerConfiguration conf) {
|
|
|
- CSQueue parentQ = q.getParent();
|
|
|
- long maxAppLifetime = conf.getMaximumLifetimePerQueue(q.getQueuePath());
|
|
|
-
|
|
|
- // If q is the root queue, then get max app lifetime from conf.
|
|
|
- if (parentQ == null) {
|
|
|
- return maxAppLifetime;
|
|
|
- }
|
|
|
-
|
|
|
- // If this is not the root queue, get this queue's max app lifetime
|
|
|
- // from the conf. The parent's max app lifetime will be used if it's
|
|
|
- // not set for this queue.
|
|
|
- // A value of 0 will override the parent's value and means no max lifetime.
|
|
|
- // A negative value means that the parent's max should be used.
|
|
|
- long parentsMaxAppLifetime = getParent().getMaximumApplicationLifetime();
|
|
|
- return (maxAppLifetime >= 0) ? maxAppLifetime : parentsMaxAppLifetime;
|
|
|
- }
|
|
|
-
|
|
|
- private long getInheritedDefaultAppLifetime(CSQueue q,
|
|
|
- CapacitySchedulerConfiguration conf, long myMaxAppLifetime) {
|
|
|
- CSQueue parentQ = q.getParent();
|
|
|
- long defaultAppLifetime = conf.getDefaultLifetimePerQueue(getQueuePath());
|
|
|
- defaultAppLifetimeWasSpecifiedInConfig =
|
|
|
- (defaultAppLifetime >= 0
|
|
|
- || (parentQ != null &&
|
|
|
- parentQ.getDefaultAppLifetimeWasSpecifiedInConfig()));
|
|
|
-
|
|
|
- // If q is the root queue, then get default app lifetime from conf.
|
|
|
- if (parentQ == null) {
|
|
|
- return defaultAppLifetime;
|
|
|
- }
|
|
|
-
|
|
|
- // If this is not the root queue, get the parent's default app lifetime. The
|
|
|
- // parent's default app lifetime will be used if not set for this queue.
|
|
|
- long parentsDefaultAppLifetime =
|
|
|
- getParent().getDefaultApplicationLifetime();
|
|
|
-
|
|
|
- // Negative value indicates default lifetime was not set at this level.
|
|
|
- // If default lifetime was not set at this level, calculate it based on
|
|
|
- // parent's default lifetime or current queue's max lifetime.
|
|
|
- if (defaultAppLifetime < 0) {
|
|
|
- // If default lifetime was not set at this level but was set somewhere in
|
|
|
- // the parent's hierarchy, set default lifetime to parent queue's default
|
|
|
- // only if parent queue's lifetime is less than current queue's max
|
|
|
- // lifetime. Otherwise, use current queue's max lifetime value for its
|
|
|
- // default lifetime.
|
|
|
- if (defaultAppLifetimeWasSpecifiedInConfig) {
|
|
|
- defaultAppLifetime =
|
|
|
- Math.min(parentsDefaultAppLifetime, myMaxAppLifetime);
|
|
|
- } else {
|
|
|
- // Default app lifetime value was not set anywhere in this queue's
|
|
|
- // hierarchy. Use current queue's max lifetime as its default.
|
|
|
- defaultAppLifetime = myMaxAppLifetime;
|
|
|
- }
|
|
|
- } // else if >= 0, default lifetime was set at this level. Just use it.
|
|
|
-
|
|
|
- if (myMaxAppLifetime > 0 &&
|
|
|
- defaultAppLifetime > myMaxAppLifetime) {
|
|
|
- throw new YarnRuntimeException(
|
|
|
- "Default lifetime " + defaultAppLifetime
|
|
|
- + " can't exceed maximum lifetime " + myMaxAppLifetime);
|
|
|
- }
|
|
|
-
|
|
|
- if (defaultAppLifetime <= 0) {
|
|
|
- defaultAppLifetime = myMaxAppLifetime;
|
|
|
- }
|
|
|
- return defaultAppLifetime;
|
|
|
- }
|
|
|
-
|
|
|
private Resource getCurrentLimitResource(String nodePartition,
|
|
|
Resource clusterResource, ResourceLimits currentResourceLimits,
|
|
|
SchedulingMode schedulingMode) {
|
|
@@ -1201,25 +994,6 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
usageTracker.getQueueUsage(), nodePartition, cluster, schedulingMode);
|
|
|
}
|
|
|
|
|
|
- public boolean accessibleToPartition(String nodePartition) {
|
|
|
- // if queue's label is *, it can access any node
|
|
|
- if (accessibleLabels != null
|
|
|
- && accessibleLabels.contains(RMNodeLabelsManager.ANY)) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- // any queue can access to a node without label
|
|
|
- if (nodePartition == null
|
|
|
- || nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- // a queue can access to a node only if it contains any label of the node
|
|
|
- if (accessibleLabels != null && accessibleLabels.contains(nodePartition)) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- // sorry, you cannot access
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public Priority getDefaultApplicationPriority() {
|
|
|
// TODO add dummy implementation
|
|
@@ -1341,7 +1115,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
LOG.info("The specified queue:" + getQueuePath()
|
|
|
+ " is already in the RUNNING state.");
|
|
|
} else {
|
|
|
- CSQueue parentQueue = getParent();
|
|
|
+ CSQueue parentQueue = parent;
|
|
|
if (parentQueue == null || parentQueue.getState() == QueueState.RUNNING) {
|
|
|
updateQueueState(QueueState.RUNNING);
|
|
|
} else {
|
|
@@ -1384,8 +1158,8 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
updateQueueState(QueueState.DRAINING);
|
|
|
}
|
|
|
LOG.info("Recover draining state for queue " + this.getQueuePath());
|
|
|
- if (getParent() != null && getParent().getState() == QueueState.STOPPED) {
|
|
|
- ((AbstractCSQueue) getParent()).recoverDrainingState();
|
|
|
+ if (parent != null && parent.getState() == QueueState.STOPPED) {
|
|
|
+ ((AbstractCSQueue) parent).recoverDrainingState();
|
|
|
}
|
|
|
} finally {
|
|
|
this.writeLock.unlock();
|
|
@@ -1402,24 +1176,24 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
public long getMaximumApplicationLifetime() {
|
|
|
- return maxApplicationLifetime;
|
|
|
+ return queueAppLifetimeSettings.getMaxApplicationLifetime();
|
|
|
}
|
|
|
|
|
|
public long getDefaultApplicationLifetime() {
|
|
|
- return defaultApplicationLifetime;
|
|
|
+ return queueAppLifetimeSettings.getDefaultApplicationLifetime();
|
|
|
}
|
|
|
|
|
|
public boolean getDefaultAppLifetimeWasSpecifiedInConfig() {
|
|
|
- return defaultAppLifetimeWasSpecifiedInConfig;
|
|
|
+ return queueAppLifetimeSettings.isDefaultAppLifetimeWasSpecifiedInConfig();
|
|
|
}
|
|
|
|
|
|
public void setMaxParallelApps(int maxParallelApps) {
|
|
|
- this.maxParallelApps = maxParallelApps;
|
|
|
+ this.queueAppLifetimeSettings.setMaxParallelApps(maxParallelApps);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public int getMaxParallelApps() {
|
|
|
- return maxParallelApps;
|
|
|
+ return this.queueAppLifetimeSettings.getMaxParallelApps();
|
|
|
}
|
|
|
|
|
|
abstract int getNumRunnableApps();
|
|
@@ -1447,7 +1221,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
ret.setResourceValue(i,
|
|
|
(long) (nResourceInformation.getValue() * ratio));
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Updating min resource for Queue: " + queuePath + " as " + ret
|
|
|
+ LOG.debug("Updating min resource for Queue: " + getQueuePath() + " as " + ret
|
|
|
.getResourceInformation(i) + ", Actual resource: "
|
|
|
+ nResourceInformation.getValue() + ", ratio: " + ratio);
|
|
|
}
|
|
@@ -1513,7 +1287,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
void updateEffectiveResources(Resource clusterResource) {
|
|
|
- for (String label : configuredNodeLabels) {
|
|
|
+ for (String label : queueNodeLabelsSettings.getConfiguredNodeLabels()) {
|
|
|
Resource resourceByLabel = labelManager.getResourceByLabel(label,
|
|
|
clusterResource);
|
|
|
Resource newEffectiveMinResource;
|
|
@@ -1549,7 +1323,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
newEffectiveMaxResource);
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Updating queue:" + queuePath
|
|
|
+ LOG.debug("Updating queue:" + getQueuePath()
|
|
|
+ " with effective minimum resource=" + newEffectiveMinResource
|
|
|
+ "and effective maximum resource="
|
|
|
+ newEffectiveMaxResource);
|