You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/08/26 15:34:50 UTC

[GitHub] [hadoop] 9uapaw opened a new pull request #3342: YARN-10897. Introduce QueuePath class

9uapaw opened a new pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342


   <!--
     Thanks for sending a pull request!
       1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
       2. Make sure your PR title starts with JIRA issue id, e.g., 'HADOOP-17799. Your PR title ...'.
   -->
   
   ### Description of PR
   
   
   ### How was this patch tested?
   
   
   ### For code changes:
   
   - [ ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
   - [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] brumi1024 commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
brumi1024 commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r704441333



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       I agree with @tomicooler, append indeed looks strange like this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] shuzirra commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
shuzirra commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r712132340



##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {
+    QueuePath initial = this;

Review comment:
       We don't need this as a filed, used only once, to set the initial "pointer" in the iterator.

##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {
+    QueuePath initial = this;
+
+    return new Iterator<QueuePath>() {
+      private boolean stop = false;
+      private QueuePath current = initial;
+
+      @Override
+      public boolean hasNext() {
+        return !stop;
+      }
+
+      @Override
+      public QueuePath next() {
+        QueuePath old = current;
+        if (current.hasParent()) {
+          current = new QueuePath(current.getParent());
+        }

Review comment:
       If you'd introduce an else branch here, and set current = null, then we could use current == null as the stop indicator, and we wouldn't need the stop variable at all.

##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {
+    QueuePath initial = this;
+
+    return new Iterator<QueuePath>() {
+      private boolean stop = false;
+      private QueuePath current = initial;
+
+      @Override
+      public boolean hasNext() {
+        return !stop;
+      }
+
+      @Override
+      public QueuePath next() {
+        QueuePath old = current;

Review comment:
       please add a hasNext check here, and throw a NoSuchElementException if we are out of parents. Currently we can make an infinite loop with this iterator since it will always return the last element if we are at the end of the list.

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePathTest.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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 QueuePathTest {
+  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<QueuePath> queuePathCollection = ImmutableList.copyOf(queuePath.reversePathIterator());
+    List<QueuePath> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.reversePathIterator());
+    List<QueuePath> rootPathCollection = ImmutableList.copyOf(rootPath.reversePathIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathCollection.get(3).getFullPath());
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0).getFullPath());
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(2).getFullPath());
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0).getFullPath());
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        rootPathCollection.get(0).getFullPath());
+  }
+}

Review comment:
       Please add test cases for equals as well, with the corner cases like empty path and null path, and null as other QueuePath.

##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {
+    QueuePath initial = this;
+
+    return new Iterator<QueuePath>() {
+      private boolean stop = false;
+      private QueuePath current = initial;

Review comment:
       Just use QueuePath.this instead

##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {
+    QueuePath initial = this;
+
+    return new Iterator<QueuePath>() {
+      private boolean stop = false;
+      private QueuePath current = initial;
+
+      @Override
+      public boolean hasNext() {
+        return !stop;
+      }
+
+      @Override
+      public QueuePath next() {
+        QueuePath old = current;
+        if (current.hasParent()) {
+          current = new QueuePath(current.getParent());
+        }
+
+        if (!old.hasParent()) {
+          stop = true;
+        }
+
+        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 strings = (QueuePath) o;

Review comment:
       Please consider using a bit more expressing name here, like otherPath or something like that, 'strings' is not too informative especially when it is an other QueuePath object and not a string collection :)

##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {

Review comment:
       I don't really like that the reverse path iterator iterates through QueuePath objects, it seems a bit overkill to me and also inconsistent with the regular iterator which iterates through the path parts (String).
   
   How are we going to use this, do we really need this many queue QueuePath objects? Wouldn't it be enough if we'd only return path parts just like the simple iterator?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] shuzirra commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
shuzirra commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-924024084


   @9uapaw Thank you for the patch, and for the update on this, LGTM+1.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r712388888



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePathTest.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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 QueuePathTest {
+  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<QueuePath> queuePathCollection = ImmutableList.copyOf(queuePath.reversePathIterator());
+    List<QueuePath> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.reversePathIterator());
+    List<QueuePath> rootPathCollection = ImmutableList.copyOf(rootPath.reversePathIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathCollection.get(3).getFullPath());
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0).getFullPath());
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(2).getFullPath());
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0).getFullPath());
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        rootPathCollection.get(0).getFullPath());
+  }
+}

