소스 검색

[YARN-10613] Config to allow Intra- and Inter-queue preemption to enable/disable conservativeDRF. Contributed by Eric Payne

Jim Brennan 4 년 전
부모
커밋
c373da9f88
7개의 변경된 파일212개의 추가작업 그리고 7개의 파일을 삭제
  1. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionContext.java
  2. 9 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java
  3. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java
  4. 32 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
  5. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  6. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java
  7. 78 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionContext.java

@@ -74,4 +74,8 @@ public interface CapacitySchedulerPreemptionContext {
 
   @Unstable
   IntraQueuePreemptionOrderPolicy getIntraQueuePreemptionOrderPolicy();
+
+  boolean getCrossQueuePreemptionConservativeDRF();
+
+  boolean getInQueuePreemptionConservativeDRF();
 }

+ 9 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoCandidatesSelector.java

@@ -116,7 +116,9 @@ public class FifoCandidatesSelector
                   .tryPreemptContainerAndDeductResToObtain(rc,
                       preemptionContext, resToObtainByPartition, c,
                       clusterResource, selectedCandidates, curCandidates,
-                      totalPreemptionAllowed, false);
+                      totalPreemptionAllowed,
+                      preemptionContext.getCrossQueuePreemptionConservativeDRF()
+                      );
               if (!preempted) {
                 continue;
               }
@@ -193,7 +195,8 @@ public class FifoCandidatesSelector
       boolean preempted = CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, preemptMap,
-              curCandidates, totalPreemptionAllowed, false);
+              curCandidates, totalPreemptionAllowed,
+              preemptionContext.getCrossQueuePreemptionConservativeDRF());
       if (preempted) {
         Resources.subtractFrom(skippedAMSize, c.getAllocatedResource());
       }
@@ -229,7 +232,8 @@ public class FifoCandidatesSelector
      CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, selectedContainers,
-              curCandidates, totalPreemptionAllowed, false);
+              curCandidates, totalPreemptionAllowed,
+              preemptionContext.getCrossQueuePreemptionConservativeDRF());
 
       if (!preemptionContext.isObserveOnly()) {
         preemptionContext.getRMContext().getDispatcher().getEventHandler()
@@ -273,7 +277,8 @@ public class FifoCandidatesSelector
       CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, selectedContainers,
-              curCandidates, totalPreemptionAllowed, false);
+              curCandidates, totalPreemptionAllowed,
+              preemptionContext.getCrossQueuePreemptionConservativeDRF());
     }
   }
 

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/IntraQueueCandidatesSelector.java

@@ -263,7 +263,8 @@ public class IntraQueueCandidatesSelector extends PreemptionCandidatesSelector {
       boolean ret = CapacitySchedulerPreemptionUtils
           .tryPreemptContainerAndDeductResToObtain(rc, preemptionContext,
               resToObtainByPartition, c, clusterResource, selectedCandidates,
-              curCandidates, totalPreemptedResourceAllowed, true);
+              curCandidates, totalPreemptedResourceAllowed,
+              preemptionContext.getInQueuePreemptionConservativeDRF());
 
       // Subtract from respective user's resource usage once a container is
       // selected for preemption.

+ 32 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java

@@ -113,6 +113,9 @@ public class ProportionalCapacityPreemptionPolicy
   private float minimumThresholdForIntraQueuePreemption;
   private IntraQueuePreemptionOrderPolicy intraQueuePreemptionOrderPolicy;
 
+  private boolean crossQueuePreemptionConservativeDRF;
+  private boolean inQueuePreemptionConservativeDRF;
+
   // Current configuration
   private CapacitySchedulerConfiguration csConfig;
 
@@ -225,6 +228,18 @@ public class ProportionalCapacityPreemptionPolicy
                 CapacitySchedulerConfiguration.DEFAULT_INTRAQUEUE_PREEMPTION_ORDER_POLICY)
             .toUpperCase());
 
+    crossQueuePreemptionConservativeDRF =  config.getBoolean(
+        CapacitySchedulerConfiguration.
+        CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        CapacitySchedulerConfiguration.
+        DEFAULT_CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF);
+
+    inQueuePreemptionConservativeDRF =  config.getBoolean(
+        CapacitySchedulerConfiguration.
+        IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        CapacitySchedulerConfiguration.
+        DEFAULT_IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF);
+
     candidatesSelectionPolicies = new ArrayList<>();
 
     // Do we need white queue-priority preemption policy?
