|
@@ -24,8 +24,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
|
|
|
|
|
-import java.util.Set;
|
|
|
|
|
|
+import java.util.*;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
|
|
|
|
|
|
public class MappingRuleValidationContextImpl
|
|
public class MappingRuleValidationContextImpl
|
|
implements MappingRuleValidationContext {
|
|
implements MappingRuleValidationContext {
|
|
@@ -51,76 +54,56 @@ public class MappingRuleValidationContextImpl
|
|
|
|
|
|
/**
|
|
/**
|
|
* This method will determine if a static queue path is valid.
|
|
* This method will determine if a static queue path is valid.
|
|
|
|
+ * We consider a path static (in the target path validation context)
|
|
|
|
+ * If non if it's parts contain any substitutable variables.
|
|
|
|
+ * eg. root.groups.bob is static, while root.groups.%user is dynamic
|
|
* @param path The static path of the queue
|
|
* @param path The static path of the queue
|
|
- * @return true of the path is valid
|
|
|
|
|
|
+ * @return true if the path is valid
|
|
* @throws YarnException if the path is invalid
|
|
* @throws YarnException if the path is invalid
|
|
*/
|
|
*/
|
|
private boolean validateStaticQueuePath(MappingQueuePath path)
|
|
private boolean validateStaticQueuePath(MappingQueuePath path)
|
|
throws YarnException {
|
|
throws YarnException {
|
|
- //Try getting queue by its full path name, if it exists it is a static
|
|
|
|
- //leaf queue indeed, without any auto creation magic
|
|
|
|
- CSQueue queue = queueManager.getQueue(path.getFullPath());
|
|
|
|
- if (queue == null) {
|
|
|
|
- //We might not be able to find the queue, because the reference was
|
|
|
|
- // ambiguous this should only happen if the queue was referenced by
|
|
|
|
- // leaf name only
|
|
|
|
- if (queueManager.isAmbiguous(path.getFullPath())) {
|
|
|
|
- throw new YarnException(
|
|
|
|
- "Target queue is an ambiguous leaf queue '" +
|
|
|
|
- path.getFullPath() + "'");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //if leaf queue does not exist,
|
|
|
|
- //we need to check if the parent exists and is a managed parent
|
|
|
|
- if (!path.hasParent()) {
|
|
|
|
- throw new YarnException(
|
|
|
|
- "Target queue does not exist and has no parent defined '" +
|
|
|
|
- path.getFullPath() + "'");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- CSQueue parentQueue = queueManager.getQueue(path.getParent());
|
|
|
|
- if (parentQueue == null) {
|
|
|
|
- if (queueManager.isAmbiguous(path.getParent())) {
|
|
|
|
- throw new YarnException("Target queue path '" + path +
|
|
|
|
- "' contains an ambiguous parent queue '" +
|
|
|
|
- path.getParent() + "' reference");
|
|
|
|
- } else {
|
|
|
|
- throw new YarnException("Target queue path '" + path + "' " +
|
|
|
|
- "contains an invalid parent queue '" + path.getParent() + "'.");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (!(parentQueue instanceof ManagedParentQueue)) {
|
|
|
|
- //If the parent path was referenced by short name, and it is not
|
|
|
|
- // managed, we look up if there is a queue under it with the leaf
|
|
|
|
- // queue's name
|
|
|
|
- String normalizedParentPath = parentQueue.getQueuePath() + "."
|
|
|
|
- + path.getLeafName();
|
|
|
|
- CSQueue normalizedQueue = queueManager.getQueue(normalizedParentPath);
|
|
|
|
- if (normalizedQueue instanceof LeafQueue) {
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (normalizedQueue == null) {
|
|
|
|
- throw new YarnException(
|
|
|
|
- "Target queue '" + path.getFullPath() + "' does not exist" +
|
|
|
|
- " and has a non-managed parent queue defined.");
|
|
|
|
- } else {
|
|
|
|
- throw new YarnException("Target queue '" + path + "' references" +
|
|
|
|
- "a non-leaf queue, target queues must always be " +
|
|
|
|
- "leaf queues.");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
|
|
+ String normalizedPath = MappingRuleValidationHelper.normalizeQueuePathRoot(
|
|
|
|
+ queueManager, path.getFullPath());
|
|
|
|
+ MappingRuleValidationHelper.ValidationResult validity =
|
|
|
|
+ MappingRuleValidationHelper.validateQueuePathAutoCreation(
|
|
|
|
+ queueManager, normalizedPath);
|
|
|
|
|
|
- } else {
|
|
|
|
- // if queue exists, validate if its an instance of leaf queue
|
|
|
|
|
|
+ switch (validity) {
|
|
|
|
+ case AMBIGUOUS_PARENT:
|
|
|
|
+ throw new YarnException("Target queue path '" + path +
|
|
|
|
+ "' contains an ambiguous parent queue '" +
|
|
|
|
+ path.getParent() + "' reference.");
|
|
|
|
+ case AMBIGUOUS_QUEUE:
|
|
|
|
+ throw new YarnException("Target queue is an ambiguous leaf queue '" +
|
|
|
|
+ path.getFullPath() + "'.");
|
|
|
|
+ case EMPTY_PATH:
|
|
|
|
+ throw new YarnException("Mapping rule did not specify a target queue.");
|
|
|
|
+ case NO_PARENT_PROVIDED:
|
|
|
|
+ throw new YarnException(
|
|
|
|
+ "Target queue does not exist and has no parent defined '" +
|
|
|
|
+ path.getFullPath() + "'.");
|
|
|
|
+ case NO_DYNAMIC_PARENT:
|
|
|
|
+ throw new YarnException("Mapping rule specified a parent queue '" +
|
|
|
|
+ path.getParent() + "', but it is not a dynamic parent queue, " +
|
|
|
|
+ "and no queue exists with name '" + path.getLeafName() +
|
|
|
|
+ "' under it.");
|
|
|
|
+ case QUEUE_EXISTS:
|
|
|
|
+ CSQueue queue = queueManager.getQueue(normalizedPath);
|
|
if (!(queue instanceof LeafQueue)) {
|
|
if (!(queue instanceof LeafQueue)) {
|
|
- throw new YarnException("Target queue '" + path + "' references" +
|
|
|
|
- "a non-leaf queue, target queues must always be " +
|
|
|
|
- "leaf queues.");
|
|
|
|
|
|
+ throw new YarnException("Target queue '" + path.getFullPath() +
|
|
|
|
+ "' but it's not a leaf queue.");
|
|
}
|
|
}
|
|
|
|
+ break;
|
|
|
|
+ case CREATABLE:
|
|
|
|
+ break;
|
|
|
|
+ default:
|
|
|
|
+ //Probably the QueueCreationValidation have
|
|
|
|
+ //new items, which are not handled here
|
|
|
|
+ throw new YarnException("Unknown queue path validation result. '" +
|
|
|
|
+ validity + "'.");
|
|
}
|
|
}
|
|
|
|
+
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -133,47 +116,91 @@ public class MappingRuleValidationContextImpl
|
|
*/
|
|
*/
|
|
private boolean validateDynamicQueuePath(MappingQueuePath path)
|
|
private boolean validateDynamicQueuePath(MappingQueuePath path)
|
|
throws YarnException{
|
|
throws YarnException{
|
|
- //if the queue is dynamic and we don't have a parent path, we cannot do
|
|
|
|
- //any validation, since the dynamic part can be substituted to anything
|
|
|
|
- //and that is the only part
|
|
|
|
- if (!path.hasParent()) {
|
|
|
|
- return true;
|
|
|
|
|
|
+ ArrayList<String> parts = new ArrayList<>();
|
|
|
|
+ Collections.addAll(parts, path.getFullPath().split("\\."));
|
|
|
|
+ //How deep is the path to be created after the root element
|
|
|
|
+
|
|
|
|
+ Iterator<String> pointer = parts.iterator();
|
|
|
|
+ if (!pointer.hasNext()) {
|
|
|
|
+ //This should not happen since we only call validateDynamicQueuePath
|
|
|
|
+ //if we have found at least ONE dynamic part, which implies the path is
|
|
|
|
+ //not empty, so if we get here, I'm really curious what the path was,
|
|
|
|
+ //that's the reason we give back a theoretically "empty" path
|
|
|
|
+ throw new YarnException("Empty queue path provided '" + path + "'");
|
|
}
|
|
}
|
|
|
|
+ StringBuilder staticPartBuffer = new StringBuilder(pointer.next());
|
|
|
|
+ String staticPartParent = null;
|
|
|
|
|
|
- String parent = path.getParent();
|
|
|
|
- //if the parent path has dynamic parts, we cannot do any more validations
|
|
|
|
- if (!isPathStatic(parent)) {
|
|
|
|
|
|
+ //If not even the root of the reference is static we cannot validate
|
|
|
|
+ if (!isPathStatic(staticPartBuffer.toString())) {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
- //We check if the parent queue exists
|
|
|
|
- CSQueue parentQueue = queueManager.getQueue(parent);
|
|
|
|
- if (parentQueue == null) {
|
|
|
|
- throw new YarnException("Target queue path '" + path + "' contains an " +
|
|
|
|
- "invalid parent queue");
|
|
|
|
|
|
+ //getting the static part of the queue, we can only validate that
|
|
|
|
+ while (pointer.hasNext()) {
|
|
|
|
+ String nextPart = pointer.next();
|
|
|
|
+ if (isPathStatic(nextPart)) {
|
|
|
|
+ staticPartParent = staticPartBuffer.toString();
|
|
|
|
+ staticPartBuffer.append(DOT).append(nextPart);
|
|
|
|
+ } else {
|
|
|
|
+ //when we find the first dynamic part, we stop the search
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ String staticPart = staticPartBuffer.toString();
|
|
|
|
|
|
- if (!(parentQueue instanceof ManagedParentQueue)) {
|
|
|
|
- if (parentQueue.getChildQueues() != null) {
|
|
|
|
- for (CSQueue queue : parentQueue.getChildQueues()) {
|
|
|
|
- if (queue instanceof LeafQueue) {
|
|
|
|
- //if a non managed parent queue has at least one leaf queue, this
|
|
|
|
- //mapping can be valid, we cannot do any more checks
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ String normalizedStaticPart =
|
|
|
|
+ MappingRuleValidationHelper.normalizeQueuePathRoot(
|
|
|
|
+ queueManager, staticPart);
|
|
|
|
+ CSQueue queue = queueManager.getQueue(normalizedStaticPart);
|
|
|
|
+ //if the static part of our queue exists, and it's not a leaf queue,
|
|
|
|
+ //we cannot do any deeper validation
|
|
|
|
+ if (queue != null) {
|
|
|
|
+ if (queue instanceof LeafQueue) {
|
|
|
|
+ throw new YarnException("Queue path '" + path +"' is invalid " +
|
|
|
|
+ "because '" + normalizedStaticPart + "' is a leaf queue, " +
|
|
|
|
+ "which can have no other queues under it.");
|
|
}
|
|
}
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
|
|
- //There is no way we can place anything into the queue referenced by the
|
|
|
|
- // rule, because we cannot auto create, and we don't have any leaf queues
|
|
|
|
- //Actually this branch is not accessible with the current queue hierarchy,
|
|
|
|
- //there should be no parents without any leaf queues. This condition says
|
|
|
|
- //for sanity checks
|
|
|
|
- throw new YarnException("Target queue path '" + path + "' has " +
|
|
|
|
- "a non-managed parent queue which has no LeafQueues either.");
|
|
|
|
|
|
+ if (staticPartParent != null) {
|
|
|
|
+ String normalizedStaticPartParent
|
|
|
|
+ = MappingRuleValidationHelper.normalizeQueuePathRoot(
|
|
|
|
+ queueManager, staticPartParent);
|
|
|
|
+ queue = queueManager.getQueue(normalizedStaticPartParent);
|
|
|
|
+ //if the parent of our static part is eligible for creation, we validate
|
|
|
|
+ //this rule
|
|
|
|
+ if (isDynamicParent(queue)) {
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
- return true;
|
|
|
|
|
|
+ //at this point we cannot find any parent which is eligible for creating
|
|
|
|
+ //this path
|
|
|
|
+ throw new YarnException("No eligible parent found on path '" + path + "'.");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This method determines if a queue is eligible for being a parent queue.
|
|
|
|
+ * Since YARN-10506 not only managed parent queues can have child queues.
|
|
|
|
+ * @param queue The queue object
|
|
|
|
+ * @return true if queues can be created under this queue otherwise false
|
|
|
|
+ */
|
|
|
|
+ private boolean isDynamicParent(CSQueue queue) {
|
|
|
|
+ if (queue == null) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (queue instanceof ManagedParentQueue) {
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (queue instanceof ParentQueue) {
|
|
|
|
+ return ((ParentQueue)queue).isEligibleForAutoQueueCreation();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return false;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -186,6 +213,9 @@ public class MappingRuleValidationContextImpl
|
|
* @throws YarnException if the provided queue path is invalid
|
|
* @throws YarnException if the provided queue path is invalid
|
|
*/
|
|
*/
|
|
public boolean validateQueuePath(String queuePath) throws YarnException {
|
|
public boolean validateQueuePath(String queuePath) throws YarnException {
|
|
|
|
+ if (queuePath == null || queuePath.isEmpty()) {
|
|
|
|
+ throw new YarnException("Queue path is empty.");
|
|
|
|
+ }
|
|
MappingQueuePath path = new MappingQueuePath(queuePath);
|
|
MappingQueuePath path = new MappingQueuePath(queuePath);
|
|
|
|
|
|
if (isPathStatic(queuePath)) {
|
|
if (isPathStatic(queuePath)) {
|
|
@@ -200,11 +230,17 @@ public class MappingRuleValidationContextImpl
|
|
* A part is dynamic if a known variable is referenced in it.
|
|
* A part is dynamic if a known variable is referenced in it.
|
|
* @param queuePath The path to check
|
|
* @param queuePath The path to check
|
|
* @return true if no dynamic parts were found
|
|
* @return true if no dynamic parts were found
|
|
|
|
+ * @throws YarnException if a path part is invalid (eg. empty)
|
|
*/
|
|
*/
|
|
- public boolean isPathStatic(String queuePath) {
|
|
|
|
|
|
+ public boolean isPathStatic(String queuePath) throws YarnException {
|
|
String[] parts = queuePath.split("\\.");
|
|
String[] parts = queuePath.split("\\.");
|
|
for (int i = 0; i < parts.length; i++) {
|
|
for (int i = 0; i < parts.length; i++) {
|
|
- if (knownVariables.contains(parts[i])) {
|
|
|
|
|
|
+ if (parts[i].isEmpty()) {
|
|
|
|
+ throw new YarnException("Path segment cannot be empty '" +
|
|
|
|
+ queuePath + "'.");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (!isPathPartStatic(parts[i])) {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -212,6 +248,20 @@ public class MappingRuleValidationContextImpl
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Method to determine if the provided queue path part is dynamic.
|
|
|
|
+ * A part is dynamic if a known variable is referenced in it.
|
|
|
|
+ * @param pathPart The path part to check
|
|
|
|
+ * @return true if part is not dynamic
|
|
|
|
+ */
|
|
|
|
+ private boolean isPathPartStatic(String pathPart) {
|
|
|
|
+ if (knownVariables.contains(pathPart)) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This method will add a known variable to the validation context, known
|
|
* This method will add a known variable to the validation context, known
|
|
* variables can be used to determine if a path is static or dynamic.
|
|
* variables can be used to determine if a path is static or dynamic.
|