|
25 | 25 | import java.util.HashSet;
|
26 | 26 | import java.util.Iterator;
|
27 | 27 | import java.util.List;
|
| 28 | +import java.util.Map; |
28 | 29 | import java.util.Set;
|
| 30 | +import java.util.concurrent.ConcurrentMap; |
29 | 31 | import java.util.concurrent.atomic.AtomicBoolean;
|
30 | 32 |
|
31 | 33 | import org.apache.hadoop.thirdparty.com.google.common.collect.Iterators;
|
32 | 34 |
|
| 35 | +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; |
33 | 36 | import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
| 37 | +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; |
34 | 38 | import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
35 | 39 | import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
36 | 40 | import org.slf4j.Logger;
|
@@ -478,4 +482,110 @@ public void testMultiNodeSorterAfterHeartbeatInterval() throws Exception {
|
478 | 482 | rm.stop();
|
479 | 483 | }
|
480 | 484 |
|
| 485 | + @Test(timeout=30000) |
| 486 | + public void testSkipAllocationOnNodeReservedByAnotherApp() throws Exception { |
| 487 | + CapacitySchedulerConfiguration newConf = |
| 488 | + new CapacitySchedulerConfiguration(conf); |
| 489 | + newConf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER, |
| 490 | + YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); |
| 491 | + newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME |
| 492 | + + ".resource-based.sorting-interval.ms", 0); |
| 493 | + newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f); |
| 494 | + newConf.set(CapacitySchedulerConfiguration.SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS, |
| 495 | + "true"); |
| 496 | + MockRM rm1 = new MockRM(newConf); |
| 497 | + |
| 498 | + rm1.start(); |
| 499 | + MockNM nm1 = rm1.registerNode("127.0.0.1:1234", 8 * GB); |
| 500 | + MockNM nm2 = rm1.registerNode("127.0.0.2:1235", 8 * GB); |
| 501 | + |
| 502 | + // launch an app to queue, AM container should be launched in nm1 |
| 503 | + RMApp app1 = MockRMAppSubmitter.submit(rm1, MockRMAppSubmissionData.Builder |
| 504 | + .createWithMemory(5 * GB, rm1) |
| 505 | + .withAppName("app") |
| 506 | + .withUser("user") |
| 507 | + .withQueue("default") |
| 508 | + .build()); |
| 509 | + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); |
| 510 | + |
| 511 | + // launch another app to queue, AM container should be launched in nm2 |
| 512 | + RMApp app2 = MockRMAppSubmitter.submit(rm1, MockRMAppSubmissionData.Builder |
| 513 | + .createWithMemory(5 * GB, rm1) |
| 514 | + .withAppName("app") |
| 515 | + .withUser("user") |
| 516 | + .withQueue("default") |
| 517 | + .build()); |
| 518 | + MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2); |
| 519 | + |
| 520 | + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); |
| 521 | + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); |
| 522 | + FiCaSchedulerApp schedulerApp1 = |
| 523 | + cs.getApplicationAttempt(am1.getApplicationAttemptId()); |
| 524 | + FiCaSchedulerApp schedulerApp2 = |
| 525 | + cs.getApplicationAttempt(am2.getApplicationAttemptId()); |
| 526 | + |
| 527 | + // Ask a container with 4 GB memory size for app1, |
| 528 | + am1.allocate("*", 4 * GB, 1, new ArrayList<>()); |
| 529 | + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); |
| 530 | + |
| 531 | + |
| 532 | + // Check containers of app1 and app2. |
| 533 | + Set<RMNode> reservedContainers = checkReservedContainers(cs, |
| 534 | + rm1.getRMContext().getRMNodes(), 1); |
| 535 | + Assert.assertEquals(1, reservedContainers.size()); |
| 536 | + RMNode nodeWithReservedContainer = reservedContainers.iterator().next(); |
| 537 | + LOG.debug("Reserved container on: {}", nodeWithReservedContainer); |
| 538 | + |
| 539 | + //Move reservation to nm1 for easier testing |
| 540 | + if (nodeWithReservedContainer.getNodeID().getHost().startsWith("127.0.0.2")) { |
| 541 | + moveReservation(cs, rm1, nm1, nm2, am1); |
| 542 | + } |
| 543 | + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); |
| 544 | + Assert.assertNull(cs.getNode(nm2.getNodeId()).getReservedContainer()); |
| 545 | + |
| 546 | + Assert.assertEquals(1, schedulerApp1.getLiveContainers().size()); |
| 547 | + Assert.assertEquals(1, schedulerApp2.getLiveContainers().size()); |
| 548 | + Assert.assertEquals(1, schedulerApp1.getReservedContainers().size()); |
| 549 | + |
| 550 | + //Make sure to have available headroom on the child queue, |
| 551 | + // see: RegularContainerAllocator#checkHeadroom, |
| 552 | + //that can make RegularContainerAllocator.preCheckForNodeCandidateSet to return |
| 553 | + // ContainerAllocation.QUEUE_SKIPPED |
| 554 | + MockNM nm3 = rm1.registerNode("127.0.0.3:1235", 3 * GB); |
| 555 | + |
| 556 | + //Allocate a container for app2, we expect this to be allocated on nm2 as |
| 557 | + // nm1 has a reservation for another app |
| 558 | + am2.allocate("*", 4 * GB, 1, new ArrayList<>()); |
| 559 | + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); |
| 560 | + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); |
| 561 | + Assert.assertNotNull(cs.getNode(nm2.getNodeId()).getReservedContainer()); |
| 562 | + |
| 563 | + rm1.close(); |
| 564 | + } |
| 565 | + |
| 566 | + private static void moveReservation(CapacityScheduler cs, |
| 567 | + MockRM rm1, MockNM nm1, MockNM nm2, MockAM am1) { |
| 568 | + RMNode sourceNode = rm1.getRMContext().getRMNodes().get(nm2.getNodeId()); |
| 569 | + RMNode targetNode = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); |
| 570 | + SchedulerApplicationAttempt firstSchedulerAppAttempt = |
| 571 | + cs.getApplicationAttempt(am1.getApplicationAttemptId()); |
| 572 | + FiCaSchedulerApp app = (FiCaSchedulerApp)firstSchedulerAppAttempt; |
| 573 | + RMContainer reservedContainer = cs.getNode(sourceNode.getNodeID()).getReservedContainer(); |
| 574 | + LOG.debug("Moving reservation"); |
| 575 | + app.moveReservation(reservedContainer, |
| 576 | + cs.getNode(sourceNode.getNodeID()), cs.getNode(targetNode.getNodeID())); |
| 577 | + } |
| 578 | + |
| 579 | + private static Set<RMNode> checkReservedContainers(CapacityScheduler cs, |
| 580 | + ConcurrentMap<NodeId, RMNode> rmNodes, int expectedNumberOfContainers) { |
| 581 | + Set<RMNode> result = new HashSet<>(); |
| 582 | + for (Map.Entry<NodeId, RMNode> entry : rmNodes.entrySet()) { |
| 583 | + if (cs.getNode(entry.getKey()).getReservedContainer() != null) { |
| 584 | + result.add(entry.getValue()); |
| 585 | + } |
| 586 | + } |
| 587 | + |
| 588 | + Assert.assertEquals(expectedNumberOfContainers, result.size()); |
| 589 | + return result; |
| 590 | + } |
481 | 591 | }
|
0 commit comments