|
@@ -30,6 +30,7 @@ import java.util.Set;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
@@ -101,6 +102,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
|
|
|
|
private final boolean allowZeroCapacitySum;
|
|
private final boolean allowZeroCapacitySum;
|
|
|
|
|
|
|
|
+ private AutoCreatedQueueTemplate autoCreatedQueueTemplate;
|
|
|
|
+
|
|
// effective min ratio per resource, it is used during updateClusterResource,
|
|
// effective min ratio per resource, it is used during updateClusterResource,
|
|
// leaf queue can use this to calculate effective resources.
|
|
// leaf queue can use this to calculate effective resources.
|
|
// This field will not be edited, reference will point to a new immutable map
|
|
// This field will not be edited, reference will point to a new immutable map
|
|
@@ -152,6 +155,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
throws IOException {
|
|
throws IOException {
|
|
writeLock.lock();
|
|
writeLock.lock();
|
|
try {
|
|
try {
|
|
|
|
+ autoCreatedQueueTemplate = new AutoCreatedQueueTemplate(
|
|
|
|
+ csConf, getQueuePath());
|
|
super.setupQueueConfigs(clusterResource, csConf);
|
|
super.setupQueueConfigs(clusterResource, csConf);
|
|
StringBuilder aclsString = new StringBuilder();
|
|
StringBuilder aclsString = new StringBuilder();
|
|
for (Map.Entry<AccessType, AccessControlList> e : acls.entrySet()) {
|
|
for (Map.Entry<AccessType, AccessControlList> e : acls.entrySet()) {
|
|
@@ -477,6 +482,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
CapacitySchedulerConfiguration dupCSConfig =
|
|
CapacitySchedulerConfiguration dupCSConfig =
|
|
new CapacitySchedulerConfiguration(
|
|
new CapacitySchedulerConfiguration(
|
|
csContext.getConfiguration(), false);
|
|
csContext.getConfiguration(), false);
|
|
|
|
+ autoCreatedQueueTemplate.setTemplateEntriesForChild(dupCSConfig,
|
|
|
|
+ childQueuePath);
|
|
if (isLeaf) {
|
|
if (isLeaf) {
|
|
// set to -1, to disable it
|
|
// set to -1, to disable it
|
|
dupCSConfig.setUserLimitFactor(childQueuePath, -1);
|
|
dupCSConfig.setUserLimitFactor(childQueuePath, -1);
|
|
@@ -647,6 +654,18 @@ public class ParentQueue extends AbstractCSQueue {
|
|
Map<String, CSQueue> currentChildQueues = getQueuesMap(childQueues);
|
|
Map<String, CSQueue> currentChildQueues = getQueuesMap(childQueues);
|
|
Map<String, CSQueue> newChildQueues = getQueuesMap(
|
|
Map<String, CSQueue> newChildQueues = getQueuesMap(
|
|
newlyParsedParentQueue.childQueues);
|
|
newlyParsedParentQueue.childQueues);
|
|
|
|
+
|
|
|
|
+ // Reinitialize dynamic queues as well, because they are not parsed
|
|
|
|
+ for (String queue : Sets.difference(currentChildQueues.keySet(),
|
|
|
|
+ newChildQueues.keySet())) {
|
|
|
|
+ CSQueue candidate = currentChildQueues.get(queue);
|
|
|
|
+ if (candidate instanceof AbstractCSQueue) {
|
|
|
|
+ if (((AbstractCSQueue) candidate).isDynamicQueue()) {
|
|
|
|
+ candidate.reinitialize(candidate, clusterResource);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
for (Map.Entry<String, CSQueue> e : newChildQueues.entrySet()) {
|
|
for (Map.Entry<String, CSQueue> e : newChildQueues.entrySet()) {
|
|
String newChildQueueName = e.getKey();
|
|
String newChildQueueName = e.getKey();
|
|
CSQueue newChildQueue = e.getValue();
|
|
CSQueue newChildQueue = e.getValue();
|
|
@@ -1217,7 +1236,9 @@ public class ParentQueue extends AbstractCSQueue {
|
|
// For dynamic queue, we will set weight to 1 every time, because it
|
|
// For dynamic queue, we will set weight to 1 every time, because it
|
|
// is possible new labels added to the parent.
|
|
// is possible new labels added to the parent.
|
|
if (((AbstractCSQueue) queue).isDynamicQueue()) {
|
|
if (((AbstractCSQueue) queue).isDynamicQueue()) {
|
|
- queue.getQueueCapacities().setWeight(nodeLabel, 1f);
|
|
|
|
|
|
+ if (queue.getQueueCapacities().getWeight(nodeLabel) == -1f) {
|
|
|
|
+ queue.getQueueCapacities().setWeight(nodeLabel, 1f);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1637,4 +1658,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|
csContext.getConfiguration().
|
|
csContext.getConfiguration().
|
|
isAutoExpiredDeletionEnabled(this.getQueuePath());
|
|
isAutoExpiredDeletionEnabled(this.getQueuePath());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public AutoCreatedQueueTemplate getAutoCreatedQueueTemplate() {
|
|
|
|
+ return autoCreatedQueueTemplate;
|
|
|
|
+ }
|
|
}
|
|
}
|