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/11/24 18:17:41 UTC

[GitHub] [hadoop] afchung opened a new pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

afchung opened a new pull request #3717:
URL: https://github.com/apache/hadoop/pull/3717


   ### Description of PR
   * Implement the `QUEUE_LENGTH_THEN_RESOURCES` `LoadComparator`
   * Add `Resource` requested to select node methods when allocating
   `OPPORTUNISTIC` containers
   * Add node capacity and node allocated resources to `ClusterNode`
   * Extend `DominantResourceCalculator` to be able to compute min-share
   * Add tests and extend existing ones to consider 3 resource dimensions
   
   ### How was this patch tested?
   * Unit tests
   * Deployment to a production cluster
   
   


-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -261,12 +391,15 @@ protected void onNewNodeAdded(
 
     if (rmNode.getState() != NodeState.DECOMMISSIONING &&
         (estimatedQueueWaitTime != -1 ||
-            comparator == LoadComparator.QUEUE_LENGTH)) {
+            comparator == LoadComparator.QUEUE_LENGTH ||
+            comparator == LoadComparator.QUEUE_LENGTH_THEN_RESOURCES)) {
       this.clusterNodes.put(rmNode.getNodeID(),

Review comment:
       Every second we are taking too many write locks of ClusterNode. IIUC that not the best optimized option. We should probably take the write lock once and update all the values like waitTime,SetQueueLength,labels etc..




-- 
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] afchung commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getAvailableResource() {
+    readLock.lock();
+    try {
+      return Resources.subtractNonNegative(capability, allocatedResource);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getCapability() {
+    readLock.lock();
+    try {
+      return this.capability;
+    }
+      finally {
+      readLock.unlock();
+    }
+  }
+
   public ClusterNode setQueueLength(int qLength) {
-    this.queueLength.set(qLength);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueLength = qLength;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueWaitTime(int wTime) {
-    this.queueWaitTime.set(wTime);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueWaitTime = wTime;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode updateTimestamp() {
-    this.timestamp = System.currentTimeMillis();
-    return this;
+    writeLock.lock();
+    try {
+      this.timestamp = System.currentTimeMillis();
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueCapacity(int capacity) {
-    this.queueCapacity = capacity;
-    return this;
+    writeLock.lock();
+    try {
+      this.queueCapacity = capacity;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setNodeLabels(Collection<String> labelsToAdd) {
-    labels.clear();
-    labels.addAll(labelsToAdd);
-    return this;
+    writeLock.lock();
+    try {
+      labels.clear();
+      labels.addAll(labelsToAdd);
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public boolean hasLabel(String label) {
-    return this.labels.contains(label);
+    readLock.lock();
+    try {
+      return this.labels.contains(label);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public long getTimestamp() {
-    return this.timestamp;
+    readLock.lock();
+    try {
+      return this.timestamp;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueLength() {
-    return this.queueLength;
+  public int getQueueLength() {
+    readLock.lock();
+    try {
+      return this.queueLength;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueWaitTime() {
-    return this.queueWaitTime;
+  public int getQueueWaitTime() {
+    readLock.lock();
+    try {
+      return this.queueWaitTime;
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public int getQueueCapacity() {
-    return this.queueCapacity;
+    readLock.lock();
+    try {
+      return this.queueCapacity;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(
+      final int incrementQLen,
+      final ResourceCalculator resourceCalculator,
+      final Resource requested) {
+    writeLock.lock();
+    try {
+      final Resource currAvailable = Resources.subtractNonNegative(
+          capability, allocatedResource);
+      if (resourceCalculator.fitsIn(requested, currAvailable)) {
+        allocatedResource = Resources.add(allocatedResource, requested);
+        return true;
+      }
+
+      if (!resourceCalculator.fitsIn(requested, capability)) {
+        // If does not fit at all, do not allocate
+        return false;
+      }
+
+      return compareAndIncrementAllocation(incrementQLen);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(

Review comment:
       How so? We need to set `queueLength` if it fits, and ignore it if it does not.




-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -295,11 +428,14 @@ protected void onExistingNodeUpdated(
 
     if (rmNode.getState() != NodeState.DECOMMISSIONING &&
         (estimatedQueueWaitTime != -1 ||
-            comparator == LoadComparator.QUEUE_LENGTH)) {
+            comparator == LoadComparator.QUEUE_LENGTH ||
+            comparator == LoadComparator.QUEUE_LENGTH_THEN_RESOURCES)) {
       clusterNode
           .setQueueWaitTime(estimatedQueueWaitTime)

Review comment:
       Every second we are taking too many write locks of ClusterNode. IIUC that not the best optimized option. We should probably take the write lock once and update all the values like waitTime,SetQueueLength,labels etc .




-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java
##########
@@ -186,6 +243,152 @@ public void testQueueLengthSort() {
     Assert.assertEquals("h4:4", nodeIds.get(2).toString());
   }
 
+  @Test
+  public void testQueueLengthThenResourcesSort() {

Review comment:
       Could you add a test case when the resource size of node changes. How its getting handled in O containers




-- 
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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 53s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 56s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  24m  7s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  10m 20s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 37s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 38s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  1s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 39s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 51s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m 49s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   9m 41s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   9m 41s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 34s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 34s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   1m 52s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 35s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  24m 31s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 42s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 100m  6s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 46s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 254m  5s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 3447b6ade702 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / d422e60e036ee26685459be3662853a2f5ddb894 |
   | 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-3717/5/testReport/ |
   | Max. process+thread count | 931 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m 24s |  |  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 _ |
   | +0 :ok: |  mvndep  |  13m  9s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  27m 47s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  12m 55s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 49s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  1s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 51s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  24m 54s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   9m 34s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   9m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 39s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 39s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 36s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/2/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 4 new + 17 unchanged - 0 fixed = 21 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   1m 52s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 47s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/2/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-common in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javadoc  |   0m 49s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/2/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-common in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  spotbugs  |   2m 13s | [/new-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/2/artifact/out/new-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.html) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  24m 29s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 43s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 100m 28s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 45s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 261m 55s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   |  |  org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor$LoadComparator.setClusterResource(Resource) unconditionally sets the field clusterResource  At NodeQueueLoadMonitor.java:clusterResource  At NodeQueueLoadMonitor.java:[line 155] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux a5cb5177c60b 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 98da8de01d912e092c2b482f4f52d4634e1852bc |
   | 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-3717/2/testReport/ |
   | Max. process+thread count | 938 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/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] goiri merged pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   


-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -64,40 +69,165 @@
    * of two Nodes are compared.
    */
   public enum LoadComparator implements Comparator<ClusterNode> {
+    /**
+     * This policy only considers queue length.
+     * When allocating, increments queue length without looking at resources
+     * available on the node, and when sorting, also only sorts by queue length.
+     */
     QUEUE_LENGTH,
-    QUEUE_WAIT_TIME;
+    /**
+     * This policy only considers the wait time of containers in the queue.
+     * Neither looks at resources nor at queue length.
+     */
+    QUEUE_WAIT_TIME,
+    /**
+     * This policy considers both queue length and resources.
+     * When allocating, first decrements resources available on a node.
+     * If resources are available, does not place OContainers on the node queue.
+     * When sorting, it first sorts by queue length,
+     * then by available resources.
+     */
+    QUEUE_LENGTH_THEN_RESOURCES;
+
+    private Resource clusterResource = Resources.none();
+    private final DominantResourceCalculator resourceCalculator =
+        new DominantResourceCalculator();
+
+    private Resource computeAvailableResource(final ClusterNode clusterNode) {
+      return Resources.subtractNonNegative(
+          clusterNode.getCapability(),
+          clusterNode.getAllocatedResource());
+    }
 
     @Override
     public int compare(ClusterNode o1, ClusterNode o2) {
-      if (getMetric(o1) == getMetric(o2)) {
-        return (int)(o2.getTimestamp() - o1.getTimestamp());
+      int diff = 0;
+      switch (this) {
+      case QUEUE_LENGTH_THEN_RESOURCES:
+        diff = o1.getQueueLength().get() - o2.getQueueLength().get();
+        if (diff != 0) {
+          break;
+        }
+
+        // NOTE: cluster resource should be
+        // set always before LoadComparator is used
+        final Resource availableResource1 = computeAvailableResource(o1);
+        final Resource availableResource2 = computeAvailableResource(o2);
+        final boolean isClusterResourceLeq0 = resourceCalculator
+            .isAnyMajorResourceZeroOrNegative(clusterResource);
+        if (!isClusterResourceLeq0) {
+          // Takes the least available resource of the two nodes,
+          // normalized to the overall cluster resource
+          final float availableRatio1 =
+              resourceCalculator.minRatio(availableResource1, clusterResource);
+          final float availableRatio2 =
+              resourceCalculator.minRatio(availableResource2, clusterResource);
+
+          // The one with more available resources should be placed first
+          diff = Precision
+              .compareTo(availableRatio2, availableRatio1, Precision.EPSILON);
+        }
+
+        if (diff == 0) {
+          // Compare absolute value if ratios are the same
+          diff = availableResource2.getVirtualCores() - availableResource1.getVirtualCores();
+        }
+
+        if (diff == 0) {
+          diff = Long.compare(availableResource2.getMemorySize(),
+              availableResource1.getMemorySize());
+        }
+        break;
+      case QUEUE_WAIT_TIME:
+      case QUEUE_LENGTH:
+      default:
+        diff = getMetric(o1) - getMetric(o2);
+        break;
+      }
+
+      if (diff == 0) {
+        return (int) (o2.getTimestamp() - o1.getTimestamp());
       }
-      return getMetric(o1) - getMetric(o2);
+      return diff;
+    }
+
+    private void setClusterResource(Resource clusterResource) {
+      this.clusterResource = clusterResource;
+    }
+
+    public ResourceCalculator getResourceCalculator() {
+      return resourceCalculator;
     }
 
     public int getMetric(ClusterNode c) {
-      return (this == QUEUE_LENGTH) ?
-          c.getQueueLength().get() : c.getQueueWaitTime().get();
+      switch (this) {
+      case QUEUE_WAIT_TIME:
+        return c.getQueueWaitTime().get();
+      case QUEUE_LENGTH:
+      case QUEUE_LENGTH_THEN_RESOURCES:
+      default:
+        return c.getQueueLength().get();
+      }
     }
 
     /**
      * Increment the metric by a delta if it is below the threshold.
      * @param c ClusterNode
      * @param incrementSize increment size
+     * @param requested the requested resource
      * @return true if the metric was below threshold and was incremented.
      */
-    public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
-      if(this == QUEUE_LENGTH) {
-        int ret = c.getQueueLength().addAndGet(incrementSize);
-        if (ret <= c.getQueueCapacity()) {
+    public boolean compareAndIncrement(
+        ClusterNode c, int incrementSize, Resource requested) {
+      if (this == QUEUE_LENGTH_THEN_RESOURCES) {
+        // Assignment and getting value is atomic
+        // Can be slightly inaccurate here, don't grab lock for performance
+        final Resource capability = c.getCapability();
+        final Resource currAllocated = c.getAllocatedResource();
+        final Resource currAvailable = Resources.subtractNonNegative(
+            capability, currAllocated);
+        if (resourceCalculator.fitsIn(requested, currAvailable)) {
+          final Resource newAllocated = Resources.add(currAllocated, requested);
+          c.setAllocatedResource(newAllocated);

Review comment:
       with multiple application running in parallel the allocated resource update could be wrong here. Do we take clusterNode level lock ??




-- 
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] afchung commented on pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   @goiri thanks for the additional round of review! Latest commit should address your comments, please let me know if any other concerns.


-- 
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] afchung commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -295,11 +428,14 @@ protected void onExistingNodeUpdated(
 
     if (rmNode.getState() != NodeState.DECOMMISSIONING &&
         (estimatedQueueWaitTime != -1 ||
-            comparator == LoadComparator.QUEUE_LENGTH)) {
+            comparator == LoadComparator.QUEUE_LENGTH ||
+            comparator == LoadComparator.QUEUE_LENGTH_THEN_RESOURCES)) {
       clusterNode
           .setQueueWaitTime(estimatedQueueWaitTime)

Review comment:
       Good catch, will address in a new iteration.




-- 
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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  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 _ |
   | +0 :ok: |  mvndep  |  13m  5s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 27s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  10m  0s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 25s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 42s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 15s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 58s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 57s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 24s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 29s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 14s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 14s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   2m  5s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 33s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 57s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m 12s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 242m  1s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux b455c0b7db98 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / fb36f73b50e128fbc49cdb096bdd07858a29f300 |
   | 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-3717/4/testReport/ |
   | Max. process+thread count | 969 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 43s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 47s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 36s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 18s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 10s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  3s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 57s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 57s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 25s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 55s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 14s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 14s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 36s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 4 new + 17 unchanged - 0 fixed = 21 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  4s |  |  the patch passed  |
   | -1 :x: |  javadoc  |   0m 54s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-yarn-common in the patch failed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.  |
   | -1 :x: |  javadoc  |   0m 53s | [/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-common in the patch failed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.  |
   | -1 :x: |  javadoc  |   0m 55s | [/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/results-javadoc-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 344 unchanged - 0 fixed = 345 total (was 344)  |
   | -1 :x: |  spotbugs  |   2m 13s | [/new-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/new-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.html) |  hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  shadedclient  |  21m 32s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 59s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m 18s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 241m 31s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | SpotBugs | module:hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager |
   |  |  org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor$LoadComparator.setClusterResource(Resource) unconditionally sets the field clusterResource  At NodeQueueLoadMonitor.java:clusterResource  At NodeQueueLoadMonitor.java:[line 155] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fa690726bd20 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / d4e172fffcbffd7c550c261a786cf53bbf48e22f |
   | 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-3717/1/testReport/ |
   | Max. process+thread count | 953 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/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] goiri commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }

Review comment:
       } finally {

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -64,39 +69,141 @@
    * of two Nodes are compared.
    */
   public enum LoadComparator implements Comparator<ClusterNode> {
+    /**
+     * This policy only considers queue length.
+     * When allocating, increments queue length without looking at resources
+     * available on the node, and when sorting, also only sorts by queue length.
+     */
     QUEUE_LENGTH,
-    QUEUE_WAIT_TIME;
+    /**
+     * This policy only considers the wait time of containers in the queue.
+     * Neither looks at resources nor at queue length.
+     */
+    QUEUE_WAIT_TIME,
+    /**
+     * This policy considers both queue length and resources.
+     * When allocating, first decrements resources available on a node.
+     * If resources are available, does not place OContainers on the node queue.
+     * When sorting, it first sorts by queue length,
+     * then by available resources.
+     */
+    QUEUE_LENGTH_THEN_RESOURCES;
+
+    private Resource clusterResource = Resources.none();
+    private final DominantResourceCalculator resourceCalculator =
+        new DominantResourceCalculator();
 
     @Override
     public int compare(ClusterNode o1, ClusterNode o2) {
-      if (getMetric(o1) == getMetric(o2)) {
-        return (int)(o2.getTimestamp() - o1.getTimestamp());
+      int diff = 0;
+      switch (this) {
+      case QUEUE_LENGTH_THEN_RESOURCES:
+        diff = o1.getQueueLength() - o2.getQueueLength();
+        if (diff != 0) {
+          break;
+        }
+
+        // NOTE: cluster resource should be

Review comment:
       Do we just go ahead with leaving a note?
   It doesn't seem very maintanable.

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getAvailableResource() {
+    readLock.lock();
+    try {
+      return Resources.subtractNonNegative(capability, allocatedResource);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getCapability() {
+    readLock.lock();
+    try {
+      return this.capability;
+    }

Review comment:
       } finally {

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -64,39 +69,141 @@
    * of two Nodes are compared.
    */
   public enum LoadComparator implements Comparator<ClusterNode> {
+    /**
+     * This policy only considers queue length.
+     * When allocating, increments queue length without looking at resources
+     * available on the node, and when sorting, also only sorts by queue length.
+     */
     QUEUE_LENGTH,
-    QUEUE_WAIT_TIME;
+    /**
+     * This policy only considers the wait time of containers in the queue.
+     * Neither looks at resources nor at queue length.
+     */
+    QUEUE_WAIT_TIME,
+    /**
+     * This policy considers both queue length and resources.
+     * When allocating, first decrements resources available on a node.
+     * If resources are available, does not place OContainers on the node queue.
+     * When sorting, it first sorts by queue length,
+     * then by available resources.
+     */
+    QUEUE_LENGTH_THEN_RESOURCES;
+
+    private Resource clusterResource = Resources.none();
+    private final DominantResourceCalculator resourceCalculator =
+        new DominantResourceCalculator();
 
     @Override
     public int compare(ClusterNode o1, ClusterNode o2) {
-      if (getMetric(o1) == getMetric(o2)) {
-        return (int)(o2.getTimestamp() - o1.getTimestamp());
+      int diff = 0;
+      switch (this) {
+      case QUEUE_LENGTH_THEN_RESOURCES:
+        diff = o1.getQueueLength() - o2.getQueueLength();

Review comment:
       This math is a little intense.
   It might be better to extract it to a function just for this so you can return things right away and have comments throughout.

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -374,7 +491,7 @@ public RMNode selectRackLocalNode(String rackName, Set<String> blacklist) {
     return null;
   }
 
-  public RMNode selectAnyNode(Set<String> blacklist) {
+  public RMNode selectAnyNode(Set<String> blacklist, Resource request) {

Review comment:
       We should add javadocs explaining what these parameters mean.

##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getAvailableResource() {
+    readLock.lock();
+    try {
+      return Resources.subtractNonNegative(capability, allocatedResource);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getCapability() {
+    readLock.lock();
+    try {
+      return this.capability;
+    }
+      finally {
+      readLock.unlock();
+    }
+  }
+
   public ClusterNode setQueueLength(int qLength) {
-    this.queueLength.set(qLength);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueLength = qLength;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueWaitTime(int wTime) {
-    this.queueWaitTime.set(wTime);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueWaitTime = wTime;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode updateTimestamp() {
-    this.timestamp = System.currentTimeMillis();
-    return this;
+    writeLock.lock();
+    try {
+      this.timestamp = System.currentTimeMillis();
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueCapacity(int capacity) {
-    this.queueCapacity = capacity;
-    return this;
+    writeLock.lock();
+    try {
+      this.queueCapacity = capacity;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setNodeLabels(Collection<String> labelsToAdd) {
-    labels.clear();
-    labels.addAll(labelsToAdd);
-    return this;
+    writeLock.lock();
+    try {
+      labels.clear();
+      labels.addAll(labelsToAdd);
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public boolean hasLabel(String label) {
-    return this.labels.contains(label);
+    readLock.lock();
+    try {
+      return this.labels.contains(label);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public long getTimestamp() {
-    return this.timestamp;
+    readLock.lock();
+    try {
+      return this.timestamp;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueLength() {
-    return this.queueLength;
+  public int getQueueLength() {
+    readLock.lock();
+    try {
+      return this.queueLength;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueWaitTime() {
-    return this.queueWaitTime;
+  public int getQueueWaitTime() {
+    readLock.lock();
+    try {
+      return this.queueWaitTime;
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public int getQueueCapacity() {
-    return this.queueCapacity;
+    readLock.lock();
+    try {
+      return this.queueCapacity;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(
+      final int incrementQLen,
+      final ResourceCalculator resourceCalculator,
+      final Resource requested) {
+    writeLock.lock();
+    try {
+      final Resource currAvailable = Resources.subtractNonNegative(
+          capability, allocatedResource);
+      if (resourceCalculator.fitsIn(requested, currAvailable)) {
+        allocatedResource = Resources.add(allocatedResource, requested);
+        return true;
+      }
+
+      if (!resourceCalculator.fitsIn(requested, capability)) {
+        // If does not fit at all, do not allocate
+        return false;
+      }
+
+      return compareAndIncrementAllocation(incrementQLen);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(

Review comment:
       Doesn't this fit in one line?




-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -295,11 +428,14 @@ protected void onExistingNodeUpdated(
 
     if (rmNode.getState() != NodeState.DECOMMISSIONING &&
         (estimatedQueueWaitTime != -1 ||
-            comparator == LoadComparator.QUEUE_LENGTH)) {
+            comparator == LoadComparator.QUEUE_LENGTH ||
+            comparator == LoadComparator.QUEUE_LENGTH_THEN_RESOURCES)) {
       clusterNode
           .setQueueWaitTime(estimatedQueueWaitTime)

Review comment:
       Every second we are taking too many write locks of ClusterNode. IIUC that not the best optimized option. We should probably take the write lock once and update all the values like waitTime,SetQueueLength,labels etc..We could update this in followup PR also..




-- 
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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 58s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 31s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  24m 15s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  10m 20s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 55s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  4s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  26m 19s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 54s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  10m 32s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |  10m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   9m  9s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   9m  9s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 39s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/9/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 1 new + 17 unchanged - 0 fixed = 18 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  6s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m 12s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  24m 45s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   5m  2s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 102m 39s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 46s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 261m 47s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux f18036f66258 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 63f11d695c16c064d268f8ad8de25952ba1fd3d6 |
   | 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-3717/9/testReport/ |
   | Max. process+thread count | 937 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/9/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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 42s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  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 _ |
   | +0 :ok: |  mvndep  |  13m 28s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 53s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 38s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 19s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 42s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 14s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 57s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 54s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 41s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 16s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 16s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   1m 33s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   2m  6s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 51s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 40s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 57s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m  1s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 51s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 242m 41s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 6154f7521f97 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b9939ed23f8d32b0c02271efaa15ae7a9b5e6cd5 |
   | 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-3717/3/testReport/ |
   | Max. process+thread count | 950 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/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] afchung commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getAvailableResource() {
+    readLock.lock();
+    try {
+      return Resources.subtractNonNegative(capability, allocatedResource);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getCapability() {
+    readLock.lock();
+    try {
+      return this.capability;
+    }
+      finally {
+      readLock.unlock();
+    }
+  }
+
   public ClusterNode setQueueLength(int qLength) {
-    this.queueLength.set(qLength);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueLength = qLength;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueWaitTime(int wTime) {
-    this.queueWaitTime.set(wTime);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueWaitTime = wTime;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode updateTimestamp() {
-    this.timestamp = System.currentTimeMillis();
-    return this;
+    writeLock.lock();
+    try {
+      this.timestamp = System.currentTimeMillis();
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueCapacity(int capacity) {
-    this.queueCapacity = capacity;
-    return this;
+    writeLock.lock();
+    try {
+      this.queueCapacity = capacity;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setNodeLabels(Collection<String> labelsToAdd) {
-    labels.clear();
-    labels.addAll(labelsToAdd);
-    return this;
+    writeLock.lock();
+    try {
+      labels.clear();
+      labels.addAll(labelsToAdd);
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public boolean hasLabel(String label) {
-    return this.labels.contains(label);
+    readLock.lock();
+    try {
+      return this.labels.contains(label);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public long getTimestamp() {
-    return this.timestamp;
+    readLock.lock();
+    try {
+      return this.timestamp;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueLength() {
-    return this.queueLength;
+  public int getQueueLength() {
+    readLock.lock();
+    try {
+      return this.queueLength;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueWaitTime() {
-    return this.queueWaitTime;
+  public int getQueueWaitTime() {
+    readLock.lock();
+    try {
+      return this.queueWaitTime;
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public int getQueueCapacity() {
-    return this.queueCapacity;
+    readLock.lock();
+    try {
+      return this.queueCapacity;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(
+      final int incrementQLen,
+      final ResourceCalculator resourceCalculator,
+      final Resource requested) {
+    writeLock.lock();
+    try {
+      final Resource currAvailable = Resources.subtractNonNegative(
+          capability, allocatedResource);
+      if (resourceCalculator.fitsIn(requested, currAvailable)) {
+        allocatedResource = Resources.add(allocatedResource, requested);
+        return true;
+      }
+
+      if (!resourceCalculator.fitsIn(requested, capability)) {
+        // If does not fit at all, do not allocate
+        return false;
+      }
+
+      return compareAndIncrementAllocation(incrementQLen);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(

Review comment:
       Fixed, good catch.




-- 
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] afchung commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -64,40 +69,165 @@
    * of two Nodes are compared.
    */
   public enum LoadComparator implements Comparator<ClusterNode> {
+    /**
+     * This policy only considers queue length.
+     * When allocating, increments queue length without looking at resources
+     * available on the node, and when sorting, also only sorts by queue length.
+     */
     QUEUE_LENGTH,
-    QUEUE_WAIT_TIME;
+    /**
+     * This policy only considers the wait time of containers in the queue.
+     * Neither looks at resources nor at queue length.
+     */
+    QUEUE_WAIT_TIME,
+    /**
+     * This policy considers both queue length and resources.
+     * When allocating, first decrements resources available on a node.
+     * If resources are available, does not place OContainers on the node queue.
+     * When sorting, it first sorts by queue length,
+     * then by available resources.
+     */
+    QUEUE_LENGTH_THEN_RESOURCES;
+
+    private Resource clusterResource = Resources.none();
+    private final DominantResourceCalculator resourceCalculator =
+        new DominantResourceCalculator();
+
+    private Resource computeAvailableResource(final ClusterNode clusterNode) {
+      return Resources.subtractNonNegative(
+          clusterNode.getCapability(),
+          clusterNode.getAllocatedResource());
+    }
 
     @Override
     public int compare(ClusterNode o1, ClusterNode o2) {
-      if (getMetric(o1) == getMetric(o2)) {
-        return (int)(o2.getTimestamp() - o1.getTimestamp());
+      int diff = 0;
+      switch (this) {
+      case QUEUE_LENGTH_THEN_RESOURCES:
+        diff = o1.getQueueLength().get() - o2.getQueueLength().get();
+        if (diff != 0) {
+          break;
+        }
+
+        // NOTE: cluster resource should be
+        // set always before LoadComparator is used
+        final Resource availableResource1 = computeAvailableResource(o1);
+        final Resource availableResource2 = computeAvailableResource(o2);
+        final boolean isClusterResourceLeq0 = resourceCalculator
+            .isAnyMajorResourceZeroOrNegative(clusterResource);
+        if (!isClusterResourceLeq0) {
+          // Takes the least available resource of the two nodes,
+          // normalized to the overall cluster resource
+          final float availableRatio1 =
+              resourceCalculator.minRatio(availableResource1, clusterResource);
+          final float availableRatio2 =
+              resourceCalculator.minRatio(availableResource2, clusterResource);
+
+          // The one with more available resources should be placed first
+          diff = Precision
+              .compareTo(availableRatio2, availableRatio1, Precision.EPSILON);
+        }
+
+        if (diff == 0) {
+          // Compare absolute value if ratios are the same
+          diff = availableResource2.getVirtualCores() - availableResource1.getVirtualCores();
+        }
+
+        if (diff == 0) {
+          diff = Long.compare(availableResource2.getMemorySize(),
+              availableResource1.getMemorySize());
+        }
+        break;
+      case QUEUE_WAIT_TIME:
+      case QUEUE_LENGTH:
+      default:
+        diff = getMetric(o1) - getMetric(o2);
+        break;
+      }
+
+      if (diff == 0) {
+        return (int) (o2.getTimestamp() - o1.getTimestamp());
       }
-      return getMetric(o1) - getMetric(o2);
+      return diff;
+    }
+
+    private void setClusterResource(Resource clusterResource) {
+      this.clusterResource = clusterResource;
+    }
+
+    public ResourceCalculator getResourceCalculator() {
+      return resourceCalculator;
     }
 
     public int getMetric(ClusterNode c) {
-      return (this == QUEUE_LENGTH) ?
-          c.getQueueLength().get() : c.getQueueWaitTime().get();
+      switch (this) {
+      case QUEUE_WAIT_TIME:
+        return c.getQueueWaitTime().get();
+      case QUEUE_LENGTH:
+      case QUEUE_LENGTH_THEN_RESOURCES:
+      default:
+        return c.getQueueLength().get();
+      }
     }
 
     /**
      * Increment the metric by a delta if it is below the threshold.
      * @param c ClusterNode
      * @param incrementSize increment size
+     * @param requested the requested resource
      * @return true if the metric was below threshold and was incremented.
      */
-    public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
-      if(this == QUEUE_LENGTH) {
-        int ret = c.getQueueLength().addAndGet(incrementSize);
-        if (ret <= c.getQueueCapacity()) {
+    public boolean compareAndIncrement(
+        ClusterNode c, int incrementSize, Resource requested) {
+      if (this == QUEUE_LENGTH_THEN_RESOURCES) {
+        // Assignment and getting value is atomic
+        // Can be slightly inaccurate here, don't grab lock for performance
+        final Resource capability = c.getCapability();
+        final Resource currAllocated = c.getAllocatedResource();
+        final Resource currAvailable = Resources.subtractNonNegative(
+            capability, currAllocated);
+        if (resourceCalculator.fitsIn(requested, currAvailable)) {
+          final Resource newAllocated = Resources.add(currAllocated, requested);
+          c.setAllocatedResource(newAllocated);

Review comment:
       Yeah, I saw that since the original code also wasn't handling parallelism correctly I thought it might've been fine to not use locks, but ensuring correctness is probably better here.




-- 
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] goiri commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/ClusterNode.java
##########
@@ -20,74 +20,239 @@
 
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
  */
 public class ClusterNode {
-  private final AtomicInteger queueLength = new AtomicInteger(0);
-  private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
+  private int queueLength = 0;
+  private int queueWaitTime = -1;
   private long timestamp;
   final NodeId nodeId;
   private int queueCapacity = 0;
   private final HashSet<String> labels;
+  private Resource capability = Resources.none();
+  private Resource allocatedResource = Resources.none();
+  private final ReentrantReadWriteLock.WriteLock writeLock;
+  private final ReentrantReadWriteLock.ReadLock readLock;
 
   public ClusterNode(NodeId nodeId) {
     this.nodeId = nodeId;
     this.labels = new HashSet<>();
+    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    writeLock = lock.writeLock();
+    readLock = lock.readLock();
     updateTimestamp();
   }
 
+  public ClusterNode setCapability(Resource nodeCapability) {
+    writeLock.lock();
+    try {
+      if (nodeCapability == null) {
+        this.capability = Resources.none();
+      } else {
+        this.capability = nodeCapability;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public ClusterNode setAllocatedResource(
+      Resource allocResource) {
+    writeLock.lock();
+    try {
+      if (allocResource == null) {
+        this.allocatedResource = Resources.none();
+      } else {
+        this.allocatedResource = allocResource;
+      }
+      return this;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public Resource getAllocatedResource() {
+    readLock.lock();
+    try {
+      return this.allocatedResource;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getAvailableResource() {
+    readLock.lock();
+    try {
+      return Resources.subtractNonNegative(capability, allocatedResource);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public Resource getCapability() {
+    readLock.lock();
+    try {
+      return this.capability;
+    }
+      finally {
+      readLock.unlock();
+    }
+  }
+
   public ClusterNode setQueueLength(int qLength) {
-    this.queueLength.set(qLength);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueLength = qLength;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueWaitTime(int wTime) {
-    this.queueWaitTime.set(wTime);
-    return this;
+    writeLock.lock();
+    try {
+      this.queueWaitTime = wTime;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode updateTimestamp() {
-    this.timestamp = System.currentTimeMillis();
-    return this;
+    writeLock.lock();
+    try {
+      this.timestamp = System.currentTimeMillis();
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setQueueCapacity(int capacity) {
-    this.queueCapacity = capacity;
-    return this;
+    writeLock.lock();
+    try {
+      this.queueCapacity = capacity;
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public ClusterNode setNodeLabels(Collection<String> labelsToAdd) {
-    labels.clear();
-    labels.addAll(labelsToAdd);
-    return this;
+    writeLock.lock();
+    try {
+      labels.clear();
+      labels.addAll(labelsToAdd);
+      return this;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public boolean hasLabel(String label) {
-    return this.labels.contains(label);
+    readLock.lock();
+    try {
+      return this.labels.contains(label);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public long getTimestamp() {
-    return this.timestamp;
+    readLock.lock();
+    try {
+      return this.timestamp;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueLength() {
-    return this.queueLength;
+  public int getQueueLength() {
+    readLock.lock();
+    try {
+      return this.queueLength;
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public AtomicInteger getQueueWaitTime() {
-    return this.queueWaitTime;
+  public int getQueueWaitTime() {
+    readLock.lock();
+    try {
+      return this.queueWaitTime;
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public int getQueueCapacity() {
-    return this.queueCapacity;
+    readLock.lock();
+    try {
+      return this.queueCapacity;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(
+      final int incrementQLen,
+      final ResourceCalculator resourceCalculator,
+      final Resource requested) {
+    writeLock.lock();
+    try {
+      final Resource currAvailable = Resources.subtractNonNegative(
+          capability, allocatedResource);
+      if (resourceCalculator.fitsIn(requested, currAvailable)) {
+        allocatedResource = Resources.add(allocatedResource, requested);
+        return true;
+      }
+
+      if (!resourceCalculator.fitsIn(requested, capability)) {
+        // If does not fit at all, do not allocate
+        return false;
+      }
+
+      return compareAndIncrementAllocation(incrementQLen);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean compareAndIncrementAllocation(

Review comment:
       I mean in terms of style, I think:
   ```
   public boolean compareAndIncrementAllocation(final int incrementQLen) {
   ```
   Fits in less than 100 characters (it may even be 120 now.




-- 
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] afchung commented on pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   @goiri and @bibinchundatt thanks for your reviews!
   @bibinchundatt I've addressed your comment by adding an additional unit test. Please let me know if there are any outstanding concerns. 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] hadoop-yetus commented on pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 42s |  |  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 _ |
   | +0 :ok: |  mvndep  |  13m  6s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 29s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 39s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 24s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 15s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 58s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 58s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 38s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 57s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 57s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 18s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 18s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 37s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/7/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 1 new + 17 unchanged - 0 fixed = 18 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  3s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  6s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 26s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 59s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m 20s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 242m 11s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 8ef6d86f48fe 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 332066afbe42d0887749a504dfa519e5397482cb |
   | 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-3717/7/testReport/ |
   | Max. process+thread count | 927 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/7/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] afchung commented on pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   @bibinchundatt thanks for the review! Added a new commit addressing your comments.


-- 
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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -261,12 +391,15 @@ protected void onNewNodeAdded(
 
     if (rmNode.getState() != NodeState.DECOMMISSIONING &&
         (estimatedQueueWaitTime != -1 ||
-            comparator == LoadComparator.QUEUE_LENGTH)) {
+            comparator == LoadComparator.QUEUE_LENGTH ||
+            comparator == LoadComparator.QUEUE_LENGTH_THEN_RESOURCES)) {
       this.clusterNodes.put(rmNode.getNodeID(),

Review comment:
       We could update this in followup PR also..




-- 
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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 38s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 50s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 40s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 34s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 23s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 42s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 15s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  4s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 58s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 56s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 44s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 55s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 13s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 13s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 36s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/8/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 1 new + 17 unchanged - 0 fixed = 18 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  3s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 50s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 47s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 35s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 58s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m 23s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 241m 59s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux c70cc0ac5048 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e345582149162d9399af5c170130d3a24bee09b1 |
   | 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-3717/8/testReport/ |
   | Max. process+thread count | 930 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/8/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] bibinchundatt commented on a change in pull request #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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



##########
File path: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
##########
@@ -64,40 +69,165 @@
    * of two Nodes are compared.
    */
   public enum LoadComparator implements Comparator<ClusterNode> {
+    /**
+     * This policy only considers queue length.
+     * When allocating, increments queue length without looking at resources
+     * available on the node, and when sorting, also only sorts by queue length.
+     */
     QUEUE_LENGTH,
-    QUEUE_WAIT_TIME;
+    /**
+     * This policy only considers the wait time of containers in the queue.
+     * Neither looks at resources nor at queue length.
+     */
+    QUEUE_WAIT_TIME,
+    /**
+     * This policy considers both queue length and resources.
+     * When allocating, first decrements resources available on a node.
+     * If resources are available, does not place OContainers on the node queue.
+     * When sorting, it first sorts by queue length,
+     * then by available resources.
+     */
+    QUEUE_LENGTH_THEN_RESOURCES;
+
+    private Resource clusterResource = Resources.none();
+    private final DominantResourceCalculator resourceCalculator =
+        new DominantResourceCalculator();
+
+    private Resource computeAvailableResource(final ClusterNode clusterNode) {
+      return Resources.subtractNonNegative(
+          clusterNode.getCapability(),
+          clusterNode.getAllocatedResource());
+    }
 
     @Override
     public int compare(ClusterNode o1, ClusterNode o2) {
-      if (getMetric(o1) == getMetric(o2)) {
-        return (int)(o2.getTimestamp() - o1.getTimestamp());
+      int diff = 0;
+      switch (this) {
+      case QUEUE_LENGTH_THEN_RESOURCES:
+        diff = o1.getQueueLength().get() - o2.getQueueLength().get();
+        if (diff != 0) {
+          break;
+        }
+
+        // NOTE: cluster resource should be
+        // set always before LoadComparator is used
+        final Resource availableResource1 = computeAvailableResource(o1);
+        final Resource availableResource2 = computeAvailableResource(o2);
+        final boolean isClusterResourceLeq0 = resourceCalculator
+            .isAnyMajorResourceZeroOrNegative(clusterResource);
+        if (!isClusterResourceLeq0) {
+          // Takes the least available resource of the two nodes,
+          // normalized to the overall cluster resource
+          final float availableRatio1 =
+              resourceCalculator.minRatio(availableResource1, clusterResource);
+          final float availableRatio2 =
+              resourceCalculator.minRatio(availableResource2, clusterResource);
+
+          // The one with more available resources should be placed first
+          diff = Precision
+              .compareTo(availableRatio2, availableRatio1, Precision.EPSILON);
+        }
+
+        if (diff == 0) {
+          // Compare absolute value if ratios are the same
+          diff = availableResource2.getVirtualCores() - availableResource1.getVirtualCores();
+        }
+
+        if (diff == 0) {
+          diff = Long.compare(availableResource2.getMemorySize(),
+              availableResource1.getMemorySize());
+        }
+        break;
+      case QUEUE_WAIT_TIME:
+      case QUEUE_LENGTH:
+      default:
+        diff = getMetric(o1) - getMetric(o2);
+        break;
+      }
+
+      if (diff == 0) {
+        return (int) (o2.getTimestamp() - o1.getTimestamp());
       }
-      return getMetric(o1) - getMetric(o2);
+      return diff;
+    }
+
+    private void setClusterResource(Resource clusterResource) {
+      this.clusterResource = clusterResource;
+    }
+
+    public ResourceCalculator getResourceCalculator() {
+      return resourceCalculator;
     }
 
     public int getMetric(ClusterNode c) {
-      return (this == QUEUE_LENGTH) ?
-          c.getQueueLength().get() : c.getQueueWaitTime().get();
+      switch (this) {
+      case QUEUE_WAIT_TIME:
+        return c.getQueueWaitTime().get();
+      case QUEUE_LENGTH:
+      case QUEUE_LENGTH_THEN_RESOURCES:
+      default:
+        return c.getQueueLength().get();
+      }
     }
 
     /**
      * Increment the metric by a delta if it is below the threshold.
      * @param c ClusterNode
      * @param incrementSize increment size
+     * @param requested the requested resource
      * @return true if the metric was below threshold and was incremented.
      */
-    public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
-      if(this == QUEUE_LENGTH) {
-        int ret = c.getQueueLength().addAndGet(incrementSize);
-        if (ret <= c.getQueueCapacity()) {
+    public boolean compareAndIncrement(
+        ClusterNode c, int incrementSize, Resource requested) {
+      if (this == QUEUE_LENGTH_THEN_RESOURCES) {
+        // Assignment and getting value is atomic
+        // Can be slightly inaccurate here, don't grab lock for performance
+        final Resource capability = c.getCapability();
+        final Resource currAllocated = c.getAllocatedResource();
+        final Resource currAvailable = Resources.subtractNonNegative(
+            capability, currAllocated);
+        if (resourceCalculator.fitsIn(requested, currAvailable)) {
+          final Resource newAllocated = Resources.add(currAllocated, requested);
+          c.setAllocatedResource(newAllocated);

Review comment:
       with multiple application running in parallel the allocated resource update could be wrong here. I think we take clusterNode level lock when we are updating / reading and during checks




-- 
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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 43s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 47s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  21m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   9m 37s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   8m 24s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 42s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m 15s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   2m  2s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 58s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   3m 53s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  21m 48s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 55s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   8m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 14s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   8m 14s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 37s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/6/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 5 new + 17 unchanged - 0 fixed = 22 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  2s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m  7s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  21m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   5m  1s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  |  96m 10s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 54s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 241m 56s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux dac3d320c1f4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e9e069e073d90310c7dbdd81aa0051ae1b532a29 |
   | 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-3717/6/testReport/ |
   | Max. process+thread count | 967 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/6/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 #3717: YARN-11005. Implement the core QUEUE_LENGTH_THEN_RESOURCES OContainer allocation policy

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


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  5s |  |  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 _ |
   | +0 :ok: |  mvndep  |  12m 38s |  |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |  26m 57s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  11m 12s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   9m 21s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   1m 46s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   2m  5s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m 53s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 44s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m 20s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  25m  3s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   1m 42s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  10m 17s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |  10m 17s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  10m 22s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |  10m 22s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m 40s | [/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/10/artifact/out/results-checkstyle-hadoop-yarn-project_hadoop-yarn.txt) |  hadoop-yarn-project/hadoop-yarn: The patch generated 10 new + 17 unchanged - 0 fixed = 27 total (was 17)  |
   | +1 :green_heart: |  mvnsite  |   2m  3s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   1m 49s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   4m 15s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  25m 15s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   4m 48s |  |  hadoop-yarn-common in the patch passed.  |
   | +1 :green_heart: |  unit  | 100m 30s |  |  hadoop-yarn-server-resourcemanager in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 45s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 263m 54s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/3717 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 0c3f2b7adc02 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3b3c0a935c67df871aae2b7eeab1432f2d90d471 |
   | 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-3717/10/testReport/ |
   | Max. process+thread count | 918 (vs. ulimit of 5500) |
   | modules | C: hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager U: hadoop-yarn-project/hadoop-yarn |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-3717/10/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