ソースを参照

YARN-853. Fixed CapacityScheduler's maximum-am-resource-percent to properly work beyond refreshing queues. Contributed by Devaraj K.
svn merge --ignore-ancestry -c 1505855 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1505856 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 年 前
コミット
10e4f3b4bd

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -717,6 +717,9 @@ Release 2.1.0-beta - 2013-07-02
     YARN-897. Ensure child queues are ordered correctly to account for
     completed containers. (Djellel Eddine Difallah via acmurthy)
 
+    YARN-853. Fixed CapacityScheduler's maximum-am-resource-percent to properly
+    work beyond refreshing queues. (Devaraj K via vinodkv)
+
   BREAKDOWN OF HADOOP-8562/YARN-191 SUBTASKS AND RELATED JIRAS
 
     YARN-158. Yarn creating package-info.java must not depend on sh.

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -151,6 +151,7 @@
       <Field name="state" />
       <Field name="userLimit" />
       <Field name="userLimitFactor" />
+      <Field name="maxAMResourcePerQueuePercent" />
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>

+ 19 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -171,9 +171,8 @@ public class LeafQueue implements CSQueue {
     maxApplicationsPerUser = 
       (int)(maxApplications * (userLimit / 100.0f) * userLimitFactor);
 
-    this.maxAMResourcePerQueuePercent = 
-        cs.getConfiguration().
-            getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath());
+    float maxAMResourcePerQueuePercent = cs.getConfiguration()
+        .getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath());
     int maxActiveApplications = 
         CSQueueUtils.computeMaxActiveApplications(
             resourceCalculator,
@@ -202,9 +201,9 @@ public class LeafQueue implements CSQueue {
         capacity, absoluteCapacity, 
         maximumCapacity, absoluteMaxCapacity, 
         userLimit, userLimitFactor, 
-        maxApplications, maxApplicationsPerUser,
-        maxActiveApplications, maxActiveApplicationsPerUser,
-        state, acls, cs.getConfiguration().getNodeLocalityDelay());
+        maxApplications, maxAMResourcePerQueuePercent, maxApplicationsPerUser,
+        maxActiveApplications, maxActiveApplicationsPerUser, state, acls, cs
+            .getConfiguration().getNodeLocalityDelay());
 
     if(LOG.isDebugEnabled()) {
       LOG.debug("LeafQueue:" + " name=" + queueName
@@ -223,10 +222,10 @@ public class LeafQueue implements CSQueue {
       float capacity, float absoluteCapacity, 
       float maximumCapacity, float absoluteMaxCapacity,
       int userLimit, float userLimitFactor,
-      int maxApplications, int maxApplicationsPerUser,
-      int maxActiveApplications, int maxActiveApplicationsPerUser,
-      QueueState state, Map<QueueACL, AccessControlList> acls, 
-      int nodeLocalityDelay)
+      int maxApplications, float maxAMResourcePerQueuePercent,
+      int maxApplicationsPerUser, int maxActiveApplications,
+      int maxActiveApplicationsPerUser, QueueState state,
+      Map<QueueACL, AccessControlList> acls, int nodeLocalityDelay)
   {
     // Sanity check
     CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
@@ -243,6 +242,7 @@ public class LeafQueue implements CSQueue {
     this.userLimitFactor = userLimitFactor;
 
     this.maxApplications = maxApplications;
+    this.maxAMResourcePerQueuePercent = maxAMResourcePerQueuePercent;
     this.maxApplicationsPerUser = maxApplicationsPerUser;
 
     this.maxActiveApplications = maxActiveApplications;
@@ -392,6 +392,14 @@ public class LeafQueue implements CSQueue {
   public float getMinimumAllocationFactor() {
     return minimumAllocationFactor;
   }
+  
+  /**
+   * Used only by tests.
+   */
+  @Private
+  public float getMaxAMResourcePerQueuePercent() {
+    return maxAMResourcePerQueuePercent;
+  }
 
   public int getMaxApplications() {
     return maxApplications;
@@ -604,6 +612,7 @@ public class LeafQueue implements CSQueue {
         newlyParsedLeafQueue.absoluteMaxCapacity, 
         newlyParsedLeafQueue.userLimit, newlyParsedLeafQueue.userLimitFactor, 
         newlyParsedLeafQueue.maxApplications,
+        newlyParsedLeafQueue.maxAMResourcePerQueuePercent,
         newlyParsedLeafQueue.getMaxApplicationsPerUser(),
         newlyParsedLeafQueue.getMaximumActiveApplications(), 
         newlyParsedLeafQueue.getMaximumActiveApplicationsPerUser(),

+ 44 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -43,7 +43,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -1961,6 +1960,50 @@ public class TestLeafQueue {
     assertEquals(0, app_0.getTotalRequiredResources(priority));
 
   }
+  
+  @Test
+  public void testMaxAMResourcePerQueuePercentAfterQueueRefresh()
+      throws Exception {
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    Resource clusterResource = Resources
+        .createResource(100 * 16 * GB, 100 * 32);
+    CapacitySchedulerContext csContext = mockCSContext(csConf, clusterResource);
+    csConf.setFloat(CapacitySchedulerConfiguration.
+        MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.1f);
+    ParentQueue root = new ParentQueue(csContext, 
+        CapacitySchedulerConfiguration.ROOT, null, null);
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 80);
+    LeafQueue a = new LeafQueue(csContext, A, root, null);
+    assertEquals(0.1f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
+    assertEquals(160, a.getMaximumActiveApplications());
+    
+    csConf.setFloat(CapacitySchedulerConfiguration.
+        MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.2f);
+    LeafQueue newA = new LeafQueue(csContext, A, root, null);
+    a.reinitialize(newA, clusterResource);
+    assertEquals(0.2f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
+    assertEquals(320, a.getMaximumActiveApplications());
+
+    Resource newClusterResource = Resources.createResource(100 * 20 * GB,
+        100 * 32);
+    a.updateClusterResource(newClusterResource);
+    //  100 * 20 * 0.2 = 400
+    assertEquals(400, a.getMaximumActiveApplications());
+  }
+
+  private CapacitySchedulerContext mockCSContext(
+      CapacitySchedulerConfiguration csConf, Resource clusterResource) {
+    CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getConf()).thenReturn(new YarnConfiguration());
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getClusterResources()).thenReturn(clusterResource);
+    when(csContext.getMinimumResourceCapability()).thenReturn(
+        Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).thenReturn(
+        Resources.createResource(2 * GB, 2));
+    return csContext;
+  }
 
   @After
   public void tearDown() throws Exception {