فهرست منبع

YARN-10897. Introduce QueuePath class. Contributed by Andras Gyori

9uapaw 3 سال پیش
والد
کامیت
9f6430c9ed

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/CSMappingPlacementRule.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 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.QueuePath;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -257,7 +258,7 @@ public class CSMappingPlacementRule extends PlacementRule {
 
   private String validateAndNormalizeQueue(
       String queueName, boolean allowCreate) throws YarnException {
-    MappingQueuePath path = new MappingQueuePath(queueName);
+    QueuePath path = new QueuePath(queueName);
 
     if (path.hasEmptyPart()) {
       throw new YarnException("Invalid path returned by rule: '" +

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/MappingRuleValidationContextImpl.java

@@ -24,6 +24,7 @@ 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.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 
 import java.util.*;
@@ -61,7 +62,7 @@ public class MappingRuleValidationContextImpl
    * @return true if the path is valid
    * @throws YarnException if the path is invalid
    */
-  private boolean validateStaticQueuePath(MappingQueuePath path)
+  private boolean validateStaticQueuePath(QueuePath path)
       throws YarnException {
     String normalizedPath = MappingRuleValidationHelper.normalizeQueuePathRoot(
         queueManager, path.getFullPath());
@@ -114,7 +115,7 @@ public class MappingRuleValidationContextImpl
    * @return true of the path is valid
    * @throws YarnException if the path is invalid
    */
-  private boolean validateDynamicQueuePath(MappingQueuePath path)
+  private boolean validateDynamicQueuePath(QueuePath path)
       throws YarnException{
     ArrayList<String> parts = new ArrayList<>();
     Collections.addAll(parts, path.getFullPath().split("\\."));
@@ -216,7 +217,7 @@ public class MappingRuleValidationContextImpl
     if (queuePath == null || queuePath.isEmpty()) {
       throw new YarnException("Queue path is empty.");
     }
-    MappingQueuePath path = new MappingQueuePath(queuePath);
+    QueuePath path = new QueuePath(queuePath);
 
     if (isPathStatic(queuePath)) {
       return validateStaticQueuePath(path);

+ 84 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/csmappingrule/MappingQueuePath.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePath.java

@@ -16,7 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule;
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Objects;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
 
@@ -24,7 +29,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.C
  * This is a helper class which represents a queue path, and has easy access
  * methods to get the path's parent or leaf part, or as a whole.
  */
-public class MappingQueuePath {
+public class QueuePath implements Iterable<String> {
+  private static final String QUEUE_REGEX_DELIMITER = "\\.";
   /**
    * The parent part of the queue path.
    */
@@ -40,7 +46,7 @@ public class MappingQueuePath {
    * @param parent Parent path of the queue
    * @param leaf Name of the leaf queue
    */
-  public MappingQueuePath(String parent, String leaf) {
+  public QueuePath(String parent, String leaf) {
     this.parent = parent;
     this.leaf = leaf;
   }
@@ -49,7 +55,7 @@ public class MappingQueuePath {
    * Constructor creates a MappingQueuePath object using the queue's full path.
    * @param fullPath Full path of the queue
    */
-  public MappingQueuePath(String fullPath) {
+  public QueuePath(String fullPath) {
     setFromFullPath(fullPath);
   }
 
@@ -74,9 +80,8 @@ public class MappingQueuePath {
    * @return true if there is at least one empty part of the path
    */
   public boolean hasEmptyPart() {
-    String[] parts = getFullPath().split("\\.");
-    for (int i = 0; i < parts.length; i++) {
-      if (parts[i].equals("")) {
+    for (String part : this) {
+      if (part.isEmpty()) {
         return true;
       }
     }
@@ -116,8 +121,80 @@ public class MappingQueuePath {
     return parent != null;
   }
 
+  /**
+   * Creates a new {@code QueuePath} from the current full path as parent, and
+   * the appended child queue path as leaf.
+   * @param childQueue path of leaf queue
+   * @return new queue path made of current full path and appended leaf path
+   */
+  public QueuePath createNewLeaf(String childQueue) {
+    return new QueuePath(getFullPath(), childQueue);
+  }
+
+  /**
+   * Returns an iterator of queue path parts, starting from the highest level
+   * (generally root).
+   * @return queue part iterator
+   */
+  @Override
+  public Iterator<String> iterator() {
+    return Arrays.asList(getFullPath().split(QUEUE_REGEX_DELIMITER)).iterator();
+  }
+
+  /**
+   * Returns an iterator that provides a way to traverse the queue path from
+   * current queue through its parents.
+   * @return queue path iterator
+   */
+  public Iterator<String> reverseIterator() {
+
+    return new Iterator<String>() {
+      private String current = getFullPath();
+
+      @Override
+      public boolean hasNext() {
+        return current != null;
+      }
+
+      @Override
+      public String next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+
+        int parentQueueNameEndIndex = current.lastIndexOf(".");
+        String old = current;
+        if (parentQueueNameEndIndex > -1) {
+          current = current.substring(0, parentQueueNameEndIndex).trim();
+        } else {
+          current = null;
+        }
+
+        return old;
+      }
+    };
+  }
+
   @Override
   public String toString() {
     return getFullPath();
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    QueuePath other = (QueuePath) o;
+    return Objects.equals(parent, other.parent) &&
+        Objects.equals(leaf, other.leaf);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(parent, leaf);
+  }
 }

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/placement/converter/LegacyMappingRuleToJson.java

@@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.csmappingrule.MappingQueuePath;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -265,7 +265,7 @@ public class LegacyMappingRuleToJson {
    */
   private ObjectNode createUserMappingRule(String match, String target) {
     ObjectNode ruleNode = createDefaultRuleNode("user");
-    MappingQueuePath targetPath = new MappingQueuePath(target);
+    QueuePath targetPath = new QueuePath(target);
 
     //We have a special token in the JSON format to match all user, replacing
     //matcher
@@ -279,8 +279,8 @@ public class LegacyMappingRuleToJson {
       ruleNode.put(JSON_NODE_POLICY, "user");
       if (targetPath.hasParent()) {
         //Parsing parent path, to be able to determine the short name of parent
-        MappingQueuePath targetParentPath =
-            new MappingQueuePath(targetPath.getParent());
+        QueuePath targetParentPath =
+            new QueuePath(targetPath.getParent());
         String parentShortName = targetParentPath.getLeafName();
 
         if (parentShortName.equals(MAPPING_PRIMARY_GROUP)) {
@@ -290,7 +290,7 @@ public class LegacyMappingRuleToJson {
           //Yep, this is confusing. The policy primaryGroupUser actually
           // appends the %primary_group.%user to the parent path, so we need to
           // remove it from the parent path to avoid duplication.
-          targetPath = new MappingQueuePath(targetParentPath.getParent(),
+          targetPath = new QueuePath(targetParentPath.getParent(),
               targetPath.getLeafName());
         } else if (parentShortName.equals(MAPPING_SECONDARY_GROUP)) {
           //%secondary_group.%user mapping
@@ -299,7 +299,7 @@ public class LegacyMappingRuleToJson {
           //Yep, this is confusing. The policy secondaryGroupUser actually
           // appends the %secondary_group.%user to the parent path, so we need
           // to remove it from the parent path to avoid duplication.
-          targetPath = new MappingQueuePath(targetParentPath.getParent(),
+          targetPath = new QueuePath(targetParentPath.getParent(),
               targetPath.getLeafName());
         }
 
@@ -339,7 +339,7 @@ public class LegacyMappingRuleToJson {
    */
   private ObjectNode createGroupMappingRule(String match, String target) {
     ObjectNode ruleNode = createDefaultRuleNode("group");
-    MappingQueuePath targetPath = new MappingQueuePath(target);
+    QueuePath targetPath = new QueuePath(target);
 
     //we simply used the source match part all valid legacy matchers are valid
     //matchers for the JSON format as well
@@ -375,7 +375,7 @@ public class LegacyMappingRuleToJson {
   private ObjectNode createApplicationNameMappingRule(
       String match, String target) {
     ObjectNode ruleNode = createDefaultRuleNode("application");
-    MappingQueuePath targetPath = new MappingQueuePath(target);
+    QueuePath targetPath = new QueuePath(target);
 
     //we simply used the source match part all valid legacy matchers are valid
     //matchers for the JSON format as well

+ 119 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueuePath.java

@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+public class TestQueuePath {
+  private static final String TEST_QUEUE = "root.level_1.level_2.level_3";
+
+  @Test
+  public void testCreation() {
+    QueuePath queuePath = new QueuePath(TEST_QUEUE);
+
+    Assert.assertEquals(TEST_QUEUE, queuePath.getFullPath());
+    Assert.assertEquals("root.level_1.level_2", queuePath.getParent());
+    Assert.assertEquals("level_3", queuePath.getLeafName());
+
+    QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT);
+    Assert.assertNull(rootPath.getParent());
+
+    QueuePath appendedPath = queuePath.createNewLeaf("level_4");
+    Assert.assertEquals(TEST_QUEUE + CapacitySchedulerConfiguration.DOT
+        + "level_4", appendedPath.getFullPath());
+    Assert.assertEquals("root.level_1.level_2.level_3", appendedPath.getParent());
+    Assert.assertEquals("level_4", appendedPath.getLeafName());
+  }
+
+  @Test
+  public void testEmptyPart() {
+    QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2");
+    QueuePath queuePathWithoutEmptyPart = new QueuePath(TEST_QUEUE);
+
+    Assert.assertTrue(queuePathWithEmptyPart.hasEmptyPart());
+    Assert.assertFalse(queuePathWithoutEmptyPart.hasEmptyPart());
+  }
+
+  @Test
+  public void testIterator() {
+    QueuePath queuePath = new QueuePath(TEST_QUEUE);
+    QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2");
+    QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT);
+
+    List<String> queuePathCollection = ImmutableList.copyOf(queuePath.iterator());
+    List<String> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.iterator());
+    List<String> rootPathCollection = ImmutableList.copyOf(rootPath.iterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, queuePathCollection.get(0));
+    Assert.assertEquals("level_3", queuePathCollection.get(3));
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(0));
+    Assert.assertEquals("level_2", queuePathWithEmptyPartCollection.get(2));
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, rootPathCollection.get(0));
+  }
+
+  @Test
+  public void testReversePathIterator() {
+    QueuePath queuePath = new QueuePath(TEST_QUEUE);
+    QueuePath queuePathWithEmptyPart = new QueuePath("root..level_2");
+    QueuePath rootPath = new QueuePath(CapacitySchedulerConfiguration.ROOT);
+
+    List<String> queuePathCollection = ImmutableList.copyOf(queuePath.reverseIterator());
+    List<String> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.reverseIterator());
+    List<String> rootPathCollection = ImmutableList.copyOf(rootPath.reverseIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathCollection.get(3));
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0));
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(2));
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0));
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        rootPathCollection.get(0));
+  }
+
+  @Test
+  public void testEquals() {
+    QueuePath queuePath = new QueuePath(TEST_QUEUE);
+    QueuePath queuePathSame = new QueuePath(TEST_QUEUE);
+
+    QueuePath empty = new QueuePath("");
+    QueuePath emptySame = new QueuePath("");
+
+    Assert.assertEquals(queuePath, queuePathSame);
+    Assert.assertEquals(empty, emptySame);
+    Assert.assertNotEquals(null, queuePath);
+  }
+}