@@ -300,7 +315,12 @@ public class ProportionalCapacityPreemptionPolicy
           selectCandidatesForResevedContainers + "\n" +
         "additional_res_balance_based_on_reserved_containers = " +
           additionalPreemptionBasedOnReservedResource + "\n" +
-        "Preemption-to-balance-queue-enabled = " + isPreemptionToBalanceRequired);
+        "Preemption-to-balance-queue-enabled = " +
+          isPreemptionToBalanceRequired + "\n" +
+        "cross-queue-preemption.conservative-drf = " +
+          crossQueuePreemptionConservativeDRF + "\n" +
+        "in-queue-preemption.conservative-drf = " +
+          inQueuePreemptionConservativeDRF);
 
     csConfig = config;
   }
@@ -425,7 +445,7 @@ public class ProportionalCapacityPreemptionPolicy
 
     return leafQueueNames;
   }
-  
+
   /**
    * This method selects and tracks containers to be preemptionCandidates. If a container
    * is in the target list for more than maxWaitTime it is killed.
@@ -784,6 +804,16 @@ public class ProportionalCapacityPreemptionPolicy
     return intraQueuePreemptionOrderPolicy;
   }
 
+  @Override
+  public boolean getCrossQueuePreemptionConservativeDRF() {
+    return crossQueuePreemptionConservativeDRF;
+  }
+
+  @Override
+  public boolean getInQueuePreemptionConservativeDRF() {
+    return inQueuePreemptionConservativeDRF;
+  }
+
   @Override
   public long getDefaultMaximumKillWaitTimeout() {
     return maxWaitTime;

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

@@ -1742,6 +1742,21 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
       + INTRA_QUEUE_PREEMPTION_CONFIG_PREFIX + "preemption-order-policy";
   public static final String DEFAULT_INTRAQUEUE_PREEMPTION_ORDER_POLICY = "userlimit_first";
 
+  /**
+   * Flag to determine whether or not to preempt containers from apps where some
+   * used resources are less than the user's user limit.
+   */
+  public static final String CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF =
+      PREEMPTION_CONFIG_PREFIX + "conservative-drf";
+  public static final Boolean DEFAULT_CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF =
+      false;
+
+  public static final String IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF =
+      PREEMPTION_CONFIG_PREFIX + INTRA_QUEUE_PREEMPTION_CONFIG_PREFIX +
+      "conservative-drf";
+  public static final Boolean DEFAULT_IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF =
+      true;
+
   /**
    * Should we allow queues continue grow after all queue reaches their
    * guaranteed capacity.

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java

@@ -19,9 +19,11 @@
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.junit.Before;
@@ -29,8 +31,10 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -174,4 +178,72 @@ public class TestProportionalCapacityPreemptionPolicyInterQueueWithDRF
         new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
             getAppAttemptId(2))));
   }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testInterQueuePreemptionWithStrictAndRelaxedDRF()
+      throws IOException {
+
+    /*
+     *              root
+     *           /  \  \
+     *          a    b  c
+     *
+     *  A / B / C have 33.3 / 33.3 / 33.4 resources
+     *  Total cluster resource have mem=61440, cpu=600
+     *
+     *  +=================+========================+
+     *  | used in queue a | user limit for queue a |
+     *  +=================+========================+
+     *  |    61440:60     |       20480:200        |
+     *  +=================+========================+
+     *  In this case, the used memory is over the user limit but the used vCores
+     *  is not. If conservative DRF is true, preemptions will not occur.
+     *  If conservative DRF is false (default) preemptions will occur.
+     */
+    String labelsConfig = "=61440:600,true;";
+    String nodesConfig = "n1= res=61440:600"; // n1 is default partition
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[61440:600 61440:600 61440:600 20480:20 0]);" + // root
+            "-a(=[20480:200 61440:600 61440:60 0:0 0]);" + // b
+            "-b(=[20480:200 61440:600 0:0 20480:20 0]);" + // a
+            "-c(=[20480:200 61440:600 0:0 0:0 0])"; // c
+    String appsConfig =
+        //queueName\t(priority,resource,host,expression,#repeat,reserved)
+        "a\t" + "(1,1024:1,n1,,60,false,0:0,user1);" + // app1 in a
+        "b\t" + "(1,0:0,n1,,0,false,20480:20,user2);"; // app2 in b
+
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        true);
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    Resource ul = Resource.newInstance(20480, 20);
+    when(((LeafQueue)(cs.getQueue("root.a")))
+        .getResourceLimitForAllUsers(any(), any(), any(), any())
+    ).thenReturn(ul);
+    policy.editSchedule();
+
+    verify(eventHandler, times(0)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+
+    reset(eventHandler);
+
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.CROSS_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        false);
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    ul = Resource.newInstance(20480, 20);
+    when(((LeafQueue)(cs.getQueue("root.a")))
+        .getResourceLimitForAllUsers(any(), any(), any(), any())
+    ).thenReturn(ul);
+    policy.editSchedule();
+
+    verify(eventHandler, times(20)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
 }

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF.java

