|
@@ -76,6 +76,7 @@ import java.util.Map;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
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.UNDEFINED;
|
|
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.UNDEFINED;
|
|
|
|
|
|
public abstract class AbstractCSQueue implements CSQueue {
|
|
public abstract class AbstractCSQueue implements CSQueue {
|
|
@@ -95,6 +96,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
final ResourceCalculator resourceCalculator;
|
|
final ResourceCalculator resourceCalculator;
|
|
Set<String> accessibleLabels;
|
|
Set<String> accessibleLabels;
|
|
|
|
+ protected Set<String> configuredNodeLabels;
|
|
Set<String> resourceTypes;
|
|
Set<String> resourceTypes;
|
|
final RMNodeLabelsManager labelManager;
|
|
final RMNodeLabelsManager labelManager;
|
|
String defaultLabelExpression;
|
|
String defaultLabelExpression;
|
|
@@ -208,7 +210,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
protected void setupConfigurableCapacities(
|
|
protected void setupConfigurableCapacities(
|
|
CapacitySchedulerConfiguration configuration) {
|
|
CapacitySchedulerConfiguration configuration) {
|
|
CSQueueUtils.loadCapacitiesByLabelsFromConf(getQueuePath(), queueCapacities,
|
|
CSQueueUtils.loadCapacitiesByLabelsFromConf(getQueuePath(), queueCapacities,
|
|
- configuration);
|
|
|
|
|
|
+ configuration, configuredNodeLabels);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -360,7 +362,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
writeLock.lock();
|
|
writeLock.lock();
|
|
try {
|
|
try {
|
|
- if (isDynamicQueue()) {
|
|
|
|
|
|
+ if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) {
|
|
setDynamicQueueProperties(configuration);
|
|
setDynamicQueueProperties(configuration);
|
|
}
|
|
}
|
|
// get labels
|
|
// get labels
|
|
@@ -386,6 +388,17 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
|
|
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ if (csContext.getCapacitySchedulerQueueManager() != null
|
|
|
|
+ && csContext.getCapacitySchedulerQueueManager()
|
|
|
|
+ .getConfiguredNodeLabels() != null) {
|
|
|
|
+ this.configuredNodeLabels = csContext.getCapacitySchedulerQueueManager()
|
|
|
|
+ .getConfiguredNodeLabels().getLabelsByQueue(getQueuePath());
|
|
|
|
+ } else {
|
|
|
|
+ // Fallback to suboptimal but correct logic
|
|
|
|
+ this.configuredNodeLabels = csContext.getConfiguration()
|
|
|
|
+ .getConfiguredNodeLabels(queuePath);
|
|
|
|
+ }
|
|
|
|
+
|
|
// After we setup labels, we can setup capacities
|
|
// After we setup labels, we can setup capacities
|
|
setupConfigurableCapacities(configuration);
|
|
setupConfigurableCapacities(configuration);
|
|
updateAbsoluteCapacities();
|
|
updateAbsoluteCapacities();
|
|
@@ -487,6 +500,19 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
if (getParent() instanceof ParentQueue) {
|
|
if (getParent() instanceof ParentQueue) {
|
|
((ParentQueue) getParent()).getAutoCreatedQueueTemplate()
|
|
((ParentQueue) getParent()).getAutoCreatedQueueTemplate()
|
|
.setTemplateEntriesForChild(configuration, getQueuePath());
|
|
.setTemplateEntriesForChild(configuration, getQueuePath());
|
|
|
|
+
|
|
|
|
+ String parentTemplate = String.format("%s.%s", getParent().getQueuePath(),
|
|
|
|
+ AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX);
|
|
|
|
+ parentTemplate = parentTemplate.substring(0, parentTemplate.lastIndexOf(
|
|
|
|
+ DOT));
|
|
|
|
+ Set<String> parentNodeLabels = csContext
|
|
|
|
+ .getCapacitySchedulerQueueManager().getConfiguredNodeLabels()
|
|
|
|
+ .getLabelsByQueue(parentTemplate);
|
|
|
|
+
|
|
|
|
+ if (parentNodeLabels != null && parentNodeLabels.size() > 1) {
|
|
|
|
+ csContext.getCapacitySchedulerQueueManager().getConfiguredNodeLabels()
|
|
|
|
+ .setLabelsByQueue(queuePath, new HashSet<>(parentNodeLabels));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -571,10 +597,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
|
|
|
|
protected void updateConfigurableResourceRequirement(String queuePath,
|
|
protected void updateConfigurableResourceRequirement(String queuePath,
|
|
Resource clusterResource) {
|
|
Resource clusterResource) {
|
|
- CapacitySchedulerConfiguration conf = csContext.getConfiguration();
|
|
|
|
- Set<String> configuredNodelabels = conf.getConfiguredNodeLabels(queuePath);
|
|
|
|
-
|
|
|
|
- for (String label : configuredNodelabels) {
|
|
|
|
|
|
+ for (String label : configuredNodeLabels) {
|
|
Resource minResource = getMinimumAbsoluteResource(queuePath, label);
|
|
Resource minResource = getMinimumAbsoluteResource(queuePath, label);
|
|
Resource maxResource = getMaximumAbsoluteResource(queuePath, label);
|
|
Resource maxResource = getMaximumAbsoluteResource(queuePath, label);
|
|
|
|
|
|
@@ -1578,9 +1601,7 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
}
|
|
}
|
|
|
|
|
|
void updateEffectiveResources(Resource clusterResource) {
|
|
void updateEffectiveResources(Resource clusterResource) {
|
|
- Set<String> configuredNodelabels =
|
|
|
|
- csContext.getConfiguration().getConfiguredNodeLabels(getQueuePath());
|
|
|
|
- for (String label : configuredNodelabels) {
|
|
|
|
|
|
+ for (String label : configuredNodeLabels) {
|
|
Resource resourceByLabel = labelManager.getResourceByLabel(label,
|
|
Resource resourceByLabel = labelManager.getResourceByLabel(label,
|
|
clusterResource);
|
|
clusterResource);
|
|
|
|
|
|
@@ -1715,5 +1736,4 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|
writeLock.unlock();
|
|
writeLock.unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
}
|
|
}
|