Browse Source

YARN-8373. RM Received RMFatalEvent of type CRITICAL_THREAD_CRASH. Contributed by Wilfred Spiegelenburg.

Sunil G 5 years ago
parent
commit
ea68756c0c

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java

@@ -33,12 +33,12 @@ import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -386,21 +386,21 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
 
   /**
    * Convenience method to sort nodes.
+   * Nodes can change while being sorted. Using a standard sort will fail
+   * without locking each node, the TreeSet handles this without locks.
    *
-   * Note that the sort is performed without holding a lock. We are sorting
-   * here instead of on the caller to allow for future optimizations (e.g.
-   * sort once every x milliseconds).
+   * @param comparator the comparator to sort the nodes with
+   * @return sorted set of nodes in the form of a TreeSet
    */
-  public List<N> sortedNodeList(Comparator<N> comparator) {
-    List<N> sortedList = null;
+  public TreeSet<N> sortedNodeSet(Comparator<N> comparator) {
+    TreeSet<N> sortedSet = new TreeSet<>(comparator);
     readLock.lock();
     try {
-      sortedList = new ArrayList(nodes.values());
+      sortedSet.addAll(nodes.values());
     } finally {
       readLock.unlock();
     }
-    Collections.sort(sortedList, comparator);
-    return sortedList;
+    return sortedSet;
   }
 
   /**

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -113,6 +113,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
@@ -1001,15 +1002,17 @@ public class FairScheduler extends
   @Deprecated
   void continuousSchedulingAttempt() throws InterruptedException {
     long start = getClock().getTime();
-    List<FSSchedulerNode> nodeIdList;
-    // Hold a lock to prevent comparator order changes due to changes of node
-    // unallocated resources
-    synchronized (this) {
-      nodeIdList = nodeTracker.sortedNodeList(nodeAvailableResourceComparator);
+    TreeSet<FSSchedulerNode> nodeIdSet;
+    // Hold a lock to prevent node changes as much as possible.
+    readLock.lock();
+    try {
+      nodeIdSet = nodeTracker.sortedNodeSet(nodeAvailableResourceComparator);
+    } finally {
+      readLock.unlock();
     }
 
     // iterate all nodes
-    for (FSSchedulerNode node : nodeIdList) {
+    for (FSSchedulerNode node : nodeIdSet) {
       try {
         if (Resources.fitsIn(minimumAllocation,
             node.getUnallocatedResource())) {

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java

@@ -335,9 +335,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         for (int j = 0; j < 100; j++) {
           for (FSSchedulerNode node : clusterNodeTracker.getAllNodes()) {
             int i = ThreadLocalRandom.current().nextInt(-30, 30);
-            synchronized (scheduler) {
               node.deductUnallocatedResource(Resource.newInstance(i * 1024, i));
-            }
           }
         }
       }