@@ -18,16 +18,20 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
 
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -111,6 +115,80 @@ public class TestProportionalCapacityPreemptionPolicyIntraQueueWithDRF
             getAppAttemptId(3))));
   }
 
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testIntraQueuePreemptionFairOrderingWithStrictAndRelaxedDRF()
+      throws IOException {
+    /**
+     * Continue to allow intra-queue preemption when only one of the user's
+     * resources is above the user limit.
+     * Queue structure is:
+     *
+     * <pre>
+     *       root
+     *     /  |
+     *    a   b
+     * </pre>
+     *
+     * Guaranteed resource of a and b are 30720:300 and 30720:300 Total cluster
+     * resource = 61440:600.
+     * Scenario: Queue B has one running app using 61720:60 resources with no
+     * pending resources, and one app with no used resources and 30720:30
+     * pending resources.
+     *
+     * The first part of the test is to show what happens when the conservative
+     * DRF property is set. Since the memory is above and the vcores is below
+     * the user limit, only the minimum number of containers is allowed.
+     * In the second part, since conservative DRF is relaxed, all containers
+     * needed are allowed to be preempted (minus the AM size).
+     */
+
+    conf.set(CapacitySchedulerConfiguration.INTRAQUEUE_PREEMPTION_ORDER_POLICY,
+        "userlimit_first");
+    conf.set(CapacitySchedulerConfiguration.PREFIX
+        + "root.b." + CapacitySchedulerConfiguration.ORDERING_POLICY, "fair");
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        true);
+
+    String labelsConfig = "=61440:600,true;";
+    String nodesConfig = // n1 has no label
+        "n1= res=61440:600";
+    String queuesConfig =
+        // guaranteed,max,used,pending,reserved
+        "root(=[61440:600 61440:600 61440:600 30720:30 0]);" + // root
+            "-a(=[30720:300 61440:600 0:0 0:0 0]);" + // a
+            "-b(=[30720:300 61440:600 61440:60 30720:30 0]);";  // b
+
+    String appsConfig =
+        "b\t" + "(1,1024:1,n1,,60,false,0:0,user1);" + // app1 in b
+        "b\t" + "(1,0:0,n1,,0,false,30720:30,user3);"; // app2 in b
+
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    Resource ul = Resource.newInstance(30720, 300);
+    when(((LeafQueue)(cs.getQueue("root.b")))
+        .getResourceLimitForAllUsers(any(), any(), any(), any())
+    ).thenReturn(ul);
+    policy.editSchedule();
+
+    verify(eventHandler, times(6)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+    reset(eventHandler);
+
+    conf.setBoolean(
+        CapacitySchedulerConfiguration.IN_QUEUE_PREEMPTION_CONSERVATIVE_DRF,
+        false);
+    buildEnv(labelsConfig, nodesConfig, queuesConfig, appsConfig);
+    when(((LeafQueue)(cs.getQueue("root.b")))
+        .getResourceLimitForAllUsers(any(), any(), any(), any())
+    ).thenReturn(ul);
+    policy.editSchedule();
+    verify(eventHandler, times(29)).handle(argThat(
+        new TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor(
+            getAppAttemptId(1))));
+  }
+
   @Test
   public void testIntraQueuePreemptionWithDominantVCoreResource()
       throws IOException {