浏览代码

YARN-5831. FairScheduler: Propagate allowPreemptionFrom flag all the way down to the app. (Yufei Gu via kasha)

Karthik Kambatla 8 年之前
父节点
当前提交
e224c96234
共有 9 个文件被更改,包括 65 次插入44 次删除
  1. 9 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
  2. 0 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  3. 22 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  4. 4 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  5. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
  6. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
  7. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
  8. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
  9. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java

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

@@ -568,6 +568,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   boolean canContainerBePreempted(RMContainer container) {
+    if (!isPreemptable()) {
+      return false;
+    }
+
     // Sanity check that the app owns this container
     if (!getLiveContainersMap().containsKey(container.getContainerId()) &&
         !newlyAllocatedContainers.contains(container)) {
@@ -581,17 +585,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       return false;
     }
 
-    // Check if any of the parent queues are not preemptable
-    // TODO (YARN-5831): Propagate the "preemptable" flag all the way down to
-    // the app to avoid recursing up every time.
-    for (FSQueue q = getQueue();
-        !q.getQueueName().equals("root");
-        q = q.getParent()) {
-      if (!q.isPreemptable()) {
-        return false;
-      }
-    }
-
     // Check if the app's allocation will be over its fairshare even
     // after preempting this container
     Resource currentUsage = getResourceUsage();
@@ -1241,4 +1234,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   public boolean equals(Object o) {
     return super.equals(o);
   }
+
+  @Override
+  public boolean isPreemptable() {
+    return getQueue().isPreemptable();
+  }
 }

+ 0 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -108,21 +108,6 @@ public class FSParentQueue extends FSQueue {
     }
   }
 
-  @Override
-  public void updatePreemptionVariables() {
-    super.updatePreemptionVariables();
-    // For child queues
-
-    readLock.lock();
-    try {
-      for (FSQueue childQueue : childQueues) {
-        childQueue.updatePreemptionVariables();
-      }
-    } finally {
-      readLock.unlock();
-    }
-  }
-
   @Override
   public Resource getDemand() {
     readLock.lock();

+ 22 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -91,6 +91,7 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.queueEntity = new PrivilegedEntity(EntityType.QUEUE, name);
     this.metrics = FSQueueMetrics.forQueue(getName(), parent, true, scheduler.getConf());
     this.parent = parent;
+    reinit(false);
   }
 
   /**
@@ -98,10 +99,19 @@ public abstract class FSQueue implements Queue, Schedulable {
    * metrics.
    * This function is invoked when a new queue is created or reloading the
    * allocation configuration.
+   *
+   * @param recursive whether child queues should be reinitialized recursively
    */