Review comment:
       This equals is a boilerplate generated by IntelliJ. As its using Objects.equals internally, I think its similar to a wrapper, therefore I do not see a real value of a test.What is your opinion?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698338724



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       getFullPath is using this. This method is used for convenience, because a static constructor would be tedious. A new QueuePath is created here, because the whole class must maintain immutability, in order to be usable in HashSets and HashMaps.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] tomicooler commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
tomicooler commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698311057



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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();

Review comment:
       I had to try this :)
   
   With a C++ background this looked dangerous - returning a reference for a temporary object. I guess the iterator will keep the list alive in Java (probably a reference?). 

##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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<QueuePath> reversePathIterator() {

Review comment:
       This is not yet used in the production code, according the Jira it will be utilised later.

##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       I think this append method is a bit misleading. It constructs a new QueuePath and does not do anything with "this", I think a constructor would be easier to understand:
   
   QueuePath(QueuePath parent, String leaf) {}
   
   (A static factory method would do the trick as well.)

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePathTest.java
##########
@@ -0,0 +1,100 @@
+/**
+ * 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 QueuePathTest {
+  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.append("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<QueuePath> queuePathCollection = ImmutableList.copyOf(queuePath.reversePathIterator());
+    List<QueuePath> queuePathWithEmptyPartCollection = ImmutableList.copyOf(queuePathWithEmptyPart.reversePathIterator());
+    List<QueuePath> rootPathCollection = ImmutableList.copyOf(rootPath.reversePathIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, queuePathCollection.get(3).getFullPath());
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0).getFullPath());
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, queuePathWithEmptyPartCollection.get(2).getFullPath());
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0).getFullPath());
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT, rootPathCollection.get(0).getFullPath());
+  }
+}

Review comment:
       Missing new line at the end of the file.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-923305855


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  4s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 50s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 56s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 48s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 50s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  19m 59s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  20m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 51s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 55s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 47s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 47s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 39s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 51s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 51s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m  1s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 34s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 187m 11s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3342 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 0196043a0008 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3b34e621458c414c59cbd584b4ef665ebd077273 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/4/testReport/ |
   | Max. process+thread count | 954 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/4/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-918015081


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  8s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 22s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  3s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 48s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 54s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 53s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 55s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 49s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 49s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 35s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/2/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 6 new + 1 unchanged - 0 fixed = 7 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 34s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 59s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  15m 10s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 100m  2s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 35s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 177m 58s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3342 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 8e66f948be1a 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 149d1de23559f858ce89a5960f527dffd248c754 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/2/testReport/ |
   | Max. process+thread count | 935 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/2/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r712920098



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePathTest.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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 QueuePathTest {
+  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<QueuePath> queuePathCollection = ImmutableList.copyOf(queuePath.reversePathIterator());
+    List<QueuePath> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.reversePathIterator());
+    List<QueuePath> rootPathCollection = ImmutableList.copyOf(rootPath.reversePathIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathCollection.get(3).getFullPath());
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0).getFullPath());
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(2).getFullPath());
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0).getFullPath());
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        rootPathCollection.get(0).getFullPath());
+  }
+}

Review comment:
       I agree with this explanation, added a testcase for this. Thanks!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698338724



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       getFullPath is using this. Append is used for convenience, because it is more terse than its static counterpart for example:
   `
   QueuePath newPath = QueuePath.createNew(oldPath.getFullPath(), childPath);
   `
   vs.
   `
   QueuePath newPath = oldPath.append(childPath);
   `
    A new QueuePath is created here, because the whole class must maintain immutability, in order to be usable in HashSets and HashMaps.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-906645036


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 10s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  30m 49s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 56s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 48s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 50s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 52s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m  5s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 48s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 38s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/1/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager: The patch generated 6 new + 1 unchanged - 0 fixed = 7 total (was 1)  |
   | +1 :green_heart: |  mvnsite  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 53s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 52s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 45s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 35s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 176m 26s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3342 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 54bd337e3c51 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 8afd52c3978066c3a730b3e3bb89ad21fd09de49 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/1/testReport/ |
   | Max. process+thread count | 928 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/1/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] tomicooler commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
tomicooler commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-918179680


   Thanks for the review fixes. +1 from my side.
   
   The hasEmptyPart could be simplified with the newly added iterator, and there are some checkStyle warnings (longer than 100 lines).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698335410



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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<QueuePath> reversePathIterator() {

Review comment:
       Indeed, it will be used in a future refactor.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r703035287



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       Lets wait for other reviews, I could change it if others find it confusing as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] tomicooler commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
tomicooler commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698384110



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(String childQueue) {

Review comment:
       a)
   `
     public QueuePath(QueuePath parent, String leaf) {
       this.parent = parent.getFullPath();
       this.leaf = leaf;
     }
   `
   
   b)
   `
     public static QueuePath createNew(QueuePath parent, String leaf) {
       new QueuePath(parent.getFullPath(), leaf);
     }
   `
   
   c)
   rename it to createNew
   
   I think it is strange that the append() creates a new object, I would not expect that (after using the Arrays.List .append and other append methods),  but I can live with it :).

##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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<QueuePath> reversePathIterator() {

Review comment:
       Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r712385607



##########
File path: 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
##########
@@ -116,8 +120,79 @@ public boolean hasParent() {
     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<QueuePath> reversePathIterator() {

Review comment:
       I had some cases where I could use it, but I agree, let us keep this simple.  




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] shuzirra merged pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
shuzirra merged pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-924016120


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  8s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 44s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 55s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  3s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 49s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 44s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 57s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 21s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  21m 39s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 53s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 48s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 38s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 51s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 55s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m 22s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  99m 41s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 188m 52s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3342 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux c5d71d7a71dd 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c90a6fc0e88cb13b1effb01243d2706b19f5e742 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/5/testReport/ |
   | Max. process+thread count | 940 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#issuecomment-922851767


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  17m  4s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 21s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 53s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 47s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 43s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 51s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m  2s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 54s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 49s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 49s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 40s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 54s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   2m  0s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m 23s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 108m  1s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 213m 30s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3342 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux ad3c8f169004 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 32bbef316008fd56de47a454950ec354083e2757 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/3/testReport/ |
   | Max. process+thread count | 955 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3342/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] tomicooler commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
tomicooler commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698373686



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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();

Review comment:
       Thanks for the clarification.
   
   This could be extracted to a function: **getFullPath().split(QUEUE_REGEX_DELIMITER)** and reused in the **hasEmptyPart()** method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] 9uapaw commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
9uapaw commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698335050



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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();

Review comment:
       I agree that returning a pointer based on a temporary object/local variable is discouraged in non-GC languages, but it is completely safe to do so in Java (there are only some very intricate, JVM distribution specific methods that allow one to create a dangling pointer) as long as you are not returning a null explicitly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] shuzirra commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
shuzirra commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r712869638



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueuePathTest.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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 QueuePathTest {
+  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<QueuePath> queuePathCollection = ImmutableList.copyOf(queuePath.reversePathIterator());
+    List<QueuePath> queuePathWithEmptyPartCollection = ImmutableList.copyOf(
+        queuePathWithEmptyPart.reversePathIterator());
+    List<QueuePath> rootPathCollection = ImmutableList.copyOf(rootPath.reversePathIterator());
+
+    Assert.assertEquals(4, queuePathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathCollection.get(3).getFullPath());
+    Assert.assertEquals(TEST_QUEUE, queuePathCollection.get(0).getFullPath());
+
+    Assert.assertEquals(3, queuePathWithEmptyPartCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        queuePathWithEmptyPartCollection.get(2).getFullPath());
+    Assert.assertEquals("root..level_2", queuePathWithEmptyPartCollection.get(0).getFullPath());
+
+    Assert.assertEquals(1, rootPathCollection.size());
+    Assert.assertEquals(CapacitySchedulerConfiguration.ROOT,
+        rootPathCollection.get(0).getFullPath());
+  }
+}

Review comment:
       It doesn't really matter what is the origin of the code, it's always good to cover it with testcases if possible. Even if we assume the IntelliJ code is perfect, the testcases also protect the code against invalid changes, so if someone alters the code, the test can point out possible errors, and since we have a custom equals, if we extend the class we might add new fileds which can result in modified equals function. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] tomicooler commented on a change in pull request #3342: YARN-10897. Introduce QueuePath class

Posted by GitBox <gi...@apache.org>.
tomicooler commented on a change in pull request #3342:
URL: https://github.com/apache/hadoop/pull/3342#discussion_r698373686



##########
File path: 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
##########
@@ -116,8 +121,79 @@ public boolean hasParent() {
     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 append(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();

Review comment:
       Thanks for the clarification. The thread can be resolved.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org