Browse Source

YARN-3769. Consider user limit when calculating total pending resource for preemption policy in Capacity Scheduler. (Eric Payne via wangda)

Wangda Tan 9 years ago
parent
commit
c7e23255ae

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

@@ -25,6 +25,9 @@ Release 2.7.3 - UNRELEASED
     YARN-3849. Too much of preemption activity causing continuos killing of containers 
     across queues. (Sunil G via wangda)
 
+    YARN-3769. Consider user limit when calculating total pending resource for 
+    preemption policy in Capacity Scheduler. (Eric Payne via wangda)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 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/ProportionalCapacityPreemptionPolicy.java

@@ -756,7 +756,8 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
       }
       if (root instanceof LeafQueue) {
         LeafQueue l = (LeafQueue) root;
-        Resource pending = l.getTotalResourcePending();
+        Resource pending =
+               l.getTotalPendingResourcesConsideringUserLimit(clusterResources);
         ret = new TempQueue(queueName, current, pending, guaranteed,
             maxCapacity, preemptionDisabled);
         if (preemptionDisabled) {

+ 28 - 6
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

@@ -1845,12 +1845,34 @@ public class LeafQueue extends AbstractCSQueue {
   }
 
   // return a single Resource capturing the overal amount of pending resources
-  public synchronized Resource getTotalResourcePending() {
-    Resource ret = BuilderUtils.newResource(0, 0);
-    for (FiCaSchedulerApp f : activeApplications) {
-      Resources.addTo(ret, f.getTotalPendingRequests());
-    }
-    return ret;
+  // Consider the headroom for each user in the queue.
+  // Total pending for the queue =
+  //   sum for each user(min( (user's headroom), sum(user's pending requests) ))
+  //  NOTE: Used for calculating pedning resources in the preemption monitor.
+  public synchronized Resource getTotalPendingResourcesConsideringUserLimit(
+      Resource resources) {
+    Map<String, Resource> userNameToHeadroom = new HashMap<String, Resource>();
+    Resource pendingConsideringUserLimit = Resource.newInstance(0, 0);
+
+    for (FiCaSchedulerApp app : activeApplications) {
+      String userName = app.getUser();
+      if (!userNameToHeadroom.containsKey(userName)) {
+        User user = getUser(userName);
+        Resource headroom = Resources.subtract(
+            computeUserLimit(app, resources, minimumAllocation, user, null),
+            user.getUsed());
+        // Make sure none of the the components of headroom is negative.
+        headroom = Resources.componentwiseMax(headroom, Resources.none());
+        userNameToHeadroom.put(userName, headroom);
+      }
+      Resource minpendingConsideringUserLimit =
+          Resources.componentwiseMin(userNameToHeadroom.get(userName),
+                                     app.getTotalPendingRequests());
+      Resources.addTo(pendingConsideringUserLimit, minpendingConsideringUserLimit);
+      Resources.subtractFrom(userNameToHeadroom.get(userName),
+                             minpendingConsideringUserLimit);
+    }
+    return pendingConsideringUserLimit;
   }
 
   @Override

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java

@@ -1142,8 +1142,10 @@ public class TestProportionalCapacityPreemptionPolicy {
     ResourceCalculator rc = mCS.getResourceCalculator();
     List<ApplicationAttemptId> appAttemptIdList = 
         new ArrayList<ApplicationAttemptId>();
-    when(lq.getTotalResourcePending()).thenReturn(pending[i]);
-    // need to set pending resource in resource usage as well
+
+    when(lq.getTotalPendingResourcesConsideringUserLimit(isA(Resource.class)))
+        .thenReturn(pending[i]); // need to set pending resource in resource
+                                 // usage as well
     ResourceUsage ru = new ResourceUsage();
     ru.setPending(pending[i]);
     ru.setUsed(used[i]);
@@ -1274,7 +1276,8 @@ public class TestProportionalCapacityPreemptionPolicy {
       }
     } else {
       System.out.println(indent + nq.getQueueName()
-          + " pen:" + ((LeafQueue) nq).getTotalResourcePending()
+          + " pen:" + ((LeafQueue) nq)
+              .getTotalPendingResourcesConsideringUserLimit(isA(Resource.class))
           + " cur:" + nq.getAbsoluteUsedCapacity()
           + " guar:" + nq.getAbsoluteCapacity()
           );

+ 334 - 0
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

@@ -2587,6 +2587,340 @@ public class TestLeafQueue {
 
   }
 
+  @Test
+  public void testGetTotalPendingResourcesConsideringUserLimitOneUser()
+      throws Exception {
+    // Manipulate queue 'e'
+    LeafQueue e = stubLeafQueue((LeafQueue)queues.get(E));
+    // Allow queue 'e' to use 100% of cluster resources (max capacity).
+    e.setMaxCapacity(1.0f);
+    // When used queue resources goes above capacity (in this case, 1%), user
+    // resource limit (used in calculating headroom) is calculated in small
+    // increments to ensure that user-limit-percent can be met for all users in
+    // a queue. Take user-limit-percent out of the equation so that user
+    // resource limit will always be calculated to its max possible value.
+    e.setUserLimit(1000);
+
+    final String user_0 = "user_0";
+
+    // Submit 2 applications for user_0
+    final ApplicationAttemptId appAttemptId_0 =
+        TestUtils.getMockApplicationAttemptId(0, 0);
+    FiCaSchedulerApp app_0 =
+        new FiCaSchedulerApp(appAttemptId_0, user_0, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_0, user_0);
+
+    final ApplicationAttemptId appAttemptId_1 =
+        TestUtils.getMockApplicationAttemptId(1, 0);
+    FiCaSchedulerApp app_1 =
+        new FiCaSchedulerApp(appAttemptId_1, user_0, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_1, user_0);  // same user
+
+    // Setup 1 node with 100GB of memory resources.
+    String host_0 = "127.0.0.1";
+    FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0,
+        100*GB);
+
+    final int numNodes = 1;
+    Resource clusterResource =
+        Resources.createResource(numNodes * (100*GB), numNodes * 128);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+
+    // Pending resource requests for app_0 and app_1 total 5GB.
+    Priority priority = TestUtils.createMockPriority(1);
+    app_0.updateResourceRequests(Collections.singletonList(
+            TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 3, true,
+                priority, recordFactory)));
+
+    app_1.updateResourceRequests(Collections.singletonList(
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true,
+            priority, recordFactory)));
+
+    // Start testing...
+
+    // Assign 1st Container of 1GB
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // With queue capacity set at 1% of 100GB and user-limit-factor set to 1.0,
+    // queue 'e' should be able to consume 1GB.
+    // The first container should be assigned to app_0 with no headroom left
+    // even though user_0's apps are still asking for a total of 4GB.
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    // Assign 2nd container of 1GB
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // user_0 has no headroom due to user-limit-factor of 1.0. However capacity
+    // scheduler will assign one container more than user-limit-factor.
+    // This container should have also gone to app_0, still with no neadroom
+    // even though app_0 and app_1 are asking for a cumulative 3GB.
+    assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    // Can't allocate 3rd container due to user-limit. Headroom still 0.
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    // Increase user-limit-factor from 1GB to 10GB (1% * 10 * 100GB = 10GB).
+    // Pending for both app_0 and app_1 are still 3GB, so user-limit-factor
+    // is no longer the limiting factor.
+    e.setUserLimitFactor(10.0f);
+    assertEquals(3*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // app_0 is now satisified, app_1 is still asking for 2GB.
+    assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
+    assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    // Get the last 2 containers for app_1, no more pending requests.
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+                                                  clusterResource).getMemory());
+
+    // Release each container from app_0
+    for (RMContainer rmContainer : app_0.getLiveContainers()) {
+      e.completedContainer(clusterResource, app_0, node_0, rmContainer,
+          ContainerStatus.newInstance(rmContainer.getContainerId(),
+              ContainerState.COMPLETE, "",
+              ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+          RMContainerEventType.KILL, null, true);
+    }
+
+    // Release each container from app_1
+    for (RMContainer rmContainer : app_1.getLiveContainers()) {
+      e.completedContainer(clusterResource, app_1, node_0, rmContainer,
+          ContainerStatus.newInstance(rmContainer.getContainerId(),
+              ContainerState.COMPLETE, "",
+              ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+          RMContainerEventType.KILL, null, true);
+    }
+  }
+
+  @Test
+  public void testGetTotalPendingResourcesConsideringUserLimitTwoUsers()
+      throws Exception {
+    // Manipulate queue 'e'
+    LeafQueue e = stubLeafQueue((LeafQueue)queues.get(E));
+    // Allow queue 'e' to use 100% of cluster resources (max capacity).
+    e.setMaxCapacity(1.0f);
+    // When used queue resources goes above capacity (in this case, 1%), user
+    // resource limit (used in calculating headroom) is calculated in small
+    // increments to ensure that user-limit-percent can be met for all users in
+    // a queue. Take user-limit-percent out of the equation so that user
+    // resource limit will always be calculated to its max possible value.
+    e.setUserLimit(1000);
+
+    final String user_0 = "user_0";
+    final String user_1 = "user_1";
+
+    // Submit 2 applications for user_0
+    final ApplicationAttemptId appAttemptId_0 =
+        TestUtils.getMockApplicationAttemptId(0, 0);
+    FiCaSchedulerApp app_0 =
+        new FiCaSchedulerApp(appAttemptId_0, user_0, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_0, user_0);
+
+    final ApplicationAttemptId appAttemptId_1 =
+        TestUtils.getMockApplicationAttemptId(1, 0);
+    FiCaSchedulerApp app_1 =
+        new FiCaSchedulerApp(appAttemptId_1, user_0, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_1, user_0);
+
+    // Submit 2 applications for user_1
+    final ApplicationAttemptId appAttemptId_2 =
+        TestUtils.getMockApplicationAttemptId(2, 0);
+    FiCaSchedulerApp app_2 =
+        new FiCaSchedulerApp(appAttemptId_2, user_1, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_2, user_1);
+
+    final ApplicationAttemptId appAttemptId_3 =
+        TestUtils.getMockApplicationAttemptId(3, 0);
+    FiCaSchedulerApp app_3 =
+        new FiCaSchedulerApp(appAttemptId_3, user_1, e,
+            mock(ActiveUsersManager.class), spyRMContext);
+    e.submitApplicationAttempt(app_3, user_1);
+
+    // Setup 1 node with 100GB of memory resources.
+    String host_0 = "127.0.0.1";
+    FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0,
+        100*GB);
+
+    final int numNodes = 1;
+    Resource clusterResource =
+        Resources.createResource(numNodes * (100*GB), numNodes * 128);
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+
+    // Pending resource requests for user_0: app_0 and app_1 total 3GB (one
+    // 1GB container for app_0 and 2 1GB containers for app_1).
+    Priority priority = TestUtils.createMockPriority(1);
+    app_0.updateResourceRequests(Collections.singletonList(
+            TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
+                priority, recordFactory)));
+
+    app_1.updateResourceRequests(Collections.singletonList(
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true,
+            priority, recordFactory)));
+
+    // Pending resource requests for user_1: app_2 and app_3 total 3GB (two
+    // 1GB containers for app_2 and one 1GB container for app_3).
+    priority = TestUtils.createMockPriority(1);
+    app_2.updateResourceRequests(Collections.singletonList(
+            TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 2, true,
+                priority, recordFactory)));
+
+    app_3.updateResourceRequests(Collections.singletonList(
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
+            priority, recordFactory)));
+
+    // Start testing...
+
+    // Assign 1st Container of 1GB
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // With queue capacity set at 1% of 100GB and user-limit-factor set to 1.0,
+    // queue 'e' should be able to consume 1GB per user
+    // The first container should be assigned to app_0. Pending considering
+    // user limit should be 1GB to account for user_1's headroom.
+    assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(0*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    // Assign 2nd container of 1GB
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // user_0 has no headroom due to user-limit-factor of 1.0. However capacity
+    // scheduler will assign one container more than user-limit-factor.
+    // This container will have gone to app_1; still with 1GB pending resources
+    // to account for the headroom of user_1.
+    // user_0's apps:
+    assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(0*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    // 3rd container.
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // Allocated to user_1's app_2 since user_1 still has user resource limit
+    // of 1GB. Available pending resources are now 0.
+    // user_0's apps:
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(1*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    // 4th container.
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // Allocated to user_1's app_2 since scheduler allocates 1 container
+    // above user_1's resource limit. Pending still 0.
+    // user_0's apps:
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(2*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // Cannot allocate 5th container because both users are above their allowed
+    // user resource limit. Values should be the same as previously.
+    // user_0's apps:
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(2*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    // Increase user-limit-factor from 1GB to 10GB (10% * 100GB = 10GB).
+    // Pending for both app_0 and app_1 are still 3GB, so user-limit-factor
+    // is no longer the limiting factor.
+    e.setUserLimitFactor(10.0f);
+    // pending considering user limit should be 2GB, since 2 containers are
+    // still being requested, one from each user.
+    assertEquals(2*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // Next container goes to user_0's app_1, since it was still pending.
+    // user_0's apps:
+    assertEquals(1*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(2*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(0*GB, app_3.getCurrentConsumption().getMemory());
+
+    e.assignContainers(clusterResource, node_0,
+        new ResourceLimits(clusterResource));
+    // Last container goes to user_1's app_3, since it was still pending.
+    // user_0's apps:
+    assertEquals(0*GB, e.getTotalPendingResourcesConsideringUserLimit(
+        clusterResource).getMemory());
+    assertEquals(1*GB, app_0.getCurrentConsumption().getMemory());
+    assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
+    // user_1's apps:
+    assertEquals(2*GB, app_2.getCurrentConsumption().getMemory());
+    assertEquals(1*GB, app_3.getCurrentConsumption().getMemory());
+
+    // Release each container from app_0
+    for (RMContainer rmContainer : app_0.getLiveContainers()) {
+      e.completedContainer(clusterResource, app_0, node_0, rmContainer,
+          ContainerStatus.newInstance(rmContainer.getContainerId(),
+              ContainerState.COMPLETE, "",
+              ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+          RMContainerEventType.KILL, null, true);
+    }
+
+    // Release each container from app_1
+    for (RMContainer rmContainer : app_1.getLiveContainers()) {
+      e.completedContainer(clusterResource, app_1, node_0, rmContainer,
+          ContainerStatus.newInstance(rmContainer.getContainerId(),
+              ContainerState.COMPLETE, "",
+              ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+          RMContainerEventType.KILL, null, true);
+    }
+  }
+
   private List<FiCaSchedulerApp> createListOfApps(int noOfApps, String user,
       LeafQueue defaultQueue) {
     List<FiCaSchedulerApp> appsLists = new ArrayList<FiCaSchedulerApp>();