-  public void init() {
+  public void reinit(boolean recursive) {
     AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
     allocConf.initFSQueue(this, scheduler);
+    updatePreemptionVariables();
+
+    if (recursive) {
+      for (FSQueue child : getChildQueues()) {
+        child.reinit(recursive);
+      }
+    }
   }
 
   public String getName() {
@@ -307,6 +317,7 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.fairSharePreemptionThreshold = fairSharePreemptionThreshold;
   }
 
+  @Override
   public boolean isPreemptable() {
     return preemptable;
   }
@@ -329,7 +340,7 @@ public abstract class FSQueue implements Queue, Schedulable {
    * Update the min/fair share preemption timeouts, threshold and preemption
    * disabled flag for this queue.
    */
-  public void updatePreemptionVariables() {
+  private void updatePreemptionVariables() {
     // For min share timeout
     minSharePreemptionTimeout = scheduler.getAllocationConfiguration()
         .getMinSharePreemptionTimeout(getName());
@@ -348,9 +359,15 @@ public abstract class FSQueue implements Queue, Schedulable {
     if (fairSharePreemptionThreshold < 0 && parent != null) {
       fairSharePreemptionThreshold = parent.getFairSharePreemptionThreshold();
     }
-    // For option whether allow preemption from this queue
-    preemptable = scheduler.getAllocationConfiguration()
-        .isPreemptable(getName());
+    // For option whether allow preemption from this queue.
+    // If the parent is non-preemptable, this queue is non-preemptable as well,
+    // otherwise get the value from the allocation file.
+    if (parent != null && !parent.isPreemptable()) {
+      preemptable = false;
+    } else {
+      preemptable = scheduler.getAllocationConfiguration()
+          .isPreemptable(getName());
+    }
   }
 
   /**

+ 4 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -73,11 +73,12 @@ public class QueueManager {
   public void initialize(Configuration conf) throws IOException,
       SAXException, AllocationConfigurationException, ParserConfigurationException {
     rootQueue = new FSParentQueue("root", scheduler, null);
-    rootQueue.init();
     queues.put(rootQueue.getName(), rootQueue);
 
     // Create the default queue
     getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
+    // Recursively reinitialize to propagate queue properties
+    rootQueue.reinit(true);
   }
   
   /**
@@ -281,11 +282,9 @@ public class QueueManager {
         queue = newParent;
       }
 
-      queue.init();
       parent.addChildQueue(queue);
       setChildResourceLimits(parent, queue, queueConf);
       queues.put(queue.getName(), queue);
-      queue.updatePreemptionVariables();
 
       // If we just created a leaf node, the newParent is null, but that's OK
       // because we only create a leaf node in the very last iteration.
@@ -496,17 +495,11 @@ public class QueueManager {
         }
       }
     }
-    rootQueue.recomputeSteadyShares();
-
-    for (FSQueue queue : queues.values()) {
-      queue.init();
-    }
 
+    // Initialize all queues recursively
+    rootQueue.reinit(true);
     // Update steady fair shares for all queues
     rootQueue.recomputeSteadyShares();
-    // Update the fair share preemption timeouts and preemption for all queues
-    // recursively
-    rootQueue.updatePreemptionVariables();
   }
 
   /**

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 
 /**
  * A Schedulable represents an entity that can be scheduled such as an
@@ -96,4 +95,11 @@ public interface Schedulable {
 
   /** Assign a fair share to this Schedulable. */
   void setFairShare(Resource fairShare);
+
+  /**
+   * Check whether the schedulable is preemptable.
+   * @return <code>true</code> if the schedulable is preemptable;
+   *         <code>false</code> otherwise
+   */
+  boolean isPreemptable();
 }

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

@@ -137,4 +137,9 @@ public class FakeSchedulable implements Schedulable {
 
   @Override
   public void updateDemand() {}
+
+  @Override
+  public boolean isPreemptable() {
+    return true;
+  }
 }

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

@@ -81,7 +81,6 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
 
     String queueName = "root.queue1";
     FSLeafQueue schedulable = new FSLeafQueue(queueName, scheduler, null);
-    schedulable.init();
     schedulable.setMaxShare(maxResource);
     assertEquals(schedulable.getMetrics().getMaxApps(), Integer.MAX_VALUE);
     assertEquals(schedulable.getMetrics().getSchedulingPolicy(),

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

@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.junit.After;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import org.junit.Before;
 import org.junit.Test;
@@ -166,6 +167,14 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     // Create and add two nodes to the cluster
     addNode(NODE_CAPACITY_MULTIPLE * 1024, NODE_CAPACITY_MULTIPLE);
     addNode(NODE_CAPACITY_MULTIPLE * 1024, NODE_CAPACITY_MULTIPLE);
+
+    // Verify if child-1 and child-2 are preemptable
+    FSQueue child1 =
+        scheduler.getQueueManager().getQueue("nonpreemptable.child-1");
+    assertFalse(child1.isPreemptable());
+    FSQueue child2 =
+        scheduler.getQueueManager().getQueue("nonpreemptable.child-2");
+    assertFalse(child2.isPreemptable());
   }
 
   private void sendEnoughNodeUpdatesToAssignFully() {
@@ -197,6 +206,10 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     scheduler.update();
     sendEnoughNodeUpdatesToAssignFully();
     assertEquals(8, greedyApp.getLiveContainers().size());
+    // Verify preemptable for queue and app attempt
+    assertTrue(
+        scheduler.getQueueManager().getQueue(queue1).isPreemptable()
+        == greedyApp.isPreemptable());
 
     // Create an app that takes up all the resources on the cluster
     ApplicationAttemptId appAttemptId2

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

@@ -345,6 +345,11 @@ public class TestSchedulingPolicy {
             ", weights:" + weights + ", demand:" + demand +
             ", minShare:" + minShare + "}";
       }
+
+      @Override
+      public boolean isPreemptable() {
+        return true;
+      }
     }
   }