|
@@ -1024,4 +1024,455 @@ public class TestNodeLabelContainerAllocation {
|
|
|
|
|
|
rm1.close();
|
|
|
}
|
|
|
+
|
|
|
+ private void checkQueueUsedCapacity(String queueName, CapacityScheduler cs,
|
|
|
+ String nodePartition, float usedCapacity, float absoluteUsedCapacity) {
|
|
|
+ float epsilon = 1e-6f;
|
|
|
+ CSQueue queue = cs.getQueue(queueName);
|
|
|
+ Assert.assertNotNull("Failed to get queue=" + queueName, queue);
|
|
|
+
|
|
|
+ Assert.assertEquals(usedCapacity, queue.getQueueCapacities()
|
|
|
+ .getUsedCapacity(nodePartition), epsilon);
|
|
|
+ Assert.assertEquals(absoluteUsedCapacity, queue.getQueueCapacities()
|
|
|
+ .getAbsoluteUsedCapacity(nodePartition), epsilon);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void doNMHeartbeat(MockRM rm, NodeId nodeId, int nHeartbeat) {
|
|
|
+ CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
+ RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nodeId);
|
|
|
+ for (int i = 0; i < nHeartbeat; i++) {
|
|
|
+ cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitSchedulerNodeJoined(MockRM rm, int expectedNodeNum)
|
|
|
+ throws InterruptedException {
|
|
|
+ int totalWaitTick = 100; // wait 10 sec at most.
|
|
|
+ while (expectedNodeNum > rm.getResourceScheduler().getNumClusterNodes()
|
|
|
+ && totalWaitTick > 0) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ totalWaitTick--;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQueueUsedCapacitiesUpdate()
|
|
|
+ throws Exception {
|
|
|
+ /**
|
|
|
+ * Test case: have a following queue structure:
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * root
|
|
|
+ * / \
|
|
|
+ * a b
|
|
|
+ * / \ (x)
|
|
|
+ * a1 a2
|
|
|
+ * (x) (x)
|
|
|
+ * </pre>
|
|
|
+ *
|
|
|
+ * Both a/b can access x, we need to verify when
|
|
|
+ * <pre>
|
|
|
+ * 1) container allocated/released in both partitioned/non-partitioned node,
|
|
|
+ * 2) clusterResource updates
|
|
|
+ * 3) queue guaranteed resource changed
|
|
|
+ * </pre>
|
|
|
+ *
|
|
|
+ * used capacity / absolute used capacity of queues are correctly updated.
|
|
|
+ */
|
|
|
+
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration(this.conf);
|
|
|
+
|
|
|
+ // Define top-level queues
|
|
|
+ csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
|
|
|
+ "b" });
|
|
|
+ csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Initially, we set A/B's resource 50:50
|
|
|
+ */
|
|
|
+ final String A = CapacitySchedulerConfiguration.ROOT + ".a";
|
|
|
+ csConf.setCapacity(A, 50);
|
|
|
+ csConf.setAccessibleNodeLabels(A, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(A, "x", 50);
|
|
|
+
|
|
|
+ csConf.setQueues(A, new String[] { "a1", "a2" });
|
|
|
+
|
|
|
+ final String A1 = A + ".a1";
|
|
|
+ csConf.setCapacity(A1, 50);
|
|
|
+ csConf.setAccessibleNodeLabels(A1, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(A1, "x", 50);
|
|
|
+
|
|
|
+ final String A2 = A + ".a2";
|
|
|
+ csConf.setCapacity(A2, 50);
|
|
|
+ csConf.setAccessibleNodeLabels(A2, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(A2, "x", 50);
|
|
|
+
|
|
|
+ final String B = CapacitySchedulerConfiguration.ROOT + ".b";
|
|
|
+ csConf.setCapacity(B, 50);
|
|
|
+ csConf.setAccessibleNodeLabels(B, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(B, "x", 50);
|
|
|
+
|
|
|
+ // set node -> label
|
|
|
+ mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
|
|
|
+ // Makes x to be non-exclusive node labels
|
|
|
+ mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
|
|
|
+ mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
|
|
|
+
|
|
|
+ // inject node label manager
|
|
|
+ MockRM rm = new MockRM(csConf) {
|
|
|
+ @Override
|
|
|
+ public RMNodeLabelsManager createNodeLabelManager() {
|
|
|
+ return mgr;
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ rm.getRMContext().setNodeLabelManager(mgr);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Before we adding any node to the cluster, used-capacity/abs-used-capacity
|
|
|
+ * should be 0
|
|
|
+ */
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0f, 0f);
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("h1:1234", 10 * GB); // label = x
|
|
|
+ MockNM nm2 = rm.registerNode("h2:1234", 10 * GB); // label = <empty>
|
|
|
+
|
|
|
+ /*
|
|
|
+ * After we adding nodes to the cluster, and before starting to use them,
|
|
|
+ * used-capacity/abs-used-capacity should be 0
|
|
|
+ */
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0f, 0f);
|
|
|
+
|
|
|
+ // app1 -> a1
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
|
|
|
+
|
|
|
+ // app1 asks for 1 partition= containers
|
|
|
+ am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
|
|
|
+
|
|
|
+ doNMHeartbeat(rm, nm2.getNodeId(), 10);
|
|
|
+
|
|
|
+ // Now check usage, app1 uses:
|
|
|
+ // a1: used(no-label) = 80%
|
|
|
+ // abs-used(no-label) = 20%
|
|
|
+ // a: used(no-label) = 40%
|
|
|
+ // abs-used(no-label) = 20%
|
|
|
+ // root: used(no-label) = 20%
|
|
|
+ // abs-used(no-label) = 20%
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.4f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.2f, 0.2f);
|
|
|
+
|
|
|
+ // app1 asks for 2 partition=x containers
|
|
|
+ am1.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "x");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 10);
|
|
|
+
|
|
|
+ // Now check usage, app1 uses:
|
|
|
+ // a1: used(x) = 80%
|
|
|
+ // abs-used(x) = 20%
|
|
|
+ // a: used(x) = 40%
|
|
|
+ // abs-used(x) = 20%
|
|
|
+ // root: used(x) = 20%
|
|
|
+ // abs-used(x) = 20%
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0.4f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.4f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0.8f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0.2f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.2f, 0.2f);
|
|
|
+
|
|
|
+ // submit an app to a2, uses 1 NON_PARTITIONED container and 1 PARTITIONED
|
|
|
+ // container
|
|
|
+ // app2 -> a2
|
|
|
+ RMApp app2 = rm.submitApp(1 * GB, "app", "user", null, "a2");
|
|
|
+ MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
|
|
|
+
|
|
|
+ // app1 asks for 1 partition= containers
|
|
|
+ am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 10);
|
|
|
+
|
|
|
+ // Now check usage, app1 uses:
|
|
|
+ // a2: used(x) = 40%
|
|
|
+ // abs-used(x) = 10%
|
|
|
+ // a: used(x) = 20%
|
|
|
+ // abs-used(x) = 10%
|
|
|
+ // root: used(x) = 10%
|
|
|
+ // abs-used(x) = 10%
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0.6f, 0.3f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.6f, 0.3f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0.8f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.8f, 0.2f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0.4f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0.4f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0.3f, 0.3f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.3f, 0.3f);
|
|
|
+
|
|
|
+ // Add nm3/nm4, double resource for both partitioned/non-partitioned
|
|
|
+ // resource, used capacity should be 1/2 of before
|
|
|
+ mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h3", 0), toSet("x")));
|
|
|
+ rm.registerNode("h3:1234", 10 * GB); // label = x
|
|
|
+ rm.registerNode("h4:1234", 10 * GB); // label = <empty>
|
|
|
+
|
|
|
+ waitSchedulerNodeJoined(rm, 4);
|
|
|
+
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0.3f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.3f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0.4f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.4f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0.2f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0.2f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0.15f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.15f, 0.15f);
|
|
|
+
|
|
|
+ // Reinitialize queue, makes A's capacity double, and B's capacity to be 0
|
|
|
+ csConf.setCapacity(A, 100); // was 50
|
|
|
+ csConf.setCapacityByLabel(A, "x", 100); // was 50
|
|
|
+ csConf.setCapacity(B, 0); // was 50
|
|
|
+ csConf.setCapacityByLabel(B, "x", 0); // was 50
|
|
|
+ cs.reinitialize(csConf, rm.getRMContext());
|
|
|
+
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0.15f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.15f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0.2f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.2f, 0.1f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0.1f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0.1f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0.15f, 0.15f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.15f, 0.15f);
|
|
|
+
|
|
|
+ // Release all task containers from a1, check usage
|
|
|
+ am1.allocate(null, Arrays.asList(
|
|
|
+ ContainerId.newContainerId(am1.getApplicationAttemptId(), 2),
|
|
|
+ ContainerId.newContainerId(am1.getApplicationAttemptId(), 3),
|
|
|
+ ContainerId.newContainerId(am1.getApplicationAttemptId(), 4)));
|
|
|
+ checkQueueUsedCapacity("a", cs, "x", 0.05f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("a", cs, "", 0.10f, 0.10f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "x", 0.0f, 0.0f);
|
|
|
+ checkQueueUsedCapacity("a1", cs, "", 0.1f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "x", 0.1f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("a2", cs, "", 0.1f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "x", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("b", cs, "", 0f, 0f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "x", 0.05f, 0.05f);
|
|
|
+ checkQueueUsedCapacity("root", cs, "", 0.10f, 0.10f);
|
|
|
+
|
|
|
+ rm.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOrderOfAllocationOnPartitions()
|
|
|
+ throws Exception {
|
|
|
+ /**
|
|
|
+ * Test case: have a following queue structure:
|
|
|
+ *
|
|
|
+ * <pre>
|
|
|
+ * root
|
|
|
+ * ________________
|
|
|
+ * / | \ \
|
|
|
+ * a (x) b (x) c d
|
|
|
+ * </pre>
|
|
|
+ *
|
|
|
+ * Both a/b can access x, we need to verify when
|
|
|
+ * <pre>
|
|
|
+ * When doing allocation on partitioned nodes,
|
|
|
+ * - Queue has accessibility to the node will go first
|
|
|
+ * - When accessibility is same
|
|
|
+ * - Queue has less used_capacity on given partition will go first
|
|
|
+ * - When used_capacity is same
|
|
|
+ * - Queue has more abs_capacity will go first
|
|
|
+ * </pre>
|
|
|
+ *
|
|
|
+ * used capacity / absolute used capacity of queues are correctly updated.
|
|
|
+ */
|
|
|
+
|
|
|
+ CapacitySchedulerConfiguration csConf =
|
|
|
+ new CapacitySchedulerConfiguration(this.conf);
|
|
|
+
|
|
|
+ // Define top-level queues
|
|
|
+ csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { "a",
|
|
|
+ "b", "c", "d" });
|
|
|
+ csConf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
|
|
|
+
|
|
|
+ final String A = CapacitySchedulerConfiguration.ROOT + ".a";
|
|
|
+ csConf.setCapacity(A, 25);
|
|
|
+ csConf.setAccessibleNodeLabels(A, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(A, "x", 30);
|
|
|
+
|
|
|
+ final String B = CapacitySchedulerConfiguration.ROOT + ".b";
|
|
|
+ csConf.setCapacity(B, 25);
|
|
|
+ csConf.setAccessibleNodeLabels(B, toSet("x"));
|
|
|
+ csConf.setCapacityByLabel(B, "x", 70);
|
|
|
+
|
|
|
+ final String C = CapacitySchedulerConfiguration.ROOT + ".c";
|
|
|
+ csConf.setCapacity(C, 25);
|
|
|
+
|
|
|
+ final String D = CapacitySchedulerConfiguration.ROOT + ".d";
|
|
|
+ csConf.setCapacity(D, 25);
|
|
|
+
|
|
|
+ // set node -> label
|
|
|
+ mgr.addToCluserNodeLabels(ImmutableSet.of("x"));
|
|
|
+ // Makes x to be non-exclusive node labels
|
|
|
+ mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("x", false)));
|
|
|
+ mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
|
|
|
+
|
|
|
+ // inject node label manager
|
|
|
+ MockRM rm = new MockRM(csConf) {
|
|
|
+ @Override
|
|
|
+ public RMNodeLabelsManager createNodeLabelManager() {
|
|
|
+ return mgr;
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ rm.getRMContext().setNodeLabelManager(mgr);
|
|
|
+ rm.start();
|
|
|
+
|
|
|
+ CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
+
|
|
|
+ MockNM nm1 = rm.registerNode("h1:1234", 10 * GB); // label = x
|
|
|
+ MockNM nm2 = rm.registerNode("h2:1234", 10 * GB); // label = <empty>
|
|
|
+
|
|
|
+ // app1 -> a
|
|
|
+ RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a");
|
|
|
+ MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
|
|
|
+
|
|
|
+ // app2 -> b
|
|
|
+ RMApp app2 = rm.submitApp(1 * GB, "app", "user", null, "b");
|
|
|
+ MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
|
|
|
+
|
|
|
+ // app3 -> c
|
|
|
+ RMApp app3 = rm.submitApp(1 * GB, "app", "user", null, "c");
|
|
|
+ MockAM am3 = MockRM.launchAndRegisterAM(app3, rm, nm2);
|
|
|
+
|
|
|
+ // app4 -> d
|
|
|
+ RMApp app4 = rm.submitApp(1 * GB, "app", "user", null, "d");
|
|
|
+ MockAM am4 = MockRM.launchAndRegisterAM(app4, rm, nm2);
|
|
|
+
|
|
|
+ // Test case 1
|
|
|
+ // Both a/b has used_capacity(x) = 0, when doing exclusive allocation, b
|
|
|
+ // will go first since b has more capacity(x)
|
|
|
+ am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
|
|
|
+ am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 1);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 2
|
|
|
+ // Do another allocation, a will go first since it has 0 use_capacity(x) and
|
|
|
+ // b has 1/7 used_capacity(x)
|
|
|
+ am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "x");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 1);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 3
|
|
|
+ // Just like above, when doing non-exclusive allocation, b will go first as well.
|
|
|
+ am1.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
|
|
|
+ am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 2);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 4
|
|
|
+ // After b allocated, we should be able to allocate non-exlusive container in a
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 2);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 5
|
|
|
+ // b/c/d asks non-exclusive container together, b will go first irrelated to
|
|
|
+ // used_capacity(x)
|
|
|
+ am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>(), "");
|
|
|
+ am3.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "");
|
|
|
+ am4.allocate("*", 1 * GB, 2, new ArrayList<ContainerId>(), "");
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 2);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am3.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am4.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 6
|
|
|
+ // After b allocated, c will go first by lexicographic order
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 1);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am3.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(0, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am4.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 7
|
|
|
+ // After c allocated, d will go first because it has less used_capacity(x)
|
|
|
+ // than c
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 2);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am3.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am4.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ // Test case 8
|
|
|
+ // After d allocated, c will go first, c/d has same use_capacity(x), so compare c/d's lexicographic order
|
|
|
+ doNMHeartbeat(rm, nm1.getNodeId(), 1);
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am1.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(3, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am2.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(2, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am3.getApplicationAttemptId()));
|
|
|
+ checkNumOfContainersInAnAppOnGivenNode(1, nm1.getNodeId(),
|
|
|
+ cs.getApplicationAttempt(am4.getApplicationAttemptId()));
|
|
|
+
|
|
|
+ }
|
|
|
